Skip to content

KAFKA-10766: Unit test cases for RocksDBRangeIterator#9717

Merged
guozhangwang merged 1 commit intoapache:trunkfrom
vamossagar12:KAFKA-10766
Mar 2, 2021
Merged

KAFKA-10766: Unit test cases for RocksDBRangeIterator#9717
guozhangwang merged 1 commit intoapache:trunkfrom
vamossagar12:KAFKA-10766

Conversation

@vamossagar12
Copy link
Copy Markdown
Contributor

This PR aims to add unit test cases for RocksDBRangeIterator which were missing.

@vamossagar12
Copy link
Copy Markdown
Contributor Author

@cadonna , this is post our discussion on prefix-scan-pr. Plz review whenever you find the time.

Copy link
Copy Markdown
Member

@cadonna cadonna left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@vamossagar12 Thank you for the PR!

Here my feedback!

Comment on lines 59 to 88
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It is not a good idea to change a class due to unit tests if it is not absolutely necessary. In this case it is definitely not necessary. See my suggestion how to change the tests.

Comment on lines 44 to 81
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

You usually do not want to mock the class under test, because you want to test it. Also partial mocks should only be used if absolutely necessary. A rule of thumb is if a partial mock is needed then most probably the design has a flaw.
In this specific case, you should mock RocksDB's iterator. For the class under test, you should test hasNext(), next(), peekNextKey() and close(), because those are the one exposed (makeNext() should actually de declared as protected, IMO).

Suggested change
final RocksDBRangeIterator rocksDBRangeIterator = mockRocksDBRangeIterator();
final String key1 = "a";
final String key2 = "b";
final String key3 = "c";
final String key4 = "d";
final String value = "value";
final Bytes key1Bytes = Bytes.wrap(key1.getBytes());
final Bytes key2Bytes = Bytes.wrap(key2.getBytes());
final Bytes key3Bytes = Bytes.wrap(key3.getBytes());
final Bytes key4Bytes = Bytes.wrap(key4.getBytes());
final byte[] valueBytes = value.getBytes();
expect(rocksDBRangeIterator.getNext())
.andReturn(KeyValue.pair(key1Bytes, valueBytes))
.andReturn(KeyValue.pair(key2Bytes, valueBytes))
.andReturn(KeyValue.pair(key3Bytes, valueBytes))
.andReturn(KeyValue.pair(key4Bytes, valueBytes));
expect(rocksDBRangeIterator.getRawLastKey())
.andReturn(key3Bytes.get())
.anyTimes();
expect(rocksDBRangeIterator.isForward())
.andReturn(true)
.anyTimes();
expect(rocksDBRangeIterator.getComparator())
.andReturn(Bytes.BYTES_LEXICO_COMPARATOR)
.anyTimes();
replay(rocksDBRangeIterator);
assertThat(rocksDBRangeIterator.makeNext().key, is(key1Bytes));
assertThat(rocksDBRangeIterator.makeNext().key, is(key2Bytes));
assertThat(rocksDBRangeIterator.makeNext().key, is(key3Bytes));
assertThat(rocksDBRangeIterator.makeNext(), is(nullValue()));
final String key1 = "a";
final String key2 = "b";
final String key3 = "c";
final String key4 = "d";
final String value = "value";
final Bytes key1Bytes = Bytes.wrap(key1.getBytes());
final Bytes key2Bytes = Bytes.wrap(key2.getBytes());
final Bytes key3Bytes = Bytes.wrap(key3.getBytes());
final Bytes key4Bytes = Bytes.wrap(key4.getBytes());
final byte[] valueBytes = value.getBytes();
final RocksIterator rocksIterator = mock(RocksIterator.class);
rocksIterator.seek(key1Bytes.get());
expect(rocksIterator.isValid())
.andReturn(true)
.andReturn(true)
.andReturn(true)
.andReturn(true)
.andReturn(false);
expect(rocksIterator.key())
.andReturn(key1Bytes.get())
.andReturn(key2Bytes.get())
.andReturn(key3Bytes.get())
.andReturn(key4Bytes.get());
expect(rocksIterator.value()).andReturn(valueBytes).times(4);
rocksIterator.next();
expectLastCall().times(4);
replay(rocksIterator);
final RocksDBRangeIterator rocksDBRangeIterator = new RocksDBRangeIterator(
STORE_NAME,
rocksIterator,
Collections.emptySet(),
key1Bytes,
key4Bytes,
true
);
assertThat(rocksDBRangeIterator.hasNext(), is(true));
assertThat(rocksDBRangeIterator.next().key, is(key1Bytes));
assertThat(rocksDBRangeIterator.hasNext(), is(true));
assertThat(rocksDBRangeIterator.next().key, is(key2Bytes));
assertThat(rocksDBRangeIterator.hasNext(), is(true));
assertThat(rocksDBRangeIterator.next().key, is(key3Bytes));
assertThat(rocksDBRangeIterator.hasNext(), is(true));
assertThat(rocksDBRangeIterator.next().key, is(key4Bytes));
assertThat(rocksDBRangeIterator.hasNext(), is(false));
verify(rocksIterator);

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@cadonna , honestly even I wasn't making the changes that I ended up making but I assumed that makeNext is the only method that needs to be tested wrt Range Iterator as other methods are inherited by it so those should have test cases separately. I have changes the test cases now.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@cadonna , did you get a chance to review these changes?

Copy link
Copy Markdown
Member

@cadonna cadonna left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@vamossagar12 Thank you for the updates!

Here my feedback!

I have mainly minor comments and one major about testing peekNextKey().

@cadonna
Copy link
Copy Markdown
Member

cadonna commented Jan 28, 2021

honestly even I wasn't making the changes that I ended up making but I assumed that makeNext is the only method that needs to be tested wrt Range Iterator as other methods are inherited by it so those should have test cases separately.

Fair point. I would agree, if we had tests for RocksDbIterator, but we haven't. I thought, it would be easier to restrict the tests to the range iterator and in a second step refactor the code to test also the parent class.

@vamossagar12
Copy link
Copy Markdown
Contributor Author

vamossagar12 commented Jan 31, 2021

@cadonna , done. Also, I need to create a new ticket to add the tests for prefix scan related changes. I will add them once the KIP-614 is merged.

@vamossagar12
Copy link
Copy Markdown
Contributor Author

@cadonna , i have added the test case for prefix scan as well.

Copy link
Copy Markdown
Member

@cadonna cadonna left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@vamossagar12 Thanks!

I have just one major comment.

Copy link
Copy Markdown
Member

@cadonna cadonna left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks, @vamossagar12 !
LGTM!

@cadonna
Copy link
Copy Markdown
Member

cadonna commented Feb 25, 2021

Call for committer approval: @guozhangwang @ableegoldman @vvcephei @mjsax @abbccdda

@guozhangwang guozhangwang merged commit 4c5867a into apache:trunk Mar 2, 2021
guozhangwang pushed a commit that referenced this pull request Mar 2, 2021
This PR aims to add unit test cases for RocksDBRangeIterator which were missing.

Reviewers: Bruno Cadonna <cadonna@confluent.io>, Guozhang Wang <guozhang@confluent.io>
@guozhangwang
Copy link
Copy Markdown
Contributor

LGTM. Merged to trunk and cherry-picked to 2.8 cc @vvcephei

ijuma added a commit to ijuma/kafka that referenced this pull request Mar 2, 2021
* apache-github/trunk: (37 commits)
  KAFKA-10357: Extract setup of changelog from Streams partition assignor (apache#10163)
  KAFKA-10251: increase timeout for consuming records (apache#10228)
  KAFKA-12394; Return `TOPIC_AUTHORIZATION_FAILED` in delete topic response if no describe permission (apache#10223)
  MINOR: Disable transactional/idempotent system tests for Raft quorums (apache#10224)
  KAFKA-10766: Unit test cases for RocksDBRangeIterator (apache#9717)
  KAFKA-12289: Adding test cases for prefix scan in InMemoryKeyValueStore (apache#10052)
  KAFKA-12268: Implement task idling semantics via currentLag API (apache#10137)
  MINOR: Time and log producer state recovery phases (apache#10241)
  MINOR: correct the error message of validating uint32 (apache#10193)
  MINOR: Format the revoking active log output in `StreamsPartitionAssignor` (apache#10242)
  KAFKA-12323 Follow-up: Refactor the unit test a bit (apache#10205)
  MINOR: Remove stack trace of the lock exception in a debug log4j (apache#10231)
  MINOR: Word count should account for extra whitespaces between words (apache#10229)
  MINOR; Small refactor in `GroupMetadata` (apache#10236)
  KAFKA-10340: Proactively close producer when cancelling source tasks (apache#10016)
  KAFKA-12329; kafka-reassign-partitions command should give a better error message when a topic does not exist (apache#10141)
  KAFKA-12254: Ensure MM2 creates topics with source topic configs (apache#10217)
  MINOR: fix kafka-metadata-shell.sh (apache#10226)
  KAFKA-12374: Add missing config sasl.mechanism.controller.protocol (apache#10199)
  KAFKA-10101: Fix edge cases in Log.recoverLog and LogManager.loadLogs (apache#8812)
  ...
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

3 participants