You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kafka.apache.org by ch...@apache.org on 2021/05/02 05:09:30 UTC
[kafka] branch trunk updated: KAFKA-12661 ConfigEntry#equal does
not compare other fields when value is NOT null (#10446)
This is an automated email from the ASF dual-hosted git repository.
chia7712 pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git
The following commit(s) were added to refs/heads/trunk by this push:
new e73731d KAFKA-12661 ConfigEntry#equal does not compare other fields when value is NOT null (#10446)
e73731d is described below
commit e73731d8486735c882a721b92df450eb4be59f15
Author: Chia-Ping Tsai <ch...@gmail.com>
AuthorDate: Sun May 2 13:07:08 2021 +0800
KAFKA-12661 ConfigEntry#equal does not compare other fields when value is NOT null (#10446)
Reviewers: Ismael Juma <is...@juma.me.uk>
---
.../org/apache/kafka/clients/admin/ConfigEntry.java | 18 ++++++++++++------
.../org/apache/kafka/clients/admin/ConfigTest.java | 15 +++++++++++++++
.../scala/unit/kafka/admin/ConfigCommandTest.scala | 9 ++++++++-
3 files changed, 35 insertions(+), 7 deletions(-)
diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/ConfigEntry.java b/clients/src/main/java/org/apache/kafka/clients/admin/ConfigEntry.java
index 0fed04f..e342666 100644
--- a/clients/src/main/java/org/apache/kafka/clients/admin/ConfigEntry.java
+++ b/clients/src/main/java/org/apache/kafka/clients/admin/ConfigEntry.java
@@ -150,11 +150,13 @@ public class ConfigEntry {
ConfigEntry that = (ConfigEntry) o;
return this.name.equals(that.name) &&
- this.value != null ? this.value.equals(that.value) : that.value == null &&
+ Objects.equals(this.value, that.value) &&
this.isSensitive == that.isSensitive &&
this.isReadOnly == that.isReadOnly &&
- this.source == that.source &&
- Objects.equals(this.synonyms, that.synonyms);
+ Objects.equals(this.source, that.source) &&
+ Objects.equals(this.synonyms, that.synonyms) &&
+ Objects.equals(this.type, that.type) &&
+ Objects.equals(this.documentation, that.documentation);
}
@Override
@@ -162,11 +164,13 @@ public class ConfigEntry {
final int prime = 31;
int result = 1;
result = prime * result + name.hashCode();
- result = prime * result + ((value == null) ? 0 : value.hashCode());
+ result = prime * result + Objects.hashCode(value);
result = prime * result + (isSensitive ? 1 : 0);
result = prime * result + (isReadOnly ? 1 : 0);
- result = prime * result + source.hashCode();
- result = prime * result + synonyms.hashCode();
+ result = prime * result + Objects.hashCode(source);
+ result = prime * result + Objects.hashCode(synonyms);
+ result = prime * result + Objects.hashCode(type);
+ result = prime * result + Objects.hashCode(documentation);
return result;
}
@@ -179,6 +183,8 @@ public class ConfigEntry {
", isSensitive=" + isSensitive +
", isReadOnly=" + isReadOnly +
", synonyms=" + synonyms +
+ ", type=" + type +
+ ", documentation=" + documentation +
")";
}
diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/ConfigTest.java b/clients/src/test/java/org/apache/kafka/clients/admin/ConfigTest.java
index 4008a54..59d1150 100644
--- a/clients/src/test/java/org/apache/kafka/clients/admin/ConfigTest.java
+++ b/clients/src/test/java/org/apache/kafka/clients/admin/ConfigTest.java
@@ -82,4 +82,19 @@ public class ConfigTest {
boolean isReadOnly, List<ConfigEntry.ConfigSynonym> synonyms) {
return new ConfigEntry(name, value, source, isSensitive, isReadOnly, synonyms, ConfigType.UNKNOWN, null);
}
+
+ @Test
+ public void testHashCodeAndEqualsWithNull() {
+ ConfigEntry ce0 = new ConfigEntry("abc", null, null, false, false, null, null, null);
+ ConfigEntry ce1 = new ConfigEntry("abc", null, null, false, false, null, null, null);
+ assertEquals(ce0, ce1);
+ assertEquals(ce0.hashCode(), ce1.hashCode());
+ }
+
+ @Test
+ public void testEquals() {
+ ConfigEntry ce0 = new ConfigEntry("abc", null, ConfigEntry.ConfigSource.DEFAULT_CONFIG, false, false, null, null, null);
+ ConfigEntry ce1 = new ConfigEntry("abc", null, ConfigEntry.ConfigSource.DYNAMIC_BROKER_CONFIG, false, false, null, null, null);
+ assertNotEquals(ce0, ce1);
+ }
}
diff --git a/core/src/test/scala/unit/kafka/admin/ConfigCommandTest.scala b/core/src/test/scala/unit/kafka/admin/ConfigCommandTest.scala
index 8603fbe..fde8faa 100644
--- a/core/src/test/scala/unit/kafka/admin/ConfigCommandTest.scala
+++ b/core/src/test/scala/unit/kafka/admin/ConfigCommandTest.scala
@@ -803,7 +803,14 @@ class ConfigCommandTest extends ZooKeeperTestHarness with Logging {
new AlterConfigOp(newConfigEntry("min.insync.replicas", "2"), AlterConfigOp.OpType.SET),
new AlterConfigOp(newConfigEntry("unclean.leader.election.enable", ""), AlterConfigOp.OpType.DELETE)
)
- assertEquals(expectedConfigOps, alterConfigOps.asScala.toSet)
+ assertEquals(expectedConfigOps.size, alterConfigOps.size)
+ expectedConfigOps.foreach { expectedOp =>
+ val actual = alterConfigOps.asScala.find(_.configEntry.name == expectedOp.configEntry.name)
+ assertNotEquals(actual, None)
+ assertEquals(expectedOp.opType, actual.get.opType)
+ assertEquals(expectedOp.configEntry.name, actual.get.configEntry.name)
+ assertEquals(expectedOp.configEntry.value, actual.get.configEntry.value)
+ }
alteredConfigs = true
alterResult
}