From f2ac0c62ef19566b4303e23f8c817e157a3fc14f Mon Sep 17 00:00:00 2001 From: Chia-Ping Tsai Date: Tue, 19 Jan 2021 00:05:44 +0800 Subject: [PATCH] KAFKA-12221 remove PowerMock from connect-json module and connect-transforms module (#9924) Reviewers: Ismael Juma --- build.gradle | 4 ---- .../kafka/connect/json/JsonConverter.java | 10 ++++++++ .../kafka/connect/json/JsonConverterTest.java | 23 ++++++++----------- 3 files changed, 19 insertions(+), 18 deletions(-) diff --git a/build.gradle b/build.gradle index f82ebecfd69..155b1e785e2 100644 --- a/build.gradle +++ b/build.gradle @@ -1889,8 +1889,6 @@ project(':connect:transforms') { testCompile libs.easymock testCompile libs.junitJupiter - testCompile libs.powermockJunit4 - testCompile libs.powermockEasymock testRuntime libs.slf4jlog4j testCompile project(':clients').sourceSets.test.output @@ -1929,8 +1927,6 @@ project(':connect:json') { testCompile libs.easymock testCompile libs.junitJupiter - testCompile libs.powermockJunit4 - testCompile libs.powermockEasymock testRuntime libs.slf4jlog4j testCompile project(':clients').sourceSets.test.output diff --git a/connect/json/src/main/java/org/apache/kafka/connect/json/JsonConverter.java b/connect/json/src/main/java/org/apache/kafka/connect/json/JsonConverter.java index 1a175381e8f..8f2f1c23a61 100644 --- a/connect/json/src/main/java/org/apache/kafka/connect/json/JsonConverter.java +++ b/connect/json/src/main/java/org/apache/kafka/connect/json/JsonConverter.java @@ -299,6 +299,16 @@ public class JsonConverter implements Converter, HeaderConverter { ); } + // visible for testing + long sizeOfFromConnectSchemaCache() { + return fromConnectSchemaCache.size(); + } + + // visible for testing + long sizeOfToConnectSchemaCache() { + return toConnectSchemaCache.size(); + } + @Override public ConfigDef config() { return JsonConverterConfig.configDef(); diff --git a/connect/json/src/test/java/org/apache/kafka/connect/json/JsonConverterTest.java b/connect/json/src/test/java/org/apache/kafka/connect/json/JsonConverterTest.java index 912d06dd75f..be556b8e915 100644 --- a/connect/json/src/test/java/org/apache/kafka/connect/json/JsonConverterTest.java +++ b/connect/json/src/test/java/org/apache/kafka/connect/json/JsonConverterTest.java @@ -21,8 +21,6 @@ import com.fasterxml.jackson.databind.JsonNode; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.node.ArrayNode; import com.fasterxml.jackson.databind.node.JsonNodeFactory; -import com.fasterxml.jackson.databind.node.ObjectNode; -import org.apache.kafka.common.cache.Cache; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.connect.data.Date; import org.apache.kafka.connect.data.Decimal; @@ -35,7 +33,6 @@ import org.apache.kafka.connect.data.Timestamp; import org.apache.kafka.connect.errors.DataException; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; -import org.powermock.reflect.Whitebox; import java.io.File; import java.io.IOException; @@ -484,22 +481,21 @@ public class JsonConverterTest { @Test public void testCacheSchemaToConnectConversion() { - Cache cache = Whitebox.getInternalState(converter, "toConnectSchemaCache"); - assertEquals(0, cache.size()); + assertEquals(0, converter.sizeOfToConnectSchemaCache()); converter.toConnectData(TOPIC, "{ \"schema\": { \"type\": \"boolean\" }, \"payload\": true }".getBytes()); - assertEquals(1, cache.size()); + assertEquals(1, converter.sizeOfToConnectSchemaCache()); converter.toConnectData(TOPIC, "{ \"schema\": { \"type\": \"boolean\" }, \"payload\": true }".getBytes()); - assertEquals(1, cache.size()); + assertEquals(1, converter.sizeOfToConnectSchemaCache()); // Different schema should also get cached converter.toConnectData(TOPIC, "{ \"schema\": { \"type\": \"boolean\", \"optional\": true }, \"payload\": true }".getBytes()); - assertEquals(2, cache.size()); + assertEquals(2, converter.sizeOfToConnectSchemaCache()); // Even equivalent, but different JSON encoding of schema, should get different cache entry converter.toConnectData(TOPIC, "{ \"schema\": { \"type\": \"boolean\", \"optional\": false }, \"payload\": true }".getBytes()); - assertEquals(3, cache.size()); + assertEquals(3, converter.sizeOfToConnectSchemaCache()); } // Schema types @@ -848,20 +844,19 @@ public class JsonConverterTest { @Test public void testCacheSchemaToJsonConversion() { - Cache cache = Whitebox.getInternalState(converter, "fromConnectSchemaCache"); - assertEquals(0, cache.size()); + assertEquals(0, converter.sizeOfFromConnectSchemaCache()); // Repeated conversion of the same schema, even if the schema object is different should return the same Java // object converter.fromConnectData(TOPIC, SchemaBuilder.bool().build(), true); - assertEquals(1, cache.size()); + assertEquals(1, converter.sizeOfFromConnectSchemaCache()); converter.fromConnectData(TOPIC, SchemaBuilder.bool().build(), true); - assertEquals(1, cache.size()); + assertEquals(1, converter.sizeOfFromConnectSchemaCache()); // Validate that a similar, but different schema correctly returns a different schema. converter.fromConnectData(TOPIC, SchemaBuilder.bool().optional().build(), true); - assertEquals(2, cache.size()); + assertEquals(2, converter.sizeOfFromConnectSchemaCache()); } @Test