kafka-users mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Guozhang Wang <wangg...@gmail.com>
Subject Re: Invalid TimeStamp Error while running WordCountDemo - kafka-0.10.0
Date Tue, 03 May 2016 14:35:53 GMT
So do you mean even with the 0.10.0 console producer producing to the input
topic (assuming it is a new topic and hence there is no old data produced
to it from tech-preview console producer) without the one-line fix, you
still see the issue?

Guozhang

On Tue, May 3, 2016 at 7:10 AM, Ramanan, Buvana (Nokia - US) <
buvana.ramanan@nokia.com> wrote:

> Guozhang,
>
> For bug scenario, I initially I produced to the topic using console
> producer of confluent's alpha release (preview for streams). And later I
> produced to it using the console producer in version 0.10.0.
>
> But yesterday after the fix, I created a new input topic, produced to it
> (with the fixed console producer in place), reflected the new topic name in
> the WordCountDemo code, recompiled & ran it without issues.
>
> Background - Prior to downloading the 0.10.0 version, I was playing with
> Confluent's alpha release, followed Quick Start instructions, and used
> console-producer to produce to the streams-file-input topic. But I had
> issues running WordCountDemo(/Job) (pasting the error message below this
> email just in case you are interested). So, I downloaded 0.10.0 on April
> 22nd and went ahead with it.
>
> Regards,
> Buvana
>
> ~/confluent-2.1.0-alpha1$ ./bin/kafka-run-class
> org.apache.kafka.streams.examples.wordcount.WordCountJob
> [2016-04-19 11:09:14,223] WARN The configuration zookeeper.connect =
> localhost:2181 was supplied but isn't a known config.
> (org.apache.kafka.clients.producer.ProducerConfig)
> [2016-04-19 11:09:14,344] WARN The configuration num.standby.replicas = 0
> was supplied but isn't a known config.
> (org.apache.kafka.clients.consumer.ConsumerConfig)
> [2016-04-19 11:09:14,344] WARN The configuration zookeeper.connect =
> localhost:2181 was supplied but isn't a known config.
> (org.apache.kafka.clients.consumer.ConsumerConfig)
> [2016-04-19 11:09:14,344] WARN The configuration
> __stream.thread.instance__ = Thread[StreamThread-1,5,main] was supplied but
> isn't a known config. (org.apache.kafka.clients.consumer.ConsumerConfig)
> [2016-04-19 11:09:14,350] WARN The configuration zookeeper.connect =
> localhost:2181 was supplied but isn't a known config.
> (org.apache.kafka.clients.consumer.ConsumerConfig)
> Exception in thread "StreamThread-1" java.lang.NoSuchMethodError:
> com.fasterxml.jackson.core.JsonGenerator.setCurrentValue(Ljava/lang/Object;)V
>     at
> com.fasterxml.jackson.databind.ser.std.MapSerializer.serialize(MapSerializer.java:445)
>     at
> com.fasterxml.jackson.databind.ser.std.MapSerializer.serialize(MapSerializer.java:29)
>     at
> com.fasterxml.jackson.databind.ser.DefaultSerializerProvider.serializeValue(DefaultSerializerProvider.java:129)
>     at
> com.fasterxml.jackson.databind.ObjectMapper._configAndWriteValue(ObjectMapper.java:3387)
>     at
> com.fasterxml.jackson.databind.ObjectMapper.writeValueAsString(ObjectMapper.java:2781)
>     at
> org.apache.kafka.streams.processor.internals.InternalTopicManager.createTopic(InternalTopicManager.java:178)
>
> -----Original Message-----
> From: EXT Guozhang Wang [mailto:wangguoz@gmail.com]
> Sent: Tuesday, May 03, 2016 12:49 AM
> To: users@kafka.apache.org
> Subject: Re: Invalid TimeStamp Error while running WordCountDemo -
> kafka-0.10.0
>
> Hi Buvana,
>
> Actually Ismael just pointed out that the new KafkaProducer already set
> the timestamp as current wall-clock time. And I cannot reproduce this issue
> from trunk either. Did you ran the console producer that is built from the
> trunk April 22nd as well?
>
> Guozhang
>
> On Mon, May 2, 2016 at 12:39 PM, Ramanan, Buvana (Nokia - US) <
> buvana.ramanan@nokia.com> wrote:
>
> > Guozhang,
> >
> > That worked!! Thanks a lot for your timely response. See you with
> > another issue.
> >
> > Regards,
> > Buvaan
> >
> > -----Original Message-----
> > From: EXT Guozhang Wang [mailto:wangguoz@gmail.com]
> > Sent: Sunday, May 01, 2016 8:15 PM
> > To: users@kafka.apache.org
> > Subject: Re: Invalid TimeStamp Error while running WordCountDemo -
> > kafka-0.10.0
> >
> > Hello Buvana,
> >
> > I realized that it is due to a recent change in Kafka Streams exposing
> > an issue in console producer that it does not set the timestamp
> > implicitly while using the new producer. Just filed
> > https://issues.apache.org/jira/browse/KAFKA-3646 for this.
> >
> > As for your case, since you are building from source code, I would
> > suggest you just modify a one-liner in kafka.producer.BaseProducer
> > (scala code), line 43 as:
> >
> >
> > val record = new ProducerRecord[Array[Byte],Array[Byte]](topic, null,
> > System.currentTimeMillis() key, value)
> >
> >
> > to explicitly set the timestamp as current system time in milliseconds.
> >
> >
> > Guozhang
> >
> >
> >
> > On Fri, Apr 29, 2016 at 10:34 AM, Ramanan, Buvana (Nokia - US) <
> > buvana.ramanan@nokia.com> wrote:
> >
> > > Hello Guozhang,
> > >
> > > thanks a lot for your response (to this and all of my previous
> > questions).
> > > Here is how I produce to the topic:
> > > cat /tmp/file-input.txt | ./kafka-console-producer.sh --broker-list
> > > localhost:9092 --topic streams-file-input
> > >
> > > Here is the content of the file:
> > > ~/kafka-0.10.0/bin$ cat /tmp/file-input.txt all streams lead to
> > > kafka hello kafka streams join kafka summit
> > >
> > > I checked the topic and made sure there are text lines in there.
> > >
> > > As for version:
> > > I downloaded kafka-0.10.0 on April 22nd and built it following the
> > > instructions. Did not have any issues with build.
> > > I was able to successfully run the wordCountDemo on April 22nd
> > >
> > > Its weird that I am not able to run it now.
> > >
> > > regards,
> > > Buvana
> > > ________________________________________
> > > From: EXT Guozhang Wang [wangguoz@gmail.com]
> > > Sent: Thursday, April 28, 2016 4:43 PM
> > > To: users@kafka.apache.org
> > > Subject: Re: Invalid TimeStamp Error while running WordCountDemo -
> > > kafka-0.10.0
> > >
> > > Hello Buvana,
> > >
> > > Could you show me the command line you used to produce the text to
> > > Kafka
> > as
> > > input? Also which version of Kafka are you using for the broker?
> > >
> > > Guozhang
> > >
> > > On Wed, Apr 27, 2016 at 12:07 PM, Ramanan, Buvana (Nokia - US) <
> > > buvana.ramanan@nokia.com> wrote:
> > >
> > > > Hello,
> > > >
> > > > I am trying to execute WordCountDemo app. I produced text to the
> > > > input topic. But when I execute the WordCountDemo, I get error.
> > > >
> > > > please help resolve the following:
> > > > ERROR Streams application error during processing in thread
> > > > [StreamThread-1]:
> > > > (org.apache.kafka.streams.processor.internals.StreamThread)
> > > > java.lang.IllegalArgumentException: Invalid timestamp -1
> > > >
> > > > broker, consumer & zk are running in the same machine. Ubuntu
> > > > 14.04,
> > java
> > > > 1.8.
> > > >
> > > > Thanks,
> > > > Buvana
> > > >
> > > > ~/kafka-0.10.0/bin$ ./kafka-run-class.sh
> > > > org.apache.kafka.streams.examples.wordcount.WordCountDemo
> > > > SLF4J: Class path contains multiple SLF4J bindings.
> > > > SLF4J: Found binding in
> > > >
> > >
> > [jar:file:/home/buvana/kafka-0.10.0/core/build/dependant-libs-2.10.6/s
> > lf4j-log4j12-1.7.18.jar!/org/slf4j/impl/StaticLoggerBinder.class]
> > > > SLF4J: Found binding in
> > > >
> > >
> > [jar:file:/home/buvana/kafka-0.10.0/tools/build/dependant-libs-2.10.6/
> > slf4j-log4j12-1.7.18.jar!/org/slf4j/impl/StaticLoggerBinder.class]
> > > > SLF4J: Found binding in
> > > >
> > >
> > [jar:file:/home/buvana/kafka-0.10.0/connect/api/build/dependant-libs/s
> > lf4j-log4j12-1.7.18.jar!/org/slf4j/impl/StaticLoggerBinder.class]
> > > > SLF4J: Found binding in
> > > >
> > >
> > [jar:file:/home/buvana/kafka-0.10.0/connect/runtime/build/dependant-li
> > bs/slf4j-log4j12-1.7.18.jar!/org/slf4j/impl/StaticLoggerBinder.class]
> > > > SLF4J: Found binding in
> > > >
> > >
> > [jar:file:/home/buvana/kafka-0.10.0/connect/file/build/dependant-libs/
> > slf4j-log4j12-1.7.18.jar!/org/slf4j/impl/StaticLoggerBinder.class]
> > > > SLF4J: Found binding in
> > > >
> > >
> > [jar:file:/home/buvana/kafka-0.10.0/connect/json/build/dependant-libs/
> > slf4j-log4j12-1.7.18.jar!/org/slf4j/impl/StaticLoggerBinder.class]
> > > > SLF4J: See http://www.slf4j.org/codes.html#multiple_bindings for
> > > > an explanation.
> > > > SLF4J: Actual binding is of type
> > > > [org.slf4j.impl.Log4jLoggerFactory]
> > > > [2016-04-27 14:56:04,967] WARN The configuration
> > > > replication.factor = 1 was supplied but isn't a known config.
> > > > (org.apache.kafka.clients.consumer.ConsumerConfig)
> > > > [2016-04-27 14:56:04,968] WARN The configuration
> > > > num.standby.replicas
> > = 0
> > > > was supplied but isn't a known config.
> > > > (org.apache.kafka.clients.consumer.ConsumerConfig)
> > > > [2016-04-27 14:56:04,968] WARN The configuration zookeeper.connect
> > > > =
> > > > localhost:2181 was supplied but isn't a known config.
> > > > (org.apache.kafka.clients.consumer.ConsumerConfig)
> > > > [2016-04-27 14:56:04,968] WARN The configuration
> > > > __stream.thread.instance__ = Thread[StreamThread-1,5,main] was
> > > > supplied
> > > but
> > > > isn't a known config.
> > (org.apache.kafka.clients.consumer.ConsumerConfig)
> > > > [2016-04-27 14:56:05,736] ERROR Streams application error during
> > > > processing in thread [StreamThread-1]:
> > > > (org.apache.kafka.streams.processor.internals.StreamThread)
> > > > java.lang.IllegalArgumentException: Invalid timestamp -1
> > > >     at
> > > >
> > >
> > org.apache.kafka.clients.producer.ProducerRecord.<init>(ProducerRecord
> > .java:60)
> > > >     at
> > > >
> > >
> > org.apache.kafka.streams.processor.internals.SinkNode.process(SinkNode
> > .java:60)
> > > >     at
> > > >
> > >
> > org.apache.kafka.streams.processor.internals.StreamTask.forward(Stream
> > Task.java:331)
> > > >     at
> > > >
> > >
> > org.apache.kafka.streams.processor.internals.ProcessorContextImpl.forw
> > ard(ProcessorContextImpl.java:169)
> > > >     at
> > > >
> > >
> > org.apache.kafka.streams.kstream.internals.KStreamMapValues$KStreamMap
> > Processor.process(KStreamMapValues.java:42)
> > > >     at
> > > >
> > >
> > org.apache.kafka.streams.processor.internals.ProcessorNode.process(Pro
> > cessorNode.java:68)
> > > >     at
> > > >
> > >
> > org.apache.kafka.streams.processor.internals.StreamTask.forward(Stream
> > Task.java:331)
> > > >     at
> > > >
> > >
> > org.apache.kafka.streams.processor.internals.ProcessorContextImpl.forw
> > ard(ProcessorContextImpl.java:169)
> > > >     at
> > > >
> > >
> > org.apache.kafka.streams.kstream.internals.KStreamAggregate$KStreamAgg
> > regateProcessor.process(KStreamAggregate.java:89)
> > > >     at
> > > >
> > >
> > org.apache.kafka.streams.processor.internals.ProcessorNode.process(Pro
> > cessorNode.java:68)
> > > >     at
> > > >
> > >
> > org.apache.kafka.streams.processor.internals.StreamTask.forward(Stream
> > Task.java:331)
> > > >     at
> > > >
> > >
> > org.apache.kafka.streams.processor.internals.ProcessorContextImpl.forw
> > ard(ProcessorContextImpl.java:169)
> > > >     at
> > > >
> > >
> > org.apache.kafka.streams.kstream.internals.KStreamMap$KStreamMapProces
> > sor.process(KStreamMap.java:43)
> > > >     at
> > > >
> > >
> > org.apache.kafka.streams.processor.internals.ProcessorNode.process(Pro
> > cessorNode.java:68)
> > > >     at
> > > >
> > >
> > org.apache.kafka.streams.processor.internals.StreamTask.forward(Stream
> > Task.java:331)
> > > >     at
> > > >
> > >
> > org.apache.kafka.streams.processor.internals.ProcessorContextImpl.forw
> > ard(ProcessorContextImpl.java:169)
> > > >     at
> > > >
> > >
> > org.apache.kafka.streams.kstream.internals.KStreamFlatMapValues$KStrea
> > mFlatMapValuesProcessor.process(KStreamFlatMapValues.java:43)
> > > >     at
> > > >
> > >
> > org.apache.kafka.streams.processor.internals.ProcessorNode.process(Pro
> > cessorNode.java:68)
> > > >     at
> > > >
> > >
> > org.apache.kafka.streams.processor.internals.StreamTask.forward(Stream
> > Task.java:331)
> > > >     at
> > > >
> > >
> > org.apache.kafka.streams.processor.internals.ProcessorContextImpl.forw
> > ard(ProcessorContextImpl.java:169)
> > > >     at
> > > >
> > >
> > org.apache.kafka.streams.processor.internals.SourceNode.process(Source
> > Node.java:56)
> > > >     at
> > > >
> > >
> > org.apache.kafka.streams.processor.internals.StreamTask.process(Stream
> > Task.java:174)
> > > >     at
> > > >
> > >
> > org.apache.kafka.streams.processor.internals.StreamThread.runLoop(Stre
> > amThread.java:350)
> > > >     at
> > > >
> > >
> > org.apache.kafka.streams.processor.internals.StreamThread.run(StreamTh
> > read.java:248)
> > > >
> > > >
> > > >
> > >
> > >
> > > --
> > > -- Guozhang
> > >
> >
> >
> >
> > --
> > -- Guozhang
> >
>
>
>
> --
> -- Guozhang
>



-- 
-- Guozhang

Mime
  • Unnamed multipart/alternative (inline, None, 0 bytes)
View raw message