KAFKA-12221 remove PowerMock from connect-json module and connect-transforms module (#9924)

Reviewers: Ismael Juma <ismael@juma.me.uk>
This commit is contained in:
Chia-Ping Tsai 2021-01-19 00:05:44 +08:00 committed by GitHub
parent 5a4bbb9f91
commit f2ac0c62ef
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
3 changed files with 19 additions and 18 deletions

View File

@ -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

View File

@ -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();

View File

@ -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<JsonNode, Schema> 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<Schema, ObjectNode> 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