mirror of https://github.com/apache/kafka.git
KAFKA-3336: Unify Serializer and Deserializer into Serialization
Author: Guozhang Wang <wangguoz@gmail.com> Reviewers: Michael G. Noll, Ismael Juma Closes #1066 from guozhangwang/K3336
This commit is contained in:
parent
f57dabbe56
commit
dea0719e99
|
|
@ -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<Double> {
|
||||
|
||||
@Override
|
||||
public void configure(Map<String, ?> 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
|
||||
}
|
||||
}
|
||||
|
|
@ -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<Double> {
|
||||
|
||||
@Override
|
||||
public void configure(Map<String, ?> 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
|
||||
}
|
||||
}
|
||||
|
|
@ -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 <T>
|
||||
*/
|
||||
public interface Serde<T> {
|
||||
|
||||
Serializer<T> serializer();
|
||||
|
||||
Deserializer<T> deserializer();
|
||||
}
|
||||
|
|
@ -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<Long> {
|
||||
@Override
|
||||
public Serializer<Long> serializer() {
|
||||
return new LongSerializer();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Deserializer<Long> deserializer() {
|
||||
return new LongDeserializer();
|
||||
}
|
||||
}
|
||||
|
||||
static public final class IntegerSerde implements Serde<Integer> {
|
||||
@Override
|
||||
public Serializer<Integer> serializer() {
|
||||
return new IntegerSerializer();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Deserializer<Integer> deserializer() {
|
||||
return new IntegerDeserializer();
|
||||
}
|
||||
}
|
||||
|
||||
static public final class DoubleSerde implements Serde<Double> {
|
||||
@Override
|
||||
public Serializer<Double> serializer() {
|
||||
return new DoubleSerializer();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Deserializer<Double> deserializer() {
|
||||
return new DoubleDeserializer();
|
||||
}
|
||||
}
|
||||
|
||||
static public final class StringSerde implements Serde<String> {
|
||||
@Override
|
||||
public Serializer<String> serializer() {
|
||||
return new StringSerializer();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Deserializer<String> deserializer() {
|
||||
return new StringDeserializer();
|
||||
}
|
||||
}
|
||||
|
||||
static public final class ByteBufferSerde implements Serde<ByteBuffer> {
|
||||
@Override
|
||||
public Serializer<ByteBuffer> serializer() {
|
||||
return new ByteBufferSerializer();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Deserializer<ByteBuffer> deserializer() {
|
||||
return new ByteBufferDeserializer();
|
||||
}
|
||||
}
|
||||
|
||||
static public final class ByteArraySerde implements Serde<byte[]> {
|
||||
@Override
|
||||
public Serializer<byte[]> serializer() {
|
||||
return new ByteArraySerializer();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Deserializer<byte[]> deserializer() {
|
||||
return new ByteArrayDeserializer();
|
||||
}
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
static public <T> Serde<T> serdeFrom(Class<T> type) {
|
||||
if (String.class.isAssignableFrom(type)) {
|
||||
return (Serde<T>) String();
|
||||
}
|
||||
|
||||
if (Integer.class.isAssignableFrom(type)) {
|
||||
return (Serde<T>) Integer();
|
||||
}
|
||||
|
||||
if (Long.class.isAssignableFrom(type)) {
|
||||
return (Serde<T>) Long();
|
||||
}
|
||||
|
||||
if (Double.class.isAssignableFrom(type)) {
|
||||
return (Serde<T>) Double();
|
||||
}
|
||||
|
||||
if (byte[].class.isAssignableFrom(type)) {
|
||||
return (Serde<T>) ByteArray();
|
||||
}
|
||||
|
||||
if (ByteBufferSerde.class.isAssignableFrom(type)) {
|
||||
return (Serde<T>) 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 <T> Serde<T> serdeFrom(final Serializer<T> serializer, final Deserializer<T> 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<T>() {
|
||||
@Override
|
||||
public Serializer<T> serializer() {
|
||||
return serializer;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Deserializer<T> deserializer() {
|
||||
return deserializer;
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
/*
|
||||
* A serde for nullable long type.
|
||||
*/
|
||||
static public Serde<Long> Long() {
|
||||
return new LongSerde();
|
||||
}
|
||||
|
||||
/*
|
||||
* A serde for nullable int type.
|
||||
*/
|
||||
static public Serde<Integer> Integer() {
|
||||
return new IntegerSerde();
|
||||
}
|
||||
|
||||
/*
|
||||
* A serde for nullable long type.
|
||||
*/
|
||||
static public Serde<Double> Double() {
|
||||
return new DoubleSerde();
|
||||
}
|
||||
|
||||
/*
|
||||
* A serde for nullable string type.
|
||||
*/
|
||||
static public Serde<String> String() {
|
||||
return new StringSerde();
|
||||
}
|
||||
|
||||
/*
|
||||
* A serde for nullable byte array type.
|
||||
*/
|
||||
static public Serde<ByteBuffer> ByteBuffer() {
|
||||
return new ByteBufferSerde();
|
||||
}
|
||||
|
||||
/*
|
||||
* A serde for nullable byte array type.
|
||||
*/
|
||||
static public Serde<byte[]> ByteArray() {
|
||||
return new ByteArraySerde();
|
||||
}
|
||||
}
|
||||
|
|
@ -24,34 +24,53 @@ import static org.junit.Assert.assertEquals;
|
|||
|
||||
public class SerializationTest {
|
||||
|
||||
private static class SerDeser<T> {
|
||||
final Serializer<T> serializer;
|
||||
final Deserializer<T> deserializer;
|
||||
final private String topic = "testTopic";
|
||||
|
||||
public SerDeser(Serializer<T> serializer, Deserializer<T> deserializer) {
|
||||
this.serializer = serializer;
|
||||
this.deserializer = deserializer;
|
||||
}
|
||||
private class DummyClass {
|
||||
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSerdeFrom() {
|
||||
Serde<Long> thisSerde = Serdes.serdeFrom(Long.class);
|
||||
Serde<Long> 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<String> encodings = new ArrayList<String>();
|
||||
encodings.add("UTF8");
|
||||
encodings.add("UTF-16");
|
||||
|
||||
for (String encoding : encodings) {
|
||||
SerDeser<String> serDeser = getStringSerDeser(encoding);
|
||||
Serializer<String> serializer = serDeser.serializer;
|
||||
Deserializer<String> deserializer = serDeser.deserializer;
|
||||
Serde<String> serDeser = getStringSerde(encoding);
|
||||
Serializer<String> serializer = serDeser.serializer();
|
||||
Deserializer<String> 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<Integer> serializer = new IntegerSerializer();
|
||||
Deserializer<Integer> deserializer = new IntegerDeserializer();
|
||||
Serializer<Integer> serializer = Serdes.Integer().serializer();
|
||||
Deserializer<Integer> 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<Long> serializer = Serdes.Long().serializer();
|
||||
Deserializer<Long> 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<Double> serializer = Serdes.Double().serializer();
|
||||
Deserializer<Double> 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<ByteBuffer> serializer = new ByteBufferSerializer();
|
||||
Deserializer<ByteBuffer> deserializer = new ByteBufferDeserializer();
|
||||
Serializer<ByteBuffer> serializer = Serdes.ByteBuffer().serializer();
|
||||
Deserializer<ByteBuffer> 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<String> getStringSerDeser(String encoder) {
|
||||
private Serde<String> getStringSerde(String encoder) {
|
||||
Map<String, Object> serializerConfigs = new HashMap<String, Object>();
|
||||
serializerConfigs.put("key.serializer.encoding", encoder);
|
||||
Serializer<String> serializer = new StringSerializer();
|
||||
Serializer<String> serializer = Serdes.String().serializer();
|
||||
serializer.configure(serializerConfigs, true);
|
||||
|
||||
Map<String, Object> deserializerConfigs = new HashMap<String, Object>();
|
||||
deserializerConfigs.put("key.deserializer.encoding", encoder);
|
||||
Deserializer<String> deserializer = new StringDeserializer();
|
||||
Deserializer<String> deserializer = Serdes.String().deserializer();
|
||||
deserializer.configure(deserializerConfigs, true);
|
||||
|
||||
return new SerDeser<String>(serializer, deserializer);
|
||||
return Serdes.serdeFrom(serializer, deserializer);
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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<String> stringSerializer = new StringSerializer();
|
||||
final Deserializer<String> stringDeserializer = new StringDeserializer();
|
||||
|
||||
// TODO: the following can be removed with a serialization factory
|
||||
Map<String, Object> serdeProps = new HashMap<>();
|
||||
|
||||
final Serializer<PageView> pageViewSerializer = new JsonPOJOSerializer<>();
|
||||
serdeProps.put("JsonPOJOClass", PageView.class);
|
||||
pageViewSerializer.configure(serdeProps, false);
|
||||
|
||||
final Deserializer<PageView> pageViewDeserializer = new JsonPOJODeserializer<>();
|
||||
serdeProps.put("JsonPOJOClass", PageView.class);
|
||||
pageViewDeserializer.configure(serdeProps, false);
|
||||
|
||||
final Deserializer<UserProfile> userProfileDeserializer = new JsonPOJODeserializer<>();
|
||||
serdeProps.put("JsonPOJOClass", UserProfile.class);
|
||||
userProfileDeserializer.configure(serdeProps, false);
|
||||
|
||||
final Serializer<UserProfile> userProfileSerializer = new JsonPOJOSerializer<>();
|
||||
serdeProps.put("JsonPOJOClass", UserProfile.class);
|
||||
userProfileSerializer.configure(serdeProps, false);
|
||||
|
||||
final Deserializer<UserProfile> userProfileDeserializer = new JsonPOJODeserializer<>();
|
||||
serdeProps.put("JsonPOJOClass", UserProfile.class);
|
||||
userProfileDeserializer.configure(serdeProps, false);
|
||||
|
||||
final Serializer<WindowedPageViewByRegion> wPageViewByRegionSerializer = new JsonPOJOSerializer<>();
|
||||
serdeProps.put("JsonPOJOClass", WindowedPageViewByRegion.class);
|
||||
wPageViewByRegionSerializer.configure(serdeProps, false);
|
||||
|
||||
final Deserializer<WindowedPageViewByRegion> wPageViewByRegionDeserializer = new JsonPOJODeserializer<>();
|
||||
serdeProps.put("JsonPOJOClass", WindowedPageViewByRegion.class);
|
||||
wPageViewByRegionDeserializer.configure(serdeProps, false);
|
||||
|
||||
final Serializer<RegionCount> regionCountSerializer = new JsonPOJOSerializer<>();
|
||||
serdeProps.put("JsonPOJOClass", RegionCount.class);
|
||||
regionCountSerializer.configure(serdeProps, false);
|
||||
|
||||
KStream<String, PageView> views = builder.stream(stringDeserializer, pageViewDeserializer, "streams-pageview-input");
|
||||
final Deserializer<RegionCount> regionCountDeserializer = new JsonPOJODeserializer<>();
|
||||
serdeProps.put("JsonPOJOClass", RegionCount.class);
|
||||
regionCountDeserializer.configure(serdeProps, false);
|
||||
|
||||
KTable<String, UserProfile> users = builder.table(stringSerializer, userProfileSerializer, stringDeserializer, userProfileDeserializer, "streams-userprofile-input");
|
||||
KStream<String, PageView> views = builder.stream(Serdes.String(), Serdes.serdeFrom(pageViewSerializer, pageViewDeserializer), "streams-pageview-input");
|
||||
|
||||
KTable<String, UserProfile> users = builder.table(Serdes.String(), Serdes.serdeFrom(userProfileSerializer, userProfileDeserializer), "streams-userprofile-input");
|
||||
|
||||
KStream<WindowedPageViewByRegion, RegionCount> regionCount = views
|
||||
.leftJoin(users, new ValueJoiner<PageView, UserProfile, PageViewByRegion>() {
|
||||
|
|
@ -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<Windowed<String>, Long, KeyValue<WindowedPageViewByRegion, RegionCount>>() {
|
||||
|
|
@ -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();
|
||||
|
|
|
|||
|
|
@ -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<String> stringSerializer = new StringSerializer();
|
||||
final Deserializer<String> stringDeserializer = new StringDeserializer();
|
||||
final Serializer<JsonNode> jsonSerializer = new JsonSerializer();
|
||||
final Deserializer<JsonNode> jsonDeserializer = new JsonDeserializer();
|
||||
final Serde<JsonNode> jsonSerde = Serdes.serdeFrom(jsonSerializer, jsonDeserializer);
|
||||
|
||||
KStream<String, JsonNode> views = builder.stream(stringDeserializer, jsonDeserializer, "streams-pageview-input");
|
||||
KStream<String, JsonNode> views = builder.stream(Serdes.String(), jsonSerde, "streams-pageview-input");
|
||||
|
||||
KTable<String, JsonNode> users = builder.table(stringSerializer, jsonSerializer, stringDeserializer, jsonDeserializer, "streams-userprofile-input");
|
||||
KTable<String, JsonNode> users = builder.table(Serdes.String(), jsonSerde, "streams-userprofile-input");
|
||||
|
||||
KTable<String, String> userRegions = users.mapValues(new ValueMapper<JsonNode, String>() {
|
||||
@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<Windowed<String>, Long, KeyValue<JsonNode, JsonNode>>() {
|
||||
|
|
@ -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();
|
||||
|
|
|
|||
|
|
@ -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");
|
||||
|
|
|
|||
|
|
@ -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<String> stringSerializer = new StringSerializer();
|
||||
final Deserializer<String> stringDeserializer = new StringDeserializer();
|
||||
final Serializer<Long> longSerializer = new LongSerializer();
|
||||
|
||||
KStream<String, String> source = builder.stream("streams-file-input");
|
||||
KStream<String, String> source = builder.stream(Serdes.String(), Serdes.String(), "streams-file-input");
|
||||
|
||||
KTable<String, Long> counts = source
|
||||
.flatMapValues(new ValueMapper<String, Iterable<String>>() {
|
||||
|
|
@ -80,9 +68,9 @@ public class WordCountDemo {
|
|||
return new KeyValue<String, String>(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();
|
||||
|
|
|
|||
|
|
@ -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");
|
||||
|
|
|
|||
|
|
@ -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.";
|
||||
|
||||
/** <code>key.serializer</code> */
|
||||
public static final String KEY_SERIALIZER_CLASS_CONFIG = ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG;
|
||||
/** <code>replication.factor</code> */
|
||||
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 <code>Serde</code> interface.";
|
||||
|
||||
/** <code>value.serializer</code> */
|
||||
public static final String VALUE_SERIALIZER_CLASS_CONFIG = ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG;
|
||||
|
||||
/** <code>key.deserializer</code> */
|
||||
public static final String KEY_DESERIALIZER_CLASS_CONFIG = ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG;
|
||||
|
||||
/** <code>value.deserializer</code> */
|
||||
public static final String VALUE_DESERIALIZER_CLASS_CONFIG = ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG;
|
||||
/** <code>replication.factor</code> */
|
||||
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 <code>Serde</code> interface.";
|
||||
|
||||
/** <code>metrics.sample.window.ms</code> */
|
||||
public static final String METRICS_SAMPLE_WINDOW_MS_CONFIG = CommonClientConfigs.METRICS_SAMPLE_WINDOW_MS_CONFIG;
|
||||
|
|
@ -121,8 +118,6 @@ public class StreamsConfig extends AbstractConfig {
|
|||
/** <code>auto.offset.reset</code> */
|
||||
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) {
|
||||
|
|
|
|||
|
|
@ -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<K, V> {
|
|||
* 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<K, V> through(String topic, Serializer<K> keySerializer, Serializer<V> valSerializer, Deserializer<K> keyDeserializer, Deserializer<V> valDeserializer);
|
||||
KStream<K, V> through(String topic, Serde<K> keySerde, Serde<V> valSerde);
|
||||
|
||||
/**
|
||||
* Sends key-value to a topic using default serializers specified in the config.
|
||||
|
|
@ -131,13 +126,13 @@ public interface KStream<K, V> {
|
|||
/**
|
||||
* 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<K> keySerializer, Serializer<V> valSerializer);
|
||||
void to(String topic, Serde<K> keySerde, Serde<V> valSerde);
|
||||
|
||||
/**
|
||||
* Applies a stateful transformation to all elements in this stream.
|
||||
|
|
@ -171,18 +166,12 @@ public interface KStream<K, V> {
|
|||
* @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 <V1> the value type of the other stream
|
||||
* @param <R> the value type of the new stream
|
||||
*/
|
||||
|
|
@ -190,12 +179,9 @@ public interface KStream<K, V> {
|
|||
KStream<K, V1> otherStream,
|
||||
ValueJoiner<V, V1, R> joiner,
|
||||
JoinWindows windows,
|
||||
Serializer<K> keySerializer,
|
||||
Serializer<V> thisValueSerializer,
|
||||
Serializer<V1> otherValueSerializer,
|
||||
Deserializer<K> keyDeserializer,
|
||||
Deserializer<V> thisValueDeserializer,
|
||||
Deserializer<V1> otherValueDeserializer);
|
||||
Serde<K> keySerde,
|
||||
Serde<V> thisValueSerde,
|
||||
Serde<V1> otherValueSerde);
|
||||
|
||||
/**
|
||||
* Combines values of this stream with another KStream using Windowed Outer Join.
|
||||
|
|
@ -203,18 +189,12 @@ public interface KStream<K, V> {
|
|||
* @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 <V1> the value type of the other stream
|
||||
* @param <R> the value type of the new stream
|
||||
*/
|
||||
|
|
@ -222,12 +202,9 @@ public interface KStream<K, V> {
|
|||
KStream<K, V1> otherStream,
|
||||
ValueJoiner<V, V1, R> joiner,
|
||||
JoinWindows windows,
|
||||
Serializer<K> keySerializer,
|
||||
Serializer<V> thisValueSerializer,
|
||||
Serializer<V1> otherValueSerializer,
|
||||
Deserializer<K> keyDeserializer,
|
||||
Deserializer<V> thisValueDeserializer,
|
||||
Deserializer<V1> otherValueDeserializer);
|
||||
Serde<K> keySerde,
|
||||
Serde<V> thisValueSerde,
|
||||
Serde<V1> otherValueSerde);
|
||||
|
||||
/**
|
||||
* Combines values of this stream with another KStream using Windowed Left Join.
|
||||
|
|
@ -235,14 +212,10 @@ public interface KStream<K, V> {
|
|||
* @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 <V1> the value type of the other stream
|
||||
* @param <R> the value type of the new stream
|
||||
*/
|
||||
|
|
@ -250,10 +223,8 @@ public interface KStream<K, V> {
|
|||
KStream<K, V1> otherStream,
|
||||
ValueJoiner<V, V1, R> joiner,
|
||||
JoinWindows windows,
|
||||
Serializer<K> keySerializer,
|
||||
Serializer<V1> otherValueSerializer,
|
||||
Deserializer<K> keyDeserializer,
|
||||
Deserializer<V1> otherValueDeserializer);
|
||||
Serde<K> keySerde,
|
||||
Serde<V1> otherValueSerde);
|
||||
|
||||
/**
|
||||
* Combines values of this stream with KTable using Left Join.
|
||||
|
|
@ -273,10 +244,8 @@ public interface KStream<K, V> {
|
|||
*/
|
||||
<W extends Window> KTable<Windowed<K>, V> reduceByKey(Reducer<V> reducer,
|
||||
Windows<W> windows,
|
||||
Serializer<K> keySerializer,
|
||||
Serializer<V> aggValueSerializer,
|
||||
Deserializer<K> keyDeserializer,
|
||||
Deserializer<V> aggValueDeserializer);
|
||||
Serde<K> keySerde,
|
||||
Serde<V> aggValueSerde);
|
||||
|
||||
/**
|
||||
* Aggregate values of this stream by key on a window basis.
|
||||
|
|
@ -284,10 +253,8 @@ public interface KStream<K, V> {
|
|||
* @param reducer the class of Reducer
|
||||
*/
|
||||
KTable<K, V> reduceByKey(Reducer<V> reducer,
|
||||
Serializer<K> keySerializer,
|
||||
Serializer<V> aggValueSerializer,
|
||||
Deserializer<K> keyDeserializer,
|
||||
Deserializer<V> aggValueDeserializer,
|
||||
Serde<K> keySerde,
|
||||
Serde<V> aggValueSerde,
|
||||
String name);
|
||||
|
||||
/**
|
||||
|
|
@ -301,10 +268,8 @@ public interface KStream<K, V> {
|
|||
<T, W extends Window> KTable<Windowed<K>, T> aggregateByKey(Initializer<T> initializer,
|
||||
Aggregator<K, V, T> aggregator,
|
||||
Windows<W> windows,
|
||||
Serializer<K> keySerializer,
|
||||
Serializer<T> aggValueSerializer,
|
||||
Deserializer<K> keyDeserializer,
|
||||
Deserializer<T> aggValueDeserializer);
|
||||
Serde<K> keySerde,
|
||||
Serde<T> aggValueSerde);
|
||||
|
||||
/**
|
||||
* Aggregate values of this stream by key without a window basis, and hence
|
||||
|
|
@ -316,10 +281,8 @@ public interface KStream<K, V> {
|
|||
*/
|
||||
<T> KTable<K, T> aggregateByKey(Initializer<T> initializer,
|
||||
Aggregator<K, V, T> aggregator,
|
||||
Serializer<K> keySerializer,
|
||||
Serializer<T> aggValueSerializer,
|
||||
Deserializer<K> keyDeserializer,
|
||||
Deserializer<T> aggValueDeserializer,
|
||||
Serde<K> keySerde,
|
||||
Serde<T> aggValueSerde,
|
||||
String name);
|
||||
|
||||
/**
|
||||
|
|
@ -328,14 +291,12 @@ public interface KStream<K, V> {
|
|||
* @param windows the specification of the aggregation window
|
||||
*/
|
||||
<W extends Window> KTable<Windowed<K>, Long> countByKey(Windows<W> windows,
|
||||
Serializer<K> keySerializer,
|
||||
Deserializer<K> keyDeserializer);
|
||||
Serde<K> keySerde);
|
||||
|
||||
/**
|
||||
* Count number of messages of this stream by key without a window basis, and hence
|
||||
* return a ever updating counting table.
|
||||
*/
|
||||
KTable<K, Long> countByKey(Serializer<K> keySerializer,
|
||||
Deserializer<K> keyDeserializer,
|
||||
KTable<K, Long> countByKey(Serde<K> keySerde,
|
||||
String name);
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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 <K, V> KStream<K, V> stream(Deserializer<K> keyDeserializer, Deserializer<V> valDeserializer, String... topics) {
|
||||
public <K, V> KStream<K, V> stream(Serde<K> keySerde, Serde<V> 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 <K, V> KTable<K, V> 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 <K, V> KTable<K, V> table(Serializer<K> keySerializer, Serializer<V> valSerializer, Deserializer<K> keyDeserializer, Deserializer<V> valDeserializer, String topic) {
|
||||
public <K, V> KTable<K, V> table(Serde<K> keySerde, Serde<V> 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<K, V> 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);
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
|||
|
|
@ -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<K, V> {
|
|||
* 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<K, V> through(String topic, Serializer<K> keySerializer, Serializer<V> valSerializer, Deserializer<K> keyDeserializer, Deserializer<V> valDeserializer);
|
||||
KTable<K, V> through(String topic, Serde<K> keySerde, Serde<V> valSerde);
|
||||
|
||||
/**
|
||||
* Sends key-value to a topic using default serializers specified in the config.
|
||||
|
|
@ -90,13 +85,13 @@ public interface KTable<K, V> {
|
|||
/**
|
||||
* 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<K> keySerializer, Serializer<V> valSerializer);
|
||||
void to(String topic, Serde<K> keySerde, Serde<V> valSerde);
|
||||
|
||||
/**
|
||||
* Creates a new instance of KStream from this KTable
|
||||
|
|
@ -152,10 +147,8 @@ public interface KTable<K, V> {
|
|||
<K1, V1> KTable<K1, V1> reduce(Reducer<V1> addReducer,
|
||||
Reducer<V1> removeReducer,
|
||||
KeyValueMapper<K, V, KeyValue<K1, V1>> selector,
|
||||
Serializer<K1> keySerializer,
|
||||
Serializer<V1> valueSerializer,
|
||||
Deserializer<K1> keyDeserializer,
|
||||
Deserializer<V1> valueDeserializer,
|
||||
Serde<K1> keySerde,
|
||||
Serde<V1> valueSerde,
|
||||
String name);
|
||||
|
||||
/**
|
||||
|
|
@ -174,12 +167,9 @@ public interface KTable<K, V> {
|
|||
Aggregator<K1, V1, T> add,
|
||||
Aggregator<K1, V1, T> remove,
|
||||
KeyValueMapper<K, V, KeyValue<K1, V1>> selector,
|
||||
Serializer<K1> keySerializer,
|
||||
Serializer<V1> valueSerializer,
|
||||
Serializer<T> aggValueSerializer,
|
||||
Deserializer<K1> keyDeserializer,
|
||||
Deserializer<V1> valueDeserializer,
|
||||
Deserializer<T> aggValueDeserializer,
|
||||
Serde<K1> keySerde,
|
||||
Serde<V1> valueSerde,
|
||||
Serde<T> aggValueSerde,
|
||||
String name);
|
||||
|
||||
/**
|
||||
|
|
@ -191,10 +181,7 @@ public interface KTable<K, V> {
|
|||
* @return the instance of KTable
|
||||
*/
|
||||
<K1> KTable<K1, Long> count(KeyValueMapper<K, V, K1> selector,
|
||||
Serializer<K1> keySerializer,
|
||||
Serializer<V> valueSerializer,
|
||||
Deserializer<K1> keyDeserializer,
|
||||
Deserializer<V> valueDeserializer,
|
||||
Serde<K1> keySerde,
|
||||
Serde<V> valueSerde,
|
||||
String name);
|
||||
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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<K, V> extends AbstractStream<K> implements KStream<K, V
|
|||
|
||||
private static final String WINDOWED_NAME = "KSTREAM-WINDOWED-";
|
||||
|
||||
private static final LongSerializer LONG_SERIALIZER = new LongSerializer();
|
||||
private static final LongDeserializer LONG_DESERIALIZER = new LongDeserializer();
|
||||
|
||||
public KStreamImpl(KStreamBuilder topology, String name, Set<String> sourceNodes) {
|
||||
super(topology, name, sourceNodes);
|
||||
}
|
||||
|
|
@ -199,18 +195,16 @@ public class KStreamImpl<K, V> extends AbstractStream<K> implements KStream<K, V
|
|||
|
||||
@Override
|
||||
public KStream<K, V> through(String topic,
|
||||
Serializer<K> keySerializer,
|
||||
Serializer<V> valSerializer,
|
||||
Deserializer<K> keyDeserializer,
|
||||
Deserializer<V> valDeserializer) {
|
||||
to(topic, keySerializer, valSerializer);
|
||||
Serde<K> keySerde,
|
||||
Serde<V> valSerde) {
|
||||
to(topic, keySerde, valSerde);
|
||||
|
||||
return topology.stream(keyDeserializer, valDeserializer, topic);
|
||||
return topology.stream(keySerde, valSerde);
|
||||
}
|
||||
|
||||
@Override
|
||||
public KStream<K, V> through(String topic) {
|
||||
return through(topic, null, null, null, null);
|
||||
return through(topic, null, null);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
@ -220,10 +214,13 @@ public class KStreamImpl<K, V> extends AbstractStream<K> implements KStream<K, V
|
|||
|
||||
@SuppressWarnings("unchecked")
|
||||
@Override
|
||||
public void to(String topic, Serializer<K> keySerializer, Serializer<V> valSerializer) {
|
||||
public void to(String topic, Serde<K> keySerde, Serde<V> valSerde) {
|
||||
String name = topology.newName(SINK_NAME);
|
||||
StreamPartitioner<K, V> streamPartitioner = null;
|
||||
|
||||
Serializer<K> keySerializer = keySerde == null ? null : keySerde.serializer();
|
||||
Serializer<V> valSerializer = keySerde == null ? null : valSerde.serializer();
|
||||
|
||||
if (keySerializer != null && keySerializer instanceof WindowedSerializer) {
|
||||
WindowedSerializer<Object> windowedSerializer = (WindowedSerializer<Object>) keySerializer;
|
||||
streamPartitioner = (StreamPartitioner<K, V>) new WindowedStreamPartitioner<Object, V>(windowedSerializer);
|
||||
|
|
@ -265,16 +262,11 @@ public class KStreamImpl<K, V> extends AbstractStream<K> implements KStream<K, V
|
|||
KStream<K, V1> other,
|
||||
ValueJoiner<V, V1, R> joiner,
|
||||
JoinWindows windows,
|
||||
Serializer<K> keySerializer,
|
||||
Serializer<V> thisValueSerializer,
|
||||
Serializer<V1> otherValueSerializer,
|
||||
Deserializer<K> keyDeserializer,
|
||||
Deserializer<V> thisValueDeserializer,
|
||||
Deserializer<V1> otherValueDeserializer) {
|
||||
Serde<K> keySerde,
|
||||
Serde<V> thisValueSerde,
|
||||
Serde<V1> 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<K, V> extends AbstractStream<K> implements KStream<K, V
|
|||
KStream<K, V1> other,
|
||||
ValueJoiner<V, V1, R> joiner,
|
||||
JoinWindows windows,
|
||||
Serializer<K> keySerializer,
|
||||
Serializer<V> thisValueSerializer,
|
||||
Serializer<V1> otherValueSerializer,
|
||||
Deserializer<K> keyDeserializer,
|
||||
Deserializer<V> thisValueDeserializer,
|
||||
Deserializer<V1> otherValueDeserializer) {
|
||||
Serde<K> keySerde,
|
||||
Serde<V> thisValueSerde,
|
||||
Serde<V1> 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<K, V> extends AbstractStream<K> implements KStream<K, V
|
|||
KStream<K, V1> other,
|
||||
ValueJoiner<V, V1, R> joiner,
|
||||
JoinWindows windows,
|
||||
Serializer<K> keySerializer,
|
||||
Serializer<V> thisValueSerializer,
|
||||
Serializer<V1> otherValueSerializer,
|
||||
Deserializer<K> keyDeserializer,
|
||||
Deserializer<V> thisValueDeserializer,
|
||||
Deserializer<V1> otherValueDeserializer,
|
||||
Serde<K> keySerde,
|
||||
Serde<V> thisValueSerde,
|
||||
Serde<V1> otherValueSerde,
|
||||
boolean outer) {
|
||||
|
||||
Set<String> allSourceNodes = ensureJoinableWith((AbstractStream<K>) 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<K, V> extends AbstractStream<K> implements KStream<K, V
|
|||
KStream<K, V1> other,
|
||||
ValueJoiner<V, V1, R> joiner,
|
||||
JoinWindows windows,
|
||||
Serializer<K> keySerializer,
|
||||
Serializer<V1> otherValueSerializer,
|
||||
Deserializer<K> keyDeserializer,
|
||||
Deserializer<V1> otherValueDeserializer) {
|
||||
Serde<K> keySerde,
|
||||
Serde<V1> otherValueSerde) {
|
||||
|
||||
Set<String> allSourceNodes = ensureJoinableWith((AbstractStream<K>) 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<K, V> extends AbstractStream<K> implements KStream<K, V
|
|||
@Override
|
||||
public <W extends Window> KTable<Windowed<K>, V> reduceByKey(Reducer<V> reducer,
|
||||
Windows<W> windows,
|
||||
Serializer<K> keySerializer,
|
||||
Serializer<V> aggValueSerializer,
|
||||
Deserializer<K> keyDeserializer,
|
||||
Deserializer<V> aggValueDeserializer) {
|
||||
Serde<K> keySerde,
|
||||
Serde<V> aggValueSerde) {
|
||||
|
||||
String reduceName = topology.newName(REDUCE_NAME);
|
||||
|
||||
KStreamWindowReduce<K, V, W> 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<K, V> extends AbstractStream<K> implements KStream<K, V
|
|||
|
||||
@Override
|
||||
public KTable<K, V> reduceByKey(Reducer<V> reducer,
|
||||
Serializer<K> keySerializer,
|
||||
Serializer<V> aggValueSerializer,
|
||||
Deserializer<K> keyDeserializer,
|
||||
Deserializer<V> aggValueDeserializer,
|
||||
Serde<K> keySerde,
|
||||
Serde<V> aggValueSerde,
|
||||
String name) {
|
||||
|
||||
String reduceName = topology.newName(REDUCE_NAME);
|
||||
|
|
@ -434,8 +412,8 @@ public class KStreamImpl<K, V> extends AbstractStream<K> implements KStream<K, V
|
|||
KStreamReduce<K, V> 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<K, V> extends AbstractStream<K> implements KStream<K, V
|
|||
public <T, W extends Window> KTable<Windowed<K>, T> aggregateByKey(Initializer<T> initializer,
|
||||
Aggregator<K, V, T> aggregator,
|
||||
Windows<W> windows,
|
||||
Serializer<K> keySerializer,
|
||||
Serializer<T> aggValueSerializer,
|
||||
Deserializer<K> keyDeserializer,
|
||||
Deserializer<T> aggValueDeserializer) {
|
||||
Serde<K> keySerde,
|
||||
Serde<T> aggValueSerde) {
|
||||
|
||||
String aggregateName = topology.newName(AGGREGATE_NAME);
|
||||
|
||||
KStreamAggProcessorSupplier<K, Windowed<K>, 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<K, V> extends AbstractStream<K> implements KStream<K, V
|
|||
@Override
|
||||
public <T> KTable<K, T> aggregateByKey(Initializer<T> initializer,
|
||||
Aggregator<K, V, T> aggregator,
|
||||
Serializer<K> keySerializer,
|
||||
Serializer<T> aggValueSerializer,
|
||||
Deserializer<K> keyDeserializer,
|
||||
Deserializer<T> aggValueDeserializer,
|
||||
Serde<K> keySerde,
|
||||
Serde<T> aggValueSerde,
|
||||
String name) {
|
||||
|
||||
String aggregateName = topology.newName(AGGREGATE_NAME);
|
||||
|
|
@ -489,8 +463,8 @@ public class KStreamImpl<K, V> extends AbstractStream<K> implements KStream<K, V
|
|||
KStreamAggProcessorSupplier<K, K, V, T> 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<K, V> extends AbstractStream<K> implements KStream<K, V
|
|||
|
||||
@Override
|
||||
public <W extends Window> KTable<Windowed<K>, Long> countByKey(Windows<W> windows,
|
||||
Serializer<K> keySerializer,
|
||||
Deserializer<K> keyDeserializer) {
|
||||
Serde<K> keySerde) {
|
||||
return this.aggregateByKey(
|
||||
new Initializer<Long>() {
|
||||
@Override
|
||||
|
|
@ -518,13 +491,12 @@ public class KStreamImpl<K, V> extends AbstractStream<K> implements KStream<K, V
|
|||
public Long apply(K aggKey, V value, Long aggregate) {
|
||||
return aggregate + 1L;
|
||||
}
|
||||
}, windows, keySerializer, LONG_SERIALIZER, keyDeserializer, LONG_DESERIALIZER);
|
||||
}, windows, keySerde, Serdes.Long());
|
||||
}
|
||||
|
||||
@Override
|
||||
public KTable<K, Long> countByKey(Serializer<K> keySerializer,
|
||||
Deserializer<K> keyDeserializer,
|
||||
String name) {
|
||||
public KTable<K, Long> countByKey(Serde<K> keySerde,
|
||||
String name) {
|
||||
return this.aggregateByKey(
|
||||
new Initializer<Long>() {
|
||||
@Override
|
||||
|
|
@ -537,6 +509,6 @@ public class KStreamImpl<K, V> extends AbstractStream<K> implements KStream<K, V
|
|||
public Long apply(K aggKey, V value, Long aggregate) {
|
||||
return aggregate + 1L;
|
||||
}
|
||||
}, keySerializer, LONG_SERIALIZER, keyDeserializer, LONG_DESERIALIZER, name);
|
||||
}, keySerde, Serdes.Long(), name);
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -17,10 +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.Serializer;
|
||||
import org.apache.kafka.common.serialization.Serde;
|
||||
import org.apache.kafka.common.serialization.Serdes;
|
||||
import org.apache.kafka.streams.kstream.Aggregator;
|
||||
import org.apache.kafka.streams.kstream.Initializer;
|
||||
import org.apache.kafka.streams.kstream.KStream;
|
||||
|
|
@ -77,15 +75,10 @@ public class KTableImpl<K, S, V> extends AbstractStream<K> implements KTable<K,
|
|||
|
||||
private static final String TOSTREAM_NAME = "KTABLE-TOSTREAM-";
|
||||
|
||||
private static final LongSerializer LONG_SERIALIZER = new LongSerializer();
|
||||
private static final LongDeserializer LONG_DESERIALIZER = new LongDeserializer();
|
||||
|
||||
public final ProcessorSupplier<?, ?> processorSupplier;
|
||||
|
||||
private final Serializer<K> keySerializer;
|
||||
private final Serializer<V> valSerializer;
|
||||
private final Deserializer<K> keyDeserializer;
|
||||
private final Deserializer<V> valDeserializer;
|
||||
private final Serde<K> keySerde;
|
||||
private final Serde<V> valSerde;
|
||||
|
||||
private boolean sendOldValues = false;
|
||||
|
||||
|
|
@ -93,23 +86,19 @@ public class KTableImpl<K, S, V> extends AbstractStream<K> implements KTable<K,
|
|||
String name,
|
||||
ProcessorSupplier<?, ?> processorSupplier,
|
||||
Set<String> 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<String> sourceNodes,
|
||||
Serializer<K> keySerializer,
|
||||
Serializer<V> valSerializer,
|
||||
Deserializer<K> keyDeserializer,
|
||||
Deserializer<V> valDeserializer) {
|
||||
Serde<K> keySerde,
|
||||
Serde<V> 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<K, S, V> extends AbstractStream<K> implements KTable<K,
|
|||
|
||||
@Override
|
||||
public KTable<K, V> through(String topic,
|
||||
Serializer<K> keySerializer,
|
||||
Serializer<V> valSerializer,
|
||||
Deserializer<K> keyDeserializer,
|
||||
Deserializer<V> valDeserializer) {
|
||||
to(topic, keySerializer, valSerializer);
|
||||
Serde<K> keySerde,
|
||||
Serde<V> valSerde) {
|
||||
to(topic, keySerde, valSerde);
|
||||
|
||||
return topology.table(keySerializer, valSerializer, keyDeserializer, valDeserializer, topic);
|
||||
return topology.table(keySerde, valSerde, topic);
|
||||
}
|
||||
|
||||
@Override
|
||||
public KTable<K, V> through(String topic) {
|
||||
return through(topic, null, null, null, null);
|
||||
return through(topic, null, null);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
@ -163,8 +150,8 @@ public class KTableImpl<K, S, V> extends AbstractStream<K> implements KTable<K,
|
|||
}
|
||||
|
||||
@Override
|
||||
public void to(String topic, Serializer<K> keySerializer, Serializer<V> valSerializer) {
|
||||
this.toStream().to(topic, keySerializer, valSerializer);
|
||||
public void to(String topic, Serde<K> keySerde, Serde<V> valSerde) {
|
||||
this.toStream().to(topic, keySerde, valSerde);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
@ -255,12 +242,9 @@ public class KTableImpl<K, S, V> extends AbstractStream<K> implements KTable<K,
|
|||
Aggregator<K1, V1, T> add,
|
||||
Aggregator<K1, V1, T> remove,
|
||||
KeyValueMapper<K, V, KeyValue<K1, V1>> selector,
|
||||
Serializer<K1> keySerializer,
|
||||
Serializer<V1> valueSerializer,
|
||||
Serializer<T> aggValueSerializer,
|
||||
Deserializer<K1> keyDeserializer,
|
||||
Deserializer<V1> valueDeserializer,
|
||||
Deserializer<T> aggValueDeserializer,
|
||||
Serde<K1> keySerde,
|
||||
Serde<V1> valueSerde,
|
||||
Serde<T> aggValueSerde,
|
||||
String name) {
|
||||
|
||||
String selectName = topology.newName(SELECT_NAME);
|
||||
|
|
@ -270,16 +254,16 @@ public class KTableImpl<K, S, V> extends AbstractStream<K> implements KTable<K,
|
|||
|
||||
String topic = name + REPARTITION_TOPIC_SUFFIX;
|
||||
|
||||
ChangedSerializer<V1> changedValueSerializer = new ChangedSerializer<>(valueSerializer);
|
||||
ChangedDeserializer<V1> changedValueDeserializer = new ChangedDeserializer<>(valueDeserializer);
|
||||
ChangedSerializer<V1> changedValueSerializer = new ChangedSerializer<>(valueSerde.serializer());
|
||||
ChangedDeserializer<V1> changedValueDeserializer = new ChangedDeserializer<>(valueSerde.deserializer());
|
||||
|
||||
KTableProcessorSupplier<K, V, KeyValue<K1, V1>> selectSupplier = new KTableRepartitionMap<>(this, selector);
|
||||
|
||||
ProcessorSupplier<K1, Change<V1>> 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<K, S, V> extends AbstractStream<K> implements KTable<K,
|
|||
|
||||
// send the aggregate key-value pairs to the intermediate topic for partitioning
|
||||
topology.addInternalTopic(topic);
|
||||
topology.addSink(sinkName, topic, keySerializer, changedValueSerializer, selectName);
|
||||
topology.addSink(sinkName, topic, keySerde.serializer(), changedValueSerializer, selectName);
|
||||
|
||||
// read the intermediate topic
|
||||
topology.addSource(sourceName, keyDeserializer, changedValueDeserializer, topic);
|
||||
topology.addSource(sourceName, keySerde.deserializer(), changedValueDeserializer, topic);
|
||||
|
||||
// aggregate the values with the aggregator and local store
|
||||
topology.addProcessor(aggregateName, aggregateSupplier, sourceName);
|
||||
|
|
@ -304,10 +288,8 @@ public class KTableImpl<K, S, V> extends AbstractStream<K> implements KTable<K,
|
|||
|
||||
@Override
|
||||
public <K1> KTable<K1, Long> count(final KeyValueMapper<K, V, K1> selector,
|
||||
Serializer<K1> keySerializer,
|
||||
Serializer<V> valueSerializer,
|
||||
Deserializer<K1> keyDeserializer,
|
||||
Deserializer<V> valueDeserializer,
|
||||
Serde<K1> keySerde,
|
||||
Serde<V> valueSerde,
|
||||
String name) {
|
||||
return this.aggregate(
|
||||
new Initializer<Long>() {
|
||||
|
|
@ -332,17 +314,15 @@ public class KTableImpl<K, S, V> extends AbstractStream<K> implements KTable<K,
|
|||
return new KeyValue<>(selector.apply(key, value), value);
|
||||
}
|
||||
},
|
||||
keySerializer, valueSerializer, LONG_SERIALIZER, keyDeserializer, valueDeserializer, LONG_DESERIALIZER, name);
|
||||
keySerde, valueSerde, Serdes.Long(), name);
|
||||
}
|
||||
|
||||
@Override
|
||||
public <K1, V1> KTable<K1, V1> reduce(Reducer<V1> addReducer,
|
||||
Reducer<V1> removeReducer,
|
||||
KeyValueMapper<K, V, KeyValue<K1, V1>> selector,
|
||||
Serializer<K1> keySerializer,
|
||||
Serializer<V1> valueSerializer,
|
||||
Deserializer<K1> keyDeserializer,
|
||||
Deserializer<V1> valueDeserializer,
|
||||
Serde<K1> keySerde,
|
||||
Serde<V1> valueSerde,
|
||||
String name) {
|
||||
|
||||
String selectName = topology.newName(SELECT_NAME);
|
||||
|
|
@ -352,16 +332,16 @@ public class KTableImpl<K, S, V> extends AbstractStream<K> implements KTable<K,
|
|||
|
||||
String topic = name + REPARTITION_TOPIC_SUFFIX;
|
||||
|
||||
ChangedSerializer<V1> changedValueSerializer = new ChangedSerializer<>(valueSerializer);
|
||||
ChangedDeserializer<V1> changedValueDeserializer = new ChangedDeserializer<>(valueDeserializer);
|
||||
ChangedSerializer<V1> changedValueSerializer = new ChangedSerializer<>(valueSerde.serializer());
|
||||
ChangedDeserializer<V1> changedValueDeserializer = new ChangedDeserializer<>(valueSerde.deserializer());
|
||||
|
||||
KTableProcessorSupplier<K, V, KeyValue<K1, V1>> selectSupplier = new KTableRepartitionMap<>(this, selector);
|
||||
|
||||
ProcessorSupplier<K1, Change<V1>> 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<K, S, V> extends AbstractStream<K> implements KTable<K,
|
|||
|
||||
// send the aggregate key-value pairs to the intermediate topic for partitioning
|
||||
topology.addInternalTopic(topic);
|
||||
topology.addSink(sinkName, topic, keySerializer, changedValueSerializer, selectName);
|
||||
topology.addSink(sinkName, topic, keySerde.serializer(), changedValueSerializer, selectName);
|
||||
|
||||
// read the intermediate topic
|
||||
topology.addSource(sourceName, keyDeserializer, changedValueDeserializer, topic);
|
||||
topology.addSource(sourceName, keySerde.deserializer(), changedValueDeserializer, topic);
|
||||
|
||||
// aggregate the values with the aggregator and local store
|
||||
topology.addProcessor(reduceName, aggregateSupplier, sourceName);
|
||||
|
|
@ -421,7 +401,7 @@ public class KTableImpl<K, S, V> extends AbstractStream<K> implements KTable<K,
|
|||
synchronized (source) {
|
||||
if (!source.isMaterialized()) {
|
||||
StateStoreSupplier storeSupplier =
|
||||
new KTableStoreSupplier<>(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();
|
||||
|
|
|
|||
|
|
@ -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<K, V> implements StateStoreSupplier {
|
||||
|
||||
private final String name;
|
||||
private final Serdes<K, V> serdes;
|
||||
private final StateSerdes<K, V> serdes;
|
||||
private final Time time;
|
||||
|
||||
protected KTableStoreSupplier(String name,
|
||||
Serializer<K> keySerializer, Deserializer<K> keyDeserializer,
|
||||
Serializer<V> valSerializer, Deserializer<V> valDeserializer,
|
||||
Serde<K> keySerde,
|
||||
Serde<V> 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;
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -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.
|
||||
|
|
|
|||
|
|
@ -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}.
|
||||
* <p>
|
||||
* 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
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -49,8 +49,8 @@ public class SinkNode<K, V> extends ProcessorNode<K, V> {
|
|||
@Override
|
||||
public void init(ProcessorContext context) {
|
||||
this.context = context;
|
||||
if (this.keySerializer == null) this.keySerializer = (Serializer<K>) context.keySerializer();
|
||||
if (this.valSerializer == null) this.valSerializer = (Serializer<V>) context.valueSerializer();
|
||||
if (this.keySerializer == null) this.keySerializer = (Serializer<K>) context.keySerde().serializer();
|
||||
if (this.valSerializer == null) this.valSerializer = (Serializer<V>) context.valueSerde().serializer();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
|||
|
|
@ -47,8 +47,8 @@ public class SourceNode<K, V> extends ProcessorNode<K, V> {
|
|||
this.context = context;
|
||||
|
||||
// if serializers are null, get the default ones from the context
|
||||
if (this.keyDeserializer == null) this.keyDeserializer = (Deserializer<K>) context.keyDeserializer();
|
||||
if (this.valDeserializer == null) this.valDeserializer = (Deserializer<V>) context.valueDeserializer();
|
||||
if (this.keyDeserializer == null) this.keyDeserializer = (Deserializer<K>) context.keySerde().deserializer();
|
||||
if (this.valDeserializer == null) this.valDeserializer = (Deserializer<V>) context.valueSerde().deserializer();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -100,7 +100,6 @@ public class StreamThread extends Thread {
|
|||
|
||||
private long lastClean;
|
||||
private long lastCommit;
|
||||
private long recordsProcessed;
|
||||
private Throwable rebalanceException = null;
|
||||
|
||||
private Map<TopicPartition, List<ConsumerRecord<byte[], byte[]>>> 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);
|
||||
|
|
|
|||
|
|
@ -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 <K> key type of serdes
|
||||
* @param <V> value type of serdes
|
||||
*/
|
||||
public final class Serdes<K, V> {
|
||||
|
||||
public static <K, V> Serdes<K, V> withBuiltinTypes(String topic, Class<K> keyClass, Class<V> valueClass) {
|
||||
Serializer<K> keySerializer = serializer(keyClass);
|
||||
Deserializer<K> keyDeserializer = deserializer(keyClass);
|
||||
Serializer<V> valueSerializer = serializer(valueClass);
|
||||
Deserializer<V> valueDeserializer = deserializer(valueClass);
|
||||
return new Serdes<>(topic, keySerializer, keyDeserializer, valueSerializer, valueDeserializer);
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
static <T> Serializer<T> serializer(Class<T> type) {
|
||||
if (String.class.isAssignableFrom(type)) return (Serializer<T>) new StringSerializer();
|
||||
if (Integer.class.isAssignableFrom(type)) return (Serializer<T>) new IntegerSerializer();
|
||||
if (Long.class.isAssignableFrom(type)) return (Serializer<T>) new LongSerializer();
|
||||
if (byte[].class.isAssignableFrom(type)) return (Serializer<T>) new ByteArraySerializer();
|
||||
throw new IllegalArgumentException("Unknown class for built-in serializer");
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
static <T> Deserializer<T> deserializer(Class<T> type) {
|
||||
if (String.class.isAssignableFrom(type)) return (Deserializer<T>) new StringDeserializer();
|
||||
if (Integer.class.isAssignableFrom(type)) return (Deserializer<T>) new IntegerDeserializer();
|
||||
if (Long.class.isAssignableFrom(type)) return (Deserializer<T>) new LongDeserializer();
|
||||
if (byte[].class.isAssignableFrom(type)) return (Deserializer<T>) new ByteArrayDeserializer();
|
||||
throw new IllegalArgumentException("Unknown class for built-in serializer");
|
||||
}
|
||||
|
||||
private final String topic;
|
||||
private Serializer<K> keySerializer;
|
||||
private Serializer<V> valueSerializer;
|
||||
private Deserializer<K> keyDeserializer;
|
||||
private Deserializer<V> valueDeserializer;
|
||||
|
||||
/**
|
||||
* Create a context for serialization using the specified serializers and deserializers which
|
||||
* <em>must</em> 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<K> keySerializer, Deserializer<K> keyDeserializer,
|
||||
Serializer<V> valueSerializer, Deserializer<V> 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<K> keyDeserializer() {
|
||||
return keyDeserializer;
|
||||
}
|
||||
|
||||
public Serializer<K> keySerializer() {
|
||||
return keySerializer;
|
||||
}
|
||||
|
||||
public Deserializer<V> valueDeserializer() {
|
||||
return valueDeserializer;
|
||||
}
|
||||
|
||||
public Serializer<V> 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);
|
||||
}
|
||||
}
|
||||
|
|
@ -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 <K> key type of serdes
|
||||
* @param <V> value type of serdes
|
||||
*/
|
||||
public final class StateSerdes<K, V> {
|
||||
|
||||
public static <K, V> StateSerdes<K, V> withBuiltinTypes(String topic, Class<K> keyClass, Class<V> valueClass) {
|
||||
return new StateSerdes<>(topic, Serdes.serdeFrom(keyClass), Serdes.serdeFrom(valueClass));
|
||||
}
|
||||
|
||||
private final String topic;
|
||||
private final Serde<K> keySerde;
|
||||
private final Serde<V> valueSerde;
|
||||
|
||||
/**
|
||||
* Create a context for serialization using the specified serializers and deserializers which
|
||||
* <em>must</em> 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<K> keySerde,
|
||||
Serde<V> 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<K> keySerde() {
|
||||
return keySerde;
|
||||
}
|
||||
|
||||
public Serde<V> valueSerde() {
|
||||
return valueSerde;
|
||||
}
|
||||
|
||||
public Deserializer<K> keyDeserializer() {
|
||||
return keySerde.deserializer();
|
||||
}
|
||||
|
||||
public Serializer<K> keySerializer() {
|
||||
return keySerde.serializer();
|
||||
}
|
||||
|
||||
public Deserializer<V> valueDeserializer() {
|
||||
return valueSerde.deserializer();
|
||||
}
|
||||
|
||||
public Serializer<V> 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);
|
||||
}
|
||||
}
|
||||
|
|
@ -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 <K> ValueFactory<K> withKeys(final Serializer<K> keySerializer, final Deserializer<K> keyDeserializer) {
|
||||
public <K> ValueFactory<K> withKeys(final Serde<K> keySerde) {
|
||||
return new ValueFactory<K>() {
|
||||
@Override
|
||||
public <V> KeyValueFactory<K, V> withValues(final Serializer<V> valueSerializer,
|
||||
final Deserializer<V> valueDeserializer) {
|
||||
final Serdes<K, V> serdes =
|
||||
new Serdes<>(name, keySerializer, keyDeserializer, valueSerializer, valueDeserializer);
|
||||
public <V> KeyValueFactory<K, V> withValues(final Serde<V> valueSerde) {
|
||||
final StateSerdes<K, V> serdes =
|
||||
new StateSerdes<>(name, keySerde, valueSerde);
|
||||
return new KeyValueFactory<K, V>() {
|
||||
@Override
|
||||
public InMemoryKeyValueFactory<K, V> inMemory() {
|
||||
|
|
@ -116,7 +109,7 @@ public class Stores {
|
|||
* @return the interface used to specify the type of values; never null
|
||||
*/
|
||||
public ValueFactory<String> 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<Integer> 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<Long> 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<Double> 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<ByteBuffer> 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<byte[]> 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 <K> ValueFactory<K> withKeys(Class<K> 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 <K> ValueFactory<K> withKeys(Serializer<K> keySerializer, Deserializer<K> keyDeserializer);
|
||||
public abstract <K> ValueFactory<K> withKeys(Serde<K> keySerde);
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
@ -181,7 +188,7 @@ public class Stores {
|
|||
* @return the interface used to specify the remaining key-value store options; never null
|
||||
*/
|
||||
public KeyValueFactory<K, String> 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<K, Integer> 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<K, Long> 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<K, Double> 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<K, ByteBuffer> 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<K, byte[]> 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 <V> KeyValueFactory<K, V> withValues(Class<V> 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 <V> KeyValueFactory<K, V> withValues(Serializer<V> valueSerializer, Deserializer<V> valueDeserializer);
|
||||
public abstract <V> KeyValueFactory<K, V> withValues(Serde<V> valueSerde);
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
@ -240,7 +261,7 @@ public class Stores {
|
|||
* @param <K> the type of keys
|
||||
* @param <V> the type of values
|
||||
*/
|
||||
public static interface KeyValueFactory<K, V> {
|
||||
public interface KeyValueFactory<K, V> {
|
||||
/**
|
||||
* 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 <K> the type of keys
|
||||
* @param <V> the type of values
|
||||
*/
|
||||
public static interface InMemoryKeyValueFactory<K, V> {
|
||||
public interface InMemoryKeyValueFactory<K, V> {
|
||||
/**
|
||||
* 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 <K> the type of keys
|
||||
* @param <V> the type of values
|
||||
*/
|
||||
public static interface PersistentKeyValueFactory<K, V> {
|
||||
public interface PersistentKeyValueFactory<K, V> {
|
||||
|
||||
/**
|
||||
* Set the persistent store as a windowed key-value store
|
||||
|
|
|
|||
|
|
@ -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<byte[], byte[]> INNER_SERDES = Serdes.withBuiltinTypes("", byte[].class, byte[].class);
|
||||
public static final StateSerdes<byte[], byte[]> INNER_SERDES = StateSerdes.withBuiltinTypes("", byte[].class, byte[].class);
|
||||
@SuppressWarnings("unchecked")
|
||||
public static final KeyValueIterator<byte[], byte[]>[] NO_ITERATORS = (KeyValueIterator<byte[], byte[]>[]) new KeyValueIterator[0];
|
||||
|
||||
public static <K> byte[] toBinaryKey(K key, final long timestamp, final int seqnum, Serdes<K, ?> serdes) {
|
||||
public static <K> byte[] toBinaryKey(K key, final long timestamp, final int seqnum, StateSerdes<K, ?> 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> K keyFromBinaryKey(byte[] binaryKey, Serdes<K, ?> serdes) {
|
||||
public static <K> K keyFromBinaryKey(byte[] binaryKey, StateSerdes<K, ?> serdes) {
|
||||
byte[] bytes = new byte[binaryKey.length - TIMESTAMP_SIZE - SEQNUM_SIZE];
|
||||
|
||||
System.arraycopy(binaryKey, 0, bytes, 0, bytes.length);
|
||||
|
|
|
|||
|
|
@ -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<K, V> implements KeyValueStore<K, V> {
|
||||
|
||||
private final KeyValueStore<K, V> inner;
|
||||
private final Serdes<K, V> serdes;
|
||||
private final StateSerdes<K, V> serdes;
|
||||
private final String storeName;
|
||||
|
||||
private StoreChangeLogger<K, V> changeLogger;
|
||||
private StoreChangeLogger.ValueGetter<K, V> getter;
|
||||
|
||||
public InMemoryKeyValueLoggedStore(final String storeName, final KeyValueStore<K, V> inner, final Serdes<K, V> serdes) {
|
||||
public InMemoryKeyValueLoggedStore(final String storeName, final KeyValueStore<K, V> inner, final StateSerdes<K, V> serdes) {
|
||||
this.storeName = storeName;
|
||||
this.inner = inner;
|
||||
this.serdes = serdes;
|
||||
|
|
|
|||
|
|
@ -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<K, V> implements StateStoreSupplier {
|
|||
|
||||
private final String name;
|
||||
private final Time time;
|
||||
private final Serdes<K, V> serdes;
|
||||
private final StateSerdes<K, V> serdes;
|
||||
|
||||
public InMemoryKeyValueStoreSupplier(String name, Serdes<K, V> serdes, Time time) {
|
||||
public InMemoryKeyValueStoreSupplier(String name, StateSerdes<K, V> serdes, Time time) {
|
||||
this.name = name;
|
||||
this.time = time;
|
||||
this.serdes = serdes;
|
||||
|
|
@ -67,7 +67,7 @@ public class InMemoryKeyValueStoreSupplier<K, V> implements StateStoreSupplier {
|
|||
private final NavigableMap<K, V> map;
|
||||
|
||||
private boolean loggingEnabled = false;
|
||||
private Serdes<K, V> serdes = null;
|
||||
private StateSerdes<K, V> serdes = null;
|
||||
|
||||
public MemoryStore(String name) {
|
||||
super();
|
||||
|
|
@ -75,7 +75,7 @@ public class InMemoryKeyValueStoreSupplier<K, V> implements StateStoreSupplier {
|
|||
this.map = new TreeMap<>();
|
||||
}
|
||||
|
||||
public KeyValueStore<K, V> enableLogging(Serdes<K, V> serdes) {
|
||||
public KeyValueStore<K, V> enableLogging(StateSerdes<K, V> serdes) {
|
||||
this.loggingEnabled = true;
|
||||
this.serdes = serdes;
|
||||
|
||||
|
|
|
|||
|
|
@ -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<K, V> implements StateStoreSupplier {
|
|||
|
||||
private final String name;
|
||||
private final int capacity;
|
||||
private final Serdes<K, V> serdes;
|
||||
private final StateSerdes<K, V> serdes;
|
||||
private final Time time;
|
||||
|
||||
public InMemoryLRUCacheStoreSupplier(String name, int capacity, Serdes<K, V> serdes, Time time) {
|
||||
public InMemoryLRUCacheStoreSupplier(String name, int capacity, StateSerdes<K, V> serdes, Time time) {
|
||||
this.name = name;
|
||||
this.capacity = capacity;
|
||||
this.serdes = serdes;
|
||||
|
|
|
|||
|
|
@ -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<K, V> implements KeyValueStore<K, V> {
|
|||
protected EldestEntryRemovalListener<K, V> listener;
|
||||
|
||||
private boolean loggingEnabled = false;
|
||||
private Serdes<K, V> serdes = null;
|
||||
private StateSerdes<K, V> serdes = null;
|
||||
|
||||
// this is used for extended MemoryNavigableLRUCache only
|
||||
public MemoryLRUCache() {}
|
||||
|
|
@ -69,7 +69,7 @@ public class MemoryLRUCache<K, V> implements KeyValueStore<K, V> {
|
|||
};
|
||||
}
|
||||
|
||||
public KeyValueStore<K, V> enableLogging(Serdes<K, V> serdes) {
|
||||
public KeyValueStore<K, V> enableLogging(StateSerdes<K, V> serdes) {
|
||||
this.loggingEnabled = true;
|
||||
this.serdes = serdes;
|
||||
|
||||
|
|
|
|||
|
|
@ -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<K, V> implements StateStoreSupplier {
|
||||
|
||||
private final String name;
|
||||
private final Serdes<K, V> serdes;
|
||||
private final StateSerdes<K, V> serdes;
|
||||
private final Time time;
|
||||
|
||||
public RocksDBKeyValueStoreSupplier(String name, Serdes<K, V> serdes, Time time) {
|
||||
public RocksDBKeyValueStoreSupplier(String name, StateSerdes<K, V> serdes, Time time) {
|
||||
this.name = name;
|
||||
this.serdes = serdes;
|
||||
this.time = time;
|
||||
|
|
|
|||
|
|
@ -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<K, V> implements KeyValueStore<K, V> {
|
|||
private final FlushOptions fOptions;
|
||||
|
||||
private ProcessorContext context;
|
||||
private Serdes<K, V> serdes;
|
||||
private StateSerdes<K, V> serdes;
|
||||
protected File dbDir;
|
||||
private RocksDB db;
|
||||
|
||||
|
|
@ -92,11 +92,11 @@ public class RocksDBStore<K, V> implements KeyValueStore<K, V> {
|
|||
return this;
|
||||
}
|
||||
|
||||
public RocksDBStore(String name, Serdes<K, V> serdes) {
|
||||
public RocksDBStore(String name, StateSerdes<K, V> serdes) {
|
||||
this(name, DB_FILE_DIR, serdes);
|
||||
}
|
||||
|
||||
public RocksDBStore(String name, String parentDir, Serdes<K, V> serdes) {
|
||||
public RocksDBStore(String name, String parentDir, StateSerdes<K, V> serdes) {
|
||||
this.name = name;
|
||||
this.parentDir = parentDir;
|
||||
this.serdes = serdes;
|
||||
|
|
@ -399,9 +399,9 @@ public class RocksDBStore<K, V> implements KeyValueStore<K, V> {
|
|||
|
||||
private static class RocksDbIterator<K, V> implements KeyValueIterator<K, V> {
|
||||
private final RocksIterator iter;
|
||||
private final Serdes<K, V> serdes;
|
||||
private final StateSerdes<K, V> serdes;
|
||||
|
||||
public RocksDbIterator(RocksIterator iter, Serdes<K, V> serdes) {
|
||||
public RocksDbIterator(RocksIterator iter, StateSerdes<K, V> serdes) {
|
||||
this.iter = iter;
|
||||
this.serdes = serdes;
|
||||
}
|
||||
|
|
@ -463,7 +463,7 @@ public class RocksDBStore<K, V> implements KeyValueStore<K, V> {
|
|||
private final Comparator<byte[]> comparator = new LexicographicComparator();
|
||||
byte[] rawToKey;
|
||||
|
||||
public RocksDBRangeIterator(RocksIterator iter, Serdes<K, V> serdes,
|
||||
public RocksDBRangeIterator(RocksIterator iter, StateSerdes<K, V> serdes,
|
||||
K from, K to) {
|
||||
super(iter, serdes);
|
||||
iter.seek(serdes.rawKey(from));
|
||||
|
|
|
|||
|
|
@ -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<K, V> implements WindowStore<K, V> {
|
|||
}
|
||||
|
||||
private static class RocksDBWindowStoreIterator<V> implements WindowStoreIterator<V> {
|
||||
private final Serdes<?, V> serdes;
|
||||
private final StateSerdes<?, V> serdes;
|
||||
private final KeyValueIterator<byte[], byte[]>[] iterators;
|
||||
private int index = 0;
|
||||
|
||||
RocksDBWindowStoreIterator(Serdes<?, V> serdes) {
|
||||
RocksDBWindowStoreIterator(StateSerdes<?, V> serdes) {
|
||||
this(serdes, WindowStoreUtils.NO_ITERATORS);
|
||||
}
|
||||
|
||||
RocksDBWindowStoreIterator(Serdes<?, V> serdes, KeyValueIterator<byte[], byte[]>[] iterators) {
|
||||
RocksDBWindowStoreIterator(StateSerdes<?, V> serdes, KeyValueIterator<byte[], byte[]>[] iterators) {
|
||||
this.serdes = serdes;
|
||||
this.iterators = iterators;
|
||||
}
|
||||
|
|
@ -114,7 +114,7 @@ public class RocksDBWindowStore<K, V> implements WindowStore<K, V> {
|
|||
private final long segmentInterval;
|
||||
private final boolean retainDuplicates;
|
||||
private final Segment[] segments;
|
||||
private final Serdes<K, V> serdes;
|
||||
private final StateSerdes<K, V> serdes;
|
||||
private final SimpleDateFormat formatter;
|
||||
private final StoreChangeLogger.ValueGetter<byte[], byte[]> getter;
|
||||
|
||||
|
|
@ -125,7 +125,7 @@ public class RocksDBWindowStore<K, V> implements WindowStore<K, V> {
|
|||
private boolean loggingEnabled = false;
|
||||
private StoreChangeLogger<byte[], byte[]> changeLogger = null;
|
||||
|
||||
public RocksDBWindowStore(String name, long retentionPeriod, int numSegments, boolean retainDuplicates, Serdes<K, V> serdes) {
|
||||
public RocksDBWindowStore(String name, long retentionPeriod, int numSegments, boolean retainDuplicates, StateSerdes<K, V> serdes) {
|
||||
this.name = name;
|
||||
|
||||
// The segment interval must be greater than MIN_SEGMENT_INTERVAL
|
||||
|
|
|
|||
|
|
@ -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<K, V> implements StateStoreSupplier {
|
|||
private final long retentionPeriod;
|
||||
private final boolean retainDuplicates;
|
||||
private final int numSegments;
|
||||
private final Serdes<K, V> serdes;
|
||||
private final StateSerdes<K, V> serdes;
|
||||
private final Time time;
|
||||
|
||||
public RocksDBWindowStoreSupplier(String name, long retentionPeriod, int numSegments, boolean retainDuplicates, Serdes<K, V> serdes, Time time) {
|
||||
public RocksDBWindowStoreSupplier(String name, long retentionPeriod, int numSegments, boolean retainDuplicates, StateSerdes<K, V> serdes, Time time) {
|
||||
this.name = name;
|
||||
this.retentionPeriod = retentionPeriod;
|
||||
this.retainDuplicates = retainDuplicates;
|
||||
|
|
|
|||
|
|
@ -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<K, V> {
|
|||
// TODO: these values should be configurable
|
||||
protected static final int DEFAULT_WRITE_BATCH_SIZE = 100;
|
||||
|
||||
protected final Serdes<K, V> serialization;
|
||||
protected final StateSerdes<K, V> serialization;
|
||||
|
||||
private final String topic;
|
||||
private final int partition;
|
||||
|
|
@ -47,16 +47,16 @@ public class StoreChangeLogger<K, V> {
|
|||
protected Set<K> dirty;
|
||||
protected Set<K> removed;
|
||||
|
||||
public StoreChangeLogger(String storeName, ProcessorContext context, Serdes<K, V> serialization) {
|
||||
public StoreChangeLogger(String storeName, ProcessorContext context, StateSerdes<K, V> serialization) {
|
||||
this(storeName, context, serialization, DEFAULT_WRITE_BATCH_SIZE, DEFAULT_WRITE_BATCH_SIZE);
|
||||
}
|
||||
|
||||
public StoreChangeLogger(String storeName, ProcessorContext context, Serdes<K, V> serialization, int maxDirty, int maxRemoved) {
|
||||
public StoreChangeLogger(String storeName, ProcessorContext context, StateSerdes<K, V> serialization, int maxDirty, int maxRemoved) {
|
||||
this(storeName, context, context.taskId().partition, serialization, maxDirty, maxRemoved);
|
||||
init();
|
||||
}
|
||||
|
||||
protected StoreChangeLogger(String storeName, ProcessorContext context, int partition, Serdes<K, V> serialization, int maxDirty, int maxRemoved) {
|
||||
protected StoreChangeLogger(String storeName, ProcessorContext context, int partition, StateSerdes<K, V> serialization, int maxDirty, int maxRemoved) {
|
||||
this.topic = ProcessorStateManager.storeChangelogTopic(context.applicationId(), storeName);
|
||||
this.context = context;
|
||||
this.partition = partition;
|
||||
|
|
|
|||
|
|
@ -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);
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -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<Integer, String>[] branches;
|
||||
MockProcessorSupplier<Integer, String>[] 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);
|
||||
|
|
|
|||
|
|
@ -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<Integer, String> isMultipleOfThree = new Predicate<Integer, String>() {
|
||||
@Override
|
||||
public boolean test(Integer key, String value) {
|
||||
|
|
@ -52,7 +48,7 @@ public class KStreamFilterTest {
|
|||
MockProcessorSupplier<Integer, String> 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<Integer, String> 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);
|
||||
|
|
|
|||
|
|
@ -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<String, String> 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);
|
||||
|
|
|
|||
|
|
@ -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<Integer, String> 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);
|
||||
|
|
|
|||
|
|
@ -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<String> stringSerde = Serdes.String();
|
||||
final private Serde<Integer> intSerde = Serdes.Integer();
|
||||
|
||||
@Test
|
||||
public void testNumProcesses() {
|
||||
final Serializer<String> stringSerializer = new StringSerializer();
|
||||
final Deserializer<String> stringDeserializer = new StringDeserializer();
|
||||
final Serializer<Integer> integerSerializer = new IntegerSerializer();
|
||||
final Deserializer<Integer> integerDeserializer = new IntegerDeserializer();
|
||||
|
||||
final KStreamBuilder builder = new KStreamBuilder();
|
||||
|
||||
KStream<String, String> source1 = builder.stream(stringDeserializer, stringDeserializer, "topic-1", "topic-2");
|
||||
KStream<String, String> source1 = builder.stream(stringSerde, stringSerde, "topic-1", "topic-2");
|
||||
|
||||
KStream<String, String> source2 = builder.stream(stringDeserializer, stringDeserializer, "topic-3", "topic-4");
|
||||
KStream<String, String> source2 = builder.stream(stringSerde, stringSerde, "topic-3", "topic-4");
|
||||
|
||||
KStream<String, String> stream1 =
|
||||
source1.filter(new Predicate<String, String>() {
|
||||
|
|
@ -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<String, Integer> stream5 = streams2[1].join(streams3[1], new ValueJoiner<Integer, Integer, Integer>() {
|
||||
@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");
|
||||
|
||||
|
|
|
|||
|
|
@ -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<Integer> intSerde = Serdes.Integer();
|
||||
final private Serde<String> stringSerde = Serdes.String();
|
||||
|
||||
private ValueJoiner<String, String, String> joiner = new ValueJoiner<String, String, String>() {
|
||||
@Override
|
||||
|
|
@ -71,10 +67,9 @@ public class KStreamKStreamJoinTest {
|
|||
MockProcessorSupplier<Integer, String> 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<Set<String>> copartitionGroups = builder.copartitionGroups();
|
||||
|
|
@ -177,10 +172,9 @@ public class KStreamKStreamJoinTest {
|
|||
MockProcessorSupplier<Integer, String> 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<Set<String>> copartitionGroups = builder.copartitionGroups();
|
||||
|
|
@ -285,10 +279,9 @@ public class KStreamKStreamJoinTest {
|
|||
MockProcessorSupplier<Integer, String> 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<Set<String>> copartitionGroups = builder.copartitionGroups();
|
||||
|
|
|
|||
|
|
@ -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<Integer> intSerde = Serdes.Integer();
|
||||
final private Serde<String> stringSerde = Serdes.String();
|
||||
|
||||
private ValueJoiner<String, String, String> joiner = new ValueJoiner<String, String, String>() {
|
||||
@Override
|
||||
|
|
@ -71,10 +67,9 @@ public class KStreamKStreamLeftJoinTest {
|
|||
MockProcessorSupplier<Integer, String> 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<Set<String>> copartitionGroups = builder.copartitionGroups();
|
||||
|
|
@ -157,10 +152,9 @@ public class KStreamKStreamLeftJoinTest {
|
|||
MockProcessorSupplier<Integer, String> 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<Set<String>> copartitionGroups = builder.copartitionGroups();
|
||||
|
|
|
|||
|
|
@ -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<Integer> intSerde = new Serdes.IntegerSerde();
|
||||
final private Serde<String> stringSerde = new Serdes.StringSerde();
|
||||
|
||||
private ValueJoiner<String, String, String> joiner = new ValueJoiner<String, String, String>() {
|
||||
@Override
|
||||
|
|
@ -81,8 +77,8 @@ public class KStreamKTableLeftJoinTest {
|
|||
MockProcessorSupplier<Integer, String> 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<Set<String>> copartitionGroups = builder.copartitionGroups();
|
||||
|
|
@ -162,8 +158,8 @@ public class KStreamKTableLeftJoinTest {
|
|||
MockProcessorSupplier<Integer, String> 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);
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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<Integer> intSerde = Serdes.Integer();
|
||||
final private Serde<String> stringSerde = Serdes.String();
|
||||
|
||||
@Test
|
||||
public void testMap() {
|
||||
|
|
@ -50,11 +50,10 @@ public class KStreamMapTest {
|
|||
|
||||
final int[] expectedKeys = new int[]{0, 1, 2, 3};
|
||||
|
||||
KStream<Integer, String> stream;
|
||||
KStream<Integer, String> stream = builder.stream(intSerde, stringSerde, topicName);
|
||||
MockProcessorSupplier<String, Integer> processor;
|
||||
|
||||
processor = new MockProcessorSupplier<>();
|
||||
stream = builder.stream(keyDeserializer, valDeserializer, topicName);
|
||||
stream.map(mapper).process(processor);
|
||||
|
||||
KStreamTestDriver driver = new KStreamTestDriver(builder);
|
||||
|
|
|
|||
|
|
@ -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<Integer> intSerde = Serdes.Integer();
|
||||
final private Serde<String> stringSerde = Serdes.String();
|
||||
|
||||
@Test
|
||||
public void testFlatMapValues() {
|
||||
|
|
@ -51,7 +51,7 @@ public class KStreamMapValuesTest {
|
|||
|
||||
KStream<Integer, String> stream;
|
||||
MockProcessorSupplier<Integer, Integer> 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);
|
||||
|
|
|
|||
|
|
@ -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<Integer> intSerde = Serdes.Integer();
|
||||
|
||||
@Test
|
||||
public void testTransform() {
|
||||
|
|
@ -71,9 +71,8 @@ public class KStreamTransformTest {
|
|||
|
||||
final int[] expectedKeys = {1, 10, 100, 1000};
|
||||
|
||||
KStream<Integer, Integer> stream;
|
||||
MockProcessorSupplier<Integer, Integer> processor = new MockProcessorSupplier<>();
|
||||
stream = builder.stream(keyDeserializer, valDeserializer, topicName);
|
||||
KStream<Integer, Integer> stream = builder.stream(intSerde, intSerde, topicName);
|
||||
stream.transform(transformerSupplier).process(processor);
|
||||
|
||||
KStreamTestDriver driver = new KStreamTestDriver(builder);
|
||||
|
|
|
|||
|
|
@ -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<Integer> intSerde = Serdes.Integer();
|
||||
|
||||
@Test
|
||||
public void testTransform() {
|
||||
|
|
@ -72,7 +72,7 @@ public class KStreamTransformValuesTest {
|
|||
|
||||
KStream<Integer, Integer> stream;
|
||||
MockProcessorSupplier<Integer, Integer> 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);
|
||||
|
|
|
|||
|
|
@ -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<String> strSerializer = new StringSerializer();
|
||||
private final Deserializer<String> strDeserializer = new StringDeserializer();
|
||||
final private Serde<String> strSerde = new Serdes.StringSerde();
|
||||
|
||||
private class StringAdd implements Aggregator<String, String, String> {
|
||||
|
||||
|
|
@ -68,13 +65,11 @@ public class KStreamWindowAggregateTest {
|
|||
final KStreamBuilder builder = new KStreamBuilder();
|
||||
String topic1 = "topic1";
|
||||
|
||||
KStream<String, String> stream1 = builder.stream(strDeserializer, strDeserializer, topic1);
|
||||
KStream<String, String> stream1 = builder.stream(strSerde, strSerde, topic1);
|
||||
KTable<Windowed<String>, String> table2 = stream1.aggregateByKey(new StringInit(), new StringAdd(),
|
||||
HoppingWindows.of("topic1-Canonized").with(10L).every(5L),
|
||||
strSerializer,
|
||||
strSerializer,
|
||||
strDeserializer,
|
||||
strDeserializer);
|
||||
strSerde,
|
||||
strSerde);
|
||||
|
||||
MockProcessorSupplier<Windowed<String>, String> proc2 = new MockProcessorSupplier<>();
|
||||
table2.toStream().process(proc2);
|
||||
|
|
@ -147,24 +142,20 @@ public class KStreamWindowAggregateTest {
|
|||
String topic1 = "topic1";
|
||||
String topic2 = "topic2";
|
||||
|
||||
KStream<String, String> stream1 = builder.stream(strDeserializer, strDeserializer, topic1);
|
||||
KStream<String, String> stream1 = builder.stream(strSerde, strSerde, topic1);
|
||||
KTable<Windowed<String>, String> table1 = stream1.aggregateByKey(new StringInit(), new StringAdd(),
|
||||
HoppingWindows.of("topic1-Canonized").with(10L).every(5L),
|
||||
strSerializer,
|
||||
strSerializer,
|
||||
strDeserializer,
|
||||
strDeserializer);
|
||||
strSerde,
|
||||
strSerde);
|
||||
|
||||
MockProcessorSupplier<Windowed<String>, String> proc1 = new MockProcessorSupplier<>();
|
||||
table1.toStream().process(proc1);
|
||||
|
||||
KStream<String, String> stream2 = builder.stream(strDeserializer, strDeserializer, topic2);
|
||||
KStream<String, String> stream2 = builder.stream(strSerde, strSerde, topic2);
|
||||
KTable<Windowed<String>, String> table2 = stream2.aggregateByKey(new StringInit(), new StringAdd(),
|
||||
HoppingWindows.of("topic2-Canonized").with(10L).every(5L),
|
||||
strSerializer,
|
||||
strSerializer,
|
||||
strDeserializer,
|
||||
strDeserializer);
|
||||
strSerde,
|
||||
strSerde);
|
||||
|
||||
MockProcessorSupplier<Windowed<String>, String> proc2 = new MockProcessorSupplier<>();
|
||||
table2.toStream().process(proc2);
|
||||
|
|
|
|||
|
|
@ -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<String> strSerializer = new StringSerializer();
|
||||
private final Deserializer<String> strDeserializer = new StringDeserializer();
|
||||
final private Serde<String> stringSerde = new Serdes.StringSerde();
|
||||
|
||||
private class StringAdd implements Aggregator<String, String, String> {
|
||||
|
||||
|
|
@ -74,15 +71,12 @@ public class KTableAggregateTest {
|
|||
final KStreamBuilder builder = new KStreamBuilder();
|
||||
String topic1 = "topic1";
|
||||
|
||||
KTable<String, String> table1 = builder.table(strSerializer, strSerializer, strDeserializer, strDeserializer, topic1);
|
||||
KTable<String, String> table2 = table1.<String, String, String>aggregate(new StringInit(), new StringAdd(), new StringRemove(),
|
||||
KTable<String, String> table1 = builder.table(stringSerde, stringSerde, topic1);
|
||||
KTable<String, String> table2 = table1.aggregate(new StringInit(), new StringAdd(), new StringRemove(),
|
||||
new NoOpKeyValueMapper<String, String>(),
|
||||
strSerializer,
|
||||
strSerializer,
|
||||
strSerializer,
|
||||
strDeserializer,
|
||||
strDeserializer,
|
||||
strDeserializer,
|
||||
stringSerde,
|
||||
stringSerde,
|
||||
stringSerde,
|
||||
"topic1-Canonized");
|
||||
|
||||
MockProcessorSupplier<String, String> proc2 = new MockProcessorSupplier<>();
|
||||
|
|
|
|||
|
|
@ -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<String> strSerializer = new StringSerializer();
|
||||
private final Deserializer<String> strDeserializer = new StringDeserializer();
|
||||
private final Serializer<Integer> intSerializer = new IntegerSerializer();
|
||||
private final Deserializer<Integer> intDeserializer = new IntegerDeserializer();
|
||||
final private Serde<Integer> intSerde = new Serdes.IntegerSerde();
|
||||
final private Serde<String> stringSerde = new Serdes.StringSerde();
|
||||
|
||||
@Test
|
||||
public void testKTable() {
|
||||
|
|
@ -51,7 +45,7 @@ public class KTableFilterTest {
|
|||
|
||||
String topic1 = "topic1";
|
||||
|
||||
KTable<String, Integer> table1 = builder.table(strSerializer, intSerializer, strDeserializer, intDeserializer, topic1);
|
||||
KTable<String, Integer> table1 = builder.table(stringSerde, intSerde, topic1);
|
||||
|
||||
KTable<String, Integer> table2 = table1.filter(new Predicate<String, Integer>() {
|
||||
@Override
|
||||
|
|
@ -93,7 +87,7 @@ public class KTableFilterTest {
|
|||
String topic1 = "topic1";
|
||||
|
||||
KTableImpl<String, Integer, Integer> table1 =
|
||||
(KTableImpl<String, Integer, Integer>) builder.table(strSerializer, intSerializer, strDeserializer, intDeserializer, topic1);
|
||||
(KTableImpl<String, Integer, Integer>) builder.table(stringSerde, intSerde, topic1);
|
||||
KTableImpl<String, Integer, Integer> table2 = (KTableImpl<String, Integer, Integer>) table1.filter(
|
||||
new Predicate<String, Integer>() {
|
||||
@Override
|
||||
|
|
@ -112,7 +106,7 @@ public class KTableFilterTest {
|
|||
KTableValueGetterSupplier<String, Integer> getterSupplier2 = table2.valueGetterSupplier();
|
||||
KTableValueGetterSupplier<String, Integer> getterSupplier3 = table3.valueGetterSupplier();
|
||||
|
||||
KStreamTestDriver driver = new KStreamTestDriver(builder, stateDir, null, null, null, null);
|
||||
KStreamTestDriver driver = new KStreamTestDriver(builder, stateDir, null, null);
|
||||
|
||||
KTableValueGetter<String, Integer> getter2 = getterSupplier2.get();
|
||||
KTableValueGetter<String, Integer> getter3 = getterSupplier3.get();
|
||||
|
|
@ -178,7 +172,7 @@ public class KTableFilterTest {
|
|||
String topic1 = "topic1";
|
||||
|
||||
KTableImpl<String, Integer, Integer> table1 =
|
||||
(KTableImpl<String, Integer, Integer>) builder.table(strSerializer, intSerializer, strDeserializer, intDeserializer, topic1);
|
||||
(KTableImpl<String, Integer, Integer>) builder.table(stringSerde, intSerde, topic1);
|
||||
KTableImpl<String, Integer, Integer> table2 = (KTableImpl<String, Integer, Integer>) table1.filter(
|
||||
new Predicate<String, Integer>() {
|
||||
@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<String, Integer, Integer> table1 =
|
||||
(KTableImpl<String, Integer, Integer>) builder.table(strSerializer, intSerializer, strDeserializer, intDeserializer, topic1);
|
||||
(KTableImpl<String, Integer, Integer>) builder.table(stringSerde, intSerde, topic1);
|
||||
KTableImpl<String, Integer, Integer> table2 = (KTableImpl<String, Integer, Integer>) table1.filter(
|
||||
new Predicate<String, Integer>() {
|
||||
@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);
|
||||
|
|
|
|||
|
|
@ -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<String> stringSerde = new Serdes.StringSerde();
|
||||
|
||||
@Test
|
||||
public void testKTable() {
|
||||
final Serializer<String> serializer = new StringSerializer();
|
||||
final Deserializer<String> deserializer = new StringDeserializer();
|
||||
final KStreamBuilder builder = new KStreamBuilder();
|
||||
|
||||
String topic1 = "topic1";
|
||||
String topic2 = "topic2";
|
||||
|
||||
KTable<String, String> table1 = builder.table(serializer, serializer, deserializer, deserializer, topic1);
|
||||
KTable<String, String> table1 = builder.table(stringSerde, stringSerde, topic1);
|
||||
|
||||
MockProcessorSupplier<String, String> proc1 = new MockProcessorSupplier<>();
|
||||
table1.toStream().process(proc1);
|
||||
|
|
@ -74,7 +72,7 @@ public class KTableImplTest {
|
|||
MockProcessorSupplier<String, Integer> proc3 = new MockProcessorSupplier<>();
|
||||
table3.toStream().process(proc3);
|
||||
|
||||
KTable<String, String> table4 = table1.through(topic2, serializer, serializer, deserializer, deserializer);
|
||||
KTable<String, String> table4 = table1.through(topic2, stringSerde, stringSerde);
|
||||
|
||||
MockProcessorSupplier<String, String> 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<String> serializer = new StringSerializer();
|
||||
final Deserializer<String> deserializer = new StringDeserializer();
|
||||
final KStreamBuilder builder = new KStreamBuilder();
|
||||
|
||||
String topic1 = "topic1";
|
||||
String topic2 = "topic2";
|
||||
|
||||
KTableImpl<String, String, String> table1 =
|
||||
(KTableImpl<String, String, String>) builder.table(serializer, serializer, deserializer, deserializer, topic1);
|
||||
(KTableImpl<String, String, String>) builder.table(stringSerde, stringSerde, topic1);
|
||||
KTableImpl<String, String, Integer> table2 = (KTableImpl<String, String, Integer>) table1.mapValues(
|
||||
new ValueMapper<String, Integer>() {
|
||||
@Override
|
||||
|
|
@ -120,14 +116,14 @@ public class KTableImplTest {
|
|||
}
|
||||
});
|
||||
KTableImpl<String, String, String> table4 = (KTableImpl<String, String, String>)
|
||||
table1.through(topic2, serializer, serializer, deserializer, deserializer);
|
||||
table1.through(topic2, stringSerde, stringSerde);
|
||||
|
||||
KTableValueGetterSupplier<String, String> getterSupplier1 = table1.valueGetterSupplier();
|
||||
KTableValueGetterSupplier<String, Integer> getterSupplier2 = table2.valueGetterSupplier();
|
||||
KTableValueGetterSupplier<String, Integer> getterSupplier3 = table3.valueGetterSupplier();
|
||||
KTableValueGetterSupplier<String, String> 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<String> serializer = new StringSerializer();
|
||||
final Deserializer<String> deserializer = new StringDeserializer();
|
||||
|
||||
String topic1 = "topic1";
|
||||
String topic2 = "topic2";
|
||||
|
||||
|
|
@ -234,9 +227,9 @@ public class KTableImplTest {
|
|||
KStreamBuilder builder = new KStreamBuilder();
|
||||
|
||||
KTableImpl<String, String, String> table1 =
|
||||
(KTableImpl<String, String, String>) builder.table(serializer, serializer, deserializer, deserializer, topic1);
|
||||
(KTableImpl<String, String, String>) builder.table(stringSerde, stringSerde, topic1);
|
||||
KTableImpl<String, String, String> table2 =
|
||||
(KTableImpl<String, String, String>) builder.table(serializer, serializer, deserializer, deserializer, topic2);
|
||||
(KTableImpl<String, String, String>) builder.table(stringSerde, stringSerde, topic2);
|
||||
|
||||
KTableImpl<String, String, Integer> table1Mapped = (KTableImpl<String, String, Integer>) table1.mapValues(
|
||||
new ValueMapper<String, Integer>() {
|
||||
|
|
@ -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<String, String, String> table1 =
|
||||
(KTableImpl<String, String, String>) builder.table(serializer, serializer, deserializer, deserializer, topic1);
|
||||
(KTableImpl<String, String, String>) builder.table(stringSerde, stringSerde, topic1);
|
||||
KTableImpl<String, String, String> table2 =
|
||||
(KTableImpl<String, String, String>) builder.table(serializer, serializer, deserializer, deserializer, topic2);
|
||||
(KTableImpl<String, String, String>) builder.table(stringSerde, stringSerde, topic2);
|
||||
|
||||
KTableImpl<String, String, Integer> table1Mapped = (KTableImpl<String, String, Integer>) table1.mapValues(
|
||||
new ValueMapper<String, Integer>() {
|
||||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -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<Integer> intSerde = new Serdes.IntegerSerde();
|
||||
final private Serde<String> stringSerde = new Serdes.StringSerde();
|
||||
|
||||
private ValueJoiner<String, String, String> joiner = new ValueJoiner<String, String, String>() {
|
||||
@Override
|
||||
|
|
@ -80,8 +76,8 @@ public class KTableKTableJoinTest {
|
|||
MockProcessorSupplier<Integer, String> 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<Integer, String> joined;
|
||||
MockProcessorSupplier<Integer, String> 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<Integer, String> joined;
|
||||
MockProcessorSupplier<Integer, String> 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();
|
||||
|
|
|
|||
|
|
@ -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<Integer> intSerde = new Serdes.IntegerSerde();
|
||||
final private Serde<String> stringSerde = new Serdes.StringSerde();
|
||||
|
||||
private ValueJoiner<String, String, String> joiner = new ValueJoiner<String, String, String>() {
|
||||
@Override
|
||||
|
|
@ -83,15 +79,11 @@ public class KTableKTableLeftJoinTest {
|
|||
|
||||
final int[] expectedKeys = new int[]{0, 1, 2, 3};
|
||||
|
||||
KTable<Integer, String> table1;
|
||||
KTable<Integer, String> table2;
|
||||
KTable<Integer, String> joined;
|
||||
KTable<Integer, String> table1 = builder.table(intSerde, stringSerde, topic1);
|
||||
KTable<Integer, String> table2 = builder.table(intSerde, stringSerde, topic2);
|
||||
KTable<Integer, String> joined = table1.leftJoin(table2, joiner);
|
||||
MockProcessorSupplier<Integer, String> 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<Set<String>> copartitionGroups = builder.copartitionGroups();
|
||||
|
|
@ -188,8 +180,8 @@ public class KTableKTableLeftJoinTest {
|
|||
KTable<Integer, String> joined;
|
||||
MockProcessorSupplier<Integer, String> 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<Integer, String> joined;
|
||||
MockProcessorSupplier<Integer, String> 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();
|
||||
|
|
|
|||
|
|
@ -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<Integer> intSerde = new Serdes.IntegerSerde();
|
||||
final private Serde<String> stringSerde = new Serdes.StringSerde();
|
||||
|
||||
private ValueJoiner<String, String, String> joiner = new ValueJoiner<String, String, String>() {
|
||||
@Override
|
||||
|
|
@ -80,8 +76,8 @@ public class KTableKTableOuterJoinTest {
|
|||
MockProcessorSupplier<Integer, String> 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<Integer, String> joined;
|
||||
MockProcessorSupplier<Integer, String> 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<Integer, String> joined;
|
||||
MockProcessorSupplier<Integer, String> 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();
|
||||
|
|
|
|||
|
|
@ -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<String> strSerializer = new StringSerializer();
|
||||
private final Deserializer<String> strDeserializer = new StringDeserializer();
|
||||
final private Serde<String> stringSerde = new Serdes.StringSerde();
|
||||
|
||||
@Test
|
||||
public void testKTable() {
|
||||
|
|
@ -50,7 +47,7 @@ public class KTableMapValuesTest {
|
|||
|
||||
String topic1 = "topic1";
|
||||
|
||||
KTable<String, String> table1 = builder.table(strSerializer, strSerializer, strDeserializer, strDeserializer, topic1);
|
||||
KTable<String, String> table1 = builder.table(stringSerde, stringSerde, topic1);
|
||||
KTable<String, Integer> table2 = table1.mapValues(new ValueMapper<String, Integer>() {
|
||||
@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<String> serializer = new StringSerializer();
|
||||
final Deserializer<String> deserializer = new StringDeserializer();
|
||||
final KStreamBuilder builder = new KStreamBuilder();
|
||||
|
||||
String topic1 = "topic1";
|
||||
String topic2 = "topic2";
|
||||
|
||||
KTableImpl<String, String, String> table1 =
|
||||
(KTableImpl<String, String, String>) builder.table(serializer, serializer, deserializer, deserializer, topic1);
|
||||
(KTableImpl<String, String, String>) builder.table(stringSerde, stringSerde, topic1);
|
||||
KTableImpl<String, String, Integer> table2 = (KTableImpl<String, String, Integer>) table1.mapValues(
|
||||
new ValueMapper<String, Integer>() {
|
||||
@Override
|
||||
|
|
@ -99,14 +94,14 @@ public class KTableMapValuesTest {
|
|||
}
|
||||
});
|
||||
KTableImpl<String, String, String> table4 = (KTableImpl<String, String, String>)
|
||||
table1.through(topic2, serializer, serializer, deserializer, deserializer);
|
||||
table1.through(topic2, stringSerde, stringSerde);
|
||||
|
||||
KTableValueGetterSupplier<String, String> getterSupplier1 = table1.valueGetterSupplier();
|
||||
KTableValueGetterSupplier<String, Integer> getterSupplier2 = table2.valueGetterSupplier();
|
||||
KTableValueGetterSupplier<String, Integer> getterSupplier3 = table3.valueGetterSupplier();
|
||||
KTableValueGetterSupplier<String, String> getterSupplier4 = table4.valueGetterSupplier();
|
||||
|
||||
KStreamTestDriver driver = new KStreamTestDriver(builder, stateDir, null, null, null, null);
|
||||
KStreamTestDriver driver = new KStreamTestDriver(builder, stateDir, null, null);
|
||||
|
||||
KTableValueGetter<String, String> 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<String> serializer = new StringSerializer();
|
||||
final Deserializer<String> deserializer = new StringDeserializer();
|
||||
final KStreamBuilder builder = new KStreamBuilder();
|
||||
|
||||
String topic1 = "topic1";
|
||||
|
||||
KTableImpl<String, String, String> table1 =
|
||||
(KTableImpl<String, String, String>) builder.table(serializer, serializer, deserializer, deserializer, topic1);
|
||||
(KTableImpl<String, String, String>) builder.table(stringSerde, stringSerde, topic1);
|
||||
KTableImpl<String, String, Integer> table2 = (KTableImpl<String, String, Integer>) table1.mapValues(
|
||||
new ValueMapper<String, Integer>() {
|
||||
@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<String> serializer = new StringSerializer();
|
||||
final Deserializer<String> deserializer = new StringDeserializer();
|
||||
final KStreamBuilder builder = new KStreamBuilder();
|
||||
|
||||
String topic1 = "topic1";
|
||||
|
||||
KTableImpl<String, String, String> table1 =
|
||||
(KTableImpl<String, String, String>) builder.table(serializer, serializer, deserializer, deserializer, topic1);
|
||||
(KTableImpl<String, String, String>) builder.table(stringSerde, stringSerde, topic1);
|
||||
KTableImpl<String, String, Integer> table2 = (KTableImpl<String, String, Integer>) table1.mapValues(
|
||||
new ValueMapper<String, Integer>() {
|
||||
@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());
|
||||
|
|
|
|||
|
|
@ -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<String> strSerializer = new StringSerializer();
|
||||
private final Deserializer<String> strDeserializer = new StringDeserializer();
|
||||
final private Serde<String> stringSerde = new Serdes.StringSerde();
|
||||
|
||||
@Test
|
||||
public void testKTable() {
|
||||
|
|
@ -47,7 +44,7 @@ public class KTableSourceTest {
|
|||
|
||||
String topic1 = "topic1";
|
||||
|
||||
KTable<String, String> table1 = builder.table(strSerializer, strSerializer, strDeserializer, strDeserializer, topic1);
|
||||
KTable<String, String> table1 = builder.table(stringSerde, stringSerde, topic1);
|
||||
|
||||
MockProcessorSupplier<String, String> proc1 = new MockProcessorSupplier<>();
|
||||
table1.toStream().process(proc1);
|
||||
|
|
@ -72,12 +69,11 @@ public class KTableSourceTest {
|
|||
|
||||
String topic1 = "topic1";
|
||||
|
||||
KTableImpl<String, String, String> table1 = (KTableImpl<String, String, String>)
|
||||
builder.table(strSerializer, strSerializer, strDeserializer, strDeserializer, topic1);
|
||||
KTableImpl<String, String, String> table1 = (KTableImpl<String, String, String>) builder.table(stringSerde, stringSerde, topic1);
|
||||
|
||||
KTableValueGetterSupplier<String, String> getterSupplier1 = table1.valueGetterSupplier();
|
||||
|
||||
KStreamTestDriver driver = new KStreamTestDriver(builder, stateDir, null, null, null, null);
|
||||
KStreamTestDriver driver = new KStreamTestDriver(builder, stateDir, null, null);
|
||||
|
||||
KTableValueGetter<String, String> getter1 = getterSupplier1.get();
|
||||
getter1.init(driver.context());
|
||||
|
|
@ -123,14 +119,13 @@ public class KTableSourceTest {
|
|||
|
||||
String topic1 = "topic1";
|
||||
|
||||
KTableImpl<String, String, String> table1 = (KTableImpl<String, String, String>)
|
||||
builder.table(strSerializer, strSerializer, strDeserializer, strDeserializer, topic1);
|
||||
KTableImpl<String, String, String> table1 = (KTableImpl<String, String, String>) builder.table(stringSerde, stringSerde, topic1);
|
||||
|
||||
MockProcessorSupplier<String, Integer> 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<String, String, String> table1 = (KTableImpl<String, String, String>)
|
||||
builder.table(strSerializer, strSerializer, strDeserializer, strDeserializer, topic1);
|
||||
KTableImpl<String, String, String> table1 = (KTableImpl<String, String, String>) 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");
|
||||
|
|
|
|||
|
|
@ -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<PartitionInfo> 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<Integer> windowedSerializer = new WindowedSerializer<>(keySerializer);
|
||||
WindowedSerializer<Integer> windowedSerializer = new WindowedSerializer<>(intSerializer);
|
||||
WindowedStreamPartitioner<Integer, String> 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);
|
||||
|
||||
|
|
|
|||
|
|
@ -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);
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -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());
|
||||
}
|
||||
});
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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());
|
||||
}
|
||||
};
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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());
|
||||
}
|
||||
});
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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());
|
||||
}
|
||||
};
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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<String, Integer> source = builder.stream(stringDeserializer, integerDeserializer, "data");
|
||||
KStream<String, Integer> source = builder.stream(stringSerde, intSerde, "data");
|
||||
|
||||
source.to("echo", stringSerializer, integerSerializer);
|
||||
source.to("echo", stringSerde, intSerde);
|
||||
|
||||
KStream<String, Integer> data = source.filter(new Predicate<String, Integer>() {
|
||||
@Override
|
||||
|
|
@ -125,15 +119,13 @@ public class SmokeTestClient extends SmokeTestUtil {
|
|||
}
|
||||
},
|
||||
UnlimitedWindows.of("uwin-min"),
|
||||
stringSerializer,
|
||||
integerSerializer,
|
||||
stringDeserializer,
|
||||
integerDeserializer
|
||||
stringSerde,
|
||||
intSerde
|
||||
).toStream().map(
|
||||
new Unwindow<String, Integer>()
|
||||
).to("min", stringSerializer, integerSerializer);
|
||||
).to("min", stringSerde, intSerde);
|
||||
|
||||
KTable<String, Integer> minTable = builder.table(stringSerializer, integerSerializer, stringDeserializer, integerDeserializer, "min");
|
||||
KTable<String, Integer> minTable = builder.table(stringSerde, intSerde, "min");
|
||||
minTable.toStream().process(SmokeTestUtil.<Integer>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<String, Integer>()
|
||||
).to("max", stringSerializer, integerSerializer);
|
||||
).to("max", stringSerde, intSerde);
|
||||
|
||||
KTable<String, Integer> maxTable = builder.table(stringSerializer, integerSerializer, stringDeserializer, integerDeserializer, "max");
|
||||
KTable<String, Integer> maxTable = builder.table(stringSerde, intSerde, "max");
|
||||
maxTable.toStream().process(SmokeTestUtil.<Integer>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<String, Long>()
|
||||
).to("sum", stringSerializer, longSerializer);
|
||||
).to("sum", stringSerde, longSerde);
|
||||
|
||||
|
||||
KTable<String, Long> sumTable = builder.table(stringSerializer, longSerializer, stringDeserializer, longDeserializer, "sum");
|
||||
KTable<String, Long> sumTable = builder.table(stringSerde, longSerde, "sum");
|
||||
sumTable.toStream().process(SmokeTestUtil.<Long>printProcessorSupplier("sum"));
|
||||
|
||||
// cnt
|
||||
data.countByKey(
|
||||
UnlimitedWindows.of("uwin-cnt"),
|
||||
stringSerializer,
|
||||
stringDeserializer
|
||||
stringSerde
|
||||
).toStream().map(
|
||||
new Unwindow<String, Long>()
|
||||
).to("cnt", stringSerializer, longSerializer);
|
||||
).to("cnt", stringSerde, longSerde);
|
||||
|
||||
KTable<String, Long> cntTable = builder.table(stringSerializer, longSerializer, stringDeserializer, longDeserializer, "cnt");
|
||||
KTable<String, Long> cntTable = builder.table(stringSerde, longSerde, "cnt");
|
||||
cntTable.toStream().process(SmokeTestUtil.<Long>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<Windowed<String>, Long, KeyValue<String, Long>>() {
|
||||
@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);
|
||||
}
|
||||
|
|
|
|||
|
|
@ -157,7 +157,7 @@ public class SmokeTestDriver extends SmokeTestUtil {
|
|||
}
|
||||
|
||||
ProducerRecord<byte[], byte[]> 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<byte[], byte[]> 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());
|
||||
|
|
|
|||
|
|
@ -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<String> stringSerializer = new StringSerializer();
|
||||
public static Serde<String> stringSerde = Serdes.String();
|
||||
|
||||
public static Deserializer<String> stringDeserializer = new StringDeserializer();
|
||||
public static Serde<Integer> intSerde = Serdes.Integer();
|
||||
|
||||
public static Serializer<Integer> integerSerializer = new IntegerSerializer();
|
||||
public static Serde<Long> longSerde = Serdes.Long();
|
||||
|
||||
public static Deserializer<Integer> integerDeserializer = new IntegerDeserializer();
|
||||
|
||||
public static Serializer<Long> longSerializer = new LongSerializer();
|
||||
|
||||
public static Deserializer<Long> longDeserializer = new LongDeserializer();
|
||||
|
||||
public static Serializer<Double> doubleSerializer = new Serializer<Double>() {
|
||||
|
||||
@Override
|
||||
public void configure(Map<String, ?> 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<Double> doubleDeserializer = new Deserializer<Double>() {
|
||||
|
||||
@Override
|
||||
public void configure(Map<String, ?> 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<Double> doubleSerde = Serdes.Double();
|
||||
|
||||
public static File createDir(String path) throws Exception {
|
||||
File dir = new File(path);
|
||||
|
|
|
|||
|
|
@ -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<K, V> {
|
||||
|
||||
private static <T> Serializer<T> unusableSerializer() {
|
||||
return new Serializer<T>() {
|
||||
@Override
|
||||
public void configure(Map<String, ?> 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 <T> Deserializer<T> unusableDeserializer() {
|
||||
return new Deserializer<T>() {
|
||||
@Override
|
||||
public void configure(Map<String, ?> 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 <em>unusable</em> 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 <K, V> KeyValueStoreTestDriver<K, V> create() {
|
||||
Serializer<K> keySerializer = unusableSerializer();
|
||||
Deserializer<K> keyDeserializer = unusableDeserializer();
|
||||
Serializer<V> valueSerializer = unusableSerializer();
|
||||
Deserializer<V> valueDeserializer = unusableDeserializer();
|
||||
Serdes<K, V> serdes = new Serdes<K, V>("unexpected", keySerializer, keyDeserializer, valueSerializer, valueDeserializer);
|
||||
return new KeyValueStoreTestDriver<K, V>(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<K, V> {
|
|||
* @return the test driver; never null
|
||||
*/
|
||||
public static <K, V> KeyValueStoreTestDriver<K, V> create(Class<K> keyClass, Class<V> valueClass) {
|
||||
Serdes<K, V> serdes = Serdes.withBuiltinTypes("unexpected", keyClass, valueClass);
|
||||
StateSerdes<K, V> serdes = StateSerdes.withBuiltinTypes("unexpected", keyClass, valueClass);
|
||||
return new KeyValueStoreTestDriver<K, V>(serdes);
|
||||
}
|
||||
|
||||
|
|
@ -215,7 +164,9 @@ public class KeyValueStoreTestDriver<K, V> {
|
|||
Deserializer<K> keyDeserializer,
|
||||
Serializer<V> valueSerializer,
|
||||
Deserializer<V> valueDeserializer) {
|
||||
Serdes<K, V> serdes = new Serdes<K, V>("unexpected", keySerializer, keyDeserializer, valueSerializer, valueDeserializer);
|
||||
StateSerdes<K, V> serdes = new StateSerdes<K, V>("unexpected",
|
||||
Serdes.serdeFrom(keySerializer, keyDeserializer),
|
||||
Serdes.serdeFrom(valueSerializer, valueDeserializer));
|
||||
return new KeyValueStoreTestDriver<K, V>(serdes);
|
||||
}
|
||||
|
||||
|
|
@ -237,7 +188,7 @@ public class KeyValueStoreTestDriver<K, V> {
|
|||
private final RecordCollector recordCollector;
|
||||
private File stateDir = null;
|
||||
|
||||
protected KeyValueStoreTestDriver(final Serdes<K, V> serdes) {
|
||||
protected KeyValueStoreTestDriver(final StateSerdes<K, V> serdes) {
|
||||
ByteArraySerializer rawSerializer = new ByteArraySerializer();
|
||||
Producer<byte[], byte[]> producer = new MockProducer<>(true, rawSerializer, rawSerializer);
|
||||
|
||||
|
|
@ -276,13 +227,10 @@ public class KeyValueStoreTestDriver<K, V> {
|
|||
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<K, V> {
|
|||
}
|
||||
}
|
||||
|
||||
private void restoreEntries(StateRestoreCallback func, Serdes<K, V> serdes) {
|
||||
private void restoreEntries(StateRestoreCallback func, StateSerdes<K, V> serdes) {
|
||||
for (KeyValue<K, V> entry : restorableEntries) {
|
||||
if (entry != null) {
|
||||
byte[] rawKey = serdes.rawKey(entry.key);
|
||||
|
|
|
|||
|
|
@ -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<K> keySer = (Serializer<K>) context.keySerializer();
|
||||
Deserializer<K> keyDeser = (Deserializer<K>) context.keyDeserializer();
|
||||
Serializer<V> valSer = (Serializer<V>) context.valueSerializer();
|
||||
Deserializer<V> valDeser = (Deserializer<V>) 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();
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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<K> keySer = (Serializer<K>) context.keySerializer();
|
||||
Deserializer<K> keyDeser = (Deserializer<K>) context.keyDeserializer();
|
||||
Serializer<V> valSer = (Serializer<V>) context.valueSerializer();
|
||||
Deserializer<V> valDeser = (Deserializer<V>) 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();
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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<K> keySer = (Serializer<K>) context.keySerializer();
|
||||
Deserializer<K> keyDeser = (Deserializer<K>) context.keyDeserializer();
|
||||
Serializer<V> valSer = (Serializer<V>) context.valueSerializer();
|
||||
Deserializer<V> valDeser = (Deserializer<V>) 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();
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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<byte[]> 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<Integer, String> serdes = Serdes.withBuiltinTypes("", Integer.class, String.class);
|
||||
private final StateSerdes<Integer, String> serdes = StateSerdes.withBuiltinTypes("", Integer.class, String.class);
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
protected <K, V> WindowStore<K, V> createWindowStore(ProcessorContext context, Serdes<K, V> serdes) {
|
||||
protected <K, V> WindowStore<K, V> createWindowStore(ProcessorContext context, StateSerdes<K, V> serdes) {
|
||||
StateStoreSupplier supplier = new RocksDBWindowStoreSupplier<>(windowName, retentionPeriod, numSegments, true, serdes, null);
|
||||
|
||||
WindowStore<K, V> store = (WindowStore<K, V>) supplier.get();
|
||||
|
|
@ -74,7 +73,7 @@ public class RocksDBWindowStoreTest {
|
|||
File baseDir = Files.createTempDirectory("test").toFile();
|
||||
try {
|
||||
final List<KeyValue<byte[], byte[]>> changeLog = new ArrayList<>();
|
||||
Producer<byte[], byte[]> producer = new MockProducer<>(true, byteArraySerializer, byteArraySerializer);
|
||||
Producer<byte[], byte[]> producer = new MockProducer<>(true, byteArraySerde.serializer(), byteArraySerde.serializer());
|
||||
RecordCollector recordCollector = new RecordCollector(producer) {
|
||||
@Override
|
||||
public <K1, V1> void send(ProducerRecord<K1, V1> record, Serializer<K1> keySerializer, Serializer<V1> valueSerializer) {
|
||||
|
|
@ -87,7 +86,7 @@ public class RocksDBWindowStoreTest {
|
|||
|
||||
MockProcessorContext context = new MockProcessorContext(
|
||||
null, baseDir,
|
||||
byteArraySerializer, byteArrayDeserializer, byteArraySerializer, byteArrayDeserializer,
|
||||
byteArraySerde, byteArraySerde,
|
||||
recordCollector);
|
||||
|
||||
WindowStore<Integer, String> store = createWindowStore(context, serdes);
|
||||
|
|
@ -170,7 +169,7 @@ public class RocksDBWindowStoreTest {
|
|||
File baseDir = Files.createTempDirectory("test").toFile();
|
||||
try {
|
||||
final List<KeyValue<byte[], byte[]>> changeLog = new ArrayList<>();
|
||||
Producer<byte[], byte[]> producer = new MockProducer<>(true, byteArraySerializer, byteArraySerializer);
|
||||
Producer<byte[], byte[]> producer = new MockProducer<>(true, byteArraySerde.serializer(), byteArraySerde.serializer());
|
||||
RecordCollector recordCollector = new RecordCollector(producer) {
|
||||
@Override
|
||||
public <K1, V1> void send(ProducerRecord<K1, V1> record, Serializer<K1> keySerializer, Serializer<V1> valueSerializer) {
|
||||
|
|
@ -183,7 +182,7 @@ public class RocksDBWindowStoreTest {
|
|||
|
||||
MockProcessorContext context = new MockProcessorContext(
|
||||
null, baseDir,
|
||||
byteArraySerializer, byteArrayDeserializer, byteArraySerializer, byteArrayDeserializer,
|
||||
byteArraySerde, byteArraySerde,
|
||||
recordCollector);
|
||||
|
||||
WindowStore<Integer, String> store = createWindowStore(context, serdes);
|
||||
|
|
@ -266,7 +265,7 @@ public class RocksDBWindowStoreTest {
|
|||
File baseDir = Files.createTempDirectory("test").toFile();
|
||||
try {
|
||||
final List<KeyValue<byte[], byte[]>> changeLog = new ArrayList<>();
|
||||
Producer<byte[], byte[]> producer = new MockProducer<>(true, byteArraySerializer, byteArraySerializer);
|
||||
Producer<byte[], byte[]> producer = new MockProducer<>(true, byteArraySerde.serializer(), byteArraySerde.serializer());
|
||||
RecordCollector recordCollector = new RecordCollector(producer) {
|
||||
@Override
|
||||
public <K1, V1> void send(ProducerRecord<K1, V1> record, Serializer<K1> keySerializer, Serializer<V1> valueSerializer) {
|
||||
|
|
@ -279,7 +278,7 @@ public class RocksDBWindowStoreTest {
|
|||
|
||||
MockProcessorContext context = new MockProcessorContext(
|
||||
null, baseDir,
|
||||
byteArraySerializer, byteArrayDeserializer, byteArraySerializer, byteArrayDeserializer,
|
||||
byteArraySerde, byteArraySerde,
|
||||
recordCollector);
|
||||
|
||||
WindowStore<Integer, String> store = createWindowStore(context, serdes);
|
||||
|
|
@ -362,7 +361,7 @@ public class RocksDBWindowStoreTest {
|
|||
File baseDir = Files.createTempDirectory("test").toFile();
|
||||
try {
|
||||
final List<KeyValue<byte[], byte[]>> changeLog = new ArrayList<>();
|
||||
Producer<byte[], byte[]> producer = new MockProducer<>(true, byteArraySerializer, byteArraySerializer);
|
||||
Producer<byte[], byte[]> producer = new MockProducer<>(true, byteArraySerde.serializer(), byteArraySerde.serializer());
|
||||
RecordCollector recordCollector = new RecordCollector(producer) {
|
||||
@Override
|
||||
public <K1, V1> void send(ProducerRecord<K1, V1> record, Serializer<K1> keySerializer, Serializer<V1> valueSerializer) {
|
||||
|
|
@ -375,7 +374,7 @@ public class RocksDBWindowStoreTest {
|
|||
|
||||
MockProcessorContext context = new MockProcessorContext(
|
||||
null, baseDir,
|
||||
byteArraySerializer, byteArrayDeserializer, byteArraySerializer, byteArrayDeserializer,
|
||||
byteArraySerde, byteArraySerde,
|
||||
recordCollector);
|
||||
|
||||
WindowStore<Integer, String> store = createWindowStore(context, serdes);
|
||||
|
|
@ -421,7 +420,7 @@ public class RocksDBWindowStoreTest {
|
|||
File baseDir = Files.createTempDirectory("test").toFile();
|
||||
try {
|
||||
final List<KeyValue<byte[], byte[]>> changeLog = new ArrayList<>();
|
||||
Producer<byte[], byte[]> producer = new MockProducer<>(true, byteArraySerializer, byteArraySerializer);
|
||||
Producer<byte[], byte[]> producer = new MockProducer<>(true, byteArraySerde.serializer(), byteArraySerde.serializer());
|
||||
RecordCollector recordCollector = new RecordCollector(producer) {
|
||||
@Override
|
||||
public <K1, V1> void send(ProducerRecord<K1, V1> record, Serializer<K1> keySerializer, Serializer<V1> valueSerializer) {
|
||||
|
|
@ -434,7 +433,7 @@ public class RocksDBWindowStoreTest {
|
|||
|
||||
MockProcessorContext context = new MockProcessorContext(
|
||||
null, baseDir,
|
||||
byteArraySerializer, byteArrayDeserializer, byteArraySerializer, byteArrayDeserializer,
|
||||
byteArraySerde, byteArraySerde,
|
||||
recordCollector);
|
||||
|
||||
WindowStore<Integer, String> store = createWindowStore(context, serdes);
|
||||
|
|
@ -538,7 +537,7 @@ public class RocksDBWindowStoreTest {
|
|||
|
||||
File baseDir = Files.createTempDirectory("test").toFile();
|
||||
try {
|
||||
Producer<byte[], byte[]> producer = new MockProducer<>(true, byteArraySerializer, byteArraySerializer);
|
||||
Producer<byte[], byte[]> producer = new MockProducer<>(true, byteArraySerde.serializer(), byteArraySerde.serializer());
|
||||
RecordCollector recordCollector = new RecordCollector(producer) {
|
||||
@Override
|
||||
public <K1, V1> void send(ProducerRecord<K1, V1> record, Serializer<K1> keySerializer, Serializer<V1> valueSerializer) {
|
||||
|
|
@ -551,7 +550,7 @@ public class RocksDBWindowStoreTest {
|
|||
|
||||
MockProcessorContext context = new MockProcessorContext(
|
||||
null, baseDir,
|
||||
byteArraySerializer, byteArrayDeserializer, byteArraySerializer, byteArrayDeserializer,
|
||||
byteArraySerde, byteArraySerde,
|
||||
recordCollector);
|
||||
|
||||
WindowStore<Integer, String> store = createWindowStore(context, serdes);
|
||||
|
|
@ -587,7 +586,7 @@ public class RocksDBWindowStoreTest {
|
|||
|
||||
File baseDir2 = Files.createTempDirectory("test").toFile();
|
||||
try {
|
||||
Producer<byte[], byte[]> producer = new MockProducer<>(true, byteArraySerializer, byteArraySerializer);
|
||||
Producer<byte[], byte[]> producer = new MockProducer<>(true, byteArraySerde.serializer(), byteArraySerde.serializer());
|
||||
RecordCollector recordCollector = new RecordCollector(producer) {
|
||||
@Override
|
||||
public <K1, V1> void send(ProducerRecord<K1, V1> record, Serializer<K1> keySerializer, Serializer<V1> valueSerializer) {
|
||||
|
|
@ -600,7 +599,7 @@ public class RocksDBWindowStoreTest {
|
|||
|
||||
MockProcessorContext context = new MockProcessorContext(
|
||||
null, baseDir,
|
||||
byteArraySerializer, byteArrayDeserializer, byteArraySerializer, byteArrayDeserializer,
|
||||
byteArraySerde, byteArraySerde,
|
||||
recordCollector);
|
||||
|
||||
WindowStore<Integer, String> store = createWindowStore(context, serdes);
|
||||
|
|
@ -642,7 +641,7 @@ public class RocksDBWindowStoreTest {
|
|||
public void testSegmentMaintenance() throws IOException {
|
||||
File baseDir = Files.createTempDirectory("test").toFile();
|
||||
try {
|
||||
Producer<byte[], byte[]> producer = new MockProducer<>(true, byteArraySerializer, byteArraySerializer);
|
||||
Producer<byte[], byte[]> producer = new MockProducer<>(true, byteArraySerde.serializer(), byteArraySerde.serializer());
|
||||
RecordCollector recordCollector = new RecordCollector(producer) {
|
||||
@Override
|
||||
public <K1, V1> void send(ProducerRecord<K1, V1> record, Serializer<K1> keySerializer, Serializer<V1> valueSerializer) {
|
||||
|
|
@ -652,7 +651,7 @@ public class RocksDBWindowStoreTest {
|
|||
|
||||
MockProcessorContext context = new MockProcessorContext(
|
||||
null, baseDir,
|
||||
byteArraySerializer, byteArrayDeserializer, byteArraySerializer, byteArrayDeserializer,
|
||||
byteArraySerde, byteArraySerde,
|
||||
recordCollector);
|
||||
|
||||
WindowStore<Integer, String> store = createWindowStore(context, serdes);
|
||||
|
|
@ -745,7 +744,7 @@ public class RocksDBWindowStoreTest {
|
|||
public void testInitialLoading() throws IOException {
|
||||
File baseDir = Files.createTempDirectory("test").toFile();
|
||||
try {
|
||||
Producer<byte[], byte[]> producer = new MockProducer<>(true, byteArraySerializer, byteArraySerializer);
|
||||
Producer<byte[], byte[]> producer = new MockProducer<>(true, byteArraySerde.serializer(), byteArraySerde.serializer());
|
||||
RecordCollector recordCollector = new RecordCollector(producer) {
|
||||
@Override
|
||||
public <K1, V1> void send(ProducerRecord<K1, V1> record, Serializer<K1> keySerializer, Serializer<V1> 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);
|
||||
|
|
|
|||
|
|
@ -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<Integer, String> logged = new HashMap<>();
|
||||
private final Map<Integer, String> 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<Integer, String> changeLogger = new StoreChangeLogger<>(topic, context, Serdes.withBuiltinTypes(topic, Integer.class, String.class), 3, 3);
|
||||
private final StoreChangeLogger<Integer, String> changeLogger = new StoreChangeLogger<>(topic, context, StateSerdes.withBuiltinTypes(topic, Integer.class, String.class), 3, 3);
|
||||
|
||||
private final StoreChangeLogger<byte[], byte[]> rawChangeLogger = new RawStoreChangeLogger(topic, context, 3, 3);
|
||||
|
||||
|
|
|
|||
|
|
@ -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();
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
|
|||
Loading…
Reference in New Issue