mirror of https://github.com/apache/kafka.git
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:
parent
cf202cb6ac
commit
50bcb34d8d
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -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 =
|
||||
|
|
|
|||
Loading…
Reference in New Issue