mirror of https://github.com/apache/kafka.git
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:
parent
5a4bbb9f91
commit
f2ac0c62ef
|
@ -1889,8 +1889,6 @@ project(':connect:transforms') {
|
||||||
|
|
||||||
testCompile libs.easymock
|
testCompile libs.easymock
|
||||||
testCompile libs.junitJupiter
|
testCompile libs.junitJupiter
|
||||||
testCompile libs.powermockJunit4
|
|
||||||
testCompile libs.powermockEasymock
|
|
||||||
|
|
||||||
testRuntime libs.slf4jlog4j
|
testRuntime libs.slf4jlog4j
|
||||||
testCompile project(':clients').sourceSets.test.output
|
testCompile project(':clients').sourceSets.test.output
|
||||||
|
@ -1929,8 +1927,6 @@ project(':connect:json') {
|
||||||
|
|
||||||
testCompile libs.easymock
|
testCompile libs.easymock
|
||||||
testCompile libs.junitJupiter
|
testCompile libs.junitJupiter
|
||||||
testCompile libs.powermockJunit4
|
|
||||||
testCompile libs.powermockEasymock
|
|
||||||
|
|
||||||
testRuntime libs.slf4jlog4j
|
testRuntime libs.slf4jlog4j
|
||||||
testCompile project(':clients').sourceSets.test.output
|
testCompile project(':clients').sourceSets.test.output
|
||||||
|
|
|
@ -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
|
@Override
|
||||||
public ConfigDef config() {
|
public ConfigDef config() {
|
||||||
return JsonConverterConfig.configDef();
|
return JsonConverterConfig.configDef();
|
||||||
|
|
|
@ -21,8 +21,6 @@ import com.fasterxml.jackson.databind.JsonNode;
|
||||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||||
import com.fasterxml.jackson.databind.node.ArrayNode;
|
import com.fasterxml.jackson.databind.node.ArrayNode;
|
||||||
import com.fasterxml.jackson.databind.node.JsonNodeFactory;
|
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.common.utils.Utils;
|
||||||
import org.apache.kafka.connect.data.Date;
|
import org.apache.kafka.connect.data.Date;
|
||||||
import org.apache.kafka.connect.data.Decimal;
|
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.apache.kafka.connect.errors.DataException;
|
||||||
import org.junit.jupiter.api.BeforeEach;
|
import org.junit.jupiter.api.BeforeEach;
|
||||||
import org.junit.jupiter.api.Test;
|
import org.junit.jupiter.api.Test;
|
||||||
import org.powermock.reflect.Whitebox;
|
|
||||||
|
|
||||||
import java.io.File;
|
import java.io.File;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
@ -484,22 +481,21 @@ public class JsonConverterTest {
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testCacheSchemaToConnectConversion() {
|
public void testCacheSchemaToConnectConversion() {
|
||||||
Cache<JsonNode, Schema> cache = Whitebox.getInternalState(converter, "toConnectSchemaCache");
|
assertEquals(0, converter.sizeOfToConnectSchemaCache());
|
||||||
assertEquals(0, cache.size());
|
|
||||||
|
|
||||||
converter.toConnectData(TOPIC, "{ \"schema\": { \"type\": \"boolean\" }, \"payload\": true }".getBytes());
|
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());
|
converter.toConnectData(TOPIC, "{ \"schema\": { \"type\": \"boolean\" }, \"payload\": true }".getBytes());
|
||||||
assertEquals(1, cache.size());
|
assertEquals(1, converter.sizeOfToConnectSchemaCache());
|
||||||
|
|
||||||
// Different schema should also get cached
|
// Different schema should also get cached
|
||||||
converter.toConnectData(TOPIC, "{ \"schema\": { \"type\": \"boolean\", \"optional\": true }, \"payload\": true }".getBytes());
|
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
|
// Even equivalent, but different JSON encoding of schema, should get different cache entry
|
||||||
converter.toConnectData(TOPIC, "{ \"schema\": { \"type\": \"boolean\", \"optional\": false }, \"payload\": true }".getBytes());
|
converter.toConnectData(TOPIC, "{ \"schema\": { \"type\": \"boolean\", \"optional\": false }, \"payload\": true }".getBytes());
|
||||||
assertEquals(3, cache.size());
|
assertEquals(3, converter.sizeOfToConnectSchemaCache());
|
||||||
}
|
}
|
||||||
|
|
||||||
// Schema types
|
// Schema types
|
||||||
|
@ -848,20 +844,19 @@ public class JsonConverterTest {
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testCacheSchemaToJsonConversion() {
|
public void testCacheSchemaToJsonConversion() {
|
||||||
Cache<Schema, ObjectNode> cache = Whitebox.getInternalState(converter, "fromConnectSchemaCache");
|
assertEquals(0, converter.sizeOfFromConnectSchemaCache());
|
||||||
assertEquals(0, cache.size());
|
|
||||||
|
|
||||||
// Repeated conversion of the same schema, even if the schema object is different should return the same Java
|
// Repeated conversion of the same schema, even if the schema object is different should return the same Java
|
||||||
// object
|
// object
|
||||||
converter.fromConnectData(TOPIC, SchemaBuilder.bool().build(), true);
|
converter.fromConnectData(TOPIC, SchemaBuilder.bool().build(), true);
|
||||||
assertEquals(1, cache.size());
|
assertEquals(1, converter.sizeOfFromConnectSchemaCache());
|
||||||
|
|
||||||
converter.fromConnectData(TOPIC, SchemaBuilder.bool().build(), true);
|
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.
|
// Validate that a similar, but different schema correctly returns a different schema.
|
||||||
converter.fromConnectData(TOPIC, SchemaBuilder.bool().optional().build(), true);
|
converter.fromConnectData(TOPIC, SchemaBuilder.bool().optional().build(), true);
|
||||||
assertEquals(2, cache.size());
|
assertEquals(2, converter.sizeOfFromConnectSchemaCache());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
|
Loading…
Reference in New Issue