Hmm. That's odd as the aggregation works ok if I use a String value for the key (and the corresponding String serde).
This error only started occurring when I tried to substitute my 'custom' key for the original String. On Tue, Dec 6, 2016 at 12:24 PM, Radek Gruchalski <ra...@gruchalski.com> wrote: > Yeah, I knew that already, this part of the error: > > > > >>> > > org.apache.kafka.streams.processor.internals. > > RecordCollector.send( > > > >>> > RecordCollector.java:73) > > points to this line: https://github.com/apache/kafka/blob/0.10.1/ > streams/src/main/java/org/apache/kafka/streams/processor/internals/ > RecordCollector.java#L73 > > which means that your error happens on the value, not the key. > > – > Best regards, > Radek Gruchalski > ra...@gruchalski.com > > > On December 6, 2016 at 9:18:53 PM, Jon Yeargers (jon.yearg...@cedexis.com) > wrote: > > 0.10.1.0 > > On Tue, Dec 6, 2016 at 11:11 AM, Radek Gruchalski <ra...@gruchalski.com> > wrote: > > > Jon, > > > > Are you using 0.10.1 or 0.10.0.1? > > > > – > > Best regards, > > Radek Gruchalski > > ra...@gruchalski.com > > > > > > On December 6, 2016 at 7:55:30 PM, Damian Guy (damian....@gmail.com) > > wrote: > > > > Hi Jon, > > > > At a glance the code looks ok, i.e, i believe the aggregate() should > have > > picked up the default Serde set in your StreamsConfig. However, you > could > > try adding the Serdes to the groupBy(..) > > > > i.e., > > rtRekey.groupByKey(new AggKeySerDe(), yourValueSerde).aggregate(...) > > > > Thanks, > > Damian > > > > On Tue, 6 Dec 2016 at 18:42 Jon Yeargers <jon.yearg...@cedexis.com> > wrote: > > > > > It's just a bunch of public 'int' and 'String' values. There's an > empty > > > constructor and a copy constructor. > > > > > > For functions I override 'equals' and the requirements for 'serde' > > (close, > > > configure, serializer and deserializer). > > > > > > @Override > > > public Serializer serializer() { > > > JsonSerializer<AggKey> jsonSerializer = new JsonSerializer<>(); > > > return jsonSerializer; > > > } > > > > > > @Override > > > public Deserializer deserializer() { > > > JsonDeserializer<AggKey> jsonDeserializer = new > > > JsonDeserializer<>(); > > > return jsonDeserializer; > > > } > > > > > > > > > > > > > > > Which relates to: > > > > > > public class JsonSerializer<T> implements Serializer<T> { > > > > > > private Gson gson = new Gson(); > > > > > > @Override > > > public void configure(Map<String, ?> map, boolean b) { > > > > > > } > > > > > > @Override > > > public byte[] serialize(String topic, T t) { > > > return gson.toJson(t).getBytes(Charset.forName("UTF-8")); > > > } > > > > > > @Override > > > public void close() { > > > > > > } > > > } > > > > > > > > > > > > public class JsonDeserializer<T> implements Deserializer<T> { > > > > > > private Gson gson = new Gson(); > > > private Class<T> deserializedClass; > > > > > > public JsonDeserializer(Class<T> deserializedClass) { > > > this.deserializedClass = deserializedClass; > > > } > > > > > > public JsonDeserializer() { > > > } > > > > > > @Override > > > @SuppressWarnings("unchecked") > > > public void configure(Map<String, ?> map, boolean b) { > > > if(deserializedClass == null) { > > > deserializedClass = (Class<T>) map.get("serializedClass"); > > > } > > > } > > > > > > @Override > > > public T deserialize(String s, byte[] bytes) { > > > if(bytes == null){ > > > return null; > > > } > > > > > > return gson.fromJson(new String(bytes),deserializedClass); > > > > > > } > > > > > > @Override > > > public void close() { > > > > > > } > > > } > > > > > > On Tue, Dec 6, 2016 at 10:37 AM, Radek Gruchalski < > ra...@gruchalski.com> > > > wrote: > > > > > > > Do you mind sharing the code of AggKey class? > > > > > > > > – > > > > Best regards, > > > > Radek Gruchalski > > > > ra...@gruchalski.com > > > > > > > > > > > > On December 6, 2016 at 7:26:51 PM, Jon Yeargers ( > > > jon.yearg...@cedexis.com) > > > > wrote: > > > > > > > > The 2nd. > > > > > > > > On Tue, Dec 6, 2016 at 10:01 AM, Radek Gruchalski < > > ra...@gruchalski.com> > > > > wrote: > > > > > > > >> Is the error happening at this stage? > > > >> > > > >> KStream<AggKey, RtDetailLogLine> rtRekey = rtDetailLines.map((key, > > > value) > > > >> -> new KeyValue<>(new AggKey(value), value)); > > > >> > > > >> or here: > > > >> > > > >> KTable<Windowed<AggKey>, BqRtDetailLogLine_aggregate> ktRtDetail = > > > >> rtRekey.groupByKey().aggregate( > > > >> BqRtDetailLogLine_aggregate::new, > > > >> new PRTAggregate(), > > > >> TimeWindows.of(60 * 60 * 1000L), > > > >> collectorSerde, "prt_minute_agg_stream"); > > > >> > > > >> – > > > >> > > > >> Best regards, > > > >> Radek Gruchalski > > > >> ra...@gruchalski.com > > > >> > > > >> > > > >> On December 6, 2016 at 6:47:38 PM, Jon Yeargers ( > > > jon.yearg...@cedexis.com) > > > >> wrote: > > > >> > > > >> If I comment out the aggregation step and just .print the .map step > I > > > >> don't hit the error. It's coming from aggregating the non-String > key. > > > >> > > > >> On Tue, Dec 6, 2016 at 9:44 AM, Radek Gruchalski < > > ra...@gruchalski.com> > > > >> wrote: > > > >> > > > >>> Jon, > > > >>> > > > >>> Looking at your code: > > > >>> > > > >>> config.put(StreamsConfig.VALUE_SERDE_CLASS_CONFIG, > > > >>> Serdes.String().getClass().getName()); > > > >>> > > > >>> and later: > > > >>> > > > >>> KStream<String, RtDetailLogLine> rtDetailLines = > > > >>> kStreamBuilder.stream(stringSerde, prtRecordSerde, TOPIC); > > > >>> > > > >>> Is RtDetailLogLine inheriting from String? It is not, as the error > > > >>> suggests. > > > >>> You may have to write your own Serializer / Deserializer for > > > >>> RtDetailLogLine. > > > >>> > > > >>> – > > > >>> Best regards, > > > >>> Radek Gruchalski > > > >>> ra...@gruchalski.com > > > >>> > > > >>> > > > >>> On December 6, 2016 at 6:28:23 PM, Jon Yeargers ( > > > >>> jon.yearg...@cedexis.com) wrote: > > > >>> > > > >>> Using 0.10.1.0 > > > >>> > > > >>> This is my topology: > > > >>> > > > >>> Properties config = new Properties(); > > > >>> config.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, BROKER_IP); > > > >>> config.put(StreamsConfig.ZOOKEEPER_CONNECT_CONFIG, ZOOKEEPER_IP); > > > >>> config.put(StreamsConfig.APPLICATION_ID_CONFIG, "PRTMinuteAgg" ); > > > >>> config.put(StreamsConfig.KEY_SERDE_CLASS_CONFIG, > > > >>> AggKey.class.getName()); > > > >>> config.put(StreamsConfig.VALUE_SERDE_CLASS_CONFIG, > > > >>> Serdes.String().getClass().getName()); > > > >>> > > > >>> JsonSerializer<BqRtDetailLogLine_aggregate> sumRecordsSerializer > = > > new > > > >>> JsonSerializer<>(); > > > >>> JsonDeserializer<BqRtDetailLogLine_aggregate> > > sumRecordsDeserializer = > > > >>> new > > > >>> JsonDeserializer<>(BqRtDetailLogLine_aggregate.class); > > > >>> Serde<BqRtDetailLogLine_aggregate> collectorSerde = > > > >>> Serdes.serdeFrom(sumRecordsSerializer,sumRecordsDeserializer); > > > >>> > > > >>> StringSerializer stringSerializer = new StringSerializer(); > > > >>> StringDeserializer stringDeserializer = new StringDeserializer(); > > > >>> Serde<String> stringSerde = > > > >>> Serdes.serdeFrom(stringSerializer,stringDeserializer); > > > >>> > > > >>> JsonDeserializer<RtDetailLogLine> prtRecordDeserializer = new > > > >>> JsonDeserializer<>(RtDetailLogLine.class); > > > >>> JsonSerializer<RtDetailLogLine> prtRecordJsonSerializer = new > > > >>> JsonSerializer<>(); > > > >>> Serde<RtDetailLogLine> prtRecordSerde = > > > >>> Serdes.serdeFrom(prtRecordJsonSerializer,prtRecordDeserializer); > > > >>> > > > >>> KStreamBuilder kStreamBuilder = new KStreamBuilder(); > > > >>> > > > >>> KStream<String, RtDetailLogLine> rtDetailLines = > > > >>> kStreamBuilder.stream(stringSerde, prtRecordSerde, TOPIC); > > > >>> > > > >>> // change the keying > > > >>> KStream<AggKey, RtDetailLogLine> rtRekey = rtDetailLines.map((key, > > > value) > > > >>> -> new KeyValue<>(new AggKey(value), value)); > > > >>> > > > >>> KTable<Windowed<AggKey>, BqRtDetailLogLine_aggregate> ktRtDetail = > > > >>> rtRekey.groupByKey().aggregate( > > > >>> BqRtDetailLogLine_aggregate::new, > > > >>> new PRTAggregate(), > > > >>> TimeWindows.of(60 * 60 * 1000L), > > > >>> collectorSerde, "prt_minute_agg_stream"); > > > >>> > > > >>> ktRtDetail.toStream().print(); > > > >>> > > > >>> KafkaStreams kafkaStreams = new KafkaStreams(kStreamBuilder, > config); > > > >>> > > > >>> kafkaStreams.start(); > > > >>> > > > >>> > > > >>> On Tue, Dec 6, 2016 at 6:48 AM, Damian Guy <damian....@gmail.com> > > > wrote: > > > >>> > > > >>> > Hi Jon, > > > >>> > > > > >>> > A couple of things: Which version are you using? > > > >>> > Can you share the code you are using to the build the topology? > > > >>> > > > > >>> > Thanks, > > > >>> > Damian > > > >>> > > > > >>> > On Tue, 6 Dec 2016 at 14:44 Jon Yeargers < > jon.yearg...@cedexis.com > > > > > > >>> wrote: > > > >>> > > > > >>> > > Im using .map to convert my (k/v) string/Object to > Object/Object > > > but > > > >>> > when I > > > >>> > > chain this to an aggregation step Im getting this exception: > > > >>> > > > > > >>> > > Exception in thread "StreamThread-1" > > java.lang.ClassCastException: > > > >>> > > com.company.prtminuteagg.types.RtDetailLogLine cannot be cast > to > > > >>> > > java.lang.String > > > >>> > > at > > > >>> > > > > > >>> > > org.apache.kafka.common.serialization. > > StringSerializer.serialize( > > > >>> > StringSerializer.java:24) > > > >>> > > at > > > >>> > > > > > >>> > > org.apache.kafka.streams.processor.internals. > > RecordCollector.send( > > > >>> > RecordCollector.java:73) > > > >>> > > at > > > >>> > > > > > >>> > > org.apache.kafka.streams.processor.internals.SinkNode. > > > >>> > process(SinkNode.java:72) > > > >>> > > at > > > >>> > > > > > >>> > > org.apache.kafka.streams.processor.internals. > > > >>> > ProcessorContextImpl.forward(ProcessorContextImpl.java:204) > > > >>> > > at > > > >>> > > > > > >>> > > org.apache.kafka.streams.kstream.internals.KStreamFilter$ > > > >>> > KStreamFilterProcessor.process(KStreamFilter.java:44) > > > >>> > > at > > > >>> > > > > > >>> > > org.apache.kafka.streams.processor.internals. > > ProcessorNode.process( > > > >>> > ProcessorNode.java:82) > > > >>> > > at > > > >>> > > > > > >>> > > org.apache.kafka.streams.processor.internals. > > > >>> > ProcessorContextImpl.forward(ProcessorContextImpl.java:204) > > > >>> > > at > > > >>> > > > > > >>> > > org.apache.kafka.streams.kstream.internals.KStreamMap$ > > > >>> > KStreamMapProcessor.process(KStreamMap.java:43) > > > >>> > > at > > > >>> > > > > > >>> > > org.apache.kafka.streams.processor.internals. > > ProcessorNode.process( > > > >>> > ProcessorNode.java:82) > > > >>> > > at > > > >>> > > > > > >>> > > org.apache.kafka.streams.processor.internals. > > > >>> > ProcessorContextImpl.forward(ProcessorContextImpl.java:204) > > > >>> > > at > > > >>> > > > > > >>> > > org.apache.kafka.streams.processor.internals. > > > >>> > SourceNode.process(SourceNode.java:66) > > > >>> > > at > > > >>> > > > > > >>> > > org.apache.kafka.streams.processor.internals. > > > >>> > StreamTask.process(StreamTask.java:181) > > > >>> > > at > > > >>> > > > > > >>> > > org.apache.kafka.streams.processor.internals. > > StreamThread.runLoop( > > > >>> > StreamThread.java:436) > > > >>> > > at > > > >>> > > > > > >>> > > org.apache.kafka.streams.processor.internals. > > > >>> > StreamThread.run(StreamThread.java:242) > > > >>> > > > > > >>> > > My key object implements Serde and returns a JsonSerializer > for > > the > > > >>> > > 'Serializer()' override. > > > >>> > > > > > >>> > > In the config for the topology Im > > > >>> > > setting: config.put(StreamsConfig.KEY_SERDE_CLASS_CONFIG, > > > >>> > > AggKey.class.getName()); > > > >>> > > > > > >>> > > Where else do I need to specify the (de)serializer for my key > > > class? > > > >>> > > > > > >>> > > > > >>> > > > >>> > > > >> > > > > > > > > > > >