diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/DoubleDeserializer.java b/clients/src/main/java/org/apache/kafka/common/serialization/DoubleDeserializer.java new file mode 100644 index 00000000000..ed4f323791d --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/serialization/DoubleDeserializer.java @@ -0,0 +1,46 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE + * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file + * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the + * License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ +package org.apache.kafka.common.serialization; + +import org.apache.kafka.common.errors.SerializationException; + +import java.util.Map; + +public class DoubleDeserializer implements Deserializer { + + @Override + public void configure(Map configs, boolean isKey) { + // nothing to do + } + + @Override + public Double deserialize(String topic, byte[] data) { + if (data == null) + return null; + if (data.length != 8) { + throw new SerializationException("Size of data received by Deserializer is not 8"); + } + + long value = 0; + for (byte b : data) { + value <<= 8; + value |= b & 0xFF; + } + return Double.longBitsToDouble(value); + } + + @Override + public void close() { + // nothing to do + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/DoubleSerializer.java b/clients/src/main/java/org/apache/kafka/common/serialization/DoubleSerializer.java new file mode 100644 index 00000000000..9d01342c596 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/serialization/DoubleSerializer.java @@ -0,0 +1,46 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE + * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file + * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the + * License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ +package org.apache.kafka.common.serialization; + +import java.util.Map; + +public class DoubleSerializer implements Serializer { + + @Override + public void configure(Map configs, boolean isKey) { + // nothing to do + } + + @Override + public byte[] serialize(String topic, Double data) { + if (data == null) + return null; + + long bits = Double.doubleToLongBits(data); + return new byte[] { + (byte) (bits >>> 56), + (byte) (bits >>> 48), + (byte) (bits >>> 40), + (byte) (bits >>> 32), + (byte) (bits >>> 24), + (byte) (bits >>> 16), + (byte) (bits >>> 8), + (byte) bits + }; + } + + @Override + public void close() { + // nothing to do + } +} \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/Serde.java b/clients/src/main/java/org/apache/kafka/common/serialization/Serde.java new file mode 100644 index 00000000000..cc7944eb891 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/serialization/Serde.java @@ -0,0 +1,26 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE + * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file + * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the + * License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package org.apache.kafka.common.serialization; + +/** + * The interface for wrapping a serializer and deserializer for the given data type. + * + * @param + */ +public interface Serde { + + Serializer serializer(); + + Deserializer deserializer(); +} diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/Serdes.java b/clients/src/main/java/org/apache/kafka/common/serialization/Serdes.java new file mode 100644 index 00000000000..f27f74fbc84 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/serialization/Serdes.java @@ -0,0 +1,193 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE + * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file + * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the + * License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package org.apache.kafka.common.serialization; + +import java.nio.ByteBuffer; + +/** + * Factory for creating serializers / deserializers. + */ +public class Serdes { + + static public final class LongSerde implements Serde { + @Override + public Serializer serializer() { + return new LongSerializer(); + } + + @Override + public Deserializer deserializer() { + return new LongDeserializer(); + } + } + + static public final class IntegerSerde implements Serde { + @Override + public Serializer serializer() { + return new IntegerSerializer(); + } + + @Override + public Deserializer deserializer() { + return new IntegerDeserializer(); + } + } + + static public final class DoubleSerde implements Serde { + @Override + public Serializer serializer() { + return new DoubleSerializer(); + } + + @Override + public Deserializer deserializer() { + return new DoubleDeserializer(); + } + } + + static public final class StringSerde implements Serde { + @Override + public Serializer serializer() { + return new StringSerializer(); + } + + @Override + public Deserializer deserializer() { + return new StringDeserializer(); + } + } + + static public final class ByteBufferSerde implements Serde { + @Override + public Serializer serializer() { + return new ByteBufferSerializer(); + } + + @Override + public Deserializer deserializer() { + return new ByteBufferDeserializer(); + } + } + + static public final class ByteArraySerde implements Serde { + @Override + public Serializer serializer() { + return new ByteArraySerializer(); + } + + @Override + public Deserializer deserializer() { + return new ByteArrayDeserializer(); + } + } + + @SuppressWarnings("unchecked") + static public Serde serdeFrom(Class type) { + if (String.class.isAssignableFrom(type)) { + return (Serde) String(); + } + + if (Integer.class.isAssignableFrom(type)) { + return (Serde) Integer(); + } + + if (Long.class.isAssignableFrom(type)) { + return (Serde) Long(); + } + + if (Double.class.isAssignableFrom(type)) { + return (Serde) Double(); + } + + if (byte[].class.isAssignableFrom(type)) { + return (Serde) ByteArray(); + } + + if (ByteBufferSerde.class.isAssignableFrom(type)) { + return (Serde) ByteBuffer(); + } + + // TODO: we can also serializes objects of type T using generic Java serialization by default + throw new IllegalArgumentException("Unknown class for built-in serializer"); + } + + /** + * Construct a serde object from separate serializer and deserializer + * + * @param serializer must not be null. + * @param deserializer must not be null. + */ + static public Serde serdeFrom(final Serializer serializer, final Deserializer deserializer) { + if (serializer == null) { + throw new IllegalArgumentException("serializer must not be null"); + } + if (deserializer == null) { + throw new IllegalArgumentException("deserializer must not be null"); + } + + return new Serde() { + @Override + public Serializer serializer() { + return serializer; + } + + @Override + public Deserializer deserializer() { + return deserializer; + } + }; + } + + /* + * A serde for nullable long type. + */ + static public Serde Long() { + return new LongSerde(); + } + + /* + * A serde for nullable int type. + */ + static public Serde Integer() { + return new IntegerSerde(); + } + + /* + * A serde for nullable long type. + */ + static public Serde Double() { + return new DoubleSerde(); + } + + /* + * A serde for nullable string type. + */ + static public Serde String() { + return new StringSerde(); + } + + /* + * A serde for nullable byte array type. + */ + static public Serde ByteBuffer() { + return new ByteBufferSerde(); + } + + /* + * A serde for nullable byte array type. + */ + static public Serde ByteArray() { + return new ByteArraySerde(); + } +} diff --git a/clients/src/test/java/org/apache/kafka/common/serialization/SerializationTest.java b/clients/src/test/java/org/apache/kafka/common/serialization/SerializationTest.java index 87d9e0aa73e..e4cd6788173 100644 --- a/clients/src/test/java/org/apache/kafka/common/serialization/SerializationTest.java +++ b/clients/src/test/java/org/apache/kafka/common/serialization/SerializationTest.java @@ -24,34 +24,53 @@ import static org.junit.Assert.assertEquals; public class SerializationTest { - private static class SerDeser { - final Serializer serializer; - final Deserializer deserializer; + final private String topic = "testTopic"; - public SerDeser(Serializer serializer, Deserializer deserializer) { - this.serializer = serializer; - this.deserializer = deserializer; - } + private class DummyClass { + + } + + @Test + public void testSerdeFrom() { + Serde thisSerde = Serdes.serdeFrom(Long.class); + Serde otherSerde = Serdes.Long(); + + Long value = 423412424L; + + assertEquals("Should get the original long after serialization and deserialization", + value, thisSerde.deserializer().deserialize(topic, otherSerde.serializer().serialize(topic, value))); + assertEquals("Should get the original long after serialization and deserialization", + value, otherSerde.deserializer().deserialize(topic, thisSerde.serializer().serialize(topic, value))); + } + + @Test(expected = IllegalArgumentException.class) + public void testSerdeFromUnknown() { + Serdes.serdeFrom(DummyClass.class); + } + + @Test(expected = IllegalArgumentException.class) + public void testSerdeFromNotNull() { + Serdes.serdeFrom(null, Serdes.Long().deserializer()); } @Test public void testStringSerializer() { String str = "my string"; - String mytopic = "testTopic"; + List encodings = new ArrayList(); encodings.add("UTF8"); encodings.add("UTF-16"); for (String encoding : encodings) { - SerDeser serDeser = getStringSerDeser(encoding); - Serializer serializer = serDeser.serializer; - Deserializer deserializer = serDeser.deserializer; + Serde serDeser = getStringSerde(encoding); + Serializer serializer = serDeser.serializer(); + Deserializer deserializer = serDeser.deserializer(); assertEquals("Should get the original string after serialization and deserialization with encoding " + encoding, - str, deserializer.deserialize(mytopic, serializer.serialize(mytopic, str))); + str, deserializer.deserialize(topic, serializer.serialize(topic, str))); assertEquals("Should support null in serialization and deserialization with encoding " + encoding, - null, deserializer.deserialize(mytopic, serializer.serialize(mytopic, null))); + null, deserializer.deserialize(topic, serializer.serialize(topic, null))); } } @@ -61,18 +80,61 @@ public class SerializationTest { 423412424, -41243432 }; - String mytopic = "testTopic"; - Serializer serializer = new IntegerSerializer(); - Deserializer deserializer = new IntegerDeserializer(); + Serializer serializer = Serdes.Integer().serializer(); + Deserializer deserializer = Serdes.Integer().deserializer(); for (Integer integer : integers) { assertEquals("Should get the original integer after serialization and deserialization", - integer, deserializer.deserialize(mytopic, serializer.serialize(mytopic, integer))); + integer, deserializer.deserialize(topic, serializer.serialize(topic, integer))); } assertEquals("Should support null in serialization and deserialization", - null, deserializer.deserialize(mytopic, serializer.serialize(mytopic, null))); + null, deserializer.deserialize(topic, serializer.serialize(topic, null))); + + serializer.close(); + deserializer.close(); + } + + @Test + public void testLongSerializer() { + Long[] longs = new Long[]{ + 922337203685477580L, + -922337203685477581L + }; + + Serializer serializer = Serdes.Long().serializer(); + Deserializer deserializer = Serdes.Long().deserializer(); + + for (Long value : longs) { + assertEquals("Should get the original long after serialization and deserialization", + value, deserializer.deserialize(topic, serializer.serialize(topic, value))); + } + + assertEquals("Should support null in serialization and deserialization", + null, deserializer.deserialize(topic, serializer.serialize(topic, null))); + + serializer.close(); + deserializer.close(); + } + + @Test + public void testDoubleSerializer() { + Double[] doubles = new Double[]{ + 5678567.12312d, + -5678567.12341d + }; + + Serializer serializer = Serdes.Double().serializer(); + Deserializer deserializer = Serdes.Double().deserializer(); + + for (Double value : doubles) { + assertEquals("Should get the original double after serialization and deserialization", + value, deserializer.deserialize(topic, serializer.serialize(topic, value))); + } + + assertEquals("Should support null in serialization and deserialization", + null, deserializer.deserialize(topic, serializer.serialize(topic, null))); serializer.close(); deserializer.close(); @@ -80,34 +142,33 @@ public class SerializationTest { @Test public void testByteBufferSerializer() { - String mytopic = "testTopic"; ByteBuffer buf = ByteBuffer.allocate(10); buf.put("my string".getBytes()); - Serializer serializer = new ByteBufferSerializer(); - Deserializer deserializer = new ByteBufferDeserializer(); + Serializer serializer = Serdes.ByteBuffer().serializer(); + Deserializer deserializer = Serdes.ByteBuffer().deserializer(); assertEquals("Should get the original ByteBuffer after serialization and deserialization", - buf, deserializer.deserialize(mytopic, serializer.serialize(mytopic, buf))); + buf, deserializer.deserialize(topic, serializer.serialize(topic, buf))); assertEquals("Should support null in serialization and deserialization", - null, deserializer.deserialize(mytopic, serializer.serialize(mytopic, null))); + null, deserializer.deserialize(topic, serializer.serialize(topic, null))); serializer.close(); deserializer.close(); } - private SerDeser getStringSerDeser(String encoder) { + private Serde getStringSerde(String encoder) { Map serializerConfigs = new HashMap(); serializerConfigs.put("key.serializer.encoding", encoder); - Serializer serializer = new StringSerializer(); + Serializer serializer = Serdes.String().serializer(); serializer.configure(serializerConfigs, true); Map deserializerConfigs = new HashMap(); deserializerConfigs.put("key.deserializer.encoding", encoder); - Deserializer deserializer = new StringDeserializer(); + Deserializer deserializer = Serdes.String().deserializer(); deserializer.configure(deserializerConfigs, true); - return new SerDeser(serializer, deserializer); + return Serdes.serdeFrom(serializer, deserializer); } } diff --git a/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewTypedDemo.java b/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewTypedDemo.java index 4f9de291c34..15083b23b98 100644 --- a/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewTypedDemo.java +++ b/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewTypedDemo.java @@ -18,9 +18,8 @@ package org.apache.kafka.streams.examples.pageview; import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.serialization.Serializer; -import org.apache.kafka.common.serialization.StringSerializer; -import org.apache.kafka.common.serialization.StringDeserializer; import org.apache.kafka.streams.KafkaStreams; import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.kstream.HoppingWindows; @@ -83,10 +82,6 @@ public class PageViewTypedDemo { props.put(StreamsConfig.APPLICATION_ID_CONFIG, "streams-pageview-typed"); props.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); props.put(StreamsConfig.ZOOKEEPER_CONNECT_CONFIG, "localhost:2181"); - props.put(StreamsConfig.KEY_SERIALIZER_CLASS_CONFIG, StringSerializer.class); - props.put(StreamsConfig.VALUE_SERIALIZER_CLASS_CONFIG, JsonPOJOSerializer.class); - props.put(StreamsConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); - props.put(StreamsConfig.VALUE_DESERIALIZER_CLASS_CONFIG, JsonPOJODeserializer.class); props.put(StreamsConfig.TIMESTAMP_EXTRACTOR_CLASS_CONFIG, JsonTimestampExtractor.class); // setting offset reset to earliest so that we can re-run the demo code with the same pre-loaded data @@ -94,35 +89,44 @@ public class PageViewTypedDemo { KStreamBuilder builder = new KStreamBuilder(); - final Serializer stringSerializer = new StringSerializer(); - final Deserializer stringDeserializer = new StringDeserializer(); - // TODO: the following can be removed with a serialization factory Map serdeProps = new HashMap<>(); + final Serializer pageViewSerializer = new JsonPOJOSerializer<>(); + serdeProps.put("JsonPOJOClass", PageView.class); + pageViewSerializer.configure(serdeProps, false); + final Deserializer pageViewDeserializer = new JsonPOJODeserializer<>(); serdeProps.put("JsonPOJOClass", PageView.class); pageViewDeserializer.configure(serdeProps, false); - final Deserializer userProfileDeserializer = new JsonPOJODeserializer<>(); - serdeProps.put("JsonPOJOClass", UserProfile.class); - userProfileDeserializer.configure(serdeProps, false); - final Serializer userProfileSerializer = new JsonPOJOSerializer<>(); serdeProps.put("JsonPOJOClass", UserProfile.class); userProfileSerializer.configure(serdeProps, false); + final Deserializer userProfileDeserializer = new JsonPOJODeserializer<>(); + serdeProps.put("JsonPOJOClass", UserProfile.class); + userProfileDeserializer.configure(serdeProps, false); + final Serializer wPageViewByRegionSerializer = new JsonPOJOSerializer<>(); serdeProps.put("JsonPOJOClass", WindowedPageViewByRegion.class); wPageViewByRegionSerializer.configure(serdeProps, false); + final Deserializer wPageViewByRegionDeserializer = new JsonPOJODeserializer<>(); + serdeProps.put("JsonPOJOClass", WindowedPageViewByRegion.class); + wPageViewByRegionDeserializer.configure(serdeProps, false); + final Serializer regionCountSerializer = new JsonPOJOSerializer<>(); serdeProps.put("JsonPOJOClass", RegionCount.class); regionCountSerializer.configure(serdeProps, false); - KStream views = builder.stream(stringDeserializer, pageViewDeserializer, "streams-pageview-input"); + final Deserializer regionCountDeserializer = new JsonPOJODeserializer<>(); + serdeProps.put("JsonPOJOClass", RegionCount.class); + regionCountDeserializer.configure(serdeProps, false); - KTable users = builder.table(stringSerializer, userProfileSerializer, stringDeserializer, userProfileDeserializer, "streams-userprofile-input"); + KStream views = builder.stream(Serdes.String(), Serdes.serdeFrom(pageViewSerializer, pageViewDeserializer), "streams-pageview-input"); + + KTable users = builder.table(Serdes.String(), Serdes.serdeFrom(userProfileSerializer, userProfileDeserializer), "streams-userprofile-input"); KStream regionCount = views .leftJoin(users, new ValueJoiner() { @@ -146,8 +150,7 @@ public class PageViewTypedDemo { return new KeyValue<>(viewRegion.region, viewRegion); } }) - .countByKey(HoppingWindows.of("GeoPageViewsWindow").with(7 * 24 * 60 * 60 * 1000), - stringSerializer, stringDeserializer) + .countByKey(HoppingWindows.of("GeoPageViewsWindow").with(7 * 24 * 60 * 60 * 1000), Serdes.String()) // TODO: we can merge ths toStream().map(...) with a single toStream(...) .toStream() .map(new KeyValueMapper, Long, KeyValue>() { @@ -166,7 +169,7 @@ public class PageViewTypedDemo { }); // write to the result topic - regionCount.to("streams-pageviewstats-typed-output", wPageViewByRegionSerializer, regionCountSerializer); + regionCount.to("streams-pageviewstats-typed-output", Serdes.serdeFrom(wPageViewByRegionSerializer, wPageViewByRegionDeserializer), Serdes.serdeFrom(regionCountSerializer, regionCountDeserializer)); KafkaStreams streams = new KafkaStreams(builder, props); streams.start(); diff --git a/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewUntypedDemo.java b/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewUntypedDemo.java index 9377095b212..5b80f64752d 100644 --- a/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewUntypedDemo.java +++ b/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewUntypedDemo.java @@ -21,9 +21,9 @@ import com.fasterxml.jackson.databind.JsonNode; import com.fasterxml.jackson.databind.node.JsonNodeFactory; import com.fasterxml.jackson.databind.node.ObjectNode; import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.serialization.Serializer; -import org.apache.kafka.common.serialization.StringSerializer; -import org.apache.kafka.common.serialization.StringDeserializer; import org.apache.kafka.connect.json.JsonSerializer; import org.apache.kafka.connect.json.JsonDeserializer; import org.apache.kafka.streams.KafkaStreams; @@ -59,10 +59,6 @@ public class PageViewUntypedDemo { props.put(StreamsConfig.APPLICATION_ID_CONFIG, "streams-pageview-untyped"); props.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); props.put(StreamsConfig.ZOOKEEPER_CONNECT_CONFIG, "localhost:2181"); - props.put(StreamsConfig.KEY_SERIALIZER_CLASS_CONFIG, StringSerializer.class); - props.put(StreamsConfig.VALUE_SERIALIZER_CLASS_CONFIG, JsonSerializer.class); - props.put(StreamsConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); - props.put(StreamsConfig.VALUE_DESERIALIZER_CLASS_CONFIG, JsonDeserializer.class); props.put(StreamsConfig.TIMESTAMP_EXTRACTOR_CLASS_CONFIG, JsonTimestampExtractor.class); // setting offset reset to earliest so that we can re-run the demo code with the same pre-loaded data @@ -70,14 +66,13 @@ public class PageViewUntypedDemo { KStreamBuilder builder = new KStreamBuilder(); - final Serializer stringSerializer = new StringSerializer(); - final Deserializer stringDeserializer = new StringDeserializer(); final Serializer jsonSerializer = new JsonSerializer(); final Deserializer jsonDeserializer = new JsonDeserializer(); + final Serde jsonSerde = Serdes.serdeFrom(jsonSerializer, jsonDeserializer); - KStream views = builder.stream(stringDeserializer, jsonDeserializer, "streams-pageview-input"); + KStream views = builder.stream(Serdes.String(), jsonSerde, "streams-pageview-input"); - KTable users = builder.table(stringSerializer, jsonSerializer, stringDeserializer, jsonDeserializer, "streams-userprofile-input"); + KTable users = builder.table(Serdes.String(), jsonSerde, "streams-userprofile-input"); KTable userRegions = users.mapValues(new ValueMapper() { @Override @@ -103,8 +98,7 @@ public class PageViewUntypedDemo { return new KeyValue<>(viewRegion.get("region").textValue(), viewRegion); } }) - .countByKey(HoppingWindows.of("GeoPageViewsWindow").with(7 * 24 * 60 * 60 * 1000), - stringSerializer, stringDeserializer) + .countByKey(HoppingWindows.of("GeoPageViewsWindow").with(7 * 24 * 60 * 60 * 1000), Serdes.String()) // TODO: we can merge ths toStream().map(...) with a single toStream(...) .toStream() .map(new KeyValueMapper, Long, KeyValue>() { @@ -122,7 +116,7 @@ public class PageViewUntypedDemo { }); // write to the result topic - regionCount.to("streams-pageviewstats-untyped-output", jsonSerializer, jsonSerializer); + regionCount.to("streams-pageviewstats-untyped-output", jsonSerde, jsonSerde); KafkaStreams streams = new KafkaStreams(builder, props); streams.start(); diff --git a/streams/examples/src/main/java/org/apache/kafka/streams/examples/pipe/PipeDemo.java b/streams/examples/src/main/java/org/apache/kafka/streams/examples/pipe/PipeDemo.java index c37c68ac3f2..619f33ddd93 100644 --- a/streams/examples/src/main/java/org/apache/kafka/streams/examples/pipe/PipeDemo.java +++ b/streams/examples/src/main/java/org/apache/kafka/streams/examples/pipe/PipeDemo.java @@ -17,8 +17,7 @@ package org.apache.kafka.streams.examples.pipe; -import org.apache.kafka.common.serialization.StringDeserializer; -import org.apache.kafka.common.serialization.StringSerializer; +import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.streams.kstream.KStreamBuilder; import org.apache.kafka.streams.KafkaStreams; import org.apache.kafka.streams.StreamsConfig; @@ -41,10 +40,8 @@ public class PipeDemo { Properties props = new Properties(); props.put(StreamsConfig.APPLICATION_ID_CONFIG, "streams-pipe"); props.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); - props.put(StreamsConfig.KEY_SERIALIZER_CLASS_CONFIG, StringSerializer.class); - props.put(StreamsConfig.VALUE_SERIALIZER_CLASS_CONFIG, StringSerializer.class); - props.put(StreamsConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); - props.put(StreamsConfig.VALUE_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); + props.put(StreamsConfig.KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass()); + props.put(StreamsConfig.VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass()); // setting offset reset to earliest so that we can re-run the demo code with the same pre-loaded data props.put(StreamsConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); diff --git a/streams/examples/src/main/java/org/apache/kafka/streams/examples/wordcount/WordCountDemo.java b/streams/examples/src/main/java/org/apache/kafka/streams/examples/wordcount/WordCountDemo.java index 03d514292ac..ebd6050c70a 100644 --- a/streams/examples/src/main/java/org/apache/kafka/streams/examples/wordcount/WordCountDemo.java +++ b/streams/examples/src/main/java/org/apache/kafka/streams/examples/wordcount/WordCountDemo.java @@ -17,11 +17,7 @@ package org.apache.kafka.streams.examples.wordcount; -import org.apache.kafka.common.serialization.Deserializer; -import org.apache.kafka.common.serialization.LongSerializer; -import org.apache.kafka.common.serialization.Serializer; -import org.apache.kafka.common.serialization.StringSerializer; -import org.apache.kafka.common.serialization.StringDeserializer; +import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.streams.KafkaStreams; import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.StreamsConfig; @@ -52,21 +48,13 @@ public class WordCountDemo { props.put(StreamsConfig.APPLICATION_ID_CONFIG, "streams-wordcount"); props.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); props.put(StreamsConfig.ZOOKEEPER_CONNECT_CONFIG, "localhost:2181"); - props.put(StreamsConfig.KEY_SERIALIZER_CLASS_CONFIG, StringSerializer.class); - props.put(StreamsConfig.VALUE_SERIALIZER_CLASS_CONFIG, StringSerializer.class); - props.put(StreamsConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); - props.put(StreamsConfig.VALUE_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); // setting offset reset to earliest so that we can re-run the demo code with the same pre-loaded data props.put(StreamsConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); KStreamBuilder builder = new KStreamBuilder(); - final Serializer stringSerializer = new StringSerializer(); - final Deserializer stringDeserializer = new StringDeserializer(); - final Serializer longSerializer = new LongSerializer(); - - KStream source = builder.stream("streams-file-input"); + KStream source = builder.stream(Serdes.String(), Serdes.String(), "streams-file-input"); KTable counts = source .flatMapValues(new ValueMapper>() { @@ -80,9 +68,9 @@ public class WordCountDemo { return new KeyValue(value, value); } }) - .countByKey(stringSerializer, stringDeserializer, "Counts"); + .countByKey(Serdes.String(), "Counts"); - counts.to("streams-wordcount-output", stringSerializer, longSerializer); + counts.to("streams-wordcount-output", Serdes.String(), Serdes.Long()); KafkaStreams streams = new KafkaStreams(builder, props); streams.start(); diff --git a/streams/examples/src/main/java/org/apache/kafka/streams/examples/wordcount/WordCountProcessorDemo.java b/streams/examples/src/main/java/org/apache/kafka/streams/examples/wordcount/WordCountProcessorDemo.java index b651b3aa402..8457415f111 100644 --- a/streams/examples/src/main/java/org/apache/kafka/streams/examples/wordcount/WordCountProcessorDemo.java +++ b/streams/examples/src/main/java/org/apache/kafka/streams/examples/wordcount/WordCountProcessorDemo.java @@ -17,8 +17,7 @@ package org.apache.kafka.streams.examples.wordcount; -import org.apache.kafka.common.serialization.StringDeserializer; -import org.apache.kafka.common.serialization.StringSerializer; +import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.streams.KafkaStreams; @@ -108,10 +107,8 @@ public class WordCountProcessorDemo { props.put(StreamsConfig.APPLICATION_ID_CONFIG, "streams-wordcount-processor"); props.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); props.put(StreamsConfig.ZOOKEEPER_CONNECT_CONFIG, "localhost:2181"); - props.put(StreamsConfig.KEY_SERIALIZER_CLASS_CONFIG, StringSerializer.class); - props.put(StreamsConfig.VALUE_SERIALIZER_CLASS_CONFIG, StringSerializer.class); - props.put(StreamsConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); - props.put(StreamsConfig.VALUE_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); + props.put(StreamsConfig.KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass()); + props.put(StreamsConfig.VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass()); // setting offset reset to earliest so that we can re-run the demo code with the same pre-loaded data props.put(StreamsConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); diff --git a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java index 52fdbd45839..4e989bee513 100644 --- a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java +++ b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java @@ -24,11 +24,12 @@ import org.apache.kafka.common.config.AbstractConfig; import org.apache.kafka.common.config.ConfigDef; import org.apache.kafka.common.config.ConfigDef.Importance; import org.apache.kafka.common.config.ConfigDef.Type; -import org.apache.kafka.common.serialization.Deserializer; -import org.apache.kafka.common.serialization.Serializer; +import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.streams.processor.DefaultPartitionGrouper; import org.apache.kafka.streams.processor.internals.StreamPartitionAssignor; import org.apache.kafka.streams.processor.internals.StreamThread; +import org.apache.kafka.streams.processor.internals.WallclockTimestampExtractor; import java.util.Map; @@ -91,17 +92,13 @@ public class StreamsConfig extends AbstractConfig { public static final String REPLICATION_FACTOR_CONFIG = "replication.factor"; public static final String REPLICATION_FACTOR_DOC = "The replication factor for change log topics and repartition topics created by the stream processing application."; - /** key.serializer */ - public static final String KEY_SERIALIZER_CLASS_CONFIG = ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG; + /** replication.factor */ + public static final String KEY_SERDE_CLASS_CONFIG = "key.serde"; + public static final String KEY_SERDE_CLASS_DOC = "Serializer / deserializer class for key that implements the Serde interface."; - /** value.serializer */ - public static final String VALUE_SERIALIZER_CLASS_CONFIG = ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG; - - /** key.deserializer */ - public static final String KEY_DESERIALIZER_CLASS_CONFIG = ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG; - - /** value.deserializer */ - public static final String VALUE_DESERIALIZER_CLASS_CONFIG = ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG; + /** replication.factor */ + public static final String VALUE_SERDE_CLASS_CONFIG = "value.serde"; + public static final String VALUE_SERDE_CLASS_DOC = "Serializer / deserializer class for value that implements the Serde interface."; /** metrics.sample.window.ms */ public static final String METRICS_SAMPLE_WINDOW_MS_CONFIG = CommonClientConfigs.METRICS_SAMPLE_WINDOW_MS_CONFIG; @@ -121,8 +118,6 @@ public class StreamsConfig extends AbstractConfig { /** auto.offset.reset */ public static final String AUTO_OFFSET_RESET_CONFIG = ConsumerConfig.AUTO_OFFSET_RESET_CONFIG; - private static final String WALLCLOCK_TIMESTAMP_EXTRACTOR = "org.apache.kafka.streams.processor.internals.WallclockTimestampExtractor"; - static { CONFIG = new ConfigDef().define(APPLICATION_ID_CONFIG, // required with no default value Type.STRING, @@ -152,32 +147,26 @@ public class StreamsConfig extends AbstractConfig { 1, Importance.MEDIUM, REPLICATION_FACTOR_DOC) - .define(KEY_SERIALIZER_CLASS_CONFIG, // required with no default value - Type.CLASS, - Importance.HIGH, - ProducerConfig.KEY_SERIALIZER_CLASS_DOC) - .define(VALUE_SERIALIZER_CLASS_CONFIG, // required with no default value - Type.CLASS, - Importance.HIGH, - ProducerConfig.VALUE_SERIALIZER_CLASS_DOC) - .define(KEY_DESERIALIZER_CLASS_CONFIG, // required with no default value - Type.CLASS, - Importance.HIGH, - ConsumerConfig.KEY_DESERIALIZER_CLASS_DOC) - .define(VALUE_DESERIALIZER_CLASS_CONFIG, // required with no default value - Type.CLASS, - Importance.HIGH, - ConsumerConfig.VALUE_DESERIALIZER_CLASS_DOC) .define(TIMESTAMP_EXTRACTOR_CLASS_CONFIG, Type.CLASS, - WALLCLOCK_TIMESTAMP_EXTRACTOR, + WallclockTimestampExtractor.class.getName(), Importance.MEDIUM, TIMESTAMP_EXTRACTOR_CLASS_DOC) .define(PARTITION_GROUPER_CLASS_CONFIG, Type.CLASS, - DefaultPartitionGrouper.class, + DefaultPartitionGrouper.class.getName(), Importance.MEDIUM, PARTITION_GROUPER_CLASS_DOC) + .define(KEY_SERDE_CLASS_CONFIG, + Type.CLASS, + Serdes.ByteArraySerde.class.getName(), + Importance.MEDIUM, + KEY_SERDE_CLASS_DOC) + .define(VALUE_SERDE_CLASS_CONFIG, + Type.CLASS, + Serdes.ByteArraySerde.class.getName(), + Importance.MEDIUM, + VALUE_SERDE_CLASS_DOC) .define(COMMIT_INTERVAL_MS_CONFIG, Type.LONG, 30000, @@ -273,8 +262,6 @@ public class StreamsConfig extends AbstractConfig { // remove properties that are not required for consumers removeStreamsSpecificConfigs(props); - props.remove(StreamsConfig.KEY_SERIALIZER_CLASS_CONFIG); - props.remove(StreamsConfig.VALUE_SERIALIZER_CLASS_CONFIG); return props; } @@ -287,8 +274,6 @@ public class StreamsConfig extends AbstractConfig { // remove properties that are not required for producers removeStreamsSpecificConfigs(props); - props.remove(StreamsConfig.KEY_DESERIALIZER_CLASS_CONFIG); - props.remove(StreamsConfig.VALUE_DESERIALIZER_CLASS_CONFIG); props.remove(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG); props.put(CommonClientConfigs.CLIENT_ID_CONFIG, clientId + "-producer"); @@ -302,23 +287,17 @@ public class StreamsConfig extends AbstractConfig { props.remove(StreamsConfig.BUFFERED_RECORDS_PER_PARTITION_CONFIG); props.remove(StreamsConfig.NUM_STREAM_THREADS_CONFIG); props.remove(StreamsConfig.TIMESTAMP_EXTRACTOR_CLASS_CONFIG); + props.remove(StreamsConfig.KEY_SERDE_CLASS_CONFIG); + props.remove(StreamsConfig.VALUE_SERDE_CLASS_CONFIG); props.remove(InternalConfig.STREAM_THREAD_INSTANCE); } - public Serializer keySerializer() { - return getConfiguredInstance(StreamsConfig.KEY_SERIALIZER_CLASS_CONFIG, Serializer.class); + public Serde keySerde() { + return getConfiguredInstance(StreamsConfig.KEY_SERDE_CLASS_CONFIG, Serde.class); } - public Serializer valueSerializer() { - return getConfiguredInstance(StreamsConfig.VALUE_SERIALIZER_CLASS_CONFIG, Serializer.class); - } - - public Deserializer keyDeserializer() { - return getConfiguredInstance(StreamsConfig.KEY_DESERIALIZER_CLASS_CONFIG, Deserializer.class); - } - - public Deserializer valueDeserializer() { - return getConfiguredInstance(StreamsConfig.VALUE_DESERIALIZER_CLASS_CONFIG, Deserializer.class); + public Serde valueSerde() { + return getConfiguredInstance(StreamsConfig.VALUE_SERDE_CLASS_CONFIG, Serde.class); } public static void main(String[] args) { diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java b/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java index 1640bdec000..1c786521783 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java @@ -17,8 +17,7 @@ package org.apache.kafka.streams.kstream; -import org.apache.kafka.common.serialization.Deserializer; -import org.apache.kafka.common.serialization.Serializer; +import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.processor.ProcessorSupplier; @@ -108,18 +107,14 @@ public interface KStream { * Sends key-value to a topic, also creates a new instance of KStream from the topic. * This is equivalent to calling to(topic) and from(topic). * - * @param topic the topic name - * @param keySerializer key serializer used to send key-value pairs, - * if not specified the default key serializer defined in the configuration will be used - * @param valSerializer value serializer used to send key-value pairs, - * if not specified the default value serializer defined in the configuration will be used - * @param keyDeserializer key deserializer used to create the new KStream, - * if not specified the default key deserializer defined in the configuration will be used - * @param valDeserializer value deserializer used to create the new KStream, - * if not specified the default value deserializer defined in the configuration will be used + * @param topic the topic name + * @param keySerde key serde used to send key-value pairs, + * if not specified the default key serde defined in the configuration will be used + * @param valSerde value serde used to send key-value pairs, + * if not specified the default value serde defined in the configuration will be used * @return the instance of KStream that consumes the given topic */ - KStream through(String topic, Serializer keySerializer, Serializer valSerializer, Deserializer keyDeserializer, Deserializer valDeserializer); + KStream through(String topic, Serde keySerde, Serde valSerde); /** * Sends key-value to a topic using default serializers specified in the config. @@ -131,13 +126,13 @@ public interface KStream { /** * Sends key-value to a topic. * - * @param topic the topic name - * @param keySerializer key serializer used to send key-value pairs, - * if not specified the default serializer defined in the configs will be used - * @param valSerializer value serializer used to send key-value pairs, - * if not specified the default serializer defined in the configs will be used + * @param topic the topic name + * @param keySerde key serde used to send key-value pairs, + * if not specified the default serde defined in the configs will be used + * @param keySerde value serde used to send key-value pairs, + * if not specified the default serde defined in the configs will be used */ - void to(String topic, Serializer keySerializer, Serializer valSerializer); + void to(String topic, Serde keySerde, Serde valSerde); /** * Applies a stateful transformation to all elements in this stream. @@ -171,18 +166,12 @@ public interface KStream { * @param otherStream the instance of KStream joined with this stream * @param joiner ValueJoiner * @param windows the specification of the join window - * @param keySerializer key serializer, - * if not specified the default serializer defined in the configs will be used - * @param thisValueSerializer value serializer for this stream, - * if not specified the default serializer defined in the configs will be used - * @param otherValueSerializer value serializer for other stream, - * if not specified the default serializer defined in the configs will be used - * @param keyDeserializer key deserializer, - * if not specified the default serializer defined in the configs will be used - * @param thisValueDeserializer value deserializer for this stream, - * if not specified the default serializer defined in the configs will be used - * @param otherValueDeserializer value deserializer for other stream, - * if not specified the default serializer defined in the configs will be used + * @param keySerde key serdes, + * if not specified the default serdes defined in the configs will be used + * @param thisValueSerde value serdes for this stream, + * if not specified the default serdes defined in the configs will be used + * @param otherValueSerde value serdes for other stream, + * if not specified the default serdes defined in the configs will be used * @param the value type of the other stream * @param the value type of the new stream */ @@ -190,12 +179,9 @@ public interface KStream { KStream otherStream, ValueJoiner joiner, JoinWindows windows, - Serializer keySerializer, - Serializer thisValueSerializer, - Serializer otherValueSerializer, - Deserializer keyDeserializer, - Deserializer thisValueDeserializer, - Deserializer otherValueDeserializer); + Serde keySerde, + Serde thisValueSerde, + Serde otherValueSerde); /** * Combines values of this stream with another KStream using Windowed Outer Join. @@ -203,18 +189,12 @@ public interface KStream { * @param otherStream the instance of KStream joined with this stream * @param joiner ValueJoiner * @param windows the specification of the join window - * @param keySerializer key serializer, - * if not specified the default serializer defined in the configs will be used - * @param thisValueSerializer value serializer for this stream, - * if not specified the default serializer defined in the configs will be used - * @param otherValueSerializer value serializer for other stream, - * if not specified the default serializer defined in the configs will be used - * @param keyDeserializer key deserializer, - * if not specified the default serializer defined in the configs will be used - * @param thisValueDeserializer value deserializer for this stream, - * if not specified the default serializer defined in the configs will be used - * @param otherValueDeserializer value deserializer for other stream, - * if not specified the default serializer defined in the configs will be used + * @param keySerde key serdes, + * if not specified the default serdes defined in the configs will be used + * @param thisValueSerde value serdes for this stream, + * if not specified the default serdes defined in the configs will be used + * @param otherValueSerde value serdes for other stream, + * if not specified the default serdes defined in the configs will be used * @param the value type of the other stream * @param the value type of the new stream */ @@ -222,12 +202,9 @@ public interface KStream { KStream otherStream, ValueJoiner joiner, JoinWindows windows, - Serializer keySerializer, - Serializer thisValueSerializer, - Serializer otherValueSerializer, - Deserializer keyDeserializer, - Deserializer thisValueDeserializer, - Deserializer otherValueDeserializer); + Serde keySerde, + Serde thisValueSerde, + Serde otherValueSerde); /** * Combines values of this stream with another KStream using Windowed Left Join. @@ -235,14 +212,10 @@ public interface KStream { * @param otherStream the instance of KStream joined with this stream * @param joiner ValueJoiner * @param windows the specification of the join window - * @param keySerializer key serializer, - * if not specified the default serializer defined in the configs will be used - * @param otherValueSerializer value serializer for other stream, - * if not specified the default serializer defined in the configs will be used - * @param keyDeserializer key deserializer, - * if not specified the default serializer defined in the configs will be used - * @param otherValueDeserializer value deserializer for other stream, - * if not specified the default serializer defined in the configs will be used + * @param keySerde key serdes, + * if not specified the default serdes defined in the configs will be used + * @param otherValueSerde value serdes for other stream, + * if not specified the default serdes defined in the configs will be used * @param the value type of the other stream * @param the value type of the new stream */ @@ -250,10 +223,8 @@ public interface KStream { KStream otherStream, ValueJoiner joiner, JoinWindows windows, - Serializer keySerializer, - Serializer otherValueSerializer, - Deserializer keyDeserializer, - Deserializer otherValueDeserializer); + Serde keySerde, + Serde otherValueSerde); /** * Combines values of this stream with KTable using Left Join. @@ -273,10 +244,8 @@ public interface KStream { */ KTable, V> reduceByKey(Reducer reducer, Windows windows, - Serializer keySerializer, - Serializer aggValueSerializer, - Deserializer keyDeserializer, - Deserializer aggValueDeserializer); + Serde keySerde, + Serde aggValueSerde); /** * Aggregate values of this stream by key on a window basis. @@ -284,10 +253,8 @@ public interface KStream { * @param reducer the class of Reducer */ KTable reduceByKey(Reducer reducer, - Serializer keySerializer, - Serializer aggValueSerializer, - Deserializer keyDeserializer, - Deserializer aggValueDeserializer, + Serde keySerde, + Serde aggValueSerde, String name); /** @@ -301,10 +268,8 @@ public interface KStream { KTable, T> aggregateByKey(Initializer initializer, Aggregator aggregator, Windows windows, - Serializer keySerializer, - Serializer aggValueSerializer, - Deserializer keyDeserializer, - Deserializer aggValueDeserializer); + Serde keySerde, + Serde aggValueSerde); /** * Aggregate values of this stream by key without a window basis, and hence @@ -316,10 +281,8 @@ public interface KStream { */ KTable aggregateByKey(Initializer initializer, Aggregator aggregator, - Serializer keySerializer, - Serializer aggValueSerializer, - Deserializer keyDeserializer, - Deserializer aggValueDeserializer, + Serde keySerde, + Serde aggValueSerde, String name); /** @@ -328,14 +291,12 @@ public interface KStream { * @param windows the specification of the aggregation window */ KTable, Long> countByKey(Windows windows, - Serializer keySerializer, - Deserializer keyDeserializer); + Serde keySerde); /** * Count number of messages of this stream by key without a window basis, and hence * return a ever updating counting table. */ - KTable countByKey(Serializer keySerializer, - Deserializer keyDeserializer, + KTable countByKey(Serde keySerde, String name); } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/KStreamBuilder.java b/streams/src/main/java/org/apache/kafka/streams/kstream/KStreamBuilder.java index 3cf198c37d3..dfd9281e711 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/KStreamBuilder.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/KStreamBuilder.java @@ -17,8 +17,7 @@ package org.apache.kafka.streams.kstream; -import org.apache.kafka.common.serialization.Deserializer; -import org.apache.kafka.common.serialization.Serializer; +import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.streams.kstream.internals.KStreamImpl; import org.apache.kafka.streams.kstream.internals.KTableImpl; import org.apache.kafka.streams.kstream.internals.KTableSource; @@ -40,7 +39,6 @@ public class KStreamBuilder extends TopologyBuilder { super(); } - // TODO: needs updated /** * Creates a KStream instance for the specified topic. * The default deserializers specified in the config are used. @@ -55,17 +53,17 @@ public class KStreamBuilder extends TopologyBuilder { /** * Creates a KStream instance for the specified topic. * - * @param keyDeserializer key deserializer used to read this source KStream, - * if not specified the default deserializer defined in the configs will be used - * @param valDeserializer value deserializer used to read this source KStream, - * if not specified the default deserializer defined in the configs will be used - * @param topics the topic names, if empty default to all the topics in the config + * @param keySerde key serde used to read this source KStream, + * if not specified the default serde defined in the configs will be used + * @param valSerde value serde used to read this source KStream, + * if not specified the default serde defined in the configs will be used + * @param topics the topic names, if empty default to all the topics in the config * @return KStream */ - public KStream stream(Deserializer keyDeserializer, Deserializer valDeserializer, String... topics) { + public KStream stream(Serde keySerde, Serde valSerde, String... topics) { String name = newName(KStreamImpl.SOURCE_NAME); - addSource(name, keyDeserializer, valDeserializer, topics); + addSource(name, keySerde == null ? null : keySerde.deserializer(), valSerde == null ? null : valSerde.deserializer(), topics); return new KStreamImpl<>(this, name, Collections.singleton(name)); } @@ -78,33 +76,29 @@ public class KStreamBuilder extends TopologyBuilder { * @return KTable */ public KTable table(String topic) { - return table(null, null, null, null, topic); + return table(null, null, topic); } /** * Creates a KTable instance for the specified topic. * - * @param keySerializer key serializer used to send key-value pairs, - * if not specified the default key serializer defined in the configuration will be used - * @param valSerializer value serializer used to send key-value pairs, - * if not specified the default value serializer defined in the configuration will be used - * @param keyDeserializer key deserializer used to read this source KStream, - * if not specified the default deserializer defined in the configs will be used - * @param valDeserializer value deserializer used to read this source KStream, - * if not specified the default deserializer defined in the configs will be used + * @param keySerde key serde used to send key-value pairs, + * if not specified the default key serde defined in the configuration will be used + * @param valSerde value serde used to send key-value pairs, + * if not specified the default value serde defined in the configuration will be used * @param topic the topic name * @return KStream */ - public KTable table(Serializer keySerializer, Serializer valSerializer, Deserializer keyDeserializer, Deserializer valDeserializer, String topic) { + public KTable table(Serde keySerde, Serde valSerde, String topic) { String source = newName(KStreamImpl.SOURCE_NAME); String name = newName(KTableImpl.SOURCE_NAME); - addSource(source, keyDeserializer, valDeserializer, topic); + addSource(source, keySerde == null ? null : keySerde.deserializer(), valSerde == null ? null : valSerde.deserializer(), topic); ProcessorSupplier processorSupplier = new KTableSource<>(topic); addProcessor(name, processorSupplier, source); - return new KTableImpl<>(this, name, processorSupplier, Collections.singleton(source), keySerializer, valSerializer, keyDeserializer, valDeserializer); + return new KTableImpl<>(this, name, processorSupplier, Collections.singleton(source), keySerde, valSerde); } /** diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/KTable.java b/streams/src/main/java/org/apache/kafka/streams/kstream/KTable.java index b44ed21cf8a..0ae515055b7 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/KTable.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/KTable.java @@ -17,8 +17,7 @@ package org.apache.kafka.streams.kstream; -import org.apache.kafka.common.serialization.Deserializer; -import org.apache.kafka.common.serialization.Serializer; +import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.streams.KeyValue; /** @@ -68,17 +67,13 @@ public interface KTable { * This is equivalent to calling to(topic) and table(topic). * * @param topic the topic name - * @param keySerializer key serializer used to send key-value pairs, - * if not specified the default key serializer defined in the configuration will be used - * @param valSerializer value serializer used to send key-value pairs, - * if not specified the default value serializer defined in the configuration will be used - * @param keyDeserializer key deserializer used to create the new KStream, - * if not specified the default key deserializer defined in the configuration will be used - * @param valDeserializer value deserializer used to create the new KStream, - * if not specified the default value deserializer defined in the configuration will be used + * @param keySerde key serde used to send key-value pairs, + * if not specified the default key serde defined in the configuration will be used + * @param valSerde value serde used to send key-value pairs, + * if not specified the default value serde defined in the configuration will be used * @return the new stream that consumes the given topic */ - KTable through(String topic, Serializer keySerializer, Serializer valSerializer, Deserializer keyDeserializer, Deserializer valDeserializer); + KTable through(String topic, Serde keySerde, Serde valSerde); /** * Sends key-value to a topic using default serializers specified in the config. @@ -90,13 +85,13 @@ public interface KTable { /** * Sends key-value to a topic. * - * @param topic the topic name - * @param keySerializer key serializer used to send key-value pairs, - * if not specified the default serializer defined in the configs will be used - * @param valSerializer value serializer used to send key-value pairs, - * if not specified the default serializer defined in the configs will be used + * @param topic the topic name + * @param keySerde key serde used to send key-value pairs, + * if not specified the default serde defined in the configs will be used + * @param valSerde value serde used to send key-value pairs, + * if not specified the default serde defined in the configs will be used */ - void to(String topic, Serializer keySerializer, Serializer valSerializer); + void to(String topic, Serde keySerde, Serde valSerde); /** * Creates a new instance of KStream from this KTable @@ -152,10 +147,8 @@ public interface KTable { KTable reduce(Reducer addReducer, Reducer removeReducer, KeyValueMapper> selector, - Serializer keySerializer, - Serializer valueSerializer, - Deserializer keyDeserializer, - Deserializer valueDeserializer, + Serde keySerde, + Serde valueSerde, String name); /** @@ -174,12 +167,9 @@ public interface KTable { Aggregator add, Aggregator remove, KeyValueMapper> selector, - Serializer keySerializer, - Serializer valueSerializer, - Serializer aggValueSerializer, - Deserializer keyDeserializer, - Deserializer valueDeserializer, - Deserializer aggValueDeserializer, + Serde keySerde, + Serde valueSerde, + Serde aggValueSerde, String name); /** @@ -191,10 +181,7 @@ public interface KTable { * @return the instance of KTable */ KTable count(KeyValueMapper selector, - Serializer keySerializer, - Serializer valueSerializer, - Deserializer keyDeserializer, - Deserializer valueDeserializer, + Serde keySerde, + Serde valueSerde, String name); - } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java index 884933b259b..b2934968483 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java @@ -17,9 +17,8 @@ package org.apache.kafka.streams.kstream.internals; -import org.apache.kafka.common.serialization.Deserializer; -import org.apache.kafka.common.serialization.LongDeserializer; -import org.apache.kafka.common.serialization.LongSerializer; +import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.serialization.Serializer; import org.apache.kafka.streams.kstream.Aggregator; import org.apache.kafka.streams.kstream.Initializer; @@ -93,9 +92,6 @@ public class KStreamImpl extends AbstractStream implements KStream sourceNodes) { super(topology, name, sourceNodes); } @@ -199,18 +195,16 @@ public class KStreamImpl extends AbstractStream implements KStream through(String topic, - Serializer keySerializer, - Serializer valSerializer, - Deserializer keyDeserializer, - Deserializer valDeserializer) { - to(topic, keySerializer, valSerializer); + Serde keySerde, + Serde valSerde) { + to(topic, keySerde, valSerde); - return topology.stream(keyDeserializer, valDeserializer, topic); + return topology.stream(keySerde, valSerde); } @Override public KStream through(String topic) { - return through(topic, null, null, null, null); + return through(topic, null, null); } @Override @@ -220,10 +214,13 @@ public class KStreamImpl extends AbstractStream implements KStream keySerializer, Serializer valSerializer) { + public void to(String topic, Serde keySerde, Serde valSerde) { String name = topology.newName(SINK_NAME); StreamPartitioner streamPartitioner = null; + Serializer keySerializer = keySerde == null ? null : keySerde.serializer(); + Serializer valSerializer = keySerde == null ? null : valSerde.serializer(); + if (keySerializer != null && keySerializer instanceof WindowedSerializer) { WindowedSerializer windowedSerializer = (WindowedSerializer) keySerializer; streamPartitioner = (StreamPartitioner) new WindowedStreamPartitioner(windowedSerializer); @@ -265,16 +262,11 @@ public class KStreamImpl extends AbstractStream implements KStream other, ValueJoiner joiner, JoinWindows windows, - Serializer keySerializer, - Serializer thisValueSerializer, - Serializer otherValueSerializer, - Deserializer keyDeserializer, - Deserializer thisValueDeserializer, - Deserializer otherValueDeserializer) { + Serde keySerde, + Serde thisValueSerde, + Serde otherValueSerde) { - return join(other, joiner, windows, - keySerializer, thisValueSerializer, otherValueSerializer, - keyDeserializer, thisValueDeserializer, otherValueDeserializer, false); + return join(other, joiner, windows, keySerde, thisValueSerde, otherValueSerde, false); } @Override @@ -282,16 +274,11 @@ public class KStreamImpl extends AbstractStream implements KStream other, ValueJoiner joiner, JoinWindows windows, - Serializer keySerializer, - Serializer thisValueSerializer, - Serializer otherValueSerializer, - Deserializer keyDeserializer, - Deserializer thisValueDeserializer, - Deserializer otherValueDeserializer) { + Serde keySerde, + Serde thisValueSerde, + Serde otherValueSerde) { - return join(other, joiner, windows, - keySerializer, thisValueSerializer, otherValueSerializer, - keyDeserializer, thisValueDeserializer, otherValueDeserializer, true); + return join(other, joiner, windows, keySerde, thisValueSerde, otherValueSerde, true); } @SuppressWarnings("unchecked") @@ -299,26 +286,23 @@ public class KStreamImpl extends AbstractStream implements KStream other, ValueJoiner joiner, JoinWindows windows, - Serializer keySerializer, - Serializer thisValueSerializer, - Serializer otherValueSerializer, - Deserializer keyDeserializer, - Deserializer thisValueDeserializer, - Deserializer otherValueDeserializer, + Serde keySerde, + Serde thisValueSerde, + Serde otherValueSerde, boolean outer) { Set allSourceNodes = ensureJoinableWith((AbstractStream) other); StateStoreSupplier thisWindow = Stores.create(windows.name() + "-this") - .withKeys(keySerializer, keyDeserializer) - .withValues(thisValueSerializer, thisValueDeserializer) + .withKeys(keySerde) + .withValues(thisValueSerde) .persistent() .windowed(windows.maintainMs(), windows.segments, true) .build(); StateStoreSupplier otherWindow = Stores.create(windows.name() + "-other") - .withKeys(keySerializer, keyDeserializer) - .withValues(otherValueSerializer, otherValueDeserializer) + .withKeys(keySerde) + .withValues(otherValueSerde) .persistent() .windowed(windows.maintainMs(), windows.segments, true) .build(); @@ -354,16 +338,14 @@ public class KStreamImpl extends AbstractStream implements KStream other, ValueJoiner joiner, JoinWindows windows, - Serializer keySerializer, - Serializer otherValueSerializer, - Deserializer keyDeserializer, - Deserializer otherValueDeserializer) { + Serde keySerde, + Serde otherValueSerde) { Set allSourceNodes = ensureJoinableWith((AbstractStream) other); StateStoreSupplier otherWindow = Stores.create(windows.name() + "-other") - .withKeys(keySerializer, keyDeserializer) - .withValues(otherValueSerializer, otherValueDeserializer) + .withKeys(keySerde) + .withValues(otherValueSerde) .persistent() .windowed(windows.maintainMs(), windows.segments, true) .build(); @@ -397,18 +379,16 @@ public class KStreamImpl extends AbstractStream implements KStream KTable, V> reduceByKey(Reducer reducer, Windows windows, - Serializer keySerializer, - Serializer aggValueSerializer, - Deserializer keyDeserializer, - Deserializer aggValueDeserializer) { + Serde keySerde, + Serde aggValueSerde) { String reduceName = topology.newName(REDUCE_NAME); KStreamWindowReduce reduceSupplier = new KStreamWindowReduce<>(windows, windows.name(), reducer); StateStoreSupplier reduceStore = Stores.create(windows.name()) - .withKeys(keySerializer, keyDeserializer) - .withValues(aggValueSerializer, aggValueDeserializer) + .withKeys(keySerde) + .withValues(aggValueSerde) .persistent() .windowed(windows.maintainMs(), windows.segments, false) .build(); @@ -423,10 +403,8 @@ public class KStreamImpl extends AbstractStream implements KStream reduceByKey(Reducer reducer, - Serializer keySerializer, - Serializer aggValueSerializer, - Deserializer keyDeserializer, - Deserializer aggValueDeserializer, + Serde keySerde, + Serde aggValueSerde, String name) { String reduceName = topology.newName(REDUCE_NAME); @@ -434,8 +412,8 @@ public class KStreamImpl extends AbstractStream implements KStream reduceSupplier = new KStreamReduce<>(name, reducer); StateStoreSupplier reduceStore = Stores.create(name) - .withKeys(keySerializer, keyDeserializer) - .withValues(aggValueSerializer, aggValueDeserializer) + .withKeys(keySerde) + .withValues(aggValueSerde) .persistent() .build(); @@ -451,18 +429,16 @@ public class KStreamImpl extends AbstractStream implements KStream KTable, T> aggregateByKey(Initializer initializer, Aggregator aggregator, Windows windows, - Serializer keySerializer, - Serializer aggValueSerializer, - Deserializer keyDeserializer, - Deserializer aggValueDeserializer) { + Serde keySerde, + Serde aggValueSerde) { String aggregateName = topology.newName(AGGREGATE_NAME); KStreamAggProcessorSupplier, V, T> aggregateSupplier = new KStreamWindowAggregate<>(windows, windows.name(), initializer, aggregator); StateStoreSupplier aggregateStore = Stores.create(windows.name()) - .withKeys(keySerializer, keyDeserializer) - .withValues(aggValueSerializer, aggValueDeserializer) + .withKeys(keySerde) + .withValues(aggValueSerde) .persistent() .windowed(windows.maintainMs(), windows.segments, false) .build(); @@ -478,10 +454,8 @@ public class KStreamImpl extends AbstractStream implements KStream KTable aggregateByKey(Initializer initializer, Aggregator aggregator, - Serializer keySerializer, - Serializer aggValueSerializer, - Deserializer keyDeserializer, - Deserializer aggValueDeserializer, + Serde keySerde, + Serde aggValueSerde, String name) { String aggregateName = topology.newName(AGGREGATE_NAME); @@ -489,8 +463,8 @@ public class KStreamImpl extends AbstractStream implements KStream aggregateSupplier = new KStreamAggregate<>(name, initializer, aggregator); StateStoreSupplier aggregateStore = Stores.create(name) - .withKeys(keySerializer, keyDeserializer) - .withValues(aggValueSerializer, aggValueDeserializer) + .withKeys(keySerde) + .withValues(aggValueSerde) .persistent() .build(); @@ -504,8 +478,7 @@ public class KStreamImpl extends AbstractStream implements KStream KTable, Long> countByKey(Windows windows, - Serializer keySerializer, - Deserializer keyDeserializer) { + Serde keySerde) { return this.aggregateByKey( new Initializer() { @Override @@ -518,13 +491,12 @@ public class KStreamImpl extends AbstractStream implements KStream countByKey(Serializer keySerializer, - Deserializer keyDeserializer, - String name) { + public KTable countByKey(Serde keySerde, + String name) { return this.aggregateByKey( new Initializer() { @Override @@ -537,6 +509,6 @@ public class KStreamImpl extends AbstractStream implements KStream extends AbstractStream implements KTable processorSupplier; - private final Serializer keySerializer; - private final Serializer valSerializer; - private final Deserializer keyDeserializer; - private final Deserializer valDeserializer; + private final Serde keySerde; + private final Serde valSerde; private boolean sendOldValues = false; @@ -93,23 +86,19 @@ public class KTableImpl extends AbstractStream implements KTable processorSupplier, Set sourceNodes) { - this(topology, name, processorSupplier, sourceNodes, null, null, null, null); + this(topology, name, processorSupplier, sourceNodes, null, null); } public KTableImpl(KStreamBuilder topology, String name, ProcessorSupplier processorSupplier, Set sourceNodes, - Serializer keySerializer, - Serializer valSerializer, - Deserializer keyDeserializer, - Deserializer valDeserializer) { + Serde keySerde, + Serde valSerde) { super(topology, name, sourceNodes); this.processorSupplier = processorSupplier; - this.keySerializer = keySerializer; - this.valSerializer = valSerializer; - this.keyDeserializer = keyDeserializer; - this.valDeserializer = valDeserializer; + this.keySerde = keySerde; + this.valSerde = valSerde; } @Override @@ -143,18 +132,16 @@ public class KTableImpl extends AbstractStream implements KTable through(String topic, - Serializer keySerializer, - Serializer valSerializer, - Deserializer keyDeserializer, - Deserializer valDeserializer) { - to(topic, keySerializer, valSerializer); + Serde keySerde, + Serde valSerde) { + to(topic, keySerde, valSerde); - return topology.table(keySerializer, valSerializer, keyDeserializer, valDeserializer, topic); + return topology.table(keySerde, valSerde, topic); } @Override public KTable through(String topic) { - return through(topic, null, null, null, null); + return through(topic, null, null); } @Override @@ -163,8 +150,8 @@ public class KTableImpl extends AbstractStream implements KTable keySerializer, Serializer valSerializer) { - this.toStream().to(topic, keySerializer, valSerializer); + public void to(String topic, Serde keySerde, Serde valSerde) { + this.toStream().to(topic, keySerde, valSerde); } @Override @@ -255,12 +242,9 @@ public class KTableImpl extends AbstractStream implements KTable add, Aggregator remove, KeyValueMapper> selector, - Serializer keySerializer, - Serializer valueSerializer, - Serializer aggValueSerializer, - Deserializer keyDeserializer, - Deserializer valueDeserializer, - Deserializer aggValueDeserializer, + Serde keySerde, + Serde valueSerde, + Serde aggValueSerde, String name) { String selectName = topology.newName(SELECT_NAME); @@ -270,16 +254,16 @@ public class KTableImpl extends AbstractStream implements KTable changedValueSerializer = new ChangedSerializer<>(valueSerializer); - ChangedDeserializer changedValueDeserializer = new ChangedDeserializer<>(valueDeserializer); + ChangedSerializer changedValueSerializer = new ChangedSerializer<>(valueSerde.serializer()); + ChangedDeserializer changedValueDeserializer = new ChangedDeserializer<>(valueSerde.deserializer()); KTableProcessorSupplier> selectSupplier = new KTableRepartitionMap<>(this, selector); ProcessorSupplier> aggregateSupplier = new KTableAggregate<>(name, initializer, add, remove); StateStoreSupplier aggregateStore = Stores.create(name) - .withKeys(keySerializer, keyDeserializer) - .withValues(aggValueSerializer, aggValueDeserializer) + .withKeys(keySerde) + .withValues(aggValueSerde) .persistent() .build(); @@ -289,10 +273,10 @@ public class KTableImpl extends AbstractStream implements KTable extends AbstractStream implements KTable KTable count(final KeyValueMapper selector, - Serializer keySerializer, - Serializer valueSerializer, - Deserializer keyDeserializer, - Deserializer valueDeserializer, + Serde keySerde, + Serde valueSerde, String name) { return this.aggregate( new Initializer() { @@ -332,17 +314,15 @@ public class KTableImpl extends AbstractStream implements KTable(selector.apply(key, value), value); } }, - keySerializer, valueSerializer, LONG_SERIALIZER, keyDeserializer, valueDeserializer, LONG_DESERIALIZER, name); + keySerde, valueSerde, Serdes.Long(), name); } @Override public KTable reduce(Reducer addReducer, Reducer removeReducer, KeyValueMapper> selector, - Serializer keySerializer, - Serializer valueSerializer, - Deserializer keyDeserializer, - Deserializer valueDeserializer, + Serde keySerde, + Serde valueSerde, String name) { String selectName = topology.newName(SELECT_NAME); @@ -352,16 +332,16 @@ public class KTableImpl extends AbstractStream implements KTable changedValueSerializer = new ChangedSerializer<>(valueSerializer); - ChangedDeserializer changedValueDeserializer = new ChangedDeserializer<>(valueDeserializer); + ChangedSerializer changedValueSerializer = new ChangedSerializer<>(valueSerde.serializer()); + ChangedDeserializer changedValueDeserializer = new ChangedDeserializer<>(valueSerde.deserializer()); KTableProcessorSupplier> selectSupplier = new KTableRepartitionMap<>(this, selector); ProcessorSupplier> aggregateSupplier = new KTableReduce<>(name, addReducer, removeReducer); StateStoreSupplier aggregateStore = Stores.create(name) - .withKeys(keySerializer, keyDeserializer) - .withValues(valueSerializer, valueDeserializer) + .withKeys(keySerde) + .withValues(valueSerde) .persistent() .build(); @@ -371,10 +351,10 @@ public class KTableImpl extends AbstractStream implements KTable extends AbstractStream implements KTable(source.topic, keySerializer, keyDeserializer, valSerializer, valDeserializer, null); + new KTableStoreSupplier<>(source.topic, keySerde, valSerde, null); // mark this state as non internal hence it is read directly from a user topic topology.addStateStore(storeSupplier, false, name); source.materialize(); diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableStoreSupplier.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableStoreSupplier.java index ffd5cf05c37..af3c0d71b52 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableStoreSupplier.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableStoreSupplier.java @@ -17,14 +17,13 @@ package org.apache.kafka.streams.kstream.internals; -import org.apache.kafka.common.serialization.Deserializer; -import org.apache.kafka.common.serialization.Serializer; +import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.common.utils.Time; import org.apache.kafka.streams.processor.StateStore; import org.apache.kafka.streams.processor.StateStoreSupplier; import org.apache.kafka.streams.state.internals.MeteredKeyValueStore; import org.apache.kafka.streams.state.internals.RocksDBStore; -import org.apache.kafka.streams.state.Serdes; +import org.apache.kafka.streams.state.StateSerdes; /** * A KTable storage. It stores all entries in a local RocksDB database. @@ -35,15 +34,15 @@ import org.apache.kafka.streams.state.Serdes; public class KTableStoreSupplier implements StateStoreSupplier { private final String name; - private final Serdes serdes; + private final StateSerdes serdes; private final Time time; protected KTableStoreSupplier(String name, - Serializer keySerializer, Deserializer keyDeserializer, - Serializer valSerializer, Deserializer valDeserializer, + Serde keySerde, + Serde valSerde, Time time) { this.name = name; - this.serdes = new Serdes<>(name, keySerializer, keyDeserializer, valSerializer, valDeserializer); + this.serdes = new StateSerdes<>(name, keySerde, valSerde); this.time = time; } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/ProcessorContext.java b/streams/src/main/java/org/apache/kafka/streams/processor/ProcessorContext.java index e9d5252b1e3..fdcff19e01b 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/ProcessorContext.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/ProcessorContext.java @@ -17,8 +17,7 @@ package org.apache.kafka.streams.processor; -import org.apache.kafka.common.serialization.Deserializer; -import org.apache.kafka.common.serialization.Serializer; +import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.streams.StreamsMetrics; import java.io.File; @@ -43,32 +42,18 @@ public interface ProcessorContext { TaskId taskId(); /** - * Returns the key serializer + * Returns the default key serde * * @return the key serializer */ - Serializer keySerializer(); + Serde keySerde(); /** - * Returns the value serializer + * Returns the default value serde * * @return the value serializer */ - Serializer valueSerializer(); - - /** - * Returns the key deserializer - * - * @return the key deserializer - */ - Deserializer keyDeserializer(); - - /** - * Returns the value deserializer - * - * @return the value deserializer - */ - Deserializer valueDeserializer(); + Serde valueSerde(); /** * Returns the state directory for the partition. diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/TopologyBuilder.java b/streams/src/main/java/org/apache/kafka/streams/processor/TopologyBuilder.java index ab7122ba1fa..7f5d6454d55 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/TopologyBuilder.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/TopologyBuilder.java @@ -194,8 +194,8 @@ public class TopologyBuilder { /** * Add a new source that consumes the named topics and forwards the messages to child processor and/or sink nodes. - * The source will use the {@link org.apache.kafka.streams.StreamsConfig#KEY_DESERIALIZER_CLASS_CONFIG default key deserializer} and - * {@link org.apache.kafka.streams.StreamsConfig#VALUE_DESERIALIZER_CLASS_CONFIG default value deserializer} specified in the + * The source will use the {@link org.apache.kafka.streams.StreamsConfig#KEY_SERDE_CLASS_CONFIG default key deserializer} and + * {@link org.apache.kafka.streams.StreamsConfig#VALUE_SERDE_CLASS_CONFIG default value deserializer} specified in the * {@link org.apache.kafka.streams.StreamsConfig stream configuration}. * * @param name the unique name of the source used to reference this node when @@ -214,10 +214,10 @@ public class TopologyBuilder { * @param name the unique name of the source used to reference this node when * {@link #addProcessor(String, ProcessorSupplier, String...) adding processor children}. * @param keyDeserializer the {@link Deserializer key deserializer} used when consuming messages; may be null if the source - * should use the {@link org.apache.kafka.streams.StreamsConfig#KEY_DESERIALIZER_CLASS_CONFIG default key deserializer} specified in the + * should use the {@link org.apache.kafka.streams.StreamsConfig#KEY_SERDE_CLASS_CONFIG default key deserializer} specified in the * {@link org.apache.kafka.streams.StreamsConfig stream configuration} * @param valDeserializer the {@link Deserializer value deserializer} used when consuming messages; may be null if the source - * should use the {@link org.apache.kafka.streams.StreamsConfig#VALUE_DESERIALIZER_CLASS_CONFIG default value deserializer} specified in the + * should use the {@link org.apache.kafka.streams.StreamsConfig#VALUE_SERDE_CLASS_CONFIG default value deserializer} specified in the * {@link org.apache.kafka.streams.StreamsConfig stream configuration} * @param topics the name of one or more Kafka topics that this source is to consume * @return this builder instance so methods can be chained together; never null @@ -242,8 +242,8 @@ public class TopologyBuilder { /** * Add a new sink that forwards messages from upstream parent processor and/or source nodes to the named Kafka topic. - * The sink will use the {@link org.apache.kafka.streams.StreamsConfig#KEY_SERIALIZER_CLASS_CONFIG default key serializer} and - * {@link org.apache.kafka.streams.StreamsConfig#VALUE_SERIALIZER_CLASS_CONFIG default value serializer} specified in the + * The sink will use the {@link org.apache.kafka.streams.StreamsConfig#KEY_SERDE_CLASS_CONFIG default key serializer} and + * {@link org.apache.kafka.streams.StreamsConfig#VALUE_SERDE_CLASS_CONFIG default value serializer} specified in the * {@link org.apache.kafka.streams.StreamsConfig stream configuration}. * * @param name the unique name of the sink @@ -262,8 +262,8 @@ public class TopologyBuilder { /** * Add a new sink that forwards messages from upstream parent processor and/or source nodes to the named Kafka topic, using * the supplied partitioner. - * The sink will use the {@link org.apache.kafka.streams.StreamsConfig#KEY_SERIALIZER_CLASS_CONFIG default key serializer} and - * {@link org.apache.kafka.streams.StreamsConfig#VALUE_SERIALIZER_CLASS_CONFIG default value serializer} specified in the + * The sink will use the {@link org.apache.kafka.streams.StreamsConfig#KEY_SERDE_CLASS_CONFIG default key serializer} and + * {@link org.apache.kafka.streams.StreamsConfig#VALUE_SERDE_CLASS_CONFIG default value serializer} specified in the * {@link org.apache.kafka.streams.StreamsConfig stream configuration}. *

* The sink will also use the specified {@link StreamPartitioner} to determine how messages are distributed among @@ -293,10 +293,10 @@ public class TopologyBuilder { * @param name the unique name of the sink * @param topic the name of the Kafka topic to which this sink should write its messages * @param keySerializer the {@link Serializer key serializer} used when consuming messages; may be null if the sink - * should use the {@link org.apache.kafka.streams.StreamsConfig#KEY_SERIALIZER_CLASS_CONFIG default key serializer} specified in the + * should use the {@link org.apache.kafka.streams.StreamsConfig#KEY_SERDE_CLASS_CONFIG default key serializer} specified in the * {@link org.apache.kafka.streams.StreamsConfig stream configuration} * @param valSerializer the {@link Serializer value serializer} used when consuming messages; may be null if the sink - * should use the {@link org.apache.kafka.streams.StreamsConfig#VALUE_SERIALIZER_CLASS_CONFIG default value serializer} specified in the + * should use the {@link org.apache.kafka.streams.StreamsConfig#VALUE_SERDE_CLASS_CONFIG default value serializer} specified in the * {@link org.apache.kafka.streams.StreamsConfig stream configuration} * @param parentNames the name of one or more source or processor nodes whose output message this sink should consume * and write to its topic @@ -316,10 +316,10 @@ public class TopologyBuilder { * @param name the unique name of the sink * @param topic the name of the Kafka topic to which this sink should write its messages * @param keySerializer the {@link Serializer key serializer} used when consuming messages; may be null if the sink - * should use the {@link org.apache.kafka.streams.StreamsConfig#KEY_SERIALIZER_CLASS_CONFIG default key serializer} specified in the + * should use the {@link org.apache.kafka.streams.StreamsConfig#KEY_SERDE_CLASS_CONFIG default key serializer} specified in the * {@link org.apache.kafka.streams.StreamsConfig stream configuration} * @param valSerializer the {@link Serializer value serializer} used when consuming messages; may be null if the sink - * should use the {@link org.apache.kafka.streams.StreamsConfig#VALUE_SERIALIZER_CLASS_CONFIG default value serializer} specified in the + * should use the {@link org.apache.kafka.streams.StreamsConfig#VALUE_SERDE_CLASS_CONFIG default value serializer} specified in the * {@link org.apache.kafka.streams.StreamsConfig stream configuration} * @param partitioner the function that should be used to determine the partition for each message processed by the sink * @param parentNames the name of one or more source or processor nodes whose output message this sink should consume diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java index f6e43d0f18a..888b89e2dea 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java @@ -17,11 +17,10 @@ package org.apache.kafka.streams.processor.internals; -import org.apache.kafka.common.serialization.Deserializer; -import org.apache.kafka.common.serialization.Serializer; +import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.streams.errors.TopologyBuilderException; import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.streams.StreamsMetrics; -import org.apache.kafka.streams.errors.TopologyBuilderException; import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.streams.processor.StateStore; import org.apache.kafka.streams.processor.StateRestoreCallback; @@ -37,10 +36,8 @@ public class ProcessorContextImpl implements ProcessorContext, RecordCollector.S private final RecordCollector collector; private final ProcessorStateManager stateMgr; - private final Serializer keySerializer; - private final Serializer valSerializer; - private final Deserializer keyDeserializer; - private final Deserializer valDeserializer; + private final Serde keySerde; + private final Serde valSerde; private boolean initialized; @@ -57,10 +54,8 @@ public class ProcessorContextImpl implements ProcessorContext, RecordCollector.S this.collector = collector; this.stateMgr = stateMgr; - this.keySerializer = config.keySerializer(); - this.valSerializer = config.valueSerializer(); - this.keyDeserializer = config.keyDeserializer(); - this.valDeserializer = config.valueDeserializer(); + this.keySerde = config.keySerde(); + this.valSerde = config.valueSerde(); this.initialized = false; } @@ -89,23 +84,13 @@ public class ProcessorContextImpl implements ProcessorContext, RecordCollector.S } @Override - public Serializer keySerializer() { - return this.keySerializer; + public Serde keySerde() { + return this.keySerde; } @Override - public Serializer valueSerializer() { - return this.valSerializer; - } - - @Override - public Deserializer keyDeserializer() { - return this.keyDeserializer; - } - - @Override - public Deserializer valueDeserializer() { - return this.valDeserializer; + public Serde valueSerde() { + return this.valSerde; } @Override diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/SinkNode.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/SinkNode.java index 7ab59ee6ca3..ffc72fd9287 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/SinkNode.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/SinkNode.java @@ -49,8 +49,8 @@ public class SinkNode extends ProcessorNode { @Override public void init(ProcessorContext context) { this.context = context; - if (this.keySerializer == null) this.keySerializer = (Serializer) context.keySerializer(); - if (this.valSerializer == null) this.valSerializer = (Serializer) context.valueSerializer(); + if (this.keySerializer == null) this.keySerializer = (Serializer) context.keySerde().serializer(); + if (this.valSerializer == null) this.valSerializer = (Serializer) context.valueSerde().serializer(); } @Override diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/SourceNode.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/SourceNode.java index fa4afafbbdf..1868c1b20b7 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/SourceNode.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/SourceNode.java @@ -47,8 +47,8 @@ public class SourceNode extends ProcessorNode { this.context = context; // if serializers are null, get the default ones from the context - if (this.keyDeserializer == null) this.keyDeserializer = (Deserializer) context.keyDeserializer(); - if (this.valDeserializer == null) this.valDeserializer = (Deserializer) context.valueDeserializer(); + if (this.keyDeserializer == null) this.keyDeserializer = (Deserializer) context.keySerde().deserializer(); + if (this.valDeserializer == null) this.valDeserializer = (Deserializer) context.valueSerde().deserializer(); } @Override diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyContextImpl.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyContextImpl.java index 0bcae18270e..3ad06e2b1d8 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyContextImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyContextImpl.java @@ -17,8 +17,7 @@ package org.apache.kafka.streams.processor.internals; -import org.apache.kafka.common.serialization.Deserializer; -import org.apache.kafka.common.serialization.Serializer; +import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.streams.StreamsMetrics; import org.apache.kafka.streams.processor.ProcessorContext; @@ -35,10 +34,8 @@ public class StandbyContextImpl implements ProcessorContext, RecordCollector.Sup private final StreamsMetrics metrics; private final ProcessorStateManager stateMgr; - private final Serializer keySerializer; - private final Serializer valSerializer; - private final Deserializer keyDeserializer; - private final Deserializer valDeserializer; + private final Serde keySerde; + private final Serde valSerde; private boolean initialized; @@ -52,10 +49,8 @@ public class StandbyContextImpl implements ProcessorContext, RecordCollector.Sup this.metrics = metrics; this.stateMgr = stateMgr; - this.keySerializer = config.keySerializer(); - this.valSerializer = config.valueSerializer(); - this.keyDeserializer = config.keyDeserializer(); - this.valDeserializer = config.valueDeserializer(); + this.keySerde = config.keySerde(); + this.valSerde = config.valueSerde(); this.initialized = false; } @@ -84,23 +79,13 @@ public class StandbyContextImpl implements ProcessorContext, RecordCollector.Sup } @Override - public Serializer keySerializer() { - return this.keySerializer; + public Serde keySerde() { + return this.keySerde; } @Override - public Serializer valueSerializer() { - return this.valSerializer; - } - - @Override - public Deserializer keyDeserializer() { - return this.keyDeserializer; - } - - @Override - public Deserializer valueDeserializer() { - return this.valDeserializer; + public Serde valueSerde() { + return this.valSerde; } @Override diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java index 491c8126fcc..e1a518d4b99 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java @@ -100,7 +100,6 @@ public class StreamThread extends Thread { private long lastClean; private long lastCommit; - private long recordsProcessed; private Throwable rebalanceException = null; private Map>> standbyRecords; @@ -201,7 +200,6 @@ public class StreamThread extends Thread { this.lastClean = Long.MAX_VALUE; // the cleaning cycle won't start until partition assignment this.lastCommit = time.milliseconds(); - this.recordsProcessed = 0; this.time = time; this.sensors = new StreamsMetricsImpl(metrics); diff --git a/streams/src/main/java/org/apache/kafka/streams/state/Serdes.java b/streams/src/main/java/org/apache/kafka/streams/state/Serdes.java deleted file mode 100644 index e92531210b1..00000000000 --- a/streams/src/main/java/org/apache/kafka/streams/state/Serdes.java +++ /dev/null @@ -1,136 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.kafka.streams.state; - -import org.apache.kafka.common.serialization.ByteArrayDeserializer; -import org.apache.kafka.common.serialization.ByteArraySerializer; -import org.apache.kafka.common.serialization.Deserializer; -import org.apache.kafka.common.serialization.IntegerDeserializer; -import org.apache.kafka.common.serialization.IntegerSerializer; -import org.apache.kafka.common.serialization.LongDeserializer; -import org.apache.kafka.common.serialization.LongSerializer; -import org.apache.kafka.common.serialization.Serializer; -import org.apache.kafka.common.serialization.StringDeserializer; -import org.apache.kafka.common.serialization.StringSerializer; - -/** - * Factory for creating serializers / deserializers for state stores in Kafka Streams. - * - * @param key type of serdes - * @param value type of serdes - */ -public final class Serdes { - - public static Serdes withBuiltinTypes(String topic, Class keyClass, Class valueClass) { - Serializer keySerializer = serializer(keyClass); - Deserializer keyDeserializer = deserializer(keyClass); - Serializer valueSerializer = serializer(valueClass); - Deserializer valueDeserializer = deserializer(valueClass); - return new Serdes<>(topic, keySerializer, keyDeserializer, valueSerializer, valueDeserializer); - } - - @SuppressWarnings("unchecked") - static Serializer serializer(Class type) { - if (String.class.isAssignableFrom(type)) return (Serializer) new StringSerializer(); - if (Integer.class.isAssignableFrom(type)) return (Serializer) new IntegerSerializer(); - if (Long.class.isAssignableFrom(type)) return (Serializer) new LongSerializer(); - if (byte[].class.isAssignableFrom(type)) return (Serializer) new ByteArraySerializer(); - throw new IllegalArgumentException("Unknown class for built-in serializer"); - } - - @SuppressWarnings("unchecked") - static Deserializer deserializer(Class type) { - if (String.class.isAssignableFrom(type)) return (Deserializer) new StringDeserializer(); - if (Integer.class.isAssignableFrom(type)) return (Deserializer) new IntegerDeserializer(); - if (Long.class.isAssignableFrom(type)) return (Deserializer) new LongDeserializer(); - if (byte[].class.isAssignableFrom(type)) return (Deserializer) new ByteArrayDeserializer(); - throw new IllegalArgumentException("Unknown class for built-in serializer"); - } - - private final String topic; - private Serializer keySerializer; - private Serializer valueSerializer; - private Deserializer keyDeserializer; - private Deserializer valueDeserializer; - - /** - * Create a context for serialization using the specified serializers and deserializers which - * must match the key and value types used as parameters for this object. - * - * @param topic the name of the topic - * @param keySerializer the serializer for keys; may be null - * @param keyDeserializer the deserializer for keys; may be null - * @param valueSerializer the serializer for values; may be null - * @param valueDeserializer the deserializer for values; may be null - */ - @SuppressWarnings("unchecked") - public Serdes(String topic, - Serializer keySerializer, Deserializer keyDeserializer, - Serializer valueSerializer, Deserializer valueDeserializer) { - this.topic = topic; - - if (keySerializer == null) - throw new NullPointerException(); - if (keyDeserializer == null) - throw new NullPointerException(); - if (valueSerializer == null) - throw new NullPointerException(); - if (valueDeserializer == null) - throw new NullPointerException(); - - this.keySerializer = keySerializer; - this.keyDeserializer = keyDeserializer; - this.valueSerializer = valueSerializer; - this.valueDeserializer = valueDeserializer; - } - - public Deserializer keyDeserializer() { - return keyDeserializer; - } - - public Serializer keySerializer() { - return keySerializer; - } - - public Deserializer valueDeserializer() { - return valueDeserializer; - } - - public Serializer valueSerializer() { - return valueSerializer; - } - - public String topic() { - return topic; - } - - public K keyFrom(byte[] rawKey) { - return keyDeserializer.deserialize(topic, rawKey); - } - - public V valueFrom(byte[] rawValue) { - return valueDeserializer.deserialize(topic, rawValue); - } - - public byte[] rawKey(K key) { - return keySerializer.serialize(topic, key); - } - - public byte[] rawValue(V value) { - return valueSerializer.serialize(topic, value); - } -} diff --git a/streams/src/main/java/org/apache/kafka/streams/state/StateSerdes.java b/streams/src/main/java/org/apache/kafka/streams/state/StateSerdes.java new file mode 100644 index 00000000000..1a41a1660fa --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/state/StateSerdes.java @@ -0,0 +1,108 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.streams.state; + +import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.common.serialization.Serdes; +import org.apache.kafka.common.serialization.Serializer; + +/** + * Factory for creating serializers / deserializers for state stores in Kafka Streams. + * + * @param key type of serdes + * @param value type of serdes + */ +public final class StateSerdes { + + public static StateSerdes withBuiltinTypes(String topic, Class keyClass, Class valueClass) { + return new StateSerdes<>(topic, Serdes.serdeFrom(keyClass), Serdes.serdeFrom(valueClass)); + } + + private final String topic; + private final Serde keySerde; + private final Serde valueSerde; + + /** + * Create a context for serialization using the specified serializers and deserializers which + * must match the key and value types used as parameters for this object; the state changelog topic + * is provided to bind this serde factory to, so that future calls for serialize / deserialize do not + * need to provide the topic name any more. + * + * @param topic the name of the topic + * @param keySerde the serde for keys; cannot be null + * @param valueSerde the serde for values; cannot be null + */ + @SuppressWarnings("unchecked") + public StateSerdes(String topic, + Serde keySerde, + Serde valueSerde) { + this.topic = topic; + + if (keySerde == null) + throw new IllegalArgumentException("key serde cannot be null"); + if (valueSerde == null) + throw new IllegalArgumentException("value serde cannot be null"); + + this.keySerde = keySerde; + this.valueSerde = valueSerde; + } + + public Serde keySerde() { + return keySerde; + } + + public Serde valueSerde() { + return valueSerde; + } + + public Deserializer keyDeserializer() { + return keySerde.deserializer(); + } + + public Serializer keySerializer() { + return keySerde.serializer(); + } + + public Deserializer valueDeserializer() { + return valueSerde.deserializer(); + } + + public Serializer valueSerializer() { + return valueSerde.serializer(); + } + + public String topic() { + return topic; + } + + public K keyFrom(byte[] rawKey) { + return keySerde.deserializer().deserialize(topic, rawKey); + } + + public V valueFrom(byte[] rawValue) { + return valueSerde.deserializer().deserialize(topic, rawValue); + } + + public byte[] rawKey(K key) { + return keySerde.serializer().serialize(topic, key); + } + + public byte[] rawValue(V value) { + return valueSerde.serializer().serialize(topic, value); + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/state/Stores.java b/streams/src/main/java/org/apache/kafka/streams/state/Stores.java index e803832ba8c..33df13f79aa 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/Stores.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/Stores.java @@ -16,22 +16,16 @@ */ package org.apache.kafka.streams.state; -import org.apache.kafka.common.serialization.ByteArrayDeserializer; -import org.apache.kafka.common.serialization.ByteArraySerializer; -import org.apache.kafka.common.serialization.Deserializer; -import org.apache.kafka.common.serialization.IntegerDeserializer; -import org.apache.kafka.common.serialization.IntegerSerializer; -import org.apache.kafka.common.serialization.LongDeserializer; -import org.apache.kafka.common.serialization.LongSerializer; -import org.apache.kafka.common.serialization.Serializer; -import org.apache.kafka.common.serialization.StringDeserializer; -import org.apache.kafka.common.serialization.StringSerializer; +import org.apache.kafka.common.serialization.Serdes; +import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.streams.processor.StateStoreSupplier; import org.apache.kafka.streams.state.internals.InMemoryKeyValueStoreSupplier; import org.apache.kafka.streams.state.internals.InMemoryLRUCacheStoreSupplier; import org.apache.kafka.streams.state.internals.RocksDBKeyValueStoreSupplier; import org.apache.kafka.streams.state.internals.RocksDBWindowStoreSupplier; +import java.nio.ByteBuffer; + /** * Factory for creating state stores in Kafka Streams. */ @@ -46,13 +40,12 @@ public class Stores { public static StoreFactory create(final String name) { return new StoreFactory() { @Override - public ValueFactory withKeys(final Serializer keySerializer, final Deserializer keyDeserializer) { + public ValueFactory withKeys(final Serde keySerde) { return new ValueFactory() { @Override - public KeyValueFactory withValues(final Serializer valueSerializer, - final Deserializer valueDeserializer) { - final Serdes serdes = - new Serdes<>(name, keySerializer, keyDeserializer, valueSerializer, valueDeserializer); + public KeyValueFactory withValues(final Serde valueSerde) { + final StateSerdes serdes = + new StateSerdes<>(name, keySerde, valueSerde); return new KeyValueFactory() { @Override public InMemoryKeyValueFactory inMemory() { @@ -116,7 +109,7 @@ public class Stores { * @return the interface used to specify the type of values; never null */ public ValueFactory withStringKeys() { - return withKeys(new StringSerializer(), new StringDeserializer()); + return withKeys(Serdes.String()); } /** @@ -125,7 +118,7 @@ public class Stores { * @return the interface used to specify the type of values; never null */ public ValueFactory withIntegerKeys() { - return withKeys(new IntegerSerializer(), new IntegerDeserializer()); + return withKeys(Serdes.Integer()); } /** @@ -134,7 +127,25 @@ public class Stores { * @return the interface used to specify the type of values; never null */ public ValueFactory withLongKeys() { - return withKeys(new LongSerializer(), new LongDeserializer()); + return withKeys(Serdes.Long()); + } + + /** + * Begin to create a {@link KeyValueStore} by specifying the keys will be {@link Double}s. + * + * @return the interface used to specify the type of values; never null + */ + public ValueFactory withDoubleKeys() { + return withKeys(Serdes.Double()); + } + + /** + * Begin to create a {@link KeyValueStore} by specifying the keys will be {@link ByteBuffer}. + * + * @return the interface used to specify the type of values; never null + */ + public ValueFactory withByteBufferKeys() { + return withKeys(Serdes.ByteBuffer()); } /** @@ -143,30 +154,26 @@ public class Stores { * @return the interface used to specify the type of values; never null */ public ValueFactory withByteArrayKeys() { - return withKeys(new ByteArraySerializer(), new ByteArrayDeserializer()); + return withKeys(Serdes.ByteArray()); } /** - * Begin to create a {@link KeyValueStore} by specifying the keys will be either {@link String}, {@link Integer}, - * {@link Long}, or {@code byte[]}. + * Begin to create a {@link KeyValueStore} by specifying the keys. * - * @param keyClass the class for the keys, which must be one of the types for which Kafka has built-in serializers and - * deserializers (e.g., {@code String.class}, {@code Integer.class}, {@code Long.class}, or - * {@code byte[].class}) + * @param keyClass the class for the keys, which must be one of the types for which Kafka has built-in serdes * @return the interface used to specify the type of values; never null */ public ValueFactory withKeys(Class keyClass) { - return withKeys(Serdes.serializer(keyClass), Serdes.deserializer(keyClass)); + return withKeys(Serdes.serdeFrom(keyClass)); } /** * Begin to create a {@link KeyValueStore} by specifying the serializer and deserializer for the keys. * - * @param keySerializer the serializer for keys; may not be null - * @param keyDeserializer the deserializer for keys; may not be null + * @param keySerde the serialization factory for keys; may not be null * @return the interface used to specify the type of values; never null */ - public abstract ValueFactory withKeys(Serializer keySerializer, Deserializer keyDeserializer); + public abstract ValueFactory withKeys(Serde keySerde); } /** @@ -181,7 +188,7 @@ public class Stores { * @return the interface used to specify the remaining key-value store options; never null */ public KeyValueFactory withStringValues() { - return withValues(new StringSerializer(), new StringDeserializer()); + return withValues(Serdes.String()); } /** @@ -190,7 +197,7 @@ public class Stores { * @return the interface used to specify the remaining key-value store options; never null */ public KeyValueFactory withIntegerValues() { - return withValues(new IntegerSerializer(), new IntegerDeserializer()); + return withValues(Serdes.Integer()); } /** @@ -199,7 +206,25 @@ public class Stores { * @return the interface used to specify the remaining key-value store options; never null */ public KeyValueFactory withLongValues() { - return withValues(new LongSerializer(), new LongDeserializer()); + return withValues(Serdes.Long()); + } + + /** + * Use {@link Double} values. + * + * @return the interface used to specify the remaining key-value store options; never null + */ + public KeyValueFactory withDoubleValues() { + return withValues(Serdes.Double()); + } + + /** + * Use {@link ByteBuffer} for values. + * + * @return the interface used to specify the remaining key-value store options; never null + */ + public KeyValueFactory withByteBufferValues() { + return withValues(Serdes.ByteBuffer()); } /** @@ -208,30 +233,26 @@ public class Stores { * @return the interface used to specify the remaining key-value store options; never null */ public KeyValueFactory withByteArrayValues() { - return withValues(new ByteArraySerializer(), new ByteArrayDeserializer()); + return withValues(Serdes.ByteArray()); } /** - * Use values of the specified type, which must be either {@link String}, {@link Integer}, {@link Long}, or {@code byte[]} - * . + * Use values of the specified type. * - * @param valueClass the class for the values, which must be one of the types for which Kafka has built-in serializers and - * deserializers (e.g., {@code String.class}, {@code Integer.class}, {@code Long.class}, or - * {@code byte[].class}) + * @param valueClass the class for the values, which must be one of the types for which Kafka has built-in serdes * @return the interface used to specify the remaining key-value store options; never null */ public KeyValueFactory withValues(Class valueClass) { - return withValues(Serdes.serializer(valueClass), Serdes.deserializer(valueClass)); + return withValues(Serdes.serdeFrom(valueClass)); } /** * Use the specified serializer and deserializer for the values. * - * @param valueSerializer the serializer for value; may not be null - * @param valueDeserializer the deserializer for values; may not be null + * @param valueSerde the serialization factory for values; may not be null * @return the interface used to specify the remaining key-value store options; never null */ - public abstract KeyValueFactory withValues(Serializer valueSerializer, Deserializer valueDeserializer); + public abstract KeyValueFactory withValues(Serde valueSerde); } /** @@ -240,7 +261,7 @@ public class Stores { * @param the type of keys * @param the type of values */ - public static interface KeyValueFactory { + public interface KeyValueFactory { /** * Keep all key-value entries in-memory, although for durability all entries are recorded in a Kafka topic that can be * read to restore the entries if they are lost. @@ -264,7 +285,7 @@ public class Stores { * @param the type of keys * @param the type of values */ - public static interface InMemoryKeyValueFactory { + public interface InMemoryKeyValueFactory { /** * Limits the in-memory key-value store to hold a maximum number of entries. The default is {@link Integer#MAX_VALUE}, which is * equivalent to not placing a limit on the number of entries. @@ -288,7 +309,7 @@ public class Stores { * @param the type of keys * @param the type of values */ - public static interface PersistentKeyValueFactory { + public interface PersistentKeyValueFactory { /** * Set the persistent store as a windowed key-value store diff --git a/streams/src/main/java/org/apache/kafka/streams/state/WindowStoreUtils.java b/streams/src/main/java/org/apache/kafka/streams/state/WindowStoreUtils.java index c6bbb232c5f..66e13387eb1 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/WindowStoreUtils.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/WindowStoreUtils.java @@ -25,11 +25,11 @@ public class WindowStoreUtils { public static final int TIMESTAMP_SIZE = 8; public static final int SEQNUM_SIZE = 4; - public static final Serdes INNER_SERDES = Serdes.withBuiltinTypes("", byte[].class, byte[].class); + public static final StateSerdes INNER_SERDES = StateSerdes.withBuiltinTypes("", byte[].class, byte[].class); @SuppressWarnings("unchecked") public static final KeyValueIterator[] NO_ITERATORS = (KeyValueIterator[]) new KeyValueIterator[0]; - public static byte[] toBinaryKey(K key, final long timestamp, final int seqnum, Serdes serdes) { + public static byte[] toBinaryKey(K key, final long timestamp, final int seqnum, StateSerdes serdes) { byte[] serializedKey = serdes.rawKey(key); ByteBuffer buf = ByteBuffer.allocate(serializedKey.length + TIMESTAMP_SIZE + SEQNUM_SIZE); @@ -40,7 +40,7 @@ public class WindowStoreUtils { return buf.array(); } - public static K keyFromBinaryKey(byte[] binaryKey, Serdes serdes) { + public static K keyFromBinaryKey(byte[] binaryKey, StateSerdes serdes) { byte[] bytes = new byte[binaryKey.length - TIMESTAMP_SIZE - SEQNUM_SIZE]; System.arraycopy(binaryKey, 0, bytes, 0, bytes.length); diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryKeyValueLoggedStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryKeyValueLoggedStore.java index d25faa83600..32116ddf725 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryKeyValueLoggedStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryKeyValueLoggedStore.java @@ -22,20 +22,20 @@ import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.streams.processor.StateStore; import org.apache.kafka.streams.state.KeyValueIterator; import org.apache.kafka.streams.state.KeyValueStore; -import org.apache.kafka.streams.state.Serdes; +import org.apache.kafka.streams.state.StateSerdes; import java.util.List; public class InMemoryKeyValueLoggedStore implements KeyValueStore { private final KeyValueStore inner; - private final Serdes serdes; + private final StateSerdes serdes; private final String storeName; private StoreChangeLogger changeLogger; private StoreChangeLogger.ValueGetter getter; - public InMemoryKeyValueLoggedStore(final String storeName, final KeyValueStore inner, final Serdes serdes) { + public InMemoryKeyValueLoggedStore(final String storeName, final KeyValueStore inner, final StateSerdes serdes) { this.storeName = storeName; this.inner = inner; this.serdes = serdes; diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryKeyValueStoreSupplier.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryKeyValueStoreSupplier.java index b96a1034bbb..4054d683f38 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryKeyValueStoreSupplier.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryKeyValueStoreSupplier.java @@ -25,7 +25,7 @@ import org.apache.kafka.streams.processor.StateStore; import org.apache.kafka.streams.processor.StateStoreSupplier; import org.apache.kafka.streams.state.KeyValueIterator; import org.apache.kafka.streams.state.KeyValueStore; -import org.apache.kafka.streams.state.Serdes; +import org.apache.kafka.streams.state.StateSerdes; import java.util.Iterator; import java.util.List; @@ -45,9 +45,9 @@ public class InMemoryKeyValueStoreSupplier implements StateStoreSupplier { private final String name; private final Time time; - private final Serdes serdes; + private final StateSerdes serdes; - public InMemoryKeyValueStoreSupplier(String name, Serdes serdes, Time time) { + public InMemoryKeyValueStoreSupplier(String name, StateSerdes serdes, Time time) { this.name = name; this.time = time; this.serdes = serdes; @@ -67,7 +67,7 @@ public class InMemoryKeyValueStoreSupplier implements StateStoreSupplier { private final NavigableMap map; private boolean loggingEnabled = false; - private Serdes serdes = null; + private StateSerdes serdes = null; public MemoryStore(String name) { super(); @@ -75,7 +75,7 @@ public class InMemoryKeyValueStoreSupplier implements StateStoreSupplier { this.map = new TreeMap<>(); } - public KeyValueStore enableLogging(Serdes serdes) { + public KeyValueStore enableLogging(StateSerdes serdes) { this.loggingEnabled = true; this.serdes = serdes; diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryLRUCacheStoreSupplier.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryLRUCacheStoreSupplier.java index 9b7936a3bcd..1c2241fdc1a 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryLRUCacheStoreSupplier.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryLRUCacheStoreSupplier.java @@ -19,7 +19,7 @@ package org.apache.kafka.streams.state.internals; import org.apache.kafka.common.utils.Time; import org.apache.kafka.streams.processor.StateStore; import org.apache.kafka.streams.processor.StateStoreSupplier; -import org.apache.kafka.streams.state.Serdes; +import org.apache.kafka.streams.state.StateSerdes; /** * An in-memory key-value store that is limited in size and retains a maximum number of most recently used entries. @@ -32,10 +32,10 @@ public class InMemoryLRUCacheStoreSupplier implements StateStoreSupplier { private final String name; private final int capacity; - private final Serdes serdes; + private final StateSerdes serdes; private final Time time; - public InMemoryLRUCacheStoreSupplier(String name, int capacity, Serdes serdes, Time time) { + public InMemoryLRUCacheStoreSupplier(String name, int capacity, StateSerdes serdes, Time time) { this.name = name; this.capacity = capacity; this.serdes = serdes; diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/MemoryLRUCache.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/MemoryLRUCache.java index bd03f03b515..a5aaa0666af 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/MemoryLRUCache.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/MemoryLRUCache.java @@ -22,7 +22,7 @@ import org.apache.kafka.streams.processor.StateRestoreCallback; import org.apache.kafka.streams.processor.StateStore; import org.apache.kafka.streams.state.KeyValueIterator; import org.apache.kafka.streams.state.KeyValueStore; -import org.apache.kafka.streams.state.Serdes; +import org.apache.kafka.streams.state.StateSerdes; import java.util.HashSet; import java.util.LinkedHashMap; @@ -43,7 +43,7 @@ public class MemoryLRUCache implements KeyValueStore { protected EldestEntryRemovalListener listener; private boolean loggingEnabled = false; - private Serdes serdes = null; + private StateSerdes serdes = null; // this is used for extended MemoryNavigableLRUCache only public MemoryLRUCache() {} @@ -69,7 +69,7 @@ public class MemoryLRUCache implements KeyValueStore { }; } - public KeyValueStore enableLogging(Serdes serdes) { + public KeyValueStore enableLogging(StateSerdes serdes) { this.loggingEnabled = true; this.serdes = serdes; diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBKeyValueStoreSupplier.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBKeyValueStoreSupplier.java index 3a4c351dcae..ec10c3fdfa6 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBKeyValueStoreSupplier.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBKeyValueStoreSupplier.java @@ -20,7 +20,7 @@ package org.apache.kafka.streams.state.internals; import org.apache.kafka.common.utils.Time; import org.apache.kafka.streams.processor.StateStore; import org.apache.kafka.streams.processor.StateStoreSupplier; -import org.apache.kafka.streams.state.Serdes; +import org.apache.kafka.streams.state.StateSerdes; /** * A {@link org.apache.kafka.streams.state.KeyValueStore} that stores all entries in a local RocksDB database. @@ -33,10 +33,10 @@ import org.apache.kafka.streams.state.Serdes; public class RocksDBKeyValueStoreSupplier implements StateStoreSupplier { private final String name; - private final Serdes serdes; + private final StateSerdes serdes; private final Time time; - public RocksDBKeyValueStoreSupplier(String name, Serdes serdes, Time time) { + public RocksDBKeyValueStoreSupplier(String name, StateSerdes serdes, Time time) { this.name = name; this.serdes = serdes; this.time = time; diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBStore.java index c295aea6e8e..30458564514 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBStore.java @@ -24,7 +24,7 @@ import org.apache.kafka.streams.processor.StateRestoreCallback; import org.apache.kafka.streams.processor.StateStore; import org.apache.kafka.streams.state.KeyValueIterator; import org.apache.kafka.streams.state.KeyValueStore; -import org.apache.kafka.streams.state.Serdes; +import org.apache.kafka.streams.state.StateSerdes; import org.rocksdb.BlockBasedTableConfig; import org.rocksdb.CompactionStyle; @@ -68,7 +68,7 @@ public class RocksDBStore implements KeyValueStore { private final FlushOptions fOptions; private ProcessorContext context; - private Serdes serdes; + private StateSerdes serdes; protected File dbDir; private RocksDB db; @@ -92,11 +92,11 @@ public class RocksDBStore implements KeyValueStore { return this; } - public RocksDBStore(String name, Serdes serdes) { + public RocksDBStore(String name, StateSerdes serdes) { this(name, DB_FILE_DIR, serdes); } - public RocksDBStore(String name, String parentDir, Serdes serdes) { + public RocksDBStore(String name, String parentDir, StateSerdes serdes) { this.name = name; this.parentDir = parentDir; this.serdes = serdes; @@ -399,9 +399,9 @@ public class RocksDBStore implements KeyValueStore { private static class RocksDbIterator implements KeyValueIterator { private final RocksIterator iter; - private final Serdes serdes; + private final StateSerdes serdes; - public RocksDbIterator(RocksIterator iter, Serdes serdes) { + public RocksDbIterator(RocksIterator iter, StateSerdes serdes) { this.iter = iter; this.serdes = serdes; } @@ -463,7 +463,7 @@ public class RocksDBStore implements KeyValueStore { private final Comparator comparator = new LexicographicComparator(); byte[] rawToKey; - public RocksDBRangeIterator(RocksIterator iter, Serdes serdes, + public RocksDBRangeIterator(RocksIterator iter, StateSerdes serdes, K from, K to) { super(iter, serdes); iter.seek(serdes.rawKey(from)); diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBWindowStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBWindowStore.java index b1605a3b251..61c2e5eec87 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBWindowStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBWindowStore.java @@ -25,7 +25,7 @@ import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.streams.processor.StateRestoreCallback; import org.apache.kafka.streams.processor.StateStore; import org.apache.kafka.streams.state.KeyValueIterator; -import org.apache.kafka.streams.state.Serdes; +import org.apache.kafka.streams.state.StateSerdes; import org.apache.kafka.streams.state.WindowStore; import org.apache.kafka.streams.state.WindowStoreIterator; import org.apache.kafka.streams.state.WindowStoreUtils; @@ -61,15 +61,15 @@ public class RocksDBWindowStore implements WindowStore { } private static class RocksDBWindowStoreIterator implements WindowStoreIterator { - private final Serdes serdes; + private final StateSerdes serdes; private final KeyValueIterator[] iterators; private int index = 0; - RocksDBWindowStoreIterator(Serdes serdes) { + RocksDBWindowStoreIterator(StateSerdes serdes) { this(serdes, WindowStoreUtils.NO_ITERATORS); } - RocksDBWindowStoreIterator(Serdes serdes, KeyValueIterator[] iterators) { + RocksDBWindowStoreIterator(StateSerdes serdes, KeyValueIterator[] iterators) { this.serdes = serdes; this.iterators = iterators; } @@ -114,7 +114,7 @@ public class RocksDBWindowStore implements WindowStore { private final long segmentInterval; private final boolean retainDuplicates; private final Segment[] segments; - private final Serdes serdes; + private final StateSerdes serdes; private final SimpleDateFormat formatter; private final StoreChangeLogger.ValueGetter getter; @@ -125,7 +125,7 @@ public class RocksDBWindowStore implements WindowStore { private boolean loggingEnabled = false; private StoreChangeLogger changeLogger = null; - public RocksDBWindowStore(String name, long retentionPeriod, int numSegments, boolean retainDuplicates, Serdes serdes) { + public RocksDBWindowStore(String name, long retentionPeriod, int numSegments, boolean retainDuplicates, StateSerdes serdes) { this.name = name; // The segment interval must be greater than MIN_SEGMENT_INTERVAL diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBWindowStoreSupplier.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBWindowStoreSupplier.java index 6823e6dbf9c..00089abebb0 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBWindowStoreSupplier.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBWindowStoreSupplier.java @@ -20,7 +20,7 @@ package org.apache.kafka.streams.state.internals; import org.apache.kafka.common.utils.Time; import org.apache.kafka.streams.processor.StateStore; import org.apache.kafka.streams.processor.StateStoreSupplier; -import org.apache.kafka.streams.state.Serdes; +import org.apache.kafka.streams.state.StateSerdes; /** * A {@link org.apache.kafka.streams.state.KeyValueStore} that stores all entries in a local RocksDB database. @@ -36,10 +36,10 @@ public class RocksDBWindowStoreSupplier implements StateStoreSupplier { private final long retentionPeriod; private final boolean retainDuplicates; private final int numSegments; - private final Serdes serdes; + private final StateSerdes serdes; private final Time time; - public RocksDBWindowStoreSupplier(String name, long retentionPeriod, int numSegments, boolean retainDuplicates, Serdes serdes, Time time) { + public RocksDBWindowStoreSupplier(String name, long retentionPeriod, int numSegments, boolean retainDuplicates, StateSerdes serdes, Time time) { this.name = name; this.retentionPeriod = retentionPeriod; this.retainDuplicates = retainDuplicates; diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/StoreChangeLogger.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/StoreChangeLogger.java index 4229f940a00..a4391179c16 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/StoreChangeLogger.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/StoreChangeLogger.java @@ -22,7 +22,7 @@ import org.apache.kafka.common.serialization.Serializer; import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.streams.processor.internals.ProcessorStateManager; import org.apache.kafka.streams.processor.internals.RecordCollector; -import org.apache.kafka.streams.state.Serdes; +import org.apache.kafka.streams.state.StateSerdes; import java.util.HashSet; import java.util.Set; @@ -36,7 +36,7 @@ public class StoreChangeLogger { // TODO: these values should be configurable protected static final int DEFAULT_WRITE_BATCH_SIZE = 100; - protected final Serdes serialization; + protected final StateSerdes serialization; private final String topic; private final int partition; @@ -47,16 +47,16 @@ public class StoreChangeLogger { protected Set dirty; protected Set removed; - public StoreChangeLogger(String storeName, ProcessorContext context, Serdes serialization) { + public StoreChangeLogger(String storeName, ProcessorContext context, StateSerdes serialization) { this(storeName, context, serialization, DEFAULT_WRITE_BATCH_SIZE, DEFAULT_WRITE_BATCH_SIZE); } - public StoreChangeLogger(String storeName, ProcessorContext context, Serdes serialization, int maxDirty, int maxRemoved) { + public StoreChangeLogger(String storeName, ProcessorContext context, StateSerdes serialization, int maxDirty, int maxRemoved) { this(storeName, context, context.taskId().partition, serialization, maxDirty, maxRemoved); init(); } - protected StoreChangeLogger(String storeName, ProcessorContext context, int partition, Serdes serialization, int maxDirty, int maxRemoved) { + protected StoreChangeLogger(String storeName, ProcessorContext context, int partition, StateSerdes serialization, int maxDirty, int maxRemoved) { this.topic = ProcessorStateManager.storeChangelogTopic(context.applicationId(), storeName); this.context = context; this.partition = partition; diff --git a/streams/src/test/java/org/apache/kafka/streams/StreamsConfigTest.java b/streams/src/test/java/org/apache/kafka/streams/StreamsConfigTest.java index 83ebe48f1c1..0dacde79f50 100644 --- a/streams/src/test/java/org/apache/kafka/streams/StreamsConfigTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/StreamsConfigTest.java @@ -18,11 +18,6 @@ package org.apache.kafka.streams; import org.apache.kafka.clients.consumer.ConsumerConfig; -import org.apache.kafka.common.serialization.IntegerDeserializer; -import org.apache.kafka.common.serialization.IntegerSerializer; -import org.apache.kafka.common.serialization.StringDeserializer; -import org.apache.kafka.common.serialization.StringSerializer; -import org.apache.kafka.streams.processor.internals.WallclockTimestampExtractor; import org.apache.kafka.streams.processor.internals.StreamThread; import org.junit.Before; import org.junit.Test; @@ -43,11 +38,6 @@ public class StreamsConfigTest { public void setUp() { props.put(StreamsConfig.APPLICATION_ID_CONFIG, "streams-config-test"); props.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); - props.put(StreamsConfig.KEY_SERIALIZER_CLASS_CONFIG, StringSerializer.class); - props.put(StreamsConfig.VALUE_SERIALIZER_CLASS_CONFIG, IntegerSerializer.class); - props.put(StreamsConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); - props.put(StreamsConfig.VALUE_DESERIALIZER_CLASS_CONFIG, IntegerDeserializer.class); - props.put(StreamsConfig.TIMESTAMP_EXTRACTOR_CLASS_CONFIG, WallclockTimestampExtractor.class); streamsConfig = new StreamsConfig(props); } diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamBranchTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamBranchTest.java index 88366fac3dd..e04a2738d9f 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamBranchTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamBranchTest.java @@ -17,8 +17,7 @@ package org.apache.kafka.streams.kstream.internals; -import org.apache.kafka.common.serialization.IntegerDeserializer; -import org.apache.kafka.common.serialization.StringDeserializer; +import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.streams.kstream.KStream; import org.apache.kafka.streams.kstream.KStreamBuilder; import org.apache.kafka.streams.kstream.Predicate; @@ -34,9 +33,6 @@ public class KStreamBranchTest { private String topicName = "topic"; - private IntegerDeserializer keyDeserializer = new IntegerDeserializer(); - private StringDeserializer valDeserializer = new StringDeserializer(); - @SuppressWarnings("unchecked") @Test public void testKStreamBranch() { @@ -67,7 +63,7 @@ public class KStreamBranchTest { KStream[] branches; MockProcessorSupplier[] processors; - stream = builder.stream(keyDeserializer, valDeserializer, topicName); + stream = builder.stream(Serdes.Integer(), Serdes.String(), topicName); branches = stream.branch(isEven, isMultipleOfThree, isOdd); assertEquals(3, branches.length); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFilterTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFilterTest.java index 3bad0416152..ecf11153dd0 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFilterTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFilterTest.java @@ -17,8 +17,7 @@ package org.apache.kafka.streams.kstream.internals; -import org.apache.kafka.common.serialization.IntegerDeserializer; -import org.apache.kafka.common.serialization.StringDeserializer; +import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.streams.kstream.KStream; import org.apache.kafka.streams.kstream.KStreamBuilder; import org.apache.kafka.streams.kstream.Predicate; @@ -33,9 +32,6 @@ public class KStreamFilterTest { private String topicName = "topic"; - private IntegerDeserializer keyDeserializer = new IntegerDeserializer(); - private StringDeserializer valDeserializer = new StringDeserializer(); - private Predicate isMultipleOfThree = new Predicate() { @Override public boolean test(Integer key, String value) { @@ -52,7 +48,7 @@ public class KStreamFilterTest { MockProcessorSupplier processor; processor = new MockProcessorSupplier<>(); - stream = builder.stream(keyDeserializer, valDeserializer, topicName); + stream = builder.stream(Serdes.Integer(), Serdes.String(), topicName); stream.filter(isMultipleOfThree).process(processor); KStreamTestDriver driver = new KStreamTestDriver(builder); @@ -72,7 +68,7 @@ public class KStreamFilterTest { MockProcessorSupplier processor; processor = new MockProcessorSupplier<>(); - stream = builder.stream(keyDeserializer, valDeserializer, topicName); + stream = builder.stream(Serdes.Integer(), Serdes.String(), topicName); stream.filterOut(isMultipleOfThree).process(processor); KStreamTestDriver driver = new KStreamTestDriver(builder); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapTest.java index 693f58e443f..bc85757ef45 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapTest.java @@ -17,8 +17,7 @@ package org.apache.kafka.streams.kstream.internals; -import org.apache.kafka.common.serialization.IntegerDeserializer; -import org.apache.kafka.common.serialization.StringDeserializer; +import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.streams.kstream.KStream; import org.apache.kafka.streams.kstream.KStreamBuilder; import org.apache.kafka.streams.KeyValue; @@ -35,9 +34,6 @@ public class KStreamFlatMapTest { private String topicName = "topic"; - private IntegerDeserializer keyDeserializer = new IntegerDeserializer(); - private StringDeserializer valDeserializer = new StringDeserializer(); - @Test public void testFlatMap() { KStreamBuilder builder = new KStreamBuilder(); @@ -60,7 +56,7 @@ public class KStreamFlatMapTest { MockProcessorSupplier processor; processor = new MockProcessorSupplier<>(); - stream = builder.stream(keyDeserializer, valDeserializer, topicName); + stream = builder.stream(Serdes.Integer(), Serdes.String(), topicName); stream.flatMap(mapper).process(processor); KStreamTestDriver driver = new KStreamTestDriver(builder); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapValuesTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapValuesTest.java index eef7933c627..a904cb15c62 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapValuesTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapValuesTest.java @@ -17,8 +17,7 @@ package org.apache.kafka.streams.kstream.internals; -import org.apache.kafka.common.serialization.IntegerDeserializer; -import org.apache.kafka.common.serialization.StringDeserializer; +import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.streams.kstream.KStream; import org.apache.kafka.streams.kstream.KStreamBuilder; import org.apache.kafka.streams.kstream.ValueMapper; @@ -34,9 +33,6 @@ public class KStreamFlatMapValuesTest { private String topicName = "topic"; - private IntegerDeserializer keyDeserializer = new IntegerDeserializer(); - private StringDeserializer valDeserializer = new StringDeserializer(); - @Test public void testFlatMapValues() { KStreamBuilder builder = new KStreamBuilder(); @@ -58,7 +54,7 @@ public class KStreamFlatMapValuesTest { MockProcessorSupplier processor; processor = new MockProcessorSupplier<>(); - stream = builder.stream(keyDeserializer, valDeserializer, topicName); + stream = builder.stream(Serdes.Integer(), Serdes.String(), topicName); stream.flatMapValues(mapper).process(processor); KStreamTestDriver driver = new KStreamTestDriver(builder); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamImplTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamImplTest.java index 3d3a9e3d6c4..38182bc355f 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamImplTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamImplTest.java @@ -17,12 +17,8 @@ package org.apache.kafka.streams.kstream.internals; -import org.apache.kafka.common.serialization.Deserializer; -import org.apache.kafka.common.serialization.IntegerDeserializer; -import org.apache.kafka.common.serialization.IntegerSerializer; -import org.apache.kafka.common.serialization.Serializer; -import org.apache.kafka.common.serialization.StringDeserializer; -import org.apache.kafka.common.serialization.StringSerializer; +import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.streams.kstream.JoinWindows; import org.apache.kafka.streams.kstream.KStream; import org.apache.kafka.streams.kstream.KStreamBuilder; @@ -39,18 +35,16 @@ import static org.junit.Assert.assertEquals; public class KStreamImplTest { + final private Serde stringSerde = Serdes.String(); + final private Serde intSerde = Serdes.Integer(); + @Test public void testNumProcesses() { - final Serializer stringSerializer = new StringSerializer(); - final Deserializer stringDeserializer = new StringDeserializer(); - final Serializer integerSerializer = new IntegerSerializer(); - final Deserializer integerDeserializer = new IntegerDeserializer(); - final KStreamBuilder builder = new KStreamBuilder(); - KStream source1 = builder.stream(stringDeserializer, stringDeserializer, "topic-1", "topic-2"); + KStream source1 = builder.stream(stringSerde, stringSerde, "topic-1", "topic-2"); - KStream source2 = builder.stream(stringDeserializer, stringDeserializer, "topic-3", "topic-4"); + KStream source2 = builder.stream(stringSerde, stringSerde, "topic-3", "topic-4"); KStream stream1 = source1.filter(new Predicate() { @@ -114,14 +108,14 @@ public class KStreamImplTest { public Integer apply(Integer value1, Integer value2) { return value1 + value2; } - }, JoinWindows.of("join-0"), stringSerializer, integerSerializer, integerSerializer, stringDeserializer, integerDeserializer, integerDeserializer); + }, JoinWindows.of("join-0"), stringSerde, intSerde, intSerde); KStream stream5 = streams2[1].join(streams3[1], new ValueJoiner() { @Override public Integer apply(Integer value1, Integer value2) { return value1 + value2; } - }, JoinWindows.of("join-1"), stringSerializer, integerSerializer, integerSerializer, stringDeserializer, integerDeserializer, integerDeserializer); + }, JoinWindows.of("join-1"), stringSerde, intSerde, intSerde); stream4.to("topic-5"); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoinTest.java index e763fd25a41..d24ab15461c 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoinTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoinTest.java @@ -17,10 +17,8 @@ package org.apache.kafka.streams.kstream.internals; -import org.apache.kafka.common.serialization.IntegerDeserializer; -import org.apache.kafka.common.serialization.IntegerSerializer; -import org.apache.kafka.common.serialization.StringDeserializer; -import org.apache.kafka.common.serialization.StringSerializer; +import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.streams.kstream.JoinWindows; import org.apache.kafka.streams.kstream.KStream; @@ -44,10 +42,8 @@ public class KStreamKStreamJoinTest { private String topic1 = "topic1"; private String topic2 = "topic2"; - private IntegerSerializer keySerializer = new IntegerSerializer(); - private StringSerializer valSerializer = new StringSerializer(); - private IntegerDeserializer keyDeserializer = new IntegerDeserializer(); - private StringDeserializer valDeserializer = new StringDeserializer(); + final private Serde intSerde = Serdes.Integer(); + final private Serde stringSerde = Serdes.String(); private ValueJoiner joiner = new ValueJoiner() { @Override @@ -71,10 +67,9 @@ public class KStreamKStreamJoinTest { MockProcessorSupplier processor; processor = new MockProcessorSupplier<>(); - stream1 = builder.stream(keyDeserializer, valDeserializer, topic1); - stream2 = builder.stream(keyDeserializer, valDeserializer, topic2); - joined = stream1.join(stream2, joiner, JoinWindows.of("test").within(100), - keySerializer, valSerializer, valSerializer, keyDeserializer, valDeserializer, valDeserializer); + stream1 = builder.stream(intSerde, stringSerde, topic1); + stream2 = builder.stream(intSerde, stringSerde, topic2); + joined = stream1.join(stream2, joiner, JoinWindows.of("test").within(100), intSerde, stringSerde, stringSerde); joined.process(processor); Collection> copartitionGroups = builder.copartitionGroups(); @@ -177,10 +172,9 @@ public class KStreamKStreamJoinTest { MockProcessorSupplier processor; processor = new MockProcessorSupplier<>(); - stream1 = builder.stream(keyDeserializer, valDeserializer, topic1); - stream2 = builder.stream(keyDeserializer, valDeserializer, topic2); - joined = stream1.outerJoin(stream2, joiner, JoinWindows.of("test").within(100), - keySerializer, valSerializer, valSerializer, keyDeserializer, valDeserializer, valDeserializer); + stream1 = builder.stream(intSerde, stringSerde, topic1); + stream2 = builder.stream(intSerde, stringSerde, topic2); + joined = stream1.outerJoin(stream2, joiner, JoinWindows.of("test").within(100), intSerde, stringSerde, stringSerde); joined.process(processor); Collection> copartitionGroups = builder.copartitionGroups(); @@ -285,10 +279,9 @@ public class KStreamKStreamJoinTest { MockProcessorSupplier processor; processor = new MockProcessorSupplier<>(); - stream1 = builder.stream(keyDeserializer, valDeserializer, topic1); - stream2 = builder.stream(keyDeserializer, valDeserializer, topic2); - joined = stream1.join(stream2, joiner, JoinWindows.of("test").within(100), - keySerializer, valSerializer, valSerializer, keyDeserializer, valDeserializer, valDeserializer); + stream1 = builder.stream(intSerde, stringSerde, topic1); + stream2 = builder.stream(intSerde, stringSerde, topic2); + joined = stream1.join(stream2, joiner, JoinWindows.of("test").within(100), intSerde, stringSerde, stringSerde); joined.process(processor); Collection> copartitionGroups = builder.copartitionGroups(); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamLeftJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamLeftJoinTest.java index 8c6e43b2eae..166e8ba0385 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamLeftJoinTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamLeftJoinTest.java @@ -17,10 +17,8 @@ package org.apache.kafka.streams.kstream.internals; -import org.apache.kafka.common.serialization.IntegerDeserializer; -import org.apache.kafka.common.serialization.IntegerSerializer; -import org.apache.kafka.common.serialization.StringDeserializer; -import org.apache.kafka.common.serialization.StringSerializer; +import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.streams.kstream.JoinWindows; import org.apache.kafka.streams.kstream.KStream; @@ -44,10 +42,8 @@ public class KStreamKStreamLeftJoinTest { private String topic1 = "topic1"; private String topic2 = "topic2"; - private IntegerSerializer keySerializer = new IntegerSerializer(); - private StringSerializer valSerializer = new StringSerializer(); - private IntegerDeserializer keyDeserializer = new IntegerDeserializer(); - private StringDeserializer valDeserializer = new StringDeserializer(); + final private Serde intSerde = Serdes.Integer(); + final private Serde stringSerde = Serdes.String(); private ValueJoiner joiner = new ValueJoiner() { @Override @@ -71,10 +67,9 @@ public class KStreamKStreamLeftJoinTest { MockProcessorSupplier processor; processor = new MockProcessorSupplier<>(); - stream1 = builder.stream(keyDeserializer, valDeserializer, topic1); - stream2 = builder.stream(keyDeserializer, valDeserializer, topic2); - joined = stream1.leftJoin(stream2, joiner, JoinWindows.of("test").within(100), - keySerializer, valSerializer, keyDeserializer, valDeserializer); + stream1 = builder.stream(intSerde, stringSerde, topic1); + stream2 = builder.stream(intSerde, stringSerde, topic2); + joined = stream1.leftJoin(stream2, joiner, JoinWindows.of("test").within(100), intSerde, stringSerde); joined.process(processor); Collection> copartitionGroups = builder.copartitionGroups(); @@ -157,10 +152,9 @@ public class KStreamKStreamLeftJoinTest { MockProcessorSupplier processor; processor = new MockProcessorSupplier<>(); - stream1 = builder.stream(keyDeserializer, valDeserializer, topic1); - stream2 = builder.stream(keyDeserializer, valDeserializer, topic2); - joined = stream1.leftJoin(stream2, joiner, JoinWindows.of("test").within(100), - keySerializer, valSerializer, keyDeserializer, valDeserializer); + stream1 = builder.stream(intSerde, stringSerde, topic1); + stream2 = builder.stream(intSerde, stringSerde, topic2); + joined = stream1.leftJoin(stream2, joiner, JoinWindows.of("test").within(100), intSerde, stringSerde); joined.process(processor); Collection> copartitionGroups = builder.copartitionGroups(); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKTableLeftJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKTableLeftJoinTest.java index f226ceee285..8e672a2311d 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKTableLeftJoinTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKTableLeftJoinTest.java @@ -18,10 +18,8 @@ package org.apache.kafka.streams.kstream.internals; import org.apache.kafka.common.KafkaException; -import org.apache.kafka.common.serialization.IntegerDeserializer; -import org.apache.kafka.common.serialization.IntegerSerializer; -import org.apache.kafka.common.serialization.StringDeserializer; -import org.apache.kafka.common.serialization.StringSerializer; +import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.streams.kstream.KStream; import org.apache.kafka.streams.kstream.KStreamBuilder; @@ -47,10 +45,8 @@ public class KStreamKTableLeftJoinTest { private String topic1 = "topic1"; private String topic2 = "topic2"; - private IntegerSerializer keySerializer = new IntegerSerializer(); - private StringSerializer valSerializer = new StringSerializer(); - private IntegerDeserializer keyDeserializer = new IntegerDeserializer(); - private StringDeserializer valDeserializer = new StringDeserializer(); + final private Serde intSerde = new Serdes.IntegerSerde(); + final private Serde stringSerde = new Serdes.StringSerde(); private ValueJoiner joiner = new ValueJoiner() { @Override @@ -81,8 +77,8 @@ public class KStreamKTableLeftJoinTest { MockProcessorSupplier processor; processor = new MockProcessorSupplier<>(); - stream = builder.stream(keyDeserializer, valDeserializer, topic1); - table = builder.table(keySerializer, valSerializer, keyDeserializer, valDeserializer, topic2); + stream = builder.stream(intSerde, stringSerde, topic1); + table = builder.table(intSerde, stringSerde, topic2); stream.leftJoin(table, joiner).process(processor); Collection> copartitionGroups = builder.copartitionGroups(); @@ -162,8 +158,8 @@ public class KStreamKTableLeftJoinTest { MockProcessorSupplier processor; processor = new MockProcessorSupplier<>(); - stream = builder.stream(keyDeserializer, valDeserializer, topic1).map(keyValueMapper); - table = builder.table(keySerializer, valSerializer, keyDeserializer, valDeserializer, topic2); + stream = builder.stream(intSerde, stringSerde, topic1).map(keyValueMapper); + table = builder.table(intSerde, stringSerde, topic2); stream.leftJoin(table, joiner).process(processor); } diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamMapTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamMapTest.java index 73c517bc658..68fa6565d50 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamMapTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamMapTest.java @@ -17,8 +17,8 @@ package org.apache.kafka.streams.kstream.internals; -import org.apache.kafka.common.serialization.IntegerDeserializer; -import org.apache.kafka.common.serialization.StringDeserializer; +import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.streams.kstream.KStream; import org.apache.kafka.streams.kstream.KStreamBuilder; import org.apache.kafka.streams.KeyValue; @@ -33,8 +33,8 @@ public class KStreamMapTest { private String topicName = "topic"; - private IntegerDeserializer keyDeserializer = new IntegerDeserializer(); - private StringDeserializer valDeserializer = new StringDeserializer(); + final private Serde intSerde = Serdes.Integer(); + final private Serde stringSerde = Serdes.String(); @Test public void testMap() { @@ -50,11 +50,10 @@ public class KStreamMapTest { final int[] expectedKeys = new int[]{0, 1, 2, 3}; - KStream stream; + KStream stream = builder.stream(intSerde, stringSerde, topicName); MockProcessorSupplier processor; processor = new MockProcessorSupplier<>(); - stream = builder.stream(keyDeserializer, valDeserializer, topicName); stream.map(mapper).process(processor); KStreamTestDriver driver = new KStreamTestDriver(builder); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamMapValuesTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamMapValuesTest.java index 68fd285870f..e671aabce39 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamMapValuesTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamMapValuesTest.java @@ -17,8 +17,8 @@ package org.apache.kafka.streams.kstream.internals; -import org.apache.kafka.common.serialization.IntegerDeserializer; -import org.apache.kafka.common.serialization.StringDeserializer; +import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.streams.kstream.KStream; import org.apache.kafka.streams.kstream.KStreamBuilder; import org.apache.kafka.streams.kstream.ValueMapper; @@ -32,8 +32,8 @@ public class KStreamMapValuesTest { private String topicName = "topic"; - private IntegerDeserializer keyDeserializer = new IntegerDeserializer(); - private StringDeserializer valDeserializer = new StringDeserializer(); + final private Serde intSerde = Serdes.Integer(); + final private Serde stringSerde = Serdes.String(); @Test public void testFlatMapValues() { @@ -51,7 +51,7 @@ public class KStreamMapValuesTest { KStream stream; MockProcessorSupplier processor = new MockProcessorSupplier<>(); - stream = builder.stream(keyDeserializer, valDeserializer, topicName); + stream = builder.stream(intSerde, stringSerde, topicName); stream.mapValues(mapper).process(processor); KStreamTestDriver driver = new KStreamTestDriver(builder); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamTransformTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamTransformTest.java index 426259f689d..4244de54164 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamTransformTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamTransformTest.java @@ -17,7 +17,8 @@ package org.apache.kafka.streams.kstream.internals; -import org.apache.kafka.common.serialization.IntegerDeserializer; +import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.streams.kstream.KStream; import org.apache.kafka.streams.kstream.KStreamBuilder; import org.apache.kafka.streams.KeyValue; @@ -34,8 +35,7 @@ public class KStreamTransformTest { private String topicName = "topic"; - private IntegerDeserializer keyDeserializer = new IntegerDeserializer(); - private IntegerDeserializer valDeserializer = new IntegerDeserializer(); + final private Serde intSerde = Serdes.Integer(); @Test public void testTransform() { @@ -71,9 +71,8 @@ public class KStreamTransformTest { final int[] expectedKeys = {1, 10, 100, 1000}; - KStream stream; MockProcessorSupplier processor = new MockProcessorSupplier<>(); - stream = builder.stream(keyDeserializer, valDeserializer, topicName); + KStream stream = builder.stream(intSerde, intSerde, topicName); stream.transform(transformerSupplier).process(processor); KStreamTestDriver driver = new KStreamTestDriver(builder); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamTransformValuesTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamTransformValuesTest.java index 7def9db45bc..52abdf728c9 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamTransformValuesTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamTransformValuesTest.java @@ -17,7 +17,8 @@ package org.apache.kafka.streams.kstream.internals; -import org.apache.kafka.common.serialization.IntegerDeserializer; +import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.streams.kstream.KStream; import org.apache.kafka.streams.kstream.KStreamBuilder; import org.apache.kafka.streams.kstream.ValueTransformer; @@ -33,8 +34,7 @@ public class KStreamTransformValuesTest { private String topicName = "topic"; - private IntegerDeserializer keyDeserializer = new IntegerDeserializer(); - private IntegerDeserializer valDeserializer = new IntegerDeserializer(); + final private Serde intSerde = Serdes.Integer(); @Test public void testTransform() { @@ -72,7 +72,7 @@ public class KStreamTransformValuesTest { KStream stream; MockProcessorSupplier processor = new MockProcessorSupplier<>(); - stream = builder.stream(keyDeserializer, valDeserializer, topicName); + stream = builder.stream(intSerde, intSerde, topicName); stream.transformValues(valueTransformerSupplier).process(processor); KStreamTestDriver driver = new KStreamTestDriver(builder); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregateTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregateTest.java index 9e0745a374c..e19510f6793 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregateTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregateTest.java @@ -17,10 +17,8 @@ package org.apache.kafka.streams.kstream.internals; -import org.apache.kafka.common.serialization.Deserializer; -import org.apache.kafka.common.serialization.Serializer; -import org.apache.kafka.common.serialization.StringDeserializer; -import org.apache.kafka.common.serialization.StringSerializer; +import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.streams.kstream.Aggregator; import org.apache.kafka.streams.kstream.HoppingWindows; @@ -41,8 +39,7 @@ import static org.junit.Assert.assertEquals; public class KStreamWindowAggregateTest { - private final Serializer strSerializer = new StringSerializer(); - private final Deserializer strDeserializer = new StringDeserializer(); + final private Serde strSerde = new Serdes.StringSerde(); private class StringAdd implements Aggregator { @@ -68,13 +65,11 @@ public class KStreamWindowAggregateTest { final KStreamBuilder builder = new KStreamBuilder(); String topic1 = "topic1"; - KStream stream1 = builder.stream(strDeserializer, strDeserializer, topic1); + KStream stream1 = builder.stream(strSerde, strSerde, topic1); KTable, String> table2 = stream1.aggregateByKey(new StringInit(), new StringAdd(), HoppingWindows.of("topic1-Canonized").with(10L).every(5L), - strSerializer, - strSerializer, - strDeserializer, - strDeserializer); + strSerde, + strSerde); MockProcessorSupplier, String> proc2 = new MockProcessorSupplier<>(); table2.toStream().process(proc2); @@ -147,24 +142,20 @@ public class KStreamWindowAggregateTest { String topic1 = "topic1"; String topic2 = "topic2"; - KStream stream1 = builder.stream(strDeserializer, strDeserializer, topic1); + KStream stream1 = builder.stream(strSerde, strSerde, topic1); KTable, String> table1 = stream1.aggregateByKey(new StringInit(), new StringAdd(), HoppingWindows.of("topic1-Canonized").with(10L).every(5L), - strSerializer, - strSerializer, - strDeserializer, - strDeserializer); + strSerde, + strSerde); MockProcessorSupplier, String> proc1 = new MockProcessorSupplier<>(); table1.toStream().process(proc1); - KStream stream2 = builder.stream(strDeserializer, strDeserializer, topic2); + KStream stream2 = builder.stream(strSerde, strSerde, topic2); KTable, String> table2 = stream2.aggregateByKey(new StringInit(), new StringAdd(), HoppingWindows.of("topic2-Canonized").with(10L).every(5L), - strSerializer, - strSerializer, - strDeserializer, - strDeserializer); + strSerde, + strSerde); MockProcessorSupplier, String> proc2 = new MockProcessorSupplier<>(); table2.toStream().process(proc2); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableAggregateTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableAggregateTest.java index ec85ed75462..fc01e5e7f60 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableAggregateTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableAggregateTest.java @@ -17,10 +17,8 @@ package org.apache.kafka.streams.kstream.internals; -import org.apache.kafka.common.serialization.Deserializer; -import org.apache.kafka.common.serialization.Serializer; -import org.apache.kafka.common.serialization.StringDeserializer; -import org.apache.kafka.common.serialization.StringSerializer; +import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.streams.kstream.Aggregator; import org.apache.kafka.streams.kstream.Initializer; @@ -38,8 +36,7 @@ import static org.junit.Assert.assertEquals; public class KTableAggregateTest { - private final Serializer strSerializer = new StringSerializer(); - private final Deserializer strDeserializer = new StringDeserializer(); + final private Serde stringSerde = new Serdes.StringSerde(); private class StringAdd implements Aggregator { @@ -74,15 +71,12 @@ public class KTableAggregateTest { final KStreamBuilder builder = new KStreamBuilder(); String topic1 = "topic1"; - KTable table1 = builder.table(strSerializer, strSerializer, strDeserializer, strDeserializer, topic1); - KTable table2 = table1.aggregate(new StringInit(), new StringAdd(), new StringRemove(), + KTable table1 = builder.table(stringSerde, stringSerde, topic1); + KTable table2 = table1.aggregate(new StringInit(), new StringAdd(), new StringRemove(), new NoOpKeyValueMapper(), - strSerializer, - strSerializer, - strSerializer, - strDeserializer, - strDeserializer, - strDeserializer, + stringSerde, + stringSerde, + stringSerde, "topic1-Canonized"); MockProcessorSupplier proc2 = new MockProcessorSupplier<>(); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableFilterTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableFilterTest.java index c43bea0c186..5491ea316e8 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableFilterTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableFilterTest.java @@ -17,12 +17,8 @@ package org.apache.kafka.streams.kstream.internals; -import org.apache.kafka.common.serialization.Deserializer; -import org.apache.kafka.common.serialization.IntegerDeserializer; -import org.apache.kafka.common.serialization.IntegerSerializer; -import org.apache.kafka.common.serialization.Serializer; -import org.apache.kafka.common.serialization.StringDeserializer; -import org.apache.kafka.common.serialization.StringSerializer; +import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.streams.kstream.KStreamBuilder; import org.apache.kafka.streams.kstream.KTable; @@ -40,10 +36,8 @@ import static org.junit.Assert.assertNull; public class KTableFilterTest { - private final Serializer strSerializer = new StringSerializer(); - private final Deserializer strDeserializer = new StringDeserializer(); - private final Serializer intSerializer = new IntegerSerializer(); - private final Deserializer intDeserializer = new IntegerDeserializer(); + final private Serde intSerde = new Serdes.IntegerSerde(); + final private Serde stringSerde = new Serdes.StringSerde(); @Test public void testKTable() { @@ -51,7 +45,7 @@ public class KTableFilterTest { String topic1 = "topic1"; - KTable table1 = builder.table(strSerializer, intSerializer, strDeserializer, intDeserializer, topic1); + KTable table1 = builder.table(stringSerde, intSerde, topic1); KTable table2 = table1.filter(new Predicate() { @Override @@ -93,7 +87,7 @@ public class KTableFilterTest { String topic1 = "topic1"; KTableImpl table1 = - (KTableImpl) builder.table(strSerializer, intSerializer, strDeserializer, intDeserializer, topic1); + (KTableImpl) builder.table(stringSerde, intSerde, topic1); KTableImpl table2 = (KTableImpl) table1.filter( new Predicate() { @Override @@ -112,7 +106,7 @@ public class KTableFilterTest { KTableValueGetterSupplier getterSupplier2 = table2.valueGetterSupplier(); KTableValueGetterSupplier getterSupplier3 = table3.valueGetterSupplier(); - KStreamTestDriver driver = new KStreamTestDriver(builder, stateDir, null, null, null, null); + KStreamTestDriver driver = new KStreamTestDriver(builder, stateDir, null, null); KTableValueGetter getter2 = getterSupplier2.get(); KTableValueGetter getter3 = getterSupplier3.get(); @@ -178,7 +172,7 @@ public class KTableFilterTest { String topic1 = "topic1"; KTableImpl table1 = - (KTableImpl) builder.table(strSerializer, intSerializer, strDeserializer, intDeserializer, topic1); + (KTableImpl) builder.table(stringSerde, intSerde, topic1); KTableImpl table2 = (KTableImpl) table1.filter( new Predicate() { @Override @@ -193,7 +187,7 @@ public class KTableFilterTest { builder.addProcessor("proc1", proc1, table1.name); builder.addProcessor("proc2", proc2, table2.name); - KStreamTestDriver driver = new KStreamTestDriver(builder, stateDir, null, null, null, null); + KStreamTestDriver driver = new KStreamTestDriver(builder, stateDir, null, null); driver.process(topic1, "A", 1); driver.process(topic1, "B", 1); @@ -233,7 +227,7 @@ public class KTableFilterTest { String topic1 = "topic1"; KTableImpl table1 = - (KTableImpl) builder.table(strSerializer, intSerializer, strDeserializer, intDeserializer, topic1); + (KTableImpl) builder.table(stringSerde, intSerde, topic1); KTableImpl table2 = (KTableImpl) table1.filter( new Predicate() { @Override @@ -250,7 +244,7 @@ public class KTableFilterTest { builder.addProcessor("proc1", proc1, table1.name); builder.addProcessor("proc2", proc2, table2.name); - KStreamTestDriver driver = new KStreamTestDriver(builder, stateDir, null, null, null, null); + KStreamTestDriver driver = new KStreamTestDriver(builder, stateDir, null, null); driver.process(topic1, "A", 1); driver.process(topic1, "B", 1); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableImplTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableImplTest.java index 2317c977cf4..20c3a287cab 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableImplTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableImplTest.java @@ -17,10 +17,8 @@ package org.apache.kafka.streams.kstream.internals; -import org.apache.kafka.common.serialization.Deserializer; -import org.apache.kafka.common.serialization.Serializer; -import org.apache.kafka.common.serialization.StringDeserializer; -import org.apache.kafka.common.serialization.StringSerializer; +import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.streams.kstream.KStreamBuilder; import org.apache.kafka.streams.kstream.KTable; @@ -40,16 +38,16 @@ import static org.junit.Assert.assertNull; public class KTableImplTest { + final private Serde stringSerde = new Serdes.StringSerde(); + @Test public void testKTable() { - final Serializer serializer = new StringSerializer(); - final Deserializer deserializer = new StringDeserializer(); final KStreamBuilder builder = new KStreamBuilder(); String topic1 = "topic1"; String topic2 = "topic2"; - KTable table1 = builder.table(serializer, serializer, deserializer, deserializer, topic1); + KTable table1 = builder.table(stringSerde, stringSerde, topic1); MockProcessorSupplier proc1 = new MockProcessorSupplier<>(); table1.toStream().process(proc1); @@ -74,7 +72,7 @@ public class KTableImplTest { MockProcessorSupplier proc3 = new MockProcessorSupplier<>(); table3.toStream().process(proc3); - KTable table4 = table1.through(topic2, serializer, serializer, deserializer, deserializer); + KTable table4 = table1.through(topic2, stringSerde, stringSerde); MockProcessorSupplier proc4 = new MockProcessorSupplier<>(); table4.toStream().process(proc4); @@ -96,15 +94,13 @@ public class KTableImplTest { public void testValueGetter() throws IOException { File stateDir = Files.createTempDirectory("test").toFile(); try { - final Serializer serializer = new StringSerializer(); - final Deserializer deserializer = new StringDeserializer(); final KStreamBuilder builder = new KStreamBuilder(); String topic1 = "topic1"; String topic2 = "topic2"; KTableImpl table1 = - (KTableImpl) builder.table(serializer, serializer, deserializer, deserializer, topic1); + (KTableImpl) builder.table(stringSerde, stringSerde, topic1); KTableImpl table2 = (KTableImpl) table1.mapValues( new ValueMapper() { @Override @@ -120,14 +116,14 @@ public class KTableImplTest { } }); KTableImpl table4 = (KTableImpl) - table1.through(topic2, serializer, serializer, deserializer, deserializer); + table1.through(topic2, stringSerde, stringSerde); KTableValueGetterSupplier getterSupplier1 = table1.valueGetterSupplier(); KTableValueGetterSupplier getterSupplier2 = table2.valueGetterSupplier(); KTableValueGetterSupplier getterSupplier3 = table3.valueGetterSupplier(); KTableValueGetterSupplier getterSupplier4 = table4.valueGetterSupplier(); - KStreamTestDriver driver = new KStreamTestDriver(builder, stateDir, null, null, null, null); + KStreamTestDriver driver = new KStreamTestDriver(builder, stateDir, null, null); // two state store should be created assertEquals(2, driver.allStateStores().size()); @@ -223,9 +219,6 @@ public class KTableImplTest { @Test public void testStateStore() throws IOException { - final Serializer serializer = new StringSerializer(); - final Deserializer deserializer = new StringDeserializer(); - String topic1 = "topic1"; String topic2 = "topic2"; @@ -234,9 +227,9 @@ public class KTableImplTest { KStreamBuilder builder = new KStreamBuilder(); KTableImpl table1 = - (KTableImpl) builder.table(serializer, serializer, deserializer, deserializer, topic1); + (KTableImpl) builder.table(stringSerde, stringSerde, topic1); KTableImpl table2 = - (KTableImpl) builder.table(serializer, serializer, deserializer, deserializer, topic2); + (KTableImpl) builder.table(stringSerde, stringSerde, topic2); KTableImpl table1Mapped = (KTableImpl) table1.mapValues( new ValueMapper() { @@ -253,7 +246,7 @@ public class KTableImplTest { } }); - KStreamTestDriver driver = new KStreamTestDriver(builder, stateDir, null, null, null, null); + KStreamTestDriver driver = new KStreamTestDriver(builder, stateDir, null, null); driver.setTime(0L); // no state store should be created @@ -267,9 +260,9 @@ public class KTableImplTest { KStreamBuilder builder = new KStreamBuilder(); KTableImpl table1 = - (KTableImpl) builder.table(serializer, serializer, deserializer, deserializer, topic1); + (KTableImpl) builder.table(stringSerde, stringSerde, topic1); KTableImpl table2 = - (KTableImpl) builder.table(serializer, serializer, deserializer, deserializer, topic2); + (KTableImpl) builder.table(stringSerde, stringSerde, topic2); KTableImpl table1Mapped = (KTableImpl) table1.mapValues( new ValueMapper() { @@ -293,7 +286,7 @@ public class KTableImplTest { } }); - KStreamTestDriver driver = new KStreamTestDriver(builder, stateDir, null, null, null, null); + KStreamTestDriver driver = new KStreamTestDriver(builder, stateDir, null, null); driver.setTime(0L); // two state store should be created diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableJoinTest.java index 12bfb9cd769..5f30574cd5a 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableJoinTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableJoinTest.java @@ -17,10 +17,8 @@ package org.apache.kafka.streams.kstream.internals; -import org.apache.kafka.common.serialization.IntegerDeserializer; -import org.apache.kafka.common.serialization.IntegerSerializer; -import org.apache.kafka.common.serialization.StringDeserializer; -import org.apache.kafka.common.serialization.StringSerializer; +import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.streams.kstream.KStreamBuilder; import org.apache.kafka.streams.kstream.KTable; @@ -47,10 +45,8 @@ public class KTableKTableJoinTest { private String topic1 = "topic1"; private String topic2 = "topic2"; - private IntegerSerializer keySerializer = new IntegerSerializer(); - private StringSerializer valSerializer = new StringSerializer(); - private IntegerDeserializer keyDeserializer = new IntegerDeserializer(); - private StringDeserializer valDeserializer = new StringDeserializer(); + final private Serde intSerde = new Serdes.IntegerSerde(); + final private Serde stringSerde = new Serdes.StringSerde(); private ValueJoiner joiner = new ValueJoiner() { @Override @@ -80,8 +76,8 @@ public class KTableKTableJoinTest { MockProcessorSupplier processor; processor = new MockProcessorSupplier<>(); - table1 = builder.table(keySerializer, valSerializer, keyDeserializer, valDeserializer, topic1); - table2 = builder.table(keySerializer, valSerializer, keyDeserializer, valDeserializer, topic2); + table1 = builder.table(intSerde, stringSerde, topic1); + table2 = builder.table(intSerde, stringSerde, topic2); joined = table1.join(table2, joiner); joined.toStream().process(processor); @@ -179,8 +175,8 @@ public class KTableKTableJoinTest { KTable joined; MockProcessorSupplier proc; - table1 = builder.table(keySerializer, valSerializer, keyDeserializer, valDeserializer, topic1); - table2 = builder.table(keySerializer, valSerializer, keyDeserializer, valDeserializer, topic2); + table1 = builder.table(intSerde, stringSerde, topic1); + table2 = builder.table(intSerde, stringSerde, topic2); joined = table1.join(table2, joiner); proc = new MockProcessorSupplier<>(); @@ -267,8 +263,8 @@ public class KTableKTableJoinTest { KTable joined; MockProcessorSupplier proc; - table1 = builder.table(keySerializer, valSerializer, keyDeserializer, valDeserializer, topic1); - table2 = builder.table(keySerializer, valSerializer, keyDeserializer, valDeserializer, topic2); + table1 = builder.table(intSerde, stringSerde, topic1); + table2 = builder.table(intSerde, stringSerde, topic2); joined = table1.join(table2, joiner); ((KTableImpl) joined).enableSendingOldValues(); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoinTest.java index e3cf22b97a5..f92c5ca8141 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoinTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoinTest.java @@ -17,10 +17,8 @@ package org.apache.kafka.streams.kstream.internals; -import org.apache.kafka.common.serialization.IntegerDeserializer; -import org.apache.kafka.common.serialization.IntegerSerializer; -import org.apache.kafka.common.serialization.StringDeserializer; -import org.apache.kafka.common.serialization.StringSerializer; +import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.streams.kstream.KStreamBuilder; import org.apache.kafka.streams.kstream.KTable; @@ -48,10 +46,8 @@ public class KTableKTableLeftJoinTest { private String topic1 = "topic1"; private String topic2 = "topic2"; - private IntegerSerializer keySerializer = new IntegerSerializer(); - private StringSerializer valSerializer = new StringSerializer(); - private IntegerDeserializer keyDeserializer = new IntegerDeserializer(); - private StringDeserializer valDeserializer = new StringDeserializer(); + final private Serde intSerde = new Serdes.IntegerSerde(); + final private Serde stringSerde = new Serdes.StringSerde(); private ValueJoiner joiner = new ValueJoiner() { @Override @@ -83,15 +79,11 @@ public class KTableKTableLeftJoinTest { final int[] expectedKeys = new int[]{0, 1, 2, 3}; - KTable table1; - KTable table2; - KTable joined; + KTable table1 = builder.table(intSerde, stringSerde, topic1); + KTable table2 = builder.table(intSerde, stringSerde, topic2); + KTable joined = table1.leftJoin(table2, joiner); MockProcessorSupplier processor; - processor = new MockProcessorSupplier<>(); - table1 = builder.table(keySerializer, valSerializer, keyDeserializer, valDeserializer, topic1); - table2 = builder.table(keySerializer, valSerializer, keyDeserializer, valDeserializer, topic2); - joined = table1.leftJoin(table2, joiner); joined.toStream().process(processor); Collection> copartitionGroups = builder.copartitionGroups(); @@ -188,8 +180,8 @@ public class KTableKTableLeftJoinTest { KTable joined; MockProcessorSupplier proc; - table1 = builder.table(keySerializer, valSerializer, keyDeserializer, valDeserializer, topic1); - table2 = builder.table(keySerializer, valSerializer, keyDeserializer, valDeserializer, topic2); + table1 = builder.table(intSerde, stringSerde, topic1); + table2 = builder.table(intSerde, stringSerde, topic2); joined = table1.leftJoin(table2, joiner); proc = new MockProcessorSupplier<>(); @@ -276,8 +268,8 @@ public class KTableKTableLeftJoinTest { KTable joined; MockProcessorSupplier proc; - table1 = builder.table(keySerializer, valSerializer, keyDeserializer, valDeserializer, topic1); - table2 = builder.table(keySerializer, valSerializer, keyDeserializer, valDeserializer, topic2); + table1 = builder.table(intSerde, stringSerde, topic1); + table2 = builder.table(intSerde, stringSerde, topic2); joined = table1.leftJoin(table2, joiner); ((KTableImpl) joined).enableSendingOldValues(); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableOuterJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableOuterJoinTest.java index feabc08612b..6cc77e0b1b0 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableOuterJoinTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableOuterJoinTest.java @@ -17,10 +17,8 @@ package org.apache.kafka.streams.kstream.internals; -import org.apache.kafka.common.serialization.IntegerDeserializer; -import org.apache.kafka.common.serialization.IntegerSerializer; -import org.apache.kafka.common.serialization.StringDeserializer; -import org.apache.kafka.common.serialization.StringSerializer; +import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.streams.kstream.KStreamBuilder; import org.apache.kafka.streams.kstream.KTable; @@ -47,10 +45,8 @@ public class KTableKTableOuterJoinTest { private String topic1 = "topic1"; private String topic2 = "topic2"; - private IntegerSerializer keySerializer = new IntegerSerializer(); - private StringSerializer valSerializer = new StringSerializer(); - private IntegerDeserializer keyDeserializer = new IntegerDeserializer(); - private StringDeserializer valDeserializer = new StringDeserializer(); + final private Serde intSerde = new Serdes.IntegerSerde(); + final private Serde stringSerde = new Serdes.StringSerde(); private ValueJoiner joiner = new ValueJoiner() { @Override @@ -80,8 +76,8 @@ public class KTableKTableOuterJoinTest { MockProcessorSupplier processor; processor = new MockProcessorSupplier<>(); - table1 = builder.table(keySerializer, valSerializer, keyDeserializer, valDeserializer, topic1); - table2 = builder.table(keySerializer, valSerializer, keyDeserializer, valDeserializer, topic2); + table1 = builder.table(intSerde, stringSerde, topic1); + table2 = builder.table(intSerde, stringSerde, topic2); joined = table1.outerJoin(table2, joiner); joined.toStream().process(processor); @@ -188,8 +184,8 @@ public class KTableKTableOuterJoinTest { KTable joined; MockProcessorSupplier proc; - table1 = builder.table(keySerializer, valSerializer, keyDeserializer, valDeserializer, topic1); - table2 = builder.table(keySerializer, valSerializer, keyDeserializer, valDeserializer, topic2); + table1 = builder.table(intSerde, stringSerde, topic1); + table2 = builder.table(intSerde, stringSerde, topic2); joined = table1.outerJoin(table2, joiner); proc = new MockProcessorSupplier<>(); @@ -284,8 +280,8 @@ public class KTableKTableOuterJoinTest { KTable joined; MockProcessorSupplier proc; - table1 = builder.table(keySerializer, valSerializer, keyDeserializer, valDeserializer, topic1); - table2 = builder.table(keySerializer, valSerializer, keyDeserializer, valDeserializer, topic2); + table1 = builder.table(intSerde, stringSerde, topic1); + table2 = builder.table(intSerde, stringSerde, topic2); joined = table1.outerJoin(table2, joiner); ((KTableImpl) joined).enableSendingOldValues(); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableMapValuesTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableMapValuesTest.java index 58f1c2a127d..aa3daeb8de5 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableMapValuesTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableMapValuesTest.java @@ -17,10 +17,8 @@ package org.apache.kafka.streams.kstream.internals; -import org.apache.kafka.common.serialization.Deserializer; -import org.apache.kafka.common.serialization.Serializer; -import org.apache.kafka.common.serialization.StringDeserializer; -import org.apache.kafka.common.serialization.StringSerializer; +import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.streams.kstream.KStreamBuilder; import org.apache.kafka.streams.kstream.KTable; @@ -41,8 +39,7 @@ import static org.junit.Assert.assertTrue; public class KTableMapValuesTest { - private final Serializer strSerializer = new StringSerializer(); - private final Deserializer strDeserializer = new StringDeserializer(); + final private Serde stringSerde = new Serdes.StringSerde(); @Test public void testKTable() { @@ -50,7 +47,7 @@ public class KTableMapValuesTest { String topic1 = "topic1"; - KTable table1 = builder.table(strSerializer, strSerializer, strDeserializer, strDeserializer, topic1); + KTable table1 = builder.table(stringSerde, stringSerde, topic1); KTable table2 = table1.mapValues(new ValueMapper() { @Override public Integer apply(String value) { @@ -75,15 +72,13 @@ public class KTableMapValuesTest { public void testValueGetter() throws IOException { File stateDir = Files.createTempDirectory("test").toFile(); try { - final Serializer serializer = new StringSerializer(); - final Deserializer deserializer = new StringDeserializer(); final KStreamBuilder builder = new KStreamBuilder(); String topic1 = "topic1"; String topic2 = "topic2"; KTableImpl table1 = - (KTableImpl) builder.table(serializer, serializer, deserializer, deserializer, topic1); + (KTableImpl) builder.table(stringSerde, stringSerde, topic1); KTableImpl table2 = (KTableImpl) table1.mapValues( new ValueMapper() { @Override @@ -99,14 +94,14 @@ public class KTableMapValuesTest { } }); KTableImpl table4 = (KTableImpl) - table1.through(topic2, serializer, serializer, deserializer, deserializer); + table1.through(topic2, stringSerde, stringSerde); KTableValueGetterSupplier getterSupplier1 = table1.valueGetterSupplier(); KTableValueGetterSupplier getterSupplier2 = table2.valueGetterSupplier(); KTableValueGetterSupplier getterSupplier3 = table3.valueGetterSupplier(); KTableValueGetterSupplier getterSupplier4 = table4.valueGetterSupplier(); - KStreamTestDriver driver = new KStreamTestDriver(builder, stateDir, null, null, null, null); + KStreamTestDriver driver = new KStreamTestDriver(builder, stateDir, null, null); KTableValueGetter getter1 = getterSupplier1.get(); getter1.init(driver.context()); @@ -201,14 +196,12 @@ public class KTableMapValuesTest { public void testNotSendingOldValue() throws IOException { File stateDir = Files.createTempDirectory("test").toFile(); try { - final Serializer serializer = new StringSerializer(); - final Deserializer deserializer = new StringDeserializer(); final KStreamBuilder builder = new KStreamBuilder(); String topic1 = "topic1"; KTableImpl table1 = - (KTableImpl) builder.table(serializer, serializer, deserializer, deserializer, topic1); + (KTableImpl) builder.table(stringSerde, stringSerde, topic1); KTableImpl table2 = (KTableImpl) table1.mapValues( new ValueMapper() { @Override @@ -221,7 +214,7 @@ public class KTableMapValuesTest { builder.addProcessor("proc", proc, table2.name); - KStreamTestDriver driver = new KStreamTestDriver(builder, stateDir, null, null, null, null); + KStreamTestDriver driver = new KStreamTestDriver(builder, stateDir, null, null); assertFalse(table1.sendingOldValueEnabled()); assertFalse(table2.sendingOldValueEnabled()); @@ -254,14 +247,12 @@ public class KTableMapValuesTest { public void testSendingOldValue() throws IOException { File stateDir = Files.createTempDirectory("test").toFile(); try { - final Serializer serializer = new StringSerializer(); - final Deserializer deserializer = new StringDeserializer(); final KStreamBuilder builder = new KStreamBuilder(); String topic1 = "topic1"; KTableImpl table1 = - (KTableImpl) builder.table(serializer, serializer, deserializer, deserializer, topic1); + (KTableImpl) builder.table(stringSerde, stringSerde, topic1); KTableImpl table2 = (KTableImpl) table1.mapValues( new ValueMapper() { @Override @@ -276,7 +267,7 @@ public class KTableMapValuesTest { builder.addProcessor("proc", proc, table2.name); - KStreamTestDriver driver = new KStreamTestDriver(builder, stateDir, null, null, null, null); + KStreamTestDriver driver = new KStreamTestDriver(builder, stateDir, null, null); assertTrue(table1.sendingOldValueEnabled()); assertTrue(table2.sendingOldValueEnabled()); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableSourceTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableSourceTest.java index 187a6f2fe2e..51276f32282 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableSourceTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableSourceTest.java @@ -17,10 +17,8 @@ package org.apache.kafka.streams.kstream.internals; -import org.apache.kafka.common.serialization.Deserializer; -import org.apache.kafka.common.serialization.Serializer; -import org.apache.kafka.common.serialization.StringDeserializer; -import org.apache.kafka.common.serialization.StringSerializer; +import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.streams.kstream.KStreamBuilder; import org.apache.kafka.streams.kstream.KTable; @@ -38,8 +36,7 @@ import static org.junit.Assert.assertTrue; public class KTableSourceTest { - private final Serializer strSerializer = new StringSerializer(); - private final Deserializer strDeserializer = new StringDeserializer(); + final private Serde stringSerde = new Serdes.StringSerde(); @Test public void testKTable() { @@ -47,7 +44,7 @@ public class KTableSourceTest { String topic1 = "topic1"; - KTable table1 = builder.table(strSerializer, strSerializer, strDeserializer, strDeserializer, topic1); + KTable table1 = builder.table(stringSerde, stringSerde, topic1); MockProcessorSupplier proc1 = new MockProcessorSupplier<>(); table1.toStream().process(proc1); @@ -72,12 +69,11 @@ public class KTableSourceTest { String topic1 = "topic1"; - KTableImpl table1 = (KTableImpl) - builder.table(strSerializer, strSerializer, strDeserializer, strDeserializer, topic1); + KTableImpl table1 = (KTableImpl) builder.table(stringSerde, stringSerde, topic1); KTableValueGetterSupplier getterSupplier1 = table1.valueGetterSupplier(); - KStreamTestDriver driver = new KStreamTestDriver(builder, stateDir, null, null, null, null); + KStreamTestDriver driver = new KStreamTestDriver(builder, stateDir, null, null); KTableValueGetter getter1 = getterSupplier1.get(); getter1.init(driver.context()); @@ -123,14 +119,13 @@ public class KTableSourceTest { String topic1 = "topic1"; - KTableImpl table1 = (KTableImpl) - builder.table(strSerializer, strSerializer, strDeserializer, strDeserializer, topic1); + KTableImpl table1 = (KTableImpl) builder.table(stringSerde, stringSerde, topic1); MockProcessorSupplier proc1 = new MockProcessorSupplier<>(); builder.addProcessor("proc1", proc1, table1.name); - KStreamTestDriver driver = new KStreamTestDriver(builder, stateDir, null, null, null, null); + KStreamTestDriver driver = new KStreamTestDriver(builder, stateDir, null, null); driver.process(topic1, "A", "01"); driver.process(topic1, "B", "01"); @@ -165,8 +160,7 @@ public class KTableSourceTest { String topic1 = "topic1"; - KTableImpl table1 = (KTableImpl) - builder.table(strSerializer, strSerializer, strDeserializer, strDeserializer, topic1); + KTableImpl table1 = (KTableImpl) builder.table(stringSerde, stringSerde, topic1); table1.enableSendingOldValues(); @@ -176,7 +170,7 @@ public class KTableSourceTest { builder.addProcessor("proc1", proc1, table1.name); - KStreamTestDriver driver = new KStreamTestDriver(builder, stateDir, null, null, null, null); + KStreamTestDriver driver = new KStreamTestDriver(builder, stateDir, null, null); driver.process(topic1, "A", "01"); driver.process(topic1, "B", "01"); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/WindowedStreamPartitionerTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/WindowedStreamPartitionerTest.java index 1b8cbb8a449..7c6d5ec0fd2 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/WindowedStreamPartitionerTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/WindowedStreamPartitionerTest.java @@ -37,8 +37,8 @@ public class WindowedStreamPartitionerTest { private String topicName = "topic"; - private IntegerSerializer keySerializer = new IntegerSerializer(); - private StringSerializer valSerializer = new StringSerializer(); + private IntegerSerializer intSerializer = new IntegerSerializer(); + private StringSerializer stringSerializer = new StringSerializer(); private List infos = Arrays.asList( new PartitionInfo(topicName, 0, Node.noNode(), new Node[0], new Node[0]), @@ -58,15 +58,15 @@ public class WindowedStreamPartitionerTest { DefaultPartitioner defaultPartitioner = new DefaultPartitioner(); - WindowedSerializer windowedSerializer = new WindowedSerializer<>(keySerializer); + WindowedSerializer windowedSerializer = new WindowedSerializer<>(intSerializer); WindowedStreamPartitioner streamPartitioner = new WindowedStreamPartitioner<>(windowedSerializer); for (int k = 0; k < 10; k++) { Integer key = rand.nextInt(); - byte[] keyBytes = keySerializer.serialize(topicName, key); + byte[] keyBytes = intSerializer.serialize(topicName, key); String value = key.toString(); - byte[] valueBytes = valSerializer.serialize(topicName, value); + byte[] valueBytes = stringSerializer.serialize(topicName, value); Integer expected = defaultPartitioner.partition("topic", key, keyBytes, value, valueBytes, cluster); diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorTopologyTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorTopologyTest.java index 12210cca867..ef081761eb4 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorTopologyTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorTopologyTest.java @@ -24,6 +24,7 @@ import static org.junit.Assert.assertNull; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.serialization.Serializer; import org.apache.kafka.common.serialization.StringDeserializer; import org.apache.kafka.common.serialization.StringSerializer; @@ -70,11 +71,9 @@ public class ProcessorTopologyTest { props.setProperty(StreamsConfig.APPLICATION_ID_CONFIG, "processor-topology-test"); props.setProperty(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9091"); props.setProperty(StreamsConfig.STATE_DIR_CONFIG, localState.getAbsolutePath()); + props.setProperty(StreamsConfig.KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass().getName()); + props.setProperty(StreamsConfig.VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass().getName()); props.setProperty(StreamsConfig.TIMESTAMP_EXTRACTOR_CLASS_CONFIG, CustomTimestampExtractor.class.getName()); - props.setProperty(StreamsConfig.KEY_SERIALIZER_CLASS_CONFIG, StringSerializer.class.getName()); - props.setProperty(StreamsConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class.getName()); - props.setProperty(StreamsConfig.VALUE_SERIALIZER_CLASS_CONFIG, StringSerializer.class.getName()); - props.setProperty(StreamsConfig.VALUE_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class.getName()); this.config = new StreamsConfig(props); } diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StandbyTaskTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StandbyTaskTest.java index 21bdaffbd8e..ea24441643e 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StandbyTaskTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StandbyTaskTest.java @@ -33,6 +33,7 @@ import org.apache.kafka.streams.processor.StateStoreSupplier; import org.apache.kafka.streams.processor.TaskId; import org.apache.kafka.streams.state.internals.OffsetCheckpoint; import org.apache.kafka.test.MockStateStoreSupplier; +import org.apache.kafka.test.MockTimestampExtractor; import org.junit.Before; import org.junit.Test; @@ -89,15 +90,11 @@ public class StandbyTaskTest { private StreamsConfig createConfig(final File baseDir) throws Exception { return new StreamsConfig(new Properties() { { - setProperty(StreamsConfig.KEY_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer"); - setProperty(StreamsConfig.KEY_DESERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArrayDeserializer"); - setProperty(StreamsConfig.VALUE_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer"); - setProperty(StreamsConfig.VALUE_DESERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArrayDeserializer"); - setProperty(StreamsConfig.TIMESTAMP_EXTRACTOR_CLASS_CONFIG, "org.apache.kafka.test.MockTimestampExtractor"); setProperty(StreamsConfig.APPLICATION_ID_CONFIG, applicationId); setProperty(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:2171"); setProperty(StreamsConfig.BUFFERED_RECORDS_PER_PARTITION_CONFIG, "3"); setProperty(StreamsConfig.STATE_DIR_CONFIG, baseDir.getCanonicalPath()); + setProperty(StreamsConfig.TIMESTAMP_EXTRACTOR_CLASS_CONFIG, MockTimestampExtractor.class.getName()); } }); } diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignorTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignorTest.java index a5990bdcf69..be851bff67b 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignorTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignorTest.java @@ -36,6 +36,7 @@ import org.apache.kafka.streams.processor.internals.assignment.AssignmentInfo; import org.apache.kafka.streams.processor.internals.assignment.SubscriptionInfo; import org.apache.kafka.test.MockProcessorSupplier; import org.apache.kafka.test.MockStateStoreSupplier; +import org.apache.kafka.test.MockTimestampExtractor; import org.junit.Test; import java.util.ArrayList; @@ -89,14 +90,10 @@ public class StreamPartitionAssignorTest { private Properties configProps() { return new Properties() { { - setProperty(StreamsConfig.KEY_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer"); - setProperty(StreamsConfig.KEY_DESERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArrayDeserializer"); - setProperty(StreamsConfig.VALUE_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer"); - setProperty(StreamsConfig.VALUE_DESERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArrayDeserializer"); - setProperty(StreamsConfig.TIMESTAMP_EXTRACTOR_CLASS_CONFIG, "org.apache.kafka.test.MockTimestampExtractor"); setProperty(StreamsConfig.APPLICATION_ID_CONFIG, "stream-partition-assignor-test"); setProperty(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:2171"); setProperty(StreamsConfig.BUFFERED_RECORDS_PER_PARTITION_CONFIG, "3"); + setProperty(StreamsConfig.TIMESTAMP_EXTRACTOR_CLASS_CONFIG, MockTimestampExtractor.class.getName()); } }; } diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java index f2ade6ba8c1..33fa5c4a091 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java @@ -33,6 +33,7 @@ import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.streams.processor.StateStoreSupplier; import org.apache.kafka.streams.processor.TaskId; import org.apache.kafka.test.MockSourceNode; +import org.apache.kafka.test.MockTimestampExtractor; import org.junit.Test; import org.junit.Before; @@ -73,15 +74,11 @@ public class StreamTaskTest { private StreamsConfig createConfig(final File baseDir) throws Exception { return new StreamsConfig(new Properties() { { - setProperty(StreamsConfig.KEY_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer"); - setProperty(StreamsConfig.KEY_DESERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArrayDeserializer"); - setProperty(StreamsConfig.VALUE_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer"); - setProperty(StreamsConfig.VALUE_DESERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArrayDeserializer"); - setProperty(StreamsConfig.TIMESTAMP_EXTRACTOR_CLASS_CONFIG, "org.apache.kafka.test.MockTimestampExtractor"); setProperty(StreamsConfig.APPLICATION_ID_CONFIG, "stream-task-test"); setProperty(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:2171"); setProperty(StreamsConfig.BUFFERED_RECORDS_PER_PARTITION_CONFIG, "3"); setProperty(StreamsConfig.STATE_DIR_CONFIG, baseDir.getCanonicalPath()); + setProperty(StreamsConfig.TIMESTAMP_EXTRACTOR_CLASS_CONFIG, MockTimestampExtractor.class.getName()); } }); } diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java index b201c078dec..e387a59d68e 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java @@ -41,6 +41,7 @@ import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.streams.processor.TaskId; import org.apache.kafka.streams.processor.TopologyBuilder; import org.apache.kafka.test.MockProcessorSupplier; +import org.apache.kafka.test.MockTimestampExtractor; import org.junit.Test; import java.io.File; @@ -113,14 +114,10 @@ public class StreamThreadTest { private Properties configProps() { return new Properties() { { - setProperty(StreamsConfig.KEY_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer"); - setProperty(StreamsConfig.KEY_DESERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArrayDeserializer"); - setProperty(StreamsConfig.VALUE_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer"); - setProperty(StreamsConfig.VALUE_DESERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArrayDeserializer"); - setProperty(StreamsConfig.TIMESTAMP_EXTRACTOR_CLASS_CONFIG, "org.apache.kafka.test.MockTimestampExtractor"); setProperty(StreamsConfig.APPLICATION_ID_CONFIG, applicationId); setProperty(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:2171"); setProperty(StreamsConfig.BUFFERED_RECORDS_PER_PARTITION_CONFIG, "3"); + setProperty(StreamsConfig.TIMESTAMP_EXTRACTOR_CLASS_CONFIG, MockTimestampExtractor.class.getName()); } }; } diff --git a/streams/src/test/java/org/apache/kafka/streams/smoketest/SmokeTestClient.java b/streams/src/test/java/org/apache/kafka/streams/smoketest/SmokeTestClient.java index 063eafeb04f..ce4956c1453 100644 --- a/streams/src/test/java/org/apache/kafka/streams/smoketest/SmokeTestClient.java +++ b/streams/src/test/java/org/apache/kafka/streams/smoketest/SmokeTestClient.java @@ -18,8 +18,6 @@ package org.apache.kafka.streams.smoketest; import org.apache.kafka.clients.consumer.ConsumerConfig; -import org.apache.kafka.common.serialization.ByteArrayDeserializer; -import org.apache.kafka.common.serialization.ByteArraySerializer; import org.apache.kafka.streams.KafkaStreams; import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.StreamsConfig; @@ -85,11 +83,7 @@ public class SmokeTestClient extends SmokeTestUtil { props.put(StreamsConfig.STATE_DIR_CONFIG, stateDir.toString()); props.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, kafka); props.put(StreamsConfig.ZOOKEEPER_CONNECT_CONFIG, zookeeper); - props.put(StreamsConfig.TIMESTAMP_EXTRACTOR_CLASS_CONFIG, TestTimestampExtractor.class); - props.put(StreamsConfig.KEY_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class); - props.put(StreamsConfig.VALUE_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class); - props.put(StreamsConfig.KEY_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class); - props.put(StreamsConfig.VALUE_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class); + props.put(StreamsConfig.TIMESTAMP_EXTRACTOR_CLASS_CONFIG, TestTimestampExtractor.class.getName()); props.put(StreamsConfig.NUM_STREAM_THREADS_CONFIG, 3); props.put(StreamsConfig.NUM_STANDBY_REPLICAS_CONFIG, 2); props.put(StreamsConfig.BUFFERED_RECORDS_PER_PARTITION_CONFIG, 100); @@ -98,9 +92,9 @@ public class SmokeTestClient extends SmokeTestUtil { KStreamBuilder builder = new KStreamBuilder(); - KStream source = builder.stream(stringDeserializer, integerDeserializer, "data"); + KStream source = builder.stream(stringSerde, intSerde, "data"); - source.to("echo", stringSerializer, integerSerializer); + source.to("echo", stringSerde, intSerde); KStream data = source.filter(new Predicate() { @Override @@ -125,15 +119,13 @@ public class SmokeTestClient extends SmokeTestUtil { } }, UnlimitedWindows.of("uwin-min"), - stringSerializer, - integerSerializer, - stringDeserializer, - integerDeserializer + stringSerde, + intSerde ).toStream().map( new Unwindow() - ).to("min", stringSerializer, integerSerializer); + ).to("min", stringSerde, intSerde); - KTable minTable = builder.table(stringSerializer, integerSerializer, stringDeserializer, integerDeserializer, "min"); + KTable minTable = builder.table(stringSerde, intSerde, "min"); minTable.toStream().process(SmokeTestUtil.printProcessorSupplier("min")); // max @@ -150,15 +142,13 @@ public class SmokeTestClient extends SmokeTestUtil { } }, UnlimitedWindows.of("uwin-max"), - stringSerializer, - integerSerializer, - stringDeserializer, - integerDeserializer + stringSerde, + intSerde ).toStream().map( new Unwindow() - ).to("max", stringSerializer, integerSerializer); + ).to("max", stringSerde, intSerde); - KTable maxTable = builder.table(stringSerializer, integerSerializer, stringDeserializer, integerDeserializer, "max"); + KTable maxTable = builder.table(stringSerde, intSerde, "max"); maxTable.toStream().process(SmokeTestUtil.printProcessorSupplier("max")); // sum @@ -175,28 +165,25 @@ public class SmokeTestClient extends SmokeTestUtil { } }, UnlimitedWindows.of("win-sum"), - stringSerializer, - longSerializer, - stringDeserializer, - longDeserializer + stringSerde, + longSerde ).toStream().map( new Unwindow() - ).to("sum", stringSerializer, longSerializer); + ).to("sum", stringSerde, longSerde); - KTable sumTable = builder.table(stringSerializer, longSerializer, stringDeserializer, longDeserializer, "sum"); + KTable sumTable = builder.table(stringSerde, longSerde, "sum"); sumTable.toStream().process(SmokeTestUtil.printProcessorSupplier("sum")); // cnt data.countByKey( UnlimitedWindows.of("uwin-cnt"), - stringSerializer, - stringDeserializer + stringSerde ).toStream().map( new Unwindow() - ).to("cnt", stringSerializer, longSerializer); + ).to("cnt", stringSerde, longSerde); - KTable cntTable = builder.table(stringSerializer, longSerializer, stringDeserializer, longDeserializer, "cnt"); + KTable cntTable = builder.table(stringSerde, longSerde, "cnt"); cntTable.toStream().process(SmokeTestUtil.printProcessorSupplier("cnt")); // dif @@ -206,7 +193,7 @@ public class SmokeTestClient extends SmokeTestUtil { return value1 - value2; } } - ).to("dif", stringSerializer, integerSerializer); + ).to("dif", stringSerde, intSerde); // avg sumTable.join( @@ -216,13 +203,12 @@ public class SmokeTestClient extends SmokeTestUtil { return (double) value1 / (double) value2; } } - ).to("avg", stringSerializer, doubleSerializer); + ).to("avg", stringSerde, doubleSerde); // windowed count data.countByKey( TumblingWindows.of("tumbling-win-cnt").with(WINDOW_SIZE), - stringSerializer, - stringDeserializer + stringSerde ).toStream().map( new KeyValueMapper, Long, KeyValue>() { @Override @@ -230,7 +216,7 @@ public class SmokeTestClient extends SmokeTestUtil { return new KeyValue<>(key.value() + "@" + key.window().start(), value); } } - ).to("wcnt", stringSerializer, longSerializer); + ).to("wcnt", stringSerde, longSerde); // test repartition Agg agg = new Agg(); @@ -239,14 +225,11 @@ public class SmokeTestClient extends SmokeTestUtil { agg.adder(), agg.remover(), agg.selector(), - stringSerializer, - longSerializer, - longSerializer, - stringDeserializer, - longDeserializer, - longDeserializer, + stringSerde, + longSerde, + longSerde, "cntByCnt" - ).to("tagg", stringSerializer, longSerializer); + ).to("tagg", stringSerde, longSerde); return new KafkaStreams(builder, props); } diff --git a/streams/src/test/java/org/apache/kafka/streams/smoketest/SmokeTestDriver.java b/streams/src/test/java/org/apache/kafka/streams/smoketest/SmokeTestDriver.java index c0a6f467487..1abf88d17eb 100644 --- a/streams/src/test/java/org/apache/kafka/streams/smoketest/SmokeTestDriver.java +++ b/streams/src/test/java/org/apache/kafka/streams/smoketest/SmokeTestDriver.java @@ -157,7 +157,7 @@ public class SmokeTestDriver extends SmokeTestUtil { } ProducerRecord record = - new ProducerRecord<>("data", stringSerializer.serialize("", key), integerSerializer.serialize("", value)); + new ProducerRecord<>("data", stringSerde.serializer().serialize("", key), intSerde.serializer().serialize("", value)); producer.send(record); @@ -233,10 +233,10 @@ public class SmokeTestDriver extends SmokeTestUtil { retryCount = 0; for (ConsumerRecord record : records) { - String key = stringDeserializer.deserialize("", record.key()); + String key = stringSerde.deserializer().deserialize("", record.key()); switch (record.topic()) { case "echo": - Integer value = integerDeserializer.deserialize("", record.value()); + Integer value = intSerde.deserializer().deserialize("", record.value()); if (value != null && value == END) { keys.remove(key); if (keys.isEmpty()) { @@ -249,28 +249,28 @@ public class SmokeTestDriver extends SmokeTestUtil { } break; case "min": - min.put(key, integerDeserializer.deserialize("", record.value())); + min.put(key, intSerde.deserializer().deserialize("", record.value())); break; case "max": - max.put(key, integerDeserializer.deserialize("", record.value())); + max.put(key, intSerde.deserializer().deserialize("", record.value())); break; case "dif": - dif.put(key, integerDeserializer.deserialize("", record.value())); + dif.put(key, intSerde.deserializer().deserialize("", record.value())); break; case "sum": - sum.put(key, longDeserializer.deserialize("", record.value())); + sum.put(key, longSerde.deserializer().deserialize("", record.value())); break; case "cnt": - cnt.put(key, longDeserializer.deserialize("", record.value())); + cnt.put(key, longSerde.deserializer().deserialize("", record.value())); break; case "avg": - avg.put(key, doubleDeserializer.deserialize("", record.value())); + avg.put(key, doubleSerde.deserializer().deserialize("", record.value())); break; case "wcnt": - wcnt.put(key, longDeserializer.deserialize("", record.value())); + wcnt.put(key, longSerde.deserializer().deserialize("", record.value())); break; case "tagg": - tagg.put(key, longDeserializer.deserialize("", record.value())); + tagg.put(key, longSerde.deserializer().deserialize("", record.value())); break; default: System.out.println("unknown topic: " + record.topic()); diff --git a/streams/src/test/java/org/apache/kafka/streams/smoketest/SmokeTestUtil.java b/streams/src/test/java/org/apache/kafka/streams/smoketest/SmokeTestUtil.java index 3f5503f9481..c5ded5eb53b 100644 --- a/streams/src/test/java/org/apache/kafka/streams/smoketest/SmokeTestUtil.java +++ b/streams/src/test/java/org/apache/kafka/streams/smoketest/SmokeTestUtil.java @@ -17,15 +17,8 @@ package org.apache.kafka.streams.smoketest; -import org.apache.kafka.common.errors.SerializationException; -import org.apache.kafka.common.serialization.Deserializer; -import org.apache.kafka.common.serialization.IntegerDeserializer; -import org.apache.kafka.common.serialization.IntegerSerializer; -import org.apache.kafka.common.serialization.LongDeserializer; -import org.apache.kafka.common.serialization.LongSerializer; -import org.apache.kafka.common.serialization.Serializer; -import org.apache.kafka.common.serialization.StringDeserializer; -import org.apache.kafka.common.serialization.StringSerializer; +import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.kstream.Aggregator; import org.apache.kafka.streams.kstream.Initializer; @@ -36,7 +29,6 @@ import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.streams.processor.ProcessorSupplier; import java.io.File; -import java.util.Map; public class SmokeTestUtil { @@ -128,74 +120,13 @@ public class SmokeTestUtil { } } - public static Serializer stringSerializer = new StringSerializer(); + public static Serde stringSerde = Serdes.String(); - public static Deserializer stringDeserializer = new StringDeserializer(); + public static Serde intSerde = Serdes.Integer(); - public static Serializer integerSerializer = new IntegerSerializer(); + public static Serde longSerde = Serdes.Long(); - public static Deserializer integerDeserializer = new IntegerDeserializer(); - - public static Serializer longSerializer = new LongSerializer(); - - public static Deserializer longDeserializer = new LongDeserializer(); - - public static Serializer doubleSerializer = new Serializer() { - - @Override - public void configure(Map configs, boolean isKey) { - } - - @Override - public byte[] serialize(String topic, Double data) { - if (data == null) - return null; - - long bits = Double.doubleToLongBits(data); - return new byte[] { - (byte) (bits >>> 56), - (byte) (bits >>> 48), - (byte) (bits >>> 40), - (byte) (bits >>> 32), - (byte) (bits >>> 24), - (byte) (bits >>> 16), - (byte) (bits >>> 8), - (byte) bits - }; - } - - @Override - public void close() { - } - }; - - public static Deserializer doubleDeserializer = new Deserializer() { - - @Override - public void configure(Map configs, boolean isKey) { - } - - @Override - public Double deserialize(String topic, byte[] data) { - if (data == null) - return null; - if (data.length != 8) { - throw new SerializationException("Size of data received by Deserializer is " + - "not 8"); - } - - long value = 0; - for (byte b : data) { - value <<= 8; - value |= b & 0xFF; - } - return Double.longBitsToDouble(value); - } - - @Override - public void close() { - } - }; + public static Serde doubleSerde = Serdes.Double(); public static File createDir(String path) throws Exception { File dir = new File(path); diff --git a/streams/src/test/java/org/apache/kafka/streams/state/KeyValueStoreTestDriver.java b/streams/src/test/java/org/apache/kafka/streams/state/KeyValueStoreTestDriver.java index d8b034f8781..0468f491f9d 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/KeyValueStoreTestDriver.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/KeyValueStoreTestDriver.java @@ -22,6 +22,7 @@ import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.common.metrics.Sensor; import org.apache.kafka.common.serialization.ByteArraySerializer; import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.serialization.Serializer; import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.StreamsConfig; @@ -129,58 +130,6 @@ import java.util.Set; */ public class KeyValueStoreTestDriver { - private static Serializer unusableSerializer() { - return new Serializer() { - @Override - public void configure(Map configs, boolean isKey) { - } - - @Override - public byte[] serialize(String topic, T data) { - throw new UnsupportedOperationException("This serializer should not be used"); - } - - @Override - public void close() { - } - }; - }; - - private static Deserializer unusableDeserializer() { - return new Deserializer() { - @Override - public void configure(Map configs, boolean isKey) { - } - - @Override - public T deserialize(String topic, byte[] data) { - throw new UnsupportedOperationException("This deserializer should not be used"); - } - - @Override - public void close() { - } - }; - }; - - /** - * Create a driver object that will have a {@link #context()} that records messages - * {@link ProcessorContext#forward(Object, Object) forwarded} by the store and that provides unusable default key and - * value serializers and deserializers. This can be used when the actual serializers and deserializers are supplied to the - * store during creation, which should eliminate the need for a store to depend on the ProcessorContext's default key and - * value serializers and deserializers. - * - * @return the test driver; never null - */ - public static KeyValueStoreTestDriver create() { - Serializer keySerializer = unusableSerializer(); - Deserializer keyDeserializer = unusableDeserializer(); - Serializer valueSerializer = unusableSerializer(); - Deserializer valueDeserializer = unusableDeserializer(); - Serdes serdes = new Serdes("unexpected", keySerializer, keyDeserializer, valueSerializer, valueDeserializer); - return new KeyValueStoreTestDriver(serdes); - } - /** * Create a driver object that will have a {@link #context()} that records messages * {@link ProcessorContext#forward(Object, Object) forwarded} by the store and that provides default serializers and @@ -195,7 +144,7 @@ public class KeyValueStoreTestDriver { * @return the test driver; never null */ public static KeyValueStoreTestDriver create(Class keyClass, Class valueClass) { - Serdes serdes = Serdes.withBuiltinTypes("unexpected", keyClass, valueClass); + StateSerdes serdes = StateSerdes.withBuiltinTypes("unexpected", keyClass, valueClass); return new KeyValueStoreTestDriver(serdes); } @@ -215,7 +164,9 @@ public class KeyValueStoreTestDriver { Deserializer keyDeserializer, Serializer valueSerializer, Deserializer valueDeserializer) { - Serdes serdes = new Serdes("unexpected", keySerializer, keyDeserializer, valueSerializer, valueDeserializer); + StateSerdes serdes = new StateSerdes("unexpected", + Serdes.serdeFrom(keySerializer, keyDeserializer), + Serdes.serdeFrom(valueSerializer, valueDeserializer)); return new KeyValueStoreTestDriver(serdes); } @@ -237,7 +188,7 @@ public class KeyValueStoreTestDriver { private final RecordCollector recordCollector; private File stateDir = null; - protected KeyValueStoreTestDriver(final Serdes serdes) { + protected KeyValueStoreTestDriver(final StateSerdes serdes) { ByteArraySerializer rawSerializer = new ByteArraySerializer(); Producer producer = new MockProducer<>(true, rawSerializer, rawSerializer); @@ -276,13 +227,10 @@ public class KeyValueStoreTestDriver { Properties props = new Properties(); props.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); props.put(StreamsConfig.TIMESTAMP_EXTRACTOR_CLASS_CONFIG, MockTimestampExtractor.class); - props.put(StreamsConfig.KEY_SERIALIZER_CLASS_CONFIG, serdes.keySerializer().getClass()); - props.put(StreamsConfig.KEY_DESERIALIZER_CLASS_CONFIG, serdes.keyDeserializer().getClass()); - props.put(StreamsConfig.VALUE_SERIALIZER_CLASS_CONFIG, serdes.valueSerializer().getClass()); - props.put(StreamsConfig.VALUE_DESERIALIZER_CLASS_CONFIG, serdes.valueDeserializer().getClass()); + props.put(StreamsConfig.KEY_SERDE_CLASS_CONFIG, serdes.keySerde().getClass()); + props.put(StreamsConfig.VALUE_SERDE_CLASS_CONFIG, serdes.valueSerde().getClass()); - this.context = new MockProcessorContext(null, this.stateDir, serdes.keySerializer(), serdes.keyDeserializer(), serdes.valueSerializer(), - serdes.valueDeserializer(), recordCollector) { + this.context = new MockProcessorContext(null, this.stateDir, serdes.keySerde(), serdes.valueSerde(), recordCollector) { @Override public TaskId taskId() { return new TaskId(0, 1); @@ -328,7 +276,7 @@ public class KeyValueStoreTestDriver { } } - private void restoreEntries(StateRestoreCallback func, Serdes serdes) { + private void restoreEntries(StateRestoreCallback func, StateSerdes serdes) { for (KeyValue entry : restorableEntries) { if (entry != null) { byte[] rawKey = serdes.rawKey(entry.key); diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/InMemoryKeyValueStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/InMemoryKeyValueStoreTest.java index 46948bd1e98..b44583d8c5c 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/InMemoryKeyValueStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/InMemoryKeyValueStoreTest.java @@ -16,8 +16,6 @@ */ package org.apache.kafka.streams.state.internals; -import org.apache.kafka.common.serialization.Deserializer; -import org.apache.kafka.common.serialization.Serializer; import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.streams.processor.StateStoreSupplier; import org.apache.kafka.streams.state.KeyValueStore; @@ -34,11 +32,7 @@ public class InMemoryKeyValueStoreTest extends AbstractKeyValueStoreTest { StateStoreSupplier supplier; if (useContextSerdes) { - Serializer keySer = (Serializer) context.keySerializer(); - Deserializer keyDeser = (Deserializer) context.keyDeserializer(); - Serializer valSer = (Serializer) context.valueSerializer(); - Deserializer valDeser = (Deserializer) context.valueDeserializer(); - supplier = Stores.create("my-store").withKeys(keySer, keyDeser).withValues(valSer, valDeser).inMemory().build(); + supplier = Stores.create("my-store").withKeys(context.keySerde()).withValues(context.valueSerde()).inMemory().build(); } else { supplier = Stores.create("my-store").withKeys(keyClass).withValues(valueClass).inMemory().build(); } diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/InMemoryLRUCacheStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/InMemoryLRUCacheStoreTest.java index a2b79e5f92f..c301223a935 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/InMemoryLRUCacheStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/InMemoryLRUCacheStoreTest.java @@ -16,8 +16,6 @@ */ package org.apache.kafka.streams.state.internals; -import org.apache.kafka.common.serialization.Deserializer; -import org.apache.kafka.common.serialization.Serializer; import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.streams.processor.StateStoreSupplier; import org.apache.kafka.streams.state.KeyValueStore; @@ -40,11 +38,7 @@ public class InMemoryLRUCacheStoreTest extends AbstractKeyValueStoreTest { StateStoreSupplier supplier; if (useContextSerdes) { - Serializer keySer = (Serializer) context.keySerializer(); - Deserializer keyDeser = (Deserializer) context.keyDeserializer(); - Serializer valSer = (Serializer) context.valueSerializer(); - Deserializer valDeser = (Deserializer) context.valueDeserializer(); - supplier = Stores.create("my-store").withKeys(keySer, keyDeser).withValues(valSer, valDeser).inMemory().maxEntries(10).build(); + supplier = Stores.create("my-store").withKeys(context.keySerde()).withValues(context.valueSerde()).inMemory().maxEntries(10).build(); } else { supplier = Stores.create("my-store").withKeys(keyClass).withValues(valueClass).inMemory().maxEntries(10).build(); } diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBKeyValueStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBKeyValueStoreTest.java index 8e8f69c4265..280255a58da 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBKeyValueStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBKeyValueStoreTest.java @@ -16,8 +16,6 @@ */ package org.apache.kafka.streams.state.internals; -import org.apache.kafka.common.serialization.Deserializer; -import org.apache.kafka.common.serialization.Serializer; import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.streams.processor.StateStoreSupplier; import org.apache.kafka.streams.state.KeyValueStore; @@ -35,11 +33,7 @@ public class RocksDBKeyValueStoreTest extends AbstractKeyValueStoreTest { StateStoreSupplier supplier; if (useContextSerdes) { - Serializer keySer = (Serializer) context.keySerializer(); - Deserializer keyDeser = (Deserializer) context.keyDeserializer(); - Serializer valSer = (Serializer) context.valueSerializer(); - Deserializer valDeser = (Deserializer) context.valueDeserializer(); - supplier = Stores.create("my-store").withKeys(keySer, keyDeser).withValues(valSer, valDeser).persistent().build(); + supplier = Stores.create("my-store").withKeys(context.keySerde()).withValues(context.valueSerde()).persistent().build(); } else { supplier = Stores.create("my-store").withKeys(keyClass).withValues(valueClass).persistent().build(); } diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBWindowStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBWindowStoreTest.java index 5a196ece84d..ffc97c3e7a4 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBWindowStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBWindowStoreTest.java @@ -20,15 +20,15 @@ package org.apache.kafka.streams.state.internals; import org.apache.kafka.clients.producer.MockProducer; import org.apache.kafka.clients.producer.Producer; import org.apache.kafka.clients.producer.ProducerRecord; -import org.apache.kafka.common.serialization.ByteArrayDeserializer; -import org.apache.kafka.common.serialization.ByteArraySerializer; +import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.serialization.Serializer; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.streams.processor.StateStoreSupplier; import org.apache.kafka.streams.processor.internals.RecordCollector; -import org.apache.kafka.streams.state.Serdes; +import org.apache.kafka.streams.state.StateSerdes; import org.apache.kafka.streams.state.WindowStore; import org.apache.kafka.streams.state.WindowStoreIterator; import org.apache.kafka.streams.state.WindowStoreUtils; @@ -51,17 +51,16 @@ import static org.junit.Assert.assertNull; public class RocksDBWindowStoreTest { - private final ByteArraySerializer byteArraySerializer = new ByteArraySerializer(); - private final ByteArrayDeserializer byteArrayDeserializer = new ByteArrayDeserializer(); + private final Serde byteArraySerde = Serdes.ByteArray(); private final String windowName = "window"; private final int numSegments = 3; private final long segmentSize = RocksDBWindowStore.MIN_SEGMENT_INTERVAL; private final long retentionPeriod = segmentSize * (numSegments - 1); private final long windowSize = 3; - private final Serdes serdes = Serdes.withBuiltinTypes("", Integer.class, String.class); + private final StateSerdes serdes = StateSerdes.withBuiltinTypes("", Integer.class, String.class); @SuppressWarnings("unchecked") - protected WindowStore createWindowStore(ProcessorContext context, Serdes serdes) { + protected WindowStore createWindowStore(ProcessorContext context, StateSerdes serdes) { StateStoreSupplier supplier = new RocksDBWindowStoreSupplier<>(windowName, retentionPeriod, numSegments, true, serdes, null); WindowStore store = (WindowStore) supplier.get(); @@ -74,7 +73,7 @@ public class RocksDBWindowStoreTest { File baseDir = Files.createTempDirectory("test").toFile(); try { final List> changeLog = new ArrayList<>(); - Producer producer = new MockProducer<>(true, byteArraySerializer, byteArraySerializer); + Producer producer = new MockProducer<>(true, byteArraySerde.serializer(), byteArraySerde.serializer()); RecordCollector recordCollector = new RecordCollector(producer) { @Override public void send(ProducerRecord record, Serializer keySerializer, Serializer valueSerializer) { @@ -87,7 +86,7 @@ public class RocksDBWindowStoreTest { MockProcessorContext context = new MockProcessorContext( null, baseDir, - byteArraySerializer, byteArrayDeserializer, byteArraySerializer, byteArrayDeserializer, + byteArraySerde, byteArraySerde, recordCollector); WindowStore store = createWindowStore(context, serdes); @@ -170,7 +169,7 @@ public class RocksDBWindowStoreTest { File baseDir = Files.createTempDirectory("test").toFile(); try { final List> changeLog = new ArrayList<>(); - Producer producer = new MockProducer<>(true, byteArraySerializer, byteArraySerializer); + Producer producer = new MockProducer<>(true, byteArraySerde.serializer(), byteArraySerde.serializer()); RecordCollector recordCollector = new RecordCollector(producer) { @Override public void send(ProducerRecord record, Serializer keySerializer, Serializer valueSerializer) { @@ -183,7 +182,7 @@ public class RocksDBWindowStoreTest { MockProcessorContext context = new MockProcessorContext( null, baseDir, - byteArraySerializer, byteArrayDeserializer, byteArraySerializer, byteArrayDeserializer, + byteArraySerde, byteArraySerde, recordCollector); WindowStore store = createWindowStore(context, serdes); @@ -266,7 +265,7 @@ public class RocksDBWindowStoreTest { File baseDir = Files.createTempDirectory("test").toFile(); try { final List> changeLog = new ArrayList<>(); - Producer producer = new MockProducer<>(true, byteArraySerializer, byteArraySerializer); + Producer producer = new MockProducer<>(true, byteArraySerde.serializer(), byteArraySerde.serializer()); RecordCollector recordCollector = new RecordCollector(producer) { @Override public void send(ProducerRecord record, Serializer keySerializer, Serializer valueSerializer) { @@ -279,7 +278,7 @@ public class RocksDBWindowStoreTest { MockProcessorContext context = new MockProcessorContext( null, baseDir, - byteArraySerializer, byteArrayDeserializer, byteArraySerializer, byteArrayDeserializer, + byteArraySerde, byteArraySerde, recordCollector); WindowStore store = createWindowStore(context, serdes); @@ -362,7 +361,7 @@ public class RocksDBWindowStoreTest { File baseDir = Files.createTempDirectory("test").toFile(); try { final List> changeLog = new ArrayList<>(); - Producer producer = new MockProducer<>(true, byteArraySerializer, byteArraySerializer); + Producer producer = new MockProducer<>(true, byteArraySerde.serializer(), byteArraySerde.serializer()); RecordCollector recordCollector = new RecordCollector(producer) { @Override public void send(ProducerRecord record, Serializer keySerializer, Serializer valueSerializer) { @@ -375,7 +374,7 @@ public class RocksDBWindowStoreTest { MockProcessorContext context = new MockProcessorContext( null, baseDir, - byteArraySerializer, byteArrayDeserializer, byteArraySerializer, byteArrayDeserializer, + byteArraySerde, byteArraySerde, recordCollector); WindowStore store = createWindowStore(context, serdes); @@ -421,7 +420,7 @@ public class RocksDBWindowStoreTest { File baseDir = Files.createTempDirectory("test").toFile(); try { final List> changeLog = new ArrayList<>(); - Producer producer = new MockProducer<>(true, byteArraySerializer, byteArraySerializer); + Producer producer = new MockProducer<>(true, byteArraySerde.serializer(), byteArraySerde.serializer()); RecordCollector recordCollector = new RecordCollector(producer) { @Override public void send(ProducerRecord record, Serializer keySerializer, Serializer valueSerializer) { @@ -434,7 +433,7 @@ public class RocksDBWindowStoreTest { MockProcessorContext context = new MockProcessorContext( null, baseDir, - byteArraySerializer, byteArrayDeserializer, byteArraySerializer, byteArrayDeserializer, + byteArraySerde, byteArraySerde, recordCollector); WindowStore store = createWindowStore(context, serdes); @@ -538,7 +537,7 @@ public class RocksDBWindowStoreTest { File baseDir = Files.createTempDirectory("test").toFile(); try { - Producer producer = new MockProducer<>(true, byteArraySerializer, byteArraySerializer); + Producer producer = new MockProducer<>(true, byteArraySerde.serializer(), byteArraySerde.serializer()); RecordCollector recordCollector = new RecordCollector(producer) { @Override public void send(ProducerRecord record, Serializer keySerializer, Serializer valueSerializer) { @@ -551,7 +550,7 @@ public class RocksDBWindowStoreTest { MockProcessorContext context = new MockProcessorContext( null, baseDir, - byteArraySerializer, byteArrayDeserializer, byteArraySerializer, byteArrayDeserializer, + byteArraySerde, byteArraySerde, recordCollector); WindowStore store = createWindowStore(context, serdes); @@ -587,7 +586,7 @@ public class RocksDBWindowStoreTest { File baseDir2 = Files.createTempDirectory("test").toFile(); try { - Producer producer = new MockProducer<>(true, byteArraySerializer, byteArraySerializer); + Producer producer = new MockProducer<>(true, byteArraySerde.serializer(), byteArraySerde.serializer()); RecordCollector recordCollector = new RecordCollector(producer) { @Override public void send(ProducerRecord record, Serializer keySerializer, Serializer valueSerializer) { @@ -600,7 +599,7 @@ public class RocksDBWindowStoreTest { MockProcessorContext context = new MockProcessorContext( null, baseDir, - byteArraySerializer, byteArrayDeserializer, byteArraySerializer, byteArrayDeserializer, + byteArraySerde, byteArraySerde, recordCollector); WindowStore store = createWindowStore(context, serdes); @@ -642,7 +641,7 @@ public class RocksDBWindowStoreTest { public void testSegmentMaintenance() throws IOException { File baseDir = Files.createTempDirectory("test").toFile(); try { - Producer producer = new MockProducer<>(true, byteArraySerializer, byteArraySerializer); + Producer producer = new MockProducer<>(true, byteArraySerde.serializer(), byteArraySerde.serializer()); RecordCollector recordCollector = new RecordCollector(producer) { @Override public void send(ProducerRecord record, Serializer keySerializer, Serializer valueSerializer) { @@ -652,7 +651,7 @@ public class RocksDBWindowStoreTest { MockProcessorContext context = new MockProcessorContext( null, baseDir, - byteArraySerializer, byteArrayDeserializer, byteArraySerializer, byteArrayDeserializer, + byteArraySerde, byteArraySerde, recordCollector); WindowStore store = createWindowStore(context, serdes); @@ -745,7 +744,7 @@ public class RocksDBWindowStoreTest { public void testInitialLoading() throws IOException { File baseDir = Files.createTempDirectory("test").toFile(); try { - Producer producer = new MockProducer<>(true, byteArraySerializer, byteArraySerializer); + Producer producer = new MockProducer<>(true, byteArraySerde.serializer(), byteArraySerde.serializer()); RecordCollector recordCollector = new RecordCollector(producer) { @Override public void send(ProducerRecord record, Serializer keySerializer, Serializer valueSerializer) { @@ -755,7 +754,7 @@ public class RocksDBWindowStoreTest { MockProcessorContext context = new MockProcessorContext( null, baseDir, - byteArraySerializer, byteArrayDeserializer, byteArraySerializer, byteArrayDeserializer, + byteArraySerde, byteArraySerde, recordCollector); File storeDir = new File(baseDir, windowName); diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/StoreChangeLoggerTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/StoreChangeLoggerTest.java index 5f014ef5139..9a477dfc10a 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/StoreChangeLoggerTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/StoreChangeLoggerTest.java @@ -31,7 +31,7 @@ import org.apache.kafka.common.serialization.StringSerializer; import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.streams.processor.StreamPartitioner; import org.apache.kafka.streams.processor.internals.RecordCollector; -import org.apache.kafka.streams.state.Serdes; +import org.apache.kafka.streams.state.StateSerdes; import org.apache.kafka.test.MockProcessorContext; import org.junit.Test; @@ -44,7 +44,7 @@ public class StoreChangeLoggerTest { private final Map logged = new HashMap<>(); private final Map written = new HashMap<>(); - private final ProcessorContext context = new MockProcessorContext(Serdes.withBuiltinTypes(topic, Integer.class, String.class), + private final ProcessorContext context = new MockProcessorContext(StateSerdes.withBuiltinTypes(topic, Integer.class, String.class), new RecordCollector(null) { @SuppressWarnings("unchecked") @Override @@ -61,7 +61,7 @@ public class StoreChangeLoggerTest { } ); - private final StoreChangeLogger changeLogger = new StoreChangeLogger<>(topic, context, Serdes.withBuiltinTypes(topic, Integer.class, String.class), 3, 3); + private final StoreChangeLogger changeLogger = new StoreChangeLogger<>(topic, context, StateSerdes.withBuiltinTypes(topic, Integer.class, String.class), 3, 3); private final StoreChangeLogger rawChangeLogger = new RawStoreChangeLogger(topic, context, 3, 3); diff --git a/streams/src/test/java/org/apache/kafka/test/KStreamTestDriver.java b/streams/src/test/java/org/apache/kafka/test/KStreamTestDriver.java index edbcb4a9faa..05713c19c3e 100644 --- a/streams/src/test/java/org/apache/kafka/test/KStreamTestDriver.java +++ b/streams/src/test/java/org/apache/kafka/test/KStreamTestDriver.java @@ -18,7 +18,8 @@ package org.apache.kafka.test; import org.apache.kafka.clients.producer.ProducerRecord; -import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.serialization.Serializer; import org.apache.kafka.streams.kstream.KStreamBuilder; import org.apache.kafka.streams.processor.ProcessorContext; @@ -42,20 +43,20 @@ public class KStreamTestDriver { private ProcessorNode currNode; public KStreamTestDriver(KStreamBuilder builder) { - this(builder, null, null, null, null, null); + this(builder, null, Serdes.ByteArray(), Serdes.ByteArray()); } public KStreamTestDriver(KStreamBuilder builder, File stateDir) { - this(builder, stateDir, null, null, null, null); + this(builder, stateDir, Serdes.ByteArray(), Serdes.ByteArray()); } public KStreamTestDriver(KStreamBuilder builder, File stateDir, - Serializer keySerializer, Deserializer keyDeserializer, - Serializer valSerializer, Deserializer valDeserializer) { + Serde keySerde, + Serde valSerde) { this.topology = builder.build("X", null); this.stateDir = stateDir; - this.context = new MockProcessorContext(this, stateDir, keySerializer, keyDeserializer, valSerializer, valDeserializer, new MockRecordCollector()); + this.context = new MockProcessorContext(this, stateDir, keySerde, valSerde, new MockRecordCollector()); for (StateStoreSupplier stateStoreSupplier : topology.stateStoreSuppliers()) { StateStore store = stateStoreSupplier.get(); diff --git a/streams/src/test/java/org/apache/kafka/test/MockProcessorContext.java b/streams/src/test/java/org/apache/kafka/test/MockProcessorContext.java index b463669595b..e57e1c7fabc 100644 --- a/streams/src/test/java/org/apache/kafka/test/MockProcessorContext.java +++ b/streams/src/test/java/org/apache/kafka/test/MockProcessorContext.java @@ -18,16 +18,15 @@ package org.apache.kafka.test; import org.apache.kafka.common.metrics.Sensor; +import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.StreamsMetrics; import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.streams.processor.StateRestoreCallback; import org.apache.kafka.streams.processor.StateStore; -import org.apache.kafka.common.serialization.Deserializer; -import org.apache.kafka.common.serialization.Serializer; import org.apache.kafka.streams.processor.TaskId; import org.apache.kafka.streams.processor.internals.RecordCollector; -import org.apache.kafka.streams.state.Serdes; +import org.apache.kafka.streams.state.StateSerdes; import java.io.File; import java.util.Collections; @@ -38,10 +37,8 @@ import java.util.Map; public class MockProcessorContext implements ProcessorContext, RecordCollector.Supplier { private final KStreamTestDriver driver; - private final Serializer keySerializer; - private final Serializer valueSerializer; - private final Deserializer keyDeserializer; - private final Deserializer valueDeserializer; + private final Serde keySerde; + private final Serde valSerde; private final RecordCollector.Supplier recordCollectorSupplier; private final File stateDir; @@ -50,21 +47,15 @@ public class MockProcessorContext implements ProcessorContext, RecordCollector.S long timestamp = -1L; - public MockProcessorContext(Serdes serdes, RecordCollector collector) { - this(null, null, serdes.keySerializer(), serdes.keyDeserializer(), serdes.valueSerializer(), serdes.valueDeserializer(), collector); - } - - public MockProcessorContext(Serializer keySerializer, Deserializer keyDeserializer, - Serializer valueSerializer, Deserializer valueDeserializer, - RecordCollector collector) { - this(null, null, keySerializer, keyDeserializer, valueSerializer, valueDeserializer, collector); + public MockProcessorContext(StateSerdes serdes, RecordCollector collector) { + this(null, null, serdes.keySerde(), serdes.valueSerde(), collector); } public MockProcessorContext(KStreamTestDriver driver, File stateDir, - Serializer keySerializer, Deserializer keyDeserializer, - Serializer valueSerializer, Deserializer valueDeserializer, + Serde keySerde, + Serde valSerde, final RecordCollector collector) { - this(driver, stateDir, keySerializer, keyDeserializer, valueSerializer, valueDeserializer, + this(driver, stateDir, keySerde, valSerde, new RecordCollector.Supplier() { @Override public RecordCollector recordCollector() { @@ -74,15 +65,13 @@ public class MockProcessorContext implements ProcessorContext, RecordCollector.S } public MockProcessorContext(KStreamTestDriver driver, File stateDir, - Serializer keySerializer, Deserializer keyDeserializer, - Serializer valueSerializer, Deserializer valueDeserializer, + Serde keySerde, + Serde valSerde, RecordCollector.Supplier collectorSupplier) { this.driver = driver; this.stateDir = stateDir; - this.keySerializer = keySerializer; - this.valueSerializer = valueSerializer; - this.keyDeserializer = keyDeserializer; - this.valueDeserializer = valueDeserializer; + this.keySerde = keySerde; + this.valSerde = valSerde; this.recordCollectorSupplier = collectorSupplier; } @@ -111,23 +100,13 @@ public class MockProcessorContext implements ProcessorContext, RecordCollector.S } @Override - public Serializer keySerializer() { - return keySerializer; + public Serde keySerde() { + return this.keySerde; } @Override - public Serializer valueSerializer() { - return valueSerializer; - } - - @Override - public Deserializer keyDeserializer() { - return keyDeserializer; - } - - @Override - public Deserializer valueDeserializer() { - return valueDeserializer; + public Serde valueSerde() { + return this.valSerde; } @Override