Skip to content

Commit f2ac0c6

Browse files
authored
KAFKA-12221 remove PowerMock from connect-json module and connect-transforms module (apache#9924)
Reviewers: Ismael Juma <[email protected]>
1 parent 5a4bbb9 commit f2ac0c6

File tree

3 files changed

+19
-18
lines changed

3 files changed

+19
-18
lines changed

build.gradle

Lines changed: 0 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -1889,8 +1889,6 @@ project(':connect:transforms') {
18891889

18901890
testCompile libs.easymock
18911891
testCompile libs.junitJupiter
1892-
testCompile libs.powermockJunit4
1893-
testCompile libs.powermockEasymock
18941892

18951893
testRuntime libs.slf4jlog4j
18961894
testCompile project(':clients').sourceSets.test.output
@@ -1929,8 +1927,6 @@ project(':connect:json') {
19291927

19301928
testCompile libs.easymock
19311929
testCompile libs.junitJupiter
1932-
testCompile libs.powermockJunit4
1933-
testCompile libs.powermockEasymock
19341930

19351931
testRuntime libs.slf4jlog4j
19361932
testCompile project(':clients').sourceSets.test.output

connect/json/src/main/java/org/apache/kafka/connect/json/JsonConverter.java

Lines changed: 10 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -299,6 +299,16 @@ public JsonConverter() {
299299
);
300300
}
301301

302+
// visible for testing
303+
long sizeOfFromConnectSchemaCache() {
304+
return fromConnectSchemaCache.size();
305+
}
306+
307+
// visible for testing
308+
long sizeOfToConnectSchemaCache() {
309+
return toConnectSchemaCache.size();
310+
}
311+
302312
@Override
303313
public ConfigDef config() {
304314
return JsonConverterConfig.configDef();

connect/json/src/test/java/org/apache/kafka/connect/json/JsonConverterTest.java

Lines changed: 9 additions & 14 deletions
Original file line numberDiff line numberDiff line change
@@ -21,8 +21,6 @@
2121
import com.fasterxml.jackson.databind.ObjectMapper;
2222
import com.fasterxml.jackson.databind.node.ArrayNode;
2323
import com.fasterxml.jackson.databind.node.JsonNodeFactory;
24-
import com.fasterxml.jackson.databind.node.ObjectNode;
25-
import org.apache.kafka.common.cache.Cache;
2624
import org.apache.kafka.common.utils.Utils;
2725
import org.apache.kafka.connect.data.Date;
2826
import org.apache.kafka.connect.data.Decimal;
@@ -35,7 +33,6 @@
3533
import org.apache.kafka.connect.errors.DataException;
3634
import org.junit.jupiter.api.BeforeEach;
3735
import org.junit.jupiter.api.Test;
38-
import org.powermock.reflect.Whitebox;
3936

4037
import java.io.File;
4138
import java.io.IOException;
@@ -484,22 +481,21 @@ public void testJsonSchemaMetadataTranslation() {
484481

485482
@Test
486483
public void testCacheSchemaToConnectConversion() {
487-
Cache<JsonNode, Schema> cache = Whitebox.getInternalState(converter, "toConnectSchemaCache");
488-
assertEquals(0, cache.size());
484+
assertEquals(0, converter.sizeOfToConnectSchemaCache());
489485

490486
converter.toConnectData(TOPIC, "{ \"schema\": { \"type\": \"boolean\" }, \"payload\": true }".getBytes());
491-
assertEquals(1, cache.size());
487+
assertEquals(1, converter.sizeOfToConnectSchemaCache());
492488

493489
converter.toConnectData(TOPIC, "{ \"schema\": { \"type\": \"boolean\" }, \"payload\": true }".getBytes());
494-
assertEquals(1, cache.size());
490+
assertEquals(1, converter.sizeOfToConnectSchemaCache());
495491

496492
// Different schema should also get cached
497493
converter.toConnectData(TOPIC, "{ \"schema\": { \"type\": \"boolean\", \"optional\": true }, \"payload\": true }".getBytes());
498-
assertEquals(2, cache.size());
494+
assertEquals(2, converter.sizeOfToConnectSchemaCache());
499495

500496
// Even equivalent, but different JSON encoding of schema, should get different cache entry
501497
converter.toConnectData(TOPIC, "{ \"schema\": { \"type\": \"boolean\", \"optional\": false }, \"payload\": true }".getBytes());
502-
assertEquals(3, cache.size());
498+
assertEquals(3, converter.sizeOfToConnectSchemaCache());
503499
}
504500

505501
// Schema types
@@ -848,20 +844,19 @@ public void noSchemaToJson() {
848844

849845
@Test
850846
public void testCacheSchemaToJsonConversion() {
851-
Cache<Schema, ObjectNode> cache = Whitebox.getInternalState(converter, "fromConnectSchemaCache");
852-
assertEquals(0, cache.size());
847+
assertEquals(0, converter.sizeOfFromConnectSchemaCache());
853848

854849
// Repeated conversion of the same schema, even if the schema object is different should return the same Java
855850
// object
856851
converter.fromConnectData(TOPIC, SchemaBuilder.bool().build(), true);
857-
assertEquals(1, cache.size());
852+
assertEquals(1, converter.sizeOfFromConnectSchemaCache());
858853

859854
converter.fromConnectData(TOPIC, SchemaBuilder.bool().build(), true);
860-
assertEquals(1, cache.size());
855+
assertEquals(1, converter.sizeOfFromConnectSchemaCache());
861856

862857
// Validate that a similar, but different schema correctly returns a different schema.
863858
converter.fromConnectData(TOPIC, SchemaBuilder.bool().optional().build(), true);
864-
assertEquals(2, cache.size());
859+
assertEquals(2, converter.sizeOfFromConnectSchemaCache());
865860
}
866861

867862
@Test

0 commit comments

Comments
 (0)