MINOR: fix potential NPE in PartitionData.equals (#9391)

the field metadata is nullable (see https://github.com/apache/kafka/blob/trunk/clients/src/main/resources/common/message/OffsetFetchResponse.json#L50)

Reviewers: David Jacot <david.jacot@gmail.com>
This commit is contained in:
Chia-Ping Tsai 2020-10-18 21:29:13 +08:00 committed by GitHub
parent cf202cb6ac
commit 50bcb34d8d
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
2 changed files with 12 additions and 11 deletions

View File

@ -91,10 +91,10 @@ public class OffsetFetchResponse extends AbstractResponse {
if (!(other instanceof PartitionData))
return false;
PartitionData otherPartition = (PartitionData) other;
return this.offset == otherPartition.offset
&& this.leaderEpoch.equals(otherPartition.leaderEpoch)
&& this.metadata.equals(otherPartition.metadata)
&& this.error.equals(otherPartition.error);
return Objects.equals(this.offset, otherPartition.offset)
&& Objects.equals(this.leaderEpoch, otherPartition.leaderEpoch)
&& Objects.equals(this.metadata, otherPartition.metadata)
&& Objects.equals(this.error, otherPartition.error);
}
@Override

View File

@ -159,14 +159,15 @@ public class OffsetFetchResponseTest {
@Test
public void testNullableMetadata() {
PartitionData pd = new PartitionData(
offset,
leaderEpochOne,
null,
Errors.UNKNOWN_TOPIC_OR_PARTITION);
// test PartitionData.equals with null metadata
assertEquals(pd, pd);
partitionDataMap.clear();
partitionDataMap.put(new TopicPartition(topicOne, partitionOne),
new PartitionData(
offset,
leaderEpochOne,
null,
Errors.UNKNOWN_TOPIC_OR_PARTITION)
);
partitionDataMap.put(new TopicPartition(topicOne, partitionOne), pd);
OffsetFetchResponse response = new OffsetFetchResponse(throttleTimeMs, Errors.GROUP_AUTHORIZATION_FAILED, partitionDataMap);
OffsetFetchResponseData expectedData =