You are viewing a plain text version of this content. The canonical link for it is here.
Posted to user@flink.apache.org by "Vieru, Mihail" <mi...@zalando.de> on 2015/12/01 16:54:35 UTC

NPE with Flink Streaming from Kafka

Hi,

we get the following NullPointerException after ~50 minutes when running a
streaming job with windowing and state that reads data from Kafka and
writes the result to local FS.
There are around 170 million messages to be processed, Flink 0.10.1 stops
at ~8 million.
Flink runs locally, started with the "start-cluster-streaming.sh" script.

12/01/2015 15:06:24    Job execution switched to status RUNNING.
12/01/2015 15:06:24    Source: Custom Source -> Map -> Map(1/1) switched to
SCHEDULED
12/01/2015 15:06:24    Source: Custom Source -> Map -> Map(1/1) switched to
DEPLOYING
12/01/2015 15:06:24    Fast TumblingTimeWindows(5000) of Reduce at
main(ItemPriceAvgPerOrder.java:108) -> Sink: Unnamed(1/1) switched to
SCHEDULED
12/01/2015 15:06:24    Fast TumblingTimeWindows(5000) of Reduce at
main(ItemPriceAvgPerOrder.java:108) -> Sink: Unnamed(1/1) switched to
DEPLOYING
12/01/2015 15:06:24    Source: Custom Source -> Map -> Map(1/1) switched to
RUNNING
12/01/2015 15:06:24    Fast TumblingTimeWindows(5000) of Reduce at
main(ItemPriceAvgPerOrder.java:108) -> Sink: Unnamed(1/1) switched to
RUNNING
12/01/2015 15:56:08    Fast TumblingTimeWindows(5000) of Reduce at
main(ItemPriceAvgPerOrder.java:108) -> Sink: Unnamed(1/1) switched to
CANCELED
12/01/2015 15:56:08    Source: Custom Source -> Map -> Map(1/1) switched to
FAILED
java.lang.Exception
    at
org.apache.flink.streaming.connectors.kafka.internals.LegacyFetcher.run(LegacyFetcher.java:242)
    at
org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer.run(FlinkKafkaConsumer.java:397)
    at
org.apache.flink.streaming.api.operators.StreamSource.run(StreamSource.java:58)
    at
org.apache.flink.streaming.runtime.tasks.SourceStreamTask.run(SourceStreamTask.java:55)
    at
org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:218)
    at org.apache.flink.runtime.taskmanager.Task.run(Task.java:584)
    at java.lang.Thread.run(Thread.java:745)
Caused by: java.lang.NullPointerException
    at
org.I0Itec.zkclient.ZkConnection.writeDataReturnStat(ZkConnection.java:115)
    at org.I0Itec.zkclient.ZkClient$10.call(ZkClient.java:817)
    at org.I0Itec.zkclient.ZkClient.retryUntilConnected(ZkClient.java:675)
    at org.I0Itec.zkclient.ZkClient.writeDataReturnStat(ZkClient.java:813)
    at org.I0Itec.zkclient.ZkClient.writeData(ZkClient.java:808)
    at org.I0Itec.zkclient.ZkClient.writeData(ZkClient.java:777)
    at kafka.utils.ZkUtils$.updatePersistentPath(ZkUtils.scala:332)
    at kafka.utils.ZkUtils.updatePersistentPath(ZkUtils.scala)
    at
org.apache.flink.streaming.connectors.kafka.internals.ZookeeperOffsetHandler.setOffsetInZooKeeper(ZookeeperOffsetHandler.java:112)
    at
org.apache.flink.streaming.connectors.kafka.internals.ZookeeperOffsetHandler.commit(ZookeeperOffsetHandler.java:80)
    at
org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer$PeriodicOffsetCommitter.run(FlinkKafkaConsumer.java:632)


Any ideas on what could cause this behaviour?

Best,
Mihail

Re: Question about DataStream serialization

Posted by Stephan Ewen <se...@apache.org>.
The object's hash it is additionally scrambled using a typical hash
function (like murmur hash) to guard against bad hash functions...

On Wed, Dec 9, 2015 at 2:37 AM, Radu Tudoran <ra...@huawei.com>
wrote:

> Hi,
>
> Is the partitioned functioned used by the ".keyBy(Object)" of the form:
>
> Object.hash % getNumberOfParallelSubtasks()
>
> ?
>
>
>
> Dr. Radu Tudoran
> Research Engineer
> IT R&D Division
>
>
> HUAWEI TECHNOLOGIES Duesseldorf GmbH
> European Research Center
> Riesstrasse 25, 80992 München
>
> E-mail: radu.tudoran@huawei.com
> Mobile: +49 15209084330
> Telephone: +49 891588344173
>
> HUAWEI TECHNOLOGIES Duesseldorf GmbH
> Hansaallee 205, 40549 Düsseldorf, Germany, www.huawei.com
> Registered Office: Düsseldorf, Register Court Düsseldorf, HRB 56063,
> Managing Director: Jingwen TAO, Wanzhou MENG, Lifang CHEN
> Sitz der Gesellschaft: Düsseldorf, Amtsgericht Düsseldorf, HRB 56063,
> Geschäftsführer: Jingwen TAO, Wanzhou MENG, Lifang CHEN
> This e-mail and its attachments contain confidential information from
> HUAWEI, which is intended only for the person or entity whose address is
> listed above. Any use of the information contained herein in any way
> (including, but not limited to, total or partial disclosure, reproduction,
> or dissemination) by persons other than the intended recipient(s) is
> prohibited. If you receive this e-mail in error, please notify the sender
> by phone or email immediately and delete it!
>
>
> -----Original Message-----
> From: Aljoscha Krettek [mailto:aljoscha@apache.org]
> Sent: Tuesday, December 08, 2015 5:00 PM
> To: user@flink.apache.org
> Subject: Re: Question about DataStream serialization
>
> Hi,
> it is not possible in an officially supported way. There is however a
> trick that you could use: You can cast the OperatorState to a KvState. This
> has a method setCurrentKey() that sets the key to be used when calling
> value() and update(). In this way you can trick the OperatorState into
> thinking that it has the key of an input element.
>
> This is an internal API, however, and could change in the future, thereby
> breaking your program.
>
> Cheers,
> Aljoscha
> > On 08 Dec 2015, at 16:31, Radu Tudoran <ra...@huawei.com> wrote:
> >
> > Hi,
> >
> > The state that is being loaded can very well be partitioned by keys.
> Assuming this scenario and that you would now that the keys go from 0 to N,
> is there some possibility to load and partitioned the initial data in the
> open function?
> >
> >
> > Dr. Radu Tudoran
> > Research Engineer
> > IT R&D Division
> >
> >
> > HUAWEI TECHNOLOGIES Duesseldorf GmbH
> > European Research Center
> > Riesstrasse 25, 80992 München
> >
> > E-mail: radu.tudoran@huawei.com
> > Mobile: +49 15209084330
> > Telephone: +49 891588344173
> >
> > HUAWEI TECHNOLOGIES Duesseldorf GmbH
> > Hansaallee 205, 40549 Düsseldorf, Germany, www.huawei.com Registered
> > Office: Düsseldorf, Register Court Düsseldorf, HRB 56063, Managing
> > Director: Jingwen TAO, Wanzhou MENG, Lifang CHEN Sitz der
> > Gesellschaft: Düsseldorf, Amtsgericht Düsseldorf, HRB 56063,
> > Geschäftsführer: Jingwen TAO, Wanzhou MENG, Lifang CHEN This e-mail
> > and its attachments contain confidential information from HUAWEI, which
> is intended only for the person or entity whose address is listed above.
> Any use of the information contained herein in any way (including, but not
> limited to, total or partial disclosure, reproduction, or dissemination) by
> persons other than the intended recipient(s) is prohibited. If you receive
> this e-mail in error, please notify the sender by phone or email
> immediately and delete it!
> >
> >
> > -----Original Message-----
> > From: Aljoscha Krettek [mailto:aljoscha@apache.org]
> > Sent: Tuesday, December 08, 2015 4:20 PM
> > To: user@flink.apache.org
> > Subject: Re: Question about DataStream serialization
> >
> > Ah, I see what’s the problem. Operator state is scoped to the key of the
> incoming element. In the open() method, no element has been received yet,
> so the key of the incoming element is basically NULL. So the open() method
> initializes state for key NULL. In flatMap() we actually have a key of
> incoming elements so we access state for a specific key, which has default
> value “0” (from the getKeyValueState() call).
> >
> > OperatorState is only useful if the state needs to be partitioned by
> key, but here it seems that the state is valid for all elements?
> >> On 08 Dec 2015, at 15:30, Radu Tudoran <ra...@huawei.com> wrote:
> >>
> >> final StreamExecutionEnvironment env = StreamExecutionEnvironment
> >>                              .getExecutionEnvironment();
> >>
> >>              DataStream<String> stream = env
> >>                              .socketTextStream("localhost", 16333, '\n')
> >>                              .map(new MapFunction<String,
> Tuple1<String>>() {
> >>                                      @Override
> >>                                      public Tuple1<String> map(String
> arg0) throws Exception {
> >>                                              return new
> Tuple1<String>(arg0);
> >>                                      }
> >>                              }).keyBy(0)
> >>                              .flatMap(new
> RichFlatMapFunction<Tuple1<String>, String>() {
> >>
> >>                                      private OperatorState<Integer>
> dataset;
> >>
> >>                                      @Override
> >>                                      public void flatMap(Tuple1<String>
> arg0,
> >>                                                      Collector<String>
> arg1) throws Exception {
> >>
> >>                                              if (dataset.value() > 0)
> >>                                                      arg1.collect("Test
> OK " + arg0);
> >>
> >>
> >>
> >>                                      }
> >>
> >>                                      @Override
> >>                                      public void open(Configuration
> parameters) throws Exception {
> >>
> >>                                              dataset =
> getRuntimeContext().getKeyValueState(
> >>
> "loadeddata", Integer.class, 0);
> >>
> >>
> >>                                               /*
> >>                                                * Simulate loading data
> >>                                                * Looks like if this
> part is  commented out and the dataset is
> >>                                                * initialize with 1 for
> example, than the non-zero value is available
> >>                                                * in the flatMap function
> >>                                                */
> >>
> >>                                                for(int i=0;i<10;i++) {
> >>
> dataset.update(dataset.value()+1);
> >>                                                }
> >>
> >>
> //System.out.println("dataset value "+dataset.value());
> >>
> >>                                      }
> >>                              });
> >>
> >>              stream.print();
> >>
> >>              env.execute("test open function");
> >
>
>

Re: Question about DataStream serialization

Posted by Aljoscha Krettek <al...@apache.org>.
Right now, it is exactly "Object.hash % getNumberOfParallelSubtasks()”...

> On 09 Dec 2015, at 02:37, Radu Tudoran <ra...@huawei.com> wrote:
> 
> Object.hash % getNumberOfParallelSubtasks()


RE: Question about DataStream serialization

Posted by Radu Tudoran <ra...@huawei.com>.
Hi,

Is the partitioned functioned used by the ".keyBy(Object)" of the form:

Object.hash % getNumberOfParallelSubtasks()

?



Dr. Radu Tudoran
Research Engineer
IT R&D Division


HUAWEI TECHNOLOGIES Duesseldorf GmbH
European Research Center
Riesstrasse 25, 80992 München

E-mail: radu.tudoran@huawei.com
Mobile: +49 15209084330
Telephone: +49 891588344173

HUAWEI TECHNOLOGIES Duesseldorf GmbH
Hansaallee 205, 40549 Düsseldorf, Germany, www.huawei.com
Registered Office: Düsseldorf, Register Court Düsseldorf, HRB 56063,
Managing Director: Jingwen TAO, Wanzhou MENG, Lifang CHEN
Sitz der Gesellschaft: Düsseldorf, Amtsgericht Düsseldorf, HRB 56063,
Geschäftsführer: Jingwen TAO, Wanzhou MENG, Lifang CHEN
This e-mail and its attachments contain confidential information from HUAWEI, which is intended only for the person or entity whose address is listed above. Any use of the information contained herein in any way (including, but not limited to, total or partial disclosure, reproduction, or dissemination) by persons other than the intended recipient(s) is prohibited. If you receive this e-mail in error, please notify the sender by phone or email immediately and delete it!


-----Original Message-----
From: Aljoscha Krettek [mailto:aljoscha@apache.org] 
Sent: Tuesday, December 08, 2015 5:00 PM
To: user@flink.apache.org
Subject: Re: Question about DataStream serialization

Hi,
it is not possible in an officially supported way. There is however a trick that you could use: You can cast the OperatorState to a KvState. This has a method setCurrentKey() that sets the key to be used when calling value() and update(). In this way you can trick the OperatorState into thinking that it has the key of an input element.

This is an internal API, however, and could change in the future, thereby breaking your program.

Cheers,
Aljoscha
> On 08 Dec 2015, at 16:31, Radu Tudoran <ra...@huawei.com> wrote:
> 
> Hi,
> 
> The state that is being loaded can very well be partitioned by keys. Assuming this scenario and that you would now that the keys go from 0 to N, is there some possibility to load and partitioned the initial data in the open function?
> 
> 
> Dr. Radu Tudoran
> Research Engineer
> IT R&D Division
> 
> 
> HUAWEI TECHNOLOGIES Duesseldorf GmbH
> European Research Center
> Riesstrasse 25, 80992 München
> 
> E-mail: radu.tudoran@huawei.com
> Mobile: +49 15209084330
> Telephone: +49 891588344173
> 
> HUAWEI TECHNOLOGIES Duesseldorf GmbH
> Hansaallee 205, 40549 Düsseldorf, Germany, www.huawei.com Registered 
> Office: Düsseldorf, Register Court Düsseldorf, HRB 56063, Managing 
> Director: Jingwen TAO, Wanzhou MENG, Lifang CHEN Sitz der 
> Gesellschaft: Düsseldorf, Amtsgericht Düsseldorf, HRB 56063,
> Geschäftsführer: Jingwen TAO, Wanzhou MENG, Lifang CHEN This e-mail 
> and its attachments contain confidential information from HUAWEI, which is intended only for the person or entity whose address is listed above. Any use of the information contained herein in any way (including, but not limited to, total or partial disclosure, reproduction, or dissemination) by persons other than the intended recipient(s) is prohibited. If you receive this e-mail in error, please notify the sender by phone or email immediately and delete it!
> 
> 
> -----Original Message-----
> From: Aljoscha Krettek [mailto:aljoscha@apache.org]
> Sent: Tuesday, December 08, 2015 4:20 PM
> To: user@flink.apache.org
> Subject: Re: Question about DataStream serialization
> 
> Ah, I see what’s the problem. Operator state is scoped to the key of the incoming element. In the open() method, no element has been received yet, so the key of the incoming element is basically NULL. So the open() method initializes state for key NULL. In flatMap() we actually have a key of incoming elements so we access state for a specific key, which has default value “0” (from the getKeyValueState() call).
> 
> OperatorState is only useful if the state needs to be partitioned by key, but here it seems that the state is valid for all elements?
>> On 08 Dec 2015, at 15:30, Radu Tudoran <ra...@huawei.com> wrote:
>> 
>> final StreamExecutionEnvironment env = StreamExecutionEnvironment
>> 				.getExecutionEnvironment();
>> 
>> 		DataStream<String> stream = env
>> 				.socketTextStream("localhost", 16333, '\n')
>> 				.map(new MapFunction<String, Tuple1<String>>() {
>> 					@Override
>> 					public Tuple1<String> map(String arg0) throws Exception {
>> 						return new Tuple1<String>(arg0);
>> 					}
>> 				}).keyBy(0)
>> 				.flatMap(new RichFlatMapFunction<Tuple1<String>, String>() {
>> 
>> 					private OperatorState<Integer> dataset;
>> 
>> 					@Override
>> 					public void flatMap(Tuple1<String> arg0,
>> 							Collector<String> arg1) throws Exception {
>> 
>> 						if (dataset.value() > 0)
>> 							arg1.collect("Test OK " + arg0);
>> 
>> 						
>> 						
>> 					}
>> 
>> 					@Override
>> 					public void open(Configuration parameters) throws Exception {
>> 
>> 						dataset = getRuntimeContext().getKeyValueState(
>> 								"loadeddata", Integer.class, 0);
>> 
>> 						
>> 						 /*
>> 						  * Simulate loading data
>> 						  * Looks like if this part is  commented out and the dataset is 
>> 						  * initialize with 1 for example, than the non-zero value is available 
>> 						  * in the flatMap function  
>> 						  */
>> 						  
>> 						  for(int i=0;i<10;i++) {
>> 						  	  dataset.update(dataset.value()+1);
>> 						  }
>> 						  
>> 						  //System.out.println("dataset value "+dataset.value());
>> 						  
>> 					}
>> 				});
>> 
>> 		stream.print();
>> 
>> 		env.execute("test open function");
> 


Re: Question about DataStream serialization

Posted by Aljoscha Krettek <al...@apache.org>.
Hi,
it is not possible in an officially supported way. There is however a trick that you could use: You can cast the OperatorState to a KvState. This has a method setCurrentKey() that sets the key to be used when calling value() and update(). In this way you can trick the OperatorState into thinking that it has the key of an input element.

This is an internal API, however, and could change in the future, thereby breaking your program.

Cheers,
Aljoscha
> On 08 Dec 2015, at 16:31, Radu Tudoran <ra...@huawei.com> wrote:
> 
> Hi,
> 
> The state that is being loaded can very well be partitioned by keys. Assuming this scenario and that you would now that the keys go from 0 to N, is there some possibility to load and partitioned the initial data in the open function?
> 
> 
> Dr. Radu Tudoran
> Research Engineer
> IT R&D Division
> 
> 
> HUAWEI TECHNOLOGIES Duesseldorf GmbH
> European Research Center
> Riesstrasse 25, 80992 München
> 
> E-mail: radu.tudoran@huawei.com
> Mobile: +49 15209084330
> Telephone: +49 891588344173
> 
> HUAWEI TECHNOLOGIES Duesseldorf GmbH
> Hansaallee 205, 40549 Düsseldorf, Germany, www.huawei.com
> Registered Office: Düsseldorf, Register Court Düsseldorf, HRB 56063,
> Managing Director: Jingwen TAO, Wanzhou MENG, Lifang CHEN
> Sitz der Gesellschaft: Düsseldorf, Amtsgericht Düsseldorf, HRB 56063,
> Geschäftsführer: Jingwen TAO, Wanzhou MENG, Lifang CHEN
> This e-mail and its attachments contain confidential information from HUAWEI, which is intended only for the person or entity whose address is listed above. Any use of the information contained herein in any way (including, but not limited to, total or partial disclosure, reproduction, or dissemination) by persons other than the intended recipient(s) is prohibited. If you receive this e-mail in error, please notify the sender by phone or email immediately and delete it!
> 
> 
> -----Original Message-----
> From: Aljoscha Krettek [mailto:aljoscha@apache.org] 
> Sent: Tuesday, December 08, 2015 4:20 PM
> To: user@flink.apache.org
> Subject: Re: Question about DataStream serialization
> 
> Ah, I see what’s the problem. Operator state is scoped to the key of the incoming element. In the open() method, no element has been received yet, so the key of the incoming element is basically NULL. So the open() method initializes state for key NULL. In flatMap() we actually have a key of incoming elements so we access state for a specific key, which has default value “0” (from the getKeyValueState() call).
> 
> OperatorState is only useful if the state needs to be partitioned by key, but here it seems that the state is valid for all elements?
>> On 08 Dec 2015, at 15:30, Radu Tudoran <ra...@huawei.com> wrote:
>> 
>> final StreamExecutionEnvironment env = StreamExecutionEnvironment
>> 				.getExecutionEnvironment();
>> 
>> 		DataStream<String> stream = env
>> 				.socketTextStream("localhost", 16333, '\n')
>> 				.map(new MapFunction<String, Tuple1<String>>() {
>> 					@Override
>> 					public Tuple1<String> map(String arg0) throws Exception {
>> 						return new Tuple1<String>(arg0);
>> 					}
>> 				}).keyBy(0)
>> 				.flatMap(new RichFlatMapFunction<Tuple1<String>, String>() {
>> 
>> 					private OperatorState<Integer> dataset;
>> 
>> 					@Override
>> 					public void flatMap(Tuple1<String> arg0,
>> 							Collector<String> arg1) throws Exception {
>> 
>> 						if (dataset.value() > 0)
>> 							arg1.collect("Test OK " + arg0);
>> 
>> 						
>> 						
>> 					}
>> 
>> 					@Override
>> 					public void open(Configuration parameters) throws Exception {
>> 
>> 						dataset = getRuntimeContext().getKeyValueState(
>> 								"loadeddata", Integer.class, 0);
>> 
>> 						
>> 						 /*
>> 						  * Simulate loading data
>> 						  * Looks like if this part is  commented out and the dataset is 
>> 						  * initialize with 1 for example, than the non-zero value is available 
>> 						  * in the flatMap function  
>> 						  */
>> 						  
>> 						  for(int i=0;i<10;i++) {
>> 						  	  dataset.update(dataset.value()+1);
>> 						  }
>> 						  
>> 						  //System.out.println("dataset value "+dataset.value());
>> 						  
>> 					}
>> 				});
>> 
>> 		stream.print();
>> 
>> 		env.execute("test open function");
> 


RE: Question about DataStream serialization

Posted by Radu Tudoran <ra...@huawei.com>.
Hi,

The state that is being loaded can very well be partitioned by keys. Assuming this scenario and that you would now that the keys go from 0 to N, is there some possibility to load and partitioned the initial data in the open function?


Dr. Radu Tudoran
Research Engineer
IT R&D Division


HUAWEI TECHNOLOGIES Duesseldorf GmbH
European Research Center
Riesstrasse 25, 80992 München

E-mail: radu.tudoran@huawei.com
Mobile: +49 15209084330
Telephone: +49 891588344173

HUAWEI TECHNOLOGIES Duesseldorf GmbH
Hansaallee 205, 40549 Düsseldorf, Germany, www.huawei.com
Registered Office: Düsseldorf, Register Court Düsseldorf, HRB 56063,
Managing Director: Jingwen TAO, Wanzhou MENG, Lifang CHEN
Sitz der Gesellschaft: Düsseldorf, Amtsgericht Düsseldorf, HRB 56063,
Geschäftsführer: Jingwen TAO, Wanzhou MENG, Lifang CHEN
This e-mail and its attachments contain confidential information from HUAWEI, which is intended only for the person or entity whose address is listed above. Any use of the information contained herein in any way (including, but not limited to, total or partial disclosure, reproduction, or dissemination) by persons other than the intended recipient(s) is prohibited. If you receive this e-mail in error, please notify the sender by phone or email immediately and delete it!


-----Original Message-----
From: Aljoscha Krettek [mailto:aljoscha@apache.org] 
Sent: Tuesday, December 08, 2015 4:20 PM
To: user@flink.apache.org
Subject: Re: Question about DataStream serialization

Ah, I see what’s the problem. Operator state is scoped to the key of the incoming element. In the open() method, no element has been received yet, so the key of the incoming element is basically NULL. So the open() method initializes state for key NULL. In flatMap() we actually have a key of incoming elements so we access state for a specific key, which has default value “0” (from the getKeyValueState() call).

OperatorState is only useful if the state needs to be partitioned by key, but here it seems that the state is valid for all elements?
> On 08 Dec 2015, at 15:30, Radu Tudoran <ra...@huawei.com> wrote:
> 
> final StreamExecutionEnvironment env = StreamExecutionEnvironment
> 				.getExecutionEnvironment();
> 
> 		DataStream<String> stream = env
> 				.socketTextStream("localhost", 16333, '\n')
> 				.map(new MapFunction<String, Tuple1<String>>() {
> 					@Override
> 					public Tuple1<String> map(String arg0) throws Exception {
> 						return new Tuple1<String>(arg0);
> 					}
> 				}).keyBy(0)
> 				.flatMap(new RichFlatMapFunction<Tuple1<String>, String>() {
> 
> 					private OperatorState<Integer> dataset;
> 
> 					@Override
> 					public void flatMap(Tuple1<String> arg0,
> 							Collector<String> arg1) throws Exception {
> 
> 						if (dataset.value() > 0)
> 							arg1.collect("Test OK " + arg0);
> 
> 						
> 						
> 					}
> 
> 					@Override
> 					public void open(Configuration parameters) throws Exception {
> 
> 						dataset = getRuntimeContext().getKeyValueState(
> 								"loadeddata", Integer.class, 0);
> 
> 						
> 						 /*
> 						  * Simulate loading data
> 						  * Looks like if this part is  commented out and the dataset is 
> 						  * initialize with 1 for example, than the non-zero value is available 
> 						  * in the flatMap function  
> 						  */
> 						  
> 						  for(int i=0;i<10;i++) {
> 						  	  dataset.update(dataset.value()+1);
> 						  }
> 						  
> 						  //System.out.println("dataset value "+dataset.value());
> 						  
> 					}
> 				});
> 
> 		stream.print();
> 
> 		env.execute("test open function");


Re: Question about DataStream serialization

Posted by Aljoscha Krettek <al...@apache.org>.
Ah, I see what’s the problem. Operator state is scoped to the key of the incoming element. In the open() method, no element has been received yet, so the key of the incoming element is basically NULL. So the open() method initializes state for key NULL. In flatMap() we actually have a key of incoming elements so we access state for a specific key, which has default value “0” (from the getKeyValueState() call).

OperatorState is only useful if the state needs to be partitioned by key, but here it seems that the state is valid for all elements?
> On 08 Dec 2015, at 15:30, Radu Tudoran <ra...@huawei.com> wrote:
> 
> final StreamExecutionEnvironment env = StreamExecutionEnvironment
> 				.getExecutionEnvironment();
> 
> 		DataStream<String> stream = env
> 				.socketTextStream("localhost", 16333, '\n')
> 				.map(new MapFunction<String, Tuple1<String>>() {
> 					@Override
> 					public Tuple1<String> map(String arg0) throws Exception {
> 						return new Tuple1<String>(arg0);
> 					}
> 				}).keyBy(0)
> 				.flatMap(new RichFlatMapFunction<Tuple1<String>, String>() {
> 
> 					private OperatorState<Integer> dataset;
> 
> 					@Override
> 					public void flatMap(Tuple1<String> arg0,
> 							Collector<String> arg1) throws Exception {
> 
> 						if (dataset.value() > 0)
> 							arg1.collect("Test OK " + arg0);
> 
> 						
> 						
> 					}
> 
> 					@Override
> 					public void open(Configuration parameters) throws Exception {
> 
> 						dataset = getRuntimeContext().getKeyValueState(
> 								"loadeddata", Integer.class, 0);
> 
> 						
> 						 /*
> 						  * Simulate loading data
> 						  * Looks like if this part is  commented out and the dataset is 
> 						  * initialize with 1 for example, than the non-zero value is available 
> 						  * in the flatMap function  
> 						  */
> 						  
> 						  for(int i=0;i<10;i++) {
> 						  	  dataset.update(dataset.value()+1);
> 						  }
> 						  
> 						  //System.out.println("dataset value "+dataset.value());
> 						  
> 					}
> 				});
> 
> 		stream.print();
> 
> 		env.execute("test open function");


RE: Question about DataStream serialization

Posted by Radu Tudoran <ra...@huawei.com>.
Hi,

I attached below a function that shows the issue and that operatorstate does not have the initialized value from the open function before the flatmap is called. You can see this as the print will not show anything. If you remove on the other hand the initialization loop and put a non zero value for the dataset flag than the print will work.



public static void main(String[] args) throws Exception {

		final StreamExecutionEnvironment env = StreamExecutionEnvironment
				.getExecutionEnvironment();

		DataStream<String> stream = env
				.socketTextStream("localhost", 16333, '\n')
				.map(new MapFunction<String, Tuple1<String>>() {
					@Override
					public Tuple1<String> map(String arg0) throws Exception {
						return new Tuple1<String>(arg0);
					}
				}).keyBy(0)
				.flatMap(new RichFlatMapFunction<Tuple1<String>, String>() {

					private OperatorState<Integer> dataset;

					@Override
					public void flatMap(Tuple1<String> arg0,
							Collector<String> arg1) throws Exception {

						if (dataset.value() > 0)
							arg1.collect("Test OK " + arg0);

						
						
					}

					@Override
					public void open(Configuration parameters) throws Exception {

						dataset = getRuntimeContext().getKeyValueState(
								"loadeddata", Integer.class, 0);

						
						 /*
						  * Simulate loading data
						  * Looks like if this part is  commented out and the dataset is 
						  * initialize with 1 for example, than the non-zero value is available 
						  * in the flatMap function  
						  */
						  
						  for(int i=0;i<10;i++) {
						  	  dataset.update(dataset.value()+1);
						  }
						  
						  //System.out.println("dataset value "+dataset.value());
						  
					}
				});

		stream.print();

		env.execute("test open function");
	}
 

Dr. Radu Tudoran
Research Engineer
IT R&D Division


HUAWEI TECHNOLOGIES Duesseldorf GmbH
European Research Center
Riesstrasse 25, 80992 München

E-mail: radu.tudoran@huawei.com
Mobile: +49 15209084330
Telephone: +49 891588344173

HUAWEI TECHNOLOGIES Duesseldorf GmbH
Hansaallee 205, 40549 Düsseldorf, Germany, www.huawei.com
Registered Office: Düsseldorf, Register Court Düsseldorf, HRB 56063,
Managing Director: Jingwen TAO, Wanzhou MENG, Lifang CHEN
Sitz der Gesellschaft: Düsseldorf, Amtsgericht Düsseldorf, HRB 56063,
Geschäftsführer: Jingwen TAO, Wanzhou MENG, Lifang CHEN
This e-mail and its attachments contain confidential information from HUAWEI, which is intended only for the person or entity whose address is listed above. Any use of the information contained herein in any way (including, but not limited to, total or partial disclosure, reproduction, or dissemination) by persons other than the intended recipient(s) is prohibited. If you receive this e-mail in error, please notify the sender by phone or email immediately and delete it!

-----Original Message-----
From: Aljoscha Krettek [mailto:aljoscha@apache.org] 
Sent: Tuesday, December 08, 2015 12:14 PM
To: user@flink.apache.org
Subject: Re: Question about DataStream serialization

Hi,
if the open() method is indeed not called before the first flatMap() call then this would be a bug. Could you please verify that this is the case and maybe provide an example where this is observable?

Cheers,
Aljoscha
> On 08 Dec 2015, at 10:41, Matthias J. Sax <mj...@apache.org> wrote:
> 
> Hi,
> 
> I think (but please someone verify) that an OperatorState is actually 
> not required -- I think that "open()" is called after a failure and 
> recovery, too. So you can use a regular member variable to store the 
> data instead of an OperatorState. In case of failure, you just re-read 
> the data as on regular start-up.
> 
> -Matthias
> 
> 
> On 12/08/2015 09:38 AM, Radu Tudoran wrote:
>> Hi,
>> 
>> Thanks for the answer - it is helpful.
>> The issue that remains is why is the open function not being executed before the flatmap to load the data in the OperatorState. 
>> 
>> I used something like - and I observe that the dataset is not 
>> initialized when being used in the flatmap function
>> 
>> env.socketTextStream
>> .map() -> to transform data to a Tuple1<String>
>> .keyby(0) -> to enable the usage of the operatorState which I saw requires keyed structured
>> .flatmap(RichFlatMapFunction<Tuple1<String>, String>   -> the function
>> {
>> private OperatorState<String> dataset; @Override public void flatMap( 
>> { Dataset -> use ...is empty } @Override public void open( { dataset 
>> -> load }
>> })
>> 
>> 
>> 
>> Dr. Radu Tudoran
>> Research Engineer
>> IT R&D Division
>> 
>> 
>> HUAWEI TECHNOLOGIES Duesseldorf GmbH
>> European Research Center
>> Riesstrasse 25, 80992 München
>> 
>> E-mail: radu.tudoran@huawei.com
>> Mobile: +49 15209084330
>> Telephone: +49 891588344173
>> 
>> HUAWEI TECHNOLOGIES Duesseldorf GmbH
>> Hansaallee 205, 40549 Düsseldorf, Germany, www.huawei.com Registered 
>> Office: Düsseldorf, Register Court Düsseldorf, HRB 56063, Managing 
>> Director: Jingwen TAO, Wanzhou MENG, Lifang CHEN Sitz der 
>> Gesellschaft: Düsseldorf, Amtsgericht Düsseldorf, HRB 56063,
>> Geschäftsführer: Jingwen TAO, Wanzhou MENG, Lifang CHEN This e-mail 
>> and its attachments contain confidential information from HUAWEI, which is intended only for the person or entity whose address is listed above. Any use of the information contained herein in any way (including, but not limited to, total or partial disclosure, reproduction, or dissemination) by persons other than the intended recipient(s) is prohibited. If you receive this e-mail in error, please notify the sender by phone or email immediately and delete it!
>> 
>> -----Original Message-----
>> From: Matthias J. Sax [mailto:mjsax@apache.org]
>> Sent: Tuesday, December 08, 2015 8:42 AM
>> To: user@flink.apache.org
>> Subject: Re: Question about DataStream serialization
>> 
>> Hi Radu,
>> 
>> you are right. The open() method is called for each parallel instance of a rich function. Thus, if all instanced use the same code, you might read the same data multiple times.
>> 
>> The easiest was to distinguish different instanced within open() is to user the RuntimeContext. If offers two methods  "int getNumberOfParallelSubtasks()" and "int getIndexOfThisSubtask()" that you can use to compute your own partitioning within open().
>> 
>> For example (just a sketch):
>> 
>> @Override
>> public void open(Configuration parameters) throws Exception {  
>> RuntimeContext context = super.getRuntimeContext();  int dop = 
>> context.getNumberOfParallelSubtasks();
>>  int idx = context.getIndexOfThisSubtask();
>> 
>>  // open file
>>  // get size of file in bytes
>> 
>>  // seek to partition #idx:
>>  long seek = fileSize * idx / dop;
>> 
>>  // read "fileSize/dop" bytes
>> }
>> 
>> Hope this helps.
>> 
>> -Matthias
>> 
>> 
>> On 12/08/2015 04:28 AM, Radu Tudoran wrote:
>>> Hi,
>>> 
>>> 
>>> 
>>> Taking the example you mentioned of using RichFlatMapFunction and in 
>>> the
>>> open() reading a file.
>>> 
>>> Would this open function be executed on each node where the 
>>> RichFlatMapFunction gets executed? (I have done some tests and I 
>>> would get the feeling it does – but I wanted to double - check )
>>> 
>>> If so, would this mean that the same data will be loaded multiple 
>>> times on each parallel instance? Is there anyway, this can be 
>>> prevented and the data to be hashed and partitioned somehow across nodes?
>>> 
>>> 
>>> 
>>> Would using the operator state help?:
>>> 
>>> “
>>> 
>>> OperatorState*<*MyList<String>*>*dataset*;*
>>> 
>>> ”
>>> 
>>> I would be curious in this case how could the open function look 
>>> like to initialize the data for this operator state:
>>> 
>>> 
>>> 
>>> 
>>> 
>>> I have tried to just read a file and write it into the dataset, but 
>>> I encountered a strange behavior that would look like the flatmap 
>>> function gets executed before the open function, which leads to 
>>> using an empty dataset in the flatmap function while when this 
>>> finish executing the dataset gets loaded. Is this an error or I am doing something wrong?
>>> 
>>> 
>>> 
>>> 
>>> 
>>> 
>>> 
>>> Dr. Radu Tudoran
>>> 
>>> Research Engineer
>>> 
>>> IT R&D Division
>>> 
>>> 
>>> 
>>> cid:image007.jpg@01CD52EB.AD060EE0
>>> 
>>> HUAWEI TECHNOLOGIES Duesseldorf GmbH
>>> 
>>> European Research Center
>>> 
>>> Riesstrasse 25, 80992 München
>>> 
>>> 
>>> 
>>> E-mail: _radu.tudoran@huawei.com_
>>> 
>>> Mobile: +49 15209084330
>>> 
>>> Telephone: +49 891588344173
>>> 
>>> 
>>> 
>>> HUAWEI TECHNOLOGIES Duesseldorf GmbH Hansaallee 205, 40549 
>>> Düsseldorf, Germany, www.huawei.com <http://www.huawei.com/> 
>>> Registered Office: Düsseldorf, Register Court Düsseldorf, HRB 56063, 
>>> Managing Director: Jingwen TAO, Wanzhou MENG, Lifang CHEN Sitz der 
>>> Gesellschaft: Düsseldorf, Amtsgericht Düsseldorf, HRB 56063,
>>> Geschäftsführer: Jingwen TAO, Wanzhou MENG, Lifang CHEN
>>> 
>>> This e-mail and its attachments contain confidential information 
>>> from HUAWEI, which is intended only for the person or entity whose 
>>> address is listed above. Any use of the information contained herein 
>>> in any way (including, but not limited to, total or partial 
>>> disclosure, reproduction, or dissemination) by persons other than 
>>> the intended
>>> recipient(s) is prohibited. If you receive this e-mail in error, 
>>> please notify the sender by phone or email immediately and delete it!
>>> 
>>> 
>>> 
>>> *From:*Robert Metzger [mailto:rmetzger@apache.org]
>>> *Sent:* Tuesday, December 01, 2015 6:21 PM
>>> *To:* user@flink.apache.org
>>> *Cc:* Goetz Brasche
>>> *Subject:* Re: Question about DataStream serialization
>>> 
>>> 
>>> 
>>> Hi Radu,
>>> 
>>> 
>>> 
>>> both emails reached the mailing list :)
>>> 
>>> 
>>> 
>>> You can not reference to DataSets or DataStreams from inside user 
>>> defined functions. Both are just abstractions for a data set or 
>>> stream, so the elements are not really inside the set.
>>> 
>>> 
>>> 
>>> We don't have any support for mixing the DataSet and DataStream API.
>>> 
>>> 
>>> 
>>> For your use case, I would recommend you to use a 
>>> RichFlatMapFunction and in the open() call read the text file.
>>> 
>>> 
>>> 
>>> 
>>> 
>>> 
>>> 
>>> On Tue, Dec 1, 2015 at 5:03 PM, Radu Tudoran 
>>> <radu.tudoran@huawei.com <ma...@huawei.com>> wrote:
>>> 
>>> 
>>> 
>>> Hello,
>>> 
>>> 
>>> 
>>> I am not sure if this message was received on the user list, if so I 
>>> apologies for duplicate messages
>>> 
>>> 
>>> 
>>> I have the following scenario
>>> 
>>> 
>>> 
>>> ·         Reading a fixed set
>>> 
>>> DataStream<String> /fixedset/ = env.readtextFile(…
>>> 
>>> ·         Reading a continuous stream of data
>>> 
>>> DataStream<String> /stream/ = ….
>>> 
>>> 
>>> 
>>> I would need that for each event read from the continuous stream to 
>>> make some operations onit and on the /fixedsettoghether/
>>> 
>>> 
>>> 
>>> 
>>> 
>>> I have tried something like
>>> 
>>> 
>>> 
>>> final myObject.referenceStaticSet = fixedset;
>>> 
>>> stream.map(new MapFunction<String, String>() {
>>> 
>>>                     @Override
>>> 
>>>                     public String map(String arg0) throws Exception 
>>> {
>>> 
>>> 
>>> 
>>>                           //for example:   final string2add = arg0;
>>> 
>>>                                                                //the 
>>> goal of below function would be to add the string2add to the 
>>> fixedset
>>> 
>>>                           myObject.referenceStaticSet = 
>>> myObject.referenceStaticSet.flatMap(new FlatMapFunction<String,
>>> String>() {
>>> 
>>> 
>>> 
>>>                                  @Override
>>> 
>>>                                  public void flatMap(String arg0, 
>>> Collector<String> arg1)
>>> 
>>> 
>>> //for example adding to the fixed set also the string2add object:  
>>> arg1.collect(string2add);
>>> 
>>> 
>>>                                }
>>> 
>>> …
>>> 
>>> }
>>> 
>>> 
>>> 
>>> However,  I get an exception (Exception in thread "main"
>>> _org.apache.flink.api.common.InvalidProgramException_: ) that object 
>>> is not serializable (Object MyClass$3@a71081 not serializable )
>>> 
>>> 
>>> 
>>> Looking into this I see that the issues is that the DataStream<> is 
>>> not serializable. What would be the solution to this issue?
>>> 
>>> 
>>> 
>>> As I mentioned before, I would like that for each event from the 
>>> continuous stream to use the initial fixed set, add the event to it 
>>> and apply an operation.
>>> 
>>> Stephan was mentioning at some point some possibility to create a 
>>> DataSet and launch a batch processing while operating in stream 
>>> mode– in case this is possible, can you give me a reference for it, 
>>> because it might be the good solution to  use in case. I am thinking 
>>> that I could keep the fixed set as a DataSet and as each new event 
>>> comes, transform it into a dataset and then join with reference set 
>>> and apply an operation
>>> 
>>> 
>>> 
>>> Regards,
>>> 
>>> 
>>> 
>>> 
>>> 
>>> 
>>> 
>>> 
>>> 
>>> Dr. Radu Tudoran
>>> 
>>> Research Engineer
>>> 
>>> IT R&D Division
>>> 
>>> 
>>> 
>>> cid:image007.jpg@01CD52EB.AD060EE0
>>> 
>>> HUAWEI TECHNOLOGIES Duesseldorf GmbH
>>> 
>>> European Research Center
>>> 
>>> Riesstrasse 25, 80992 München
>>> 
>>> 
>>> 
>>> E-mail: _radu.tudoran@huawei.com <ma...@huawei.com>_
>>> 
>>> Mobile: +49 15209084330 <tel:%2B49%2015209084330>
>>> 
>>> Telephone: +49 891588344173 <tel:%2B49%20891588344173>
>>> 
>>> 
>>> 
>>> HUAWEI TECHNOLOGIES Duesseldorf GmbH Hansaallee 205, 40549 
>>> Düsseldorf, Germany, www.huawei.com <http://www.huawei.com/> 
>>> Registered Office: Düsseldorf, Register Court Düsseldorf, HRB 56063, 
>>> Managing Director: Jingwen TAO, Wanzhou MENG, Lifang CHEN Sitz der 
>>> Gesellschaft: Düsseldorf, Amtsgericht Düsseldorf, HRB 56063,
>>> Geschäftsführer: Jingwen TAO, Wanzhou MENG, Lifang CHEN
>>> 
>>> This e-mail and its attachments contain confidential information 
>>> from HUAWEI, which is intended only for the person or entity whose 
>>> address is listed above. Any use of the information contained herein 
>>> in any way (including, but not limited to, total or partial 
>>> disclosure, reproduction, or dissemination) by persons other than 
>>> the intended
>>> recipient(s) is prohibited. If you receive this e-mail in error, 
>>> please notify the sender by phone or email immediately and delete it!
>>> 
>>> 
>>> 
>>> *From:*Vieru, Mihail [mailto:mihail.vieru@zalando.de 
>>> <ma...@zalando.de>]
>>> *Sent:* Tuesday, December 01, 2015 4:55 PM
>>> *To:* user@flink.apache.org <ma...@flink.apache.org>
>>> *Subject:* NPE with Flink Streaming from Kafka
>>> 
>>> 
>>> 
>>> Hi,
>>> 
>>> we get the following NullPointerException after ~50 minutes when 
>>> running a streaming job with windowing and state that reads data 
>>> from Kafka and writes the result to local FS.
>>> 
>>> There are around 170 million messages to be processed, Flink 0.10.1 
>>> stops at ~8 million.
>>> 
>>> Flink runs locally, started with the "start-cluster-streaming.sh" script.
>>> 
>>> 
>>> 12/01/2015 15:06:24    Job execution switched to status RUNNING.
>>> 12/01/2015 15:06:24    Source: Custom Source -> Map -> Map(1/1) switched
>>> to SCHEDULED
>>> 12/01/2015 15:06:24    Source: Custom Source -> Map -> Map(1/1) switched
>>> to DEPLOYING
>>> 12/01/2015 15:06:24    Fast TumblingTimeWindows(5000) of Reduce at
>>> main(ItemPriceAvgPerOrder.java:108) -> Sink: Unnamed(1/1) switched 
>>> to SCHEDULED
>>> 12/01/2015 15:06:24    Fast TumblingTimeWindows(5000) of Reduce at
>>> main(ItemPriceAvgPerOrder.java:108) -> Sink: Unnamed(1/1) switched 
>>> to DEPLOYING
>>> 12/01/2015 15:06:24    Source: Custom Source -> Map -> Map(1/1) switched
>>> to RUNNING
>>> 12/01/2015 15:06:24    Fast TumblingTimeWindows(5000) of Reduce at
>>> main(ItemPriceAvgPerOrder.java:108) -> Sink: Unnamed(1/1) switched 
>>> to RUNNING
>>> 12/01/2015 15:56:08    Fast TumblingTimeWindows(5000) of Reduce at
>>> main(ItemPriceAvgPerOrder.java:108) -> Sink: Unnamed(1/1) switched 
>>> to CANCELED
>>> 12/01/2015 15:56:08    Source: Custom Source -> Map -> Map(1/1) switched
>>> to FAILED
>>> java.lang.Exception
>>>    at
>>> org.apache.flink.streaming.connectors.kafka.internals.LegacyFetcher.run(LegacyFetcher.java:242)
>>>    at
>>> org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer.run(FlinkKafkaConsumer.java:397)
>>>    at
>>> org.apache.flink.streaming.api.operators.StreamSource.run(StreamSource.java:58)
>>>    at
>>> org.apache.flink.streaming.runtime.tasks.SourceStreamTask.run(SourceStreamTask.java:55)
>>>    at
>>> org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:218)
>>>    at org.apache.flink.runtime.taskmanager.Task.run(Task.java:584)
>>>    at java.lang.Thread.run(Thread.java:745)
>>> Caused by: java.lang.NullPointerException
>>>    at
>>> org.I0Itec.zkclient.ZkConnection.writeDataReturnStat(ZkConnection.java:115)
>>>    at org.I0Itec.zkclient.ZkClient$10.call(ZkClient.java:817)
>>>    at org.I0Itec.zkclient.ZkClient.retryUntilConnected(ZkClient.java:675)
>>>    at org.I0Itec.zkclient.ZkClient.writeDataReturnStat(ZkClient.java:813)
>>>    at org.I0Itec.zkclient.ZkClient.writeData(ZkClient.java:808)
>>>    at org.I0Itec.zkclient.ZkClient.writeData(ZkClient.java:777)
>>>    at kafka.utils.ZkUtils$.updatePersistentPath(ZkUtils.scala:332)
>>>    at kafka.utils.ZkUtils.updatePersistentPath(ZkUtils.scala)
>>>    at
>>> org.apache.flink.streaming.connectors.kafka.internals.ZookeeperOffsetHandler.setOffsetInZooKeeper(ZookeeperOffsetHandler.java:112)
>>>    at
>>> org.apache.flink.streaming.connectors.kafka.internals.ZookeeperOffsetHandler.commit(ZookeeperOffsetHandler.java:80)
>>>    at
>>> org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer$Perio
>>> di
>>> cOffsetCommitter.run(FlinkKafkaConsumer.java:632)
>>> 
>>> Any ideas on what could cause this behaviour?
>>> 
>>> 
>>> 
>>> Best,
>>> 
>>> Mihail
>>> 
>>> 
>>> 
>> 
> 


Re: Question about DataStream serialization

Posted by Aljoscha Krettek <al...@apache.org>.
Hi,
if the open() method is indeed not called before the first flatMap() call then this would be a bug. Could you please verify that this is the case and maybe provide an example where this is observable?

Cheers,
Aljoscha
> On 08 Dec 2015, at 10:41, Matthias J. Sax <mj...@apache.org> wrote:
> 
> Hi,
> 
> I think (but please someone verify) that an OperatorState is actually
> not required -- I think that "open()" is called after a failure and
> recovery, too. So you can use a regular member variable to store the
> data instead of an OperatorState. In case of failure, you just re-read
> the data as on regular start-up.
> 
> -Matthias
> 
> 
> On 12/08/2015 09:38 AM, Radu Tudoran wrote:
>> Hi,
>> 
>> Thanks for the answer - it is helpful.
>> The issue that remains is why is the open function not being executed before the flatmap to load the data in the OperatorState. 
>> 
>> I used something like - and I observe that the dataset is not initialized when being used in the flatmap function
>> 
>> env.socketTextStream
>> .map() -> to transform data to a Tuple1<String>
>> .keyby(0) -> to enable the usage of the operatorState which I saw requires keyed structured
>> .flatmap(RichFlatMapFunction<Tuple1<String>, String>   -> the function
>> {
>> private OperatorState<String> dataset;
>> @Override
>> public void flatMap(
>> {
>> Dataset -> use ...is empty
>> }
>> @Override
>> public void open(
>> {
>> dataset -> load 
>> }
>> })
>> 
>> 
>> 
>> Dr. Radu Tudoran
>> Research Engineer
>> IT R&D Division
>> 
>> 
>> HUAWEI TECHNOLOGIES Duesseldorf GmbH
>> European Research Center
>> Riesstrasse 25, 80992 München
>> 
>> E-mail: radu.tudoran@huawei.com
>> Mobile: +49 15209084330
>> Telephone: +49 891588344173
>> 
>> HUAWEI TECHNOLOGIES Duesseldorf GmbH
>> Hansaallee 205, 40549 Düsseldorf, Germany, www.huawei.com
>> Registered Office: Düsseldorf, Register Court Düsseldorf, HRB 56063,
>> Managing Director: Jingwen TAO, Wanzhou MENG, Lifang CHEN
>> Sitz der Gesellschaft: Düsseldorf, Amtsgericht Düsseldorf, HRB 56063,
>> Geschäftsführer: Jingwen TAO, Wanzhou MENG, Lifang CHEN
>> This e-mail and its attachments contain confidential information from HUAWEI, which is intended only for the person or entity whose address is listed above. Any use of the information contained herein in any way (including, but not limited to, total or partial disclosure, reproduction, or dissemination) by persons other than the intended recipient(s) is prohibited. If you receive this e-mail in error, please notify the sender by phone or email immediately and delete it!
>> 
>> -----Original Message-----
>> From: Matthias J. Sax [mailto:mjsax@apache.org] 
>> Sent: Tuesday, December 08, 2015 8:42 AM
>> To: user@flink.apache.org
>> Subject: Re: Question about DataStream serialization
>> 
>> Hi Radu,
>> 
>> you are right. The open() method is called for each parallel instance of a rich function. Thus, if all instanced use the same code, you might read the same data multiple times.
>> 
>> The easiest was to distinguish different instanced within open() is to user the RuntimeContext. If offers two methods  "int getNumberOfParallelSubtasks()" and "int getIndexOfThisSubtask()" that you can use to compute your own partitioning within open().
>> 
>> For example (just a sketch):
>> 
>> @Override
>> public void open(Configuration parameters) throws Exception {
>>  RuntimeContext context = super.getRuntimeContext();
>>  int dop = context.getNumberOfParallelSubtasks();
>>  int idx = context.getIndexOfThisSubtask();
>> 
>>  // open file
>>  // get size of file in bytes
>> 
>>  // seek to partition #idx:
>>  long seek = fileSize * idx / dop;
>> 
>>  // read "fileSize/dop" bytes
>> }
>> 
>> Hope this helps.
>> 
>> -Matthias
>> 
>> 
>> On 12/08/2015 04:28 AM, Radu Tudoran wrote:
>>> Hi,
>>> 
>>> 
>>> 
>>> Taking the example you mentioned of using RichFlatMapFunction and in 
>>> the
>>> open() reading a file.
>>> 
>>> Would this open function be executed on each node where the 
>>> RichFlatMapFunction gets executed? (I have done some tests and I would 
>>> get the feeling it does – but I wanted to double - check )
>>> 
>>> If so, would this mean that the same data will be loaded multiple 
>>> times on each parallel instance? Is there anyway, this can be 
>>> prevented and the data to be hashed and partitioned somehow across nodes?
>>> 
>>> 
>>> 
>>> Would using the operator state help?:
>>> 
>>> “
>>> 
>>> OperatorState*<*MyList<String>*>*dataset*;*
>>> 
>>> ”
>>> 
>>> I would be curious in this case how could the open function look like 
>>> to initialize the data for this operator state:
>>> 
>>> 
>>> 
>>> 
>>> 
>>> I have tried to just read a file and write it into the dataset, but I 
>>> encountered a strange behavior that would look like the flatmap 
>>> function gets executed before the open function, which leads to using 
>>> an empty dataset in the flatmap function while when this finish 
>>> executing the dataset gets loaded. Is this an error or I am doing something wrong?
>>> 
>>> 
>>> 
>>> 
>>> 
>>> 
>>> 
>>> Dr. Radu Tudoran
>>> 
>>> Research Engineer
>>> 
>>> IT R&D Division
>>> 
>>> 
>>> 
>>> cid:image007.jpg@01CD52EB.AD060EE0
>>> 
>>> HUAWEI TECHNOLOGIES Duesseldorf GmbH
>>> 
>>> European Research Center
>>> 
>>> Riesstrasse 25, 80992 München
>>> 
>>> 
>>> 
>>> E-mail: _radu.tudoran@huawei.com_
>>> 
>>> Mobile: +49 15209084330
>>> 
>>> Telephone: +49 891588344173
>>> 
>>> 
>>> 
>>> HUAWEI TECHNOLOGIES Duesseldorf GmbH
>>> Hansaallee 205, 40549 Düsseldorf, Germany, www.huawei.com 
>>> <http://www.huawei.com/> Registered Office: Düsseldorf, Register Court 
>>> Düsseldorf, HRB 56063, Managing Director: Jingwen TAO, Wanzhou MENG, 
>>> Lifang CHEN Sitz der Gesellschaft: Düsseldorf, Amtsgericht Düsseldorf, 
>>> HRB 56063,
>>> Geschäftsführer: Jingwen TAO, Wanzhou MENG, Lifang CHEN
>>> 
>>> This e-mail and its attachments contain confidential information from 
>>> HUAWEI, which is intended only for the person or entity whose address 
>>> is listed above. Any use of the information contained herein in any 
>>> way (including, but not limited to, total or partial disclosure, 
>>> reproduction, or dissemination) by persons other than the intended
>>> recipient(s) is prohibited. If you receive this e-mail in error, 
>>> please notify the sender by phone or email immediately and delete it!
>>> 
>>> 
>>> 
>>> *From:*Robert Metzger [mailto:rmetzger@apache.org]
>>> *Sent:* Tuesday, December 01, 2015 6:21 PM
>>> *To:* user@flink.apache.org
>>> *Cc:* Goetz Brasche
>>> *Subject:* Re: Question about DataStream serialization
>>> 
>>> 
>>> 
>>> Hi Radu,
>>> 
>>> 
>>> 
>>> both emails reached the mailing list :)
>>> 
>>> 
>>> 
>>> You can not reference to DataSets or DataStreams from inside user 
>>> defined functions. Both are just abstractions for a data set or 
>>> stream, so the elements are not really inside the set.
>>> 
>>> 
>>> 
>>> We don't have any support for mixing the DataSet and DataStream API.
>>> 
>>> 
>>> 
>>> For your use case, I would recommend you to use a RichFlatMapFunction 
>>> and in the open() call read the text file.
>>> 
>>> 
>>> 
>>> 
>>> 
>>> 
>>> 
>>> On Tue, Dec 1, 2015 at 5:03 PM, Radu Tudoran <radu.tudoran@huawei.com 
>>> <ma...@huawei.com>> wrote:
>>> 
>>> 
>>> 
>>> Hello,
>>> 
>>> 
>>> 
>>> I am not sure if this message was received on the user list, if so I 
>>> apologies for duplicate messages
>>> 
>>> 
>>> 
>>> I have the following scenario
>>> 
>>> 
>>> 
>>> ·         Reading a fixed set
>>> 
>>> DataStream<String> /fixedset/ = env.readtextFile(…
>>> 
>>> ·         Reading a continuous stream of data
>>> 
>>> DataStream<String> /stream/ = ….
>>> 
>>> 
>>> 
>>> I would need that for each event read from the continuous stream to 
>>> make some operations onit and on the /fixedsettoghether/
>>> 
>>> 
>>> 
>>> 
>>> 
>>> I have tried something like
>>> 
>>> 
>>> 
>>> final myObject.referenceStaticSet = fixedset;
>>> 
>>> stream.map(new MapFunction<String, String>() {
>>> 
>>>                     @Override
>>> 
>>>                     public String map(String arg0) throws Exception {
>>> 
>>> 
>>> 
>>>                           //for example:   final string2add = arg0;
>>> 
>>>                                                                //the 
>>> goal of below function would be to add the string2add to the fixedset
>>> 
>>>                           myObject.referenceStaticSet = 
>>> myObject.referenceStaticSet.flatMap(new FlatMapFunction<String, 
>>> String>() {
>>> 
>>> 
>>> 
>>>                                  @Override
>>> 
>>>                                  public void flatMap(String arg0, 
>>> Collector<String> arg1)
>>> 
>>> 
>>> //for example adding to the fixed set also the string2add object:  
>>> arg1.collect(string2add);
>>> 
>>> 
>>>                                }
>>> 
>>> …
>>> 
>>> }
>>> 
>>> 
>>> 
>>> However,  I get an exception (Exception in thread "main"
>>> _org.apache.flink.api.common.InvalidProgramException_: ) that object 
>>> is not serializable (Object MyClass$3@a71081 not serializable )
>>> 
>>> 
>>> 
>>> Looking into this I see that the issues is that the DataStream<> is 
>>> not serializable. What would be the solution to this issue?
>>> 
>>> 
>>> 
>>> As I mentioned before, I would like that for each event from the 
>>> continuous stream to use the initial fixed set, add the event to it 
>>> and apply an operation.
>>> 
>>> Stephan was mentioning at some point some possibility to create a 
>>> DataSet and launch a batch processing while operating in stream mode– 
>>> in case this is possible, can you give me a reference for it, because 
>>> it might be the good solution to  use in case. I am thinking that I 
>>> could keep the fixed set as a DataSet and as each new event comes, 
>>> transform it into a dataset and then join with reference set and apply 
>>> an operation
>>> 
>>> 
>>> 
>>> Regards,
>>> 
>>> 
>>> 
>>> 
>>> 
>>> 
>>> 
>>> 
>>> 
>>> Dr. Radu Tudoran
>>> 
>>> Research Engineer
>>> 
>>> IT R&D Division
>>> 
>>> 
>>> 
>>> cid:image007.jpg@01CD52EB.AD060EE0
>>> 
>>> HUAWEI TECHNOLOGIES Duesseldorf GmbH
>>> 
>>> European Research Center
>>> 
>>> Riesstrasse 25, 80992 München
>>> 
>>> 
>>> 
>>> E-mail: _radu.tudoran@huawei.com <ma...@huawei.com>_
>>> 
>>> Mobile: +49 15209084330 <tel:%2B49%2015209084330>
>>> 
>>> Telephone: +49 891588344173 <tel:%2B49%20891588344173>
>>> 
>>> 
>>> 
>>> HUAWEI TECHNOLOGIES Duesseldorf GmbH
>>> Hansaallee 205, 40549 Düsseldorf, Germany, www.huawei.com 
>>> <http://www.huawei.com/> Registered Office: Düsseldorf, Register Court 
>>> Düsseldorf, HRB 56063, Managing Director: Jingwen TAO, Wanzhou MENG, 
>>> Lifang CHEN Sitz der Gesellschaft: Düsseldorf, Amtsgericht Düsseldorf, 
>>> HRB 56063,
>>> Geschäftsführer: Jingwen TAO, Wanzhou MENG, Lifang CHEN
>>> 
>>> This e-mail and its attachments contain confidential information from 
>>> HUAWEI, which is intended only for the person or entity whose address 
>>> is listed above. Any use of the information contained herein in any 
>>> way (including, but not limited to, total or partial disclosure, 
>>> reproduction, or dissemination) by persons other than the intended
>>> recipient(s) is prohibited. If you receive this e-mail in error, 
>>> please notify the sender by phone or email immediately and delete it!
>>> 
>>> 
>>> 
>>> *From:*Vieru, Mihail [mailto:mihail.vieru@zalando.de 
>>> <ma...@zalando.de>]
>>> *Sent:* Tuesday, December 01, 2015 4:55 PM
>>> *To:* user@flink.apache.org <ma...@flink.apache.org>
>>> *Subject:* NPE with Flink Streaming from Kafka
>>> 
>>> 
>>> 
>>> Hi,
>>> 
>>> we get the following NullPointerException after ~50 minutes when 
>>> running a streaming job with windowing and state that reads data from 
>>> Kafka and writes the result to local FS.
>>> 
>>> There are around 170 million messages to be processed, Flink 0.10.1 
>>> stops at ~8 million.
>>> 
>>> Flink runs locally, started with the "start-cluster-streaming.sh" script.
>>> 
>>> 
>>> 12/01/2015 15:06:24    Job execution switched to status RUNNING.
>>> 12/01/2015 15:06:24    Source: Custom Source -> Map -> Map(1/1) switched
>>> to SCHEDULED
>>> 12/01/2015 15:06:24    Source: Custom Source -> Map -> Map(1/1) switched
>>> to DEPLOYING
>>> 12/01/2015 15:06:24    Fast TumblingTimeWindows(5000) of Reduce at
>>> main(ItemPriceAvgPerOrder.java:108) -> Sink: Unnamed(1/1) switched to 
>>> SCHEDULED
>>> 12/01/2015 15:06:24    Fast TumblingTimeWindows(5000) of Reduce at
>>> main(ItemPriceAvgPerOrder.java:108) -> Sink: Unnamed(1/1) switched to 
>>> DEPLOYING
>>> 12/01/2015 15:06:24    Source: Custom Source -> Map -> Map(1/1) switched
>>> to RUNNING
>>> 12/01/2015 15:06:24    Fast TumblingTimeWindows(5000) of Reduce at
>>> main(ItemPriceAvgPerOrder.java:108) -> Sink: Unnamed(1/1) switched to 
>>> RUNNING
>>> 12/01/2015 15:56:08    Fast TumblingTimeWindows(5000) of Reduce at
>>> main(ItemPriceAvgPerOrder.java:108) -> Sink: Unnamed(1/1) switched to 
>>> CANCELED
>>> 12/01/2015 15:56:08    Source: Custom Source -> Map -> Map(1/1) switched
>>> to FAILED
>>> java.lang.Exception
>>>    at
>>> org.apache.flink.streaming.connectors.kafka.internals.LegacyFetcher.run(LegacyFetcher.java:242)
>>>    at
>>> org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer.run(FlinkKafkaConsumer.java:397)
>>>    at
>>> org.apache.flink.streaming.api.operators.StreamSource.run(StreamSource.java:58)
>>>    at
>>> org.apache.flink.streaming.runtime.tasks.SourceStreamTask.run(SourceStreamTask.java:55)
>>>    at
>>> org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:218)
>>>    at org.apache.flink.runtime.taskmanager.Task.run(Task.java:584)
>>>    at java.lang.Thread.run(Thread.java:745)
>>> Caused by: java.lang.NullPointerException
>>>    at
>>> org.I0Itec.zkclient.ZkConnection.writeDataReturnStat(ZkConnection.java:115)
>>>    at org.I0Itec.zkclient.ZkClient$10.call(ZkClient.java:817)
>>>    at org.I0Itec.zkclient.ZkClient.retryUntilConnected(ZkClient.java:675)
>>>    at org.I0Itec.zkclient.ZkClient.writeDataReturnStat(ZkClient.java:813)
>>>    at org.I0Itec.zkclient.ZkClient.writeData(ZkClient.java:808)
>>>    at org.I0Itec.zkclient.ZkClient.writeData(ZkClient.java:777)
>>>    at kafka.utils.ZkUtils$.updatePersistentPath(ZkUtils.scala:332)
>>>    at kafka.utils.ZkUtils.updatePersistentPath(ZkUtils.scala)
>>>    at
>>> org.apache.flink.streaming.connectors.kafka.internals.ZookeeperOffsetHandler.setOffsetInZooKeeper(ZookeeperOffsetHandler.java:112)
>>>    at
>>> org.apache.flink.streaming.connectors.kafka.internals.ZookeeperOffsetHandler.commit(ZookeeperOffsetHandler.java:80)
>>>    at
>>> org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer$Periodi
>>> cOffsetCommitter.run(FlinkKafkaConsumer.java:632)
>>> 
>>> Any ideas on what could cause this behaviour?
>>> 
>>> 
>>> 
>>> Best,
>>> 
>>> Mihail
>>> 
>>> 
>>> 
>> 
> 


Re: Question about DataStream serialization

Posted by "Matthias J. Sax" <mj...@apache.org>.
Hi,

I think (but please someone verify) that an OperatorState is actually
not required -- I think that "open()" is called after a failure and
recovery, too. So you can use a regular member variable to store the
data instead of an OperatorState. In case of failure, you just re-read
the data as on regular start-up.

-Matthias


On 12/08/2015 09:38 AM, Radu Tudoran wrote:
> Hi,
> 
> Thanks for the answer - it is helpful.
> The issue that remains is why is the open function not being executed before the flatmap to load the data in the OperatorState. 
> 
> I used something like - and I observe that the dataset is not initialized when being used in the flatmap function
> 
> env.socketTextStream
> .map() -> to transform data to a Tuple1<String>
> .keyby(0) -> to enable the usage of the operatorState which I saw requires keyed structured
> .flatmap(RichFlatMapFunction<Tuple1<String>, String>   -> the function
> {
> private OperatorState<String> dataset;
> @Override
> public void flatMap(
> {
> Dataset -> use ...is empty
> }
> @Override
> public void open(
> {
> dataset -> load 
> }
> })
> 
> 
> 
> Dr. Radu Tudoran
> Research Engineer
> IT R&D Division
> 
> 
> HUAWEI TECHNOLOGIES Duesseldorf GmbH
> European Research Center
> Riesstrasse 25, 80992 München
> 
> E-mail: radu.tudoran@huawei.com
> Mobile: +49 15209084330
> Telephone: +49 891588344173
> 
> HUAWEI TECHNOLOGIES Duesseldorf GmbH
> Hansaallee 205, 40549 Düsseldorf, Germany, www.huawei.com
> Registered Office: Düsseldorf, Register Court Düsseldorf, HRB 56063,
> Managing Director: Jingwen TAO, Wanzhou MENG, Lifang CHEN
> Sitz der Gesellschaft: Düsseldorf, Amtsgericht Düsseldorf, HRB 56063,
> Geschäftsführer: Jingwen TAO, Wanzhou MENG, Lifang CHEN
> This e-mail and its attachments contain confidential information from HUAWEI, which is intended only for the person or entity whose address is listed above. Any use of the information contained herein in any way (including, but not limited to, total or partial disclosure, reproduction, or dissemination) by persons other than the intended recipient(s) is prohibited. If you receive this e-mail in error, please notify the sender by phone or email immediately and delete it!
> 
> -----Original Message-----
> From: Matthias J. Sax [mailto:mjsax@apache.org] 
> Sent: Tuesday, December 08, 2015 8:42 AM
> To: user@flink.apache.org
> Subject: Re: Question about DataStream serialization
> 
> Hi Radu,
> 
> you are right. The open() method is called for each parallel instance of a rich function. Thus, if all instanced use the same code, you might read the same data multiple times.
> 
> The easiest was to distinguish different instanced within open() is to user the RuntimeContext. If offers two methods  "int getNumberOfParallelSubtasks()" and "int getIndexOfThisSubtask()" that you can use to compute your own partitioning within open().
> 
> For example (just a sketch):
> 
> @Override
> public void open(Configuration parameters) throws Exception {
>   RuntimeContext context = super.getRuntimeContext();
>   int dop = context.getNumberOfParallelSubtasks();
>   int idx = context.getIndexOfThisSubtask();
> 
>   // open file
>   // get size of file in bytes
> 
>   // seek to partition #idx:
>   long seek = fileSize * idx / dop;
> 
>   // read "fileSize/dop" bytes
> }
> 
> Hope this helps.
> 
> -Matthias
> 
> 
> On 12/08/2015 04:28 AM, Radu Tudoran wrote:
>> Hi,
>>
>>  
>>
>> Taking the example you mentioned of using RichFlatMapFunction and in 
>> the
>> open() reading a file.
>>
>> Would this open function be executed on each node where the 
>> RichFlatMapFunction gets executed? (I have done some tests and I would 
>> get the feeling it does – but I wanted to double - check )
>>
>> If so, would this mean that the same data will be loaded multiple 
>> times on each parallel instance? Is there anyway, this can be 
>> prevented and the data to be hashed and partitioned somehow across nodes?
>>
>>  
>>
>> Would using the operator state help?:
>>
>> “
>>
>> OperatorState*<*MyList<String>*>*dataset*;*
>>
>> ”
>>
>> I would be curious in this case how could the open function look like 
>> to initialize the data for this operator state:
>>
>>  
>>
>>  
>>
>> I have tried to just read a file and write it into the dataset, but I 
>> encountered a strange behavior that would look like the flatmap 
>> function gets executed before the open function, which leads to using 
>> an empty dataset in the flatmap function while when this finish 
>> executing the dataset gets loaded. Is this an error or I am doing something wrong?
>>
>>  
>>
>>  
>>
>>  
>>
>> Dr. Radu Tudoran
>>
>> Research Engineer
>>
>> IT R&D Division
>>
>>  
>>
>> cid:image007.jpg@01CD52EB.AD060EE0
>>
>> HUAWEI TECHNOLOGIES Duesseldorf GmbH
>>
>> European Research Center
>>
>> Riesstrasse 25, 80992 München
>>
>>  
>>
>> E-mail: _radu.tudoran@huawei.com_
>>
>> Mobile: +49 15209084330
>>
>> Telephone: +49 891588344173
>>
>>  
>>
>> HUAWEI TECHNOLOGIES Duesseldorf GmbH
>> Hansaallee 205, 40549 Düsseldorf, Germany, www.huawei.com 
>> <http://www.huawei.com/> Registered Office: Düsseldorf, Register Court 
>> Düsseldorf, HRB 56063, Managing Director: Jingwen TAO, Wanzhou MENG, 
>> Lifang CHEN Sitz der Gesellschaft: Düsseldorf, Amtsgericht Düsseldorf, 
>> HRB 56063,
>> Geschäftsführer: Jingwen TAO, Wanzhou MENG, Lifang CHEN
>>
>> This e-mail and its attachments contain confidential information from 
>> HUAWEI, which is intended only for the person or entity whose address 
>> is listed above. Any use of the information contained herein in any 
>> way (including, but not limited to, total or partial disclosure, 
>> reproduction, or dissemination) by persons other than the intended
>> recipient(s) is prohibited. If you receive this e-mail in error, 
>> please notify the sender by phone or email immediately and delete it!
>>
>>  
>>
>> *From:*Robert Metzger [mailto:rmetzger@apache.org]
>> *Sent:* Tuesday, December 01, 2015 6:21 PM
>> *To:* user@flink.apache.org
>> *Cc:* Goetz Brasche
>> *Subject:* Re: Question about DataStream serialization
>>
>>  
>>
>> Hi Radu,
>>
>>  
>>
>> both emails reached the mailing list :)
>>
>>  
>>
>> You can not reference to DataSets or DataStreams from inside user 
>> defined functions. Both are just abstractions for a data set or 
>> stream, so the elements are not really inside the set.
>>
>>  
>>
>> We don't have any support for mixing the DataSet and DataStream API.
>>
>>  
>>
>> For your use case, I would recommend you to use a RichFlatMapFunction 
>> and in the open() call read the text file.
>>
>>  
>>
>>  
>>
>>  
>>
>> On Tue, Dec 1, 2015 at 5:03 PM, Radu Tudoran <radu.tudoran@huawei.com 
>> <ma...@huawei.com>> wrote:
>>
>>  
>>
>> Hello,
>>
>>  
>>
>> I am not sure if this message was received on the user list, if so I 
>> apologies for duplicate messages
>>
>>  
>>
>> I have the following scenario
>>
>>  
>>
>> ·         Reading a fixed set
>>
>> DataStream<String> /fixedset/ = env.readtextFile(…
>>
>> ·         Reading a continuous stream of data
>>
>> DataStream<String> /stream/ = ….
>>
>>  
>>
>> I would need that for each event read from the continuous stream to 
>> make some operations onit and on the /fixedsettoghether/
>>
>>  
>>
>>  
>>
>> I have tried something like
>>
>>  
>>
>> final myObject.referenceStaticSet = fixedset;
>>
>> stream.map(new MapFunction<String, String>() {
>>
>>                      @Override
>>
>>                      public String map(String arg0) throws Exception {
>>
>>                           
>>
>>                            //for example:   final string2add = arg0;
>>
>>                                                                 //the 
>> goal of below function would be to add the string2add to the fixedset
>>
>>                            myObject.referenceStaticSet = 
>> myObject.referenceStaticSet.flatMap(new FlatMapFunction<String, 
>> String>() {
>>
>>                           
>>
>>                                   @Override
>>
>>                                   public void flatMap(String arg0, 
>> Collector<String> arg1)
>>
>>                                                                                
>> //for example adding to the fixed set also the string2add object:  
>> arg1.collect(string2add);
>>
>>                                                
>>                                 }
>>
>> …
>>
>> }
>>
>>  
>>
>> However,  I get an exception (Exception in thread "main"
>> _org.apache.flink.api.common.InvalidProgramException_: ) that object 
>> is not serializable (Object MyClass$3@a71081 not serializable )
>>
>>  
>>
>> Looking into this I see that the issues is that the DataStream<> is 
>> not serializable. What would be the solution to this issue?
>>
>>  
>>
>> As I mentioned before, I would like that for each event from the 
>> continuous stream to use the initial fixed set, add the event to it 
>> and apply an operation.
>>
>> Stephan was mentioning at some point some possibility to create a 
>> DataSet and launch a batch processing while operating in stream mode– 
>> in case this is possible, can you give me a reference for it, because 
>> it might be the good solution to  use in case. I am thinking that I 
>> could keep the fixed set as a DataSet and as each new event comes, 
>> transform it into a dataset and then join with reference set and apply 
>> an operation
>>
>>  
>>
>> Regards,
>>
>>  
>>
>>  
>>
>>  
>>
>>  
>>
>> Dr. Radu Tudoran
>>
>> Research Engineer
>>
>> IT R&D Division
>>
>>  
>>
>> cid:image007.jpg@01CD52EB.AD060EE0
>>
>> HUAWEI TECHNOLOGIES Duesseldorf GmbH
>>
>> European Research Center
>>
>> Riesstrasse 25, 80992 München
>>
>>  
>>
>> E-mail: _radu.tudoran@huawei.com <ma...@huawei.com>_
>>
>> Mobile: +49 15209084330 <tel:%2B49%2015209084330>
>>
>> Telephone: +49 891588344173 <tel:%2B49%20891588344173>
>>
>>  
>>
>> HUAWEI TECHNOLOGIES Duesseldorf GmbH
>> Hansaallee 205, 40549 Düsseldorf, Germany, www.huawei.com 
>> <http://www.huawei.com/> Registered Office: Düsseldorf, Register Court 
>> Düsseldorf, HRB 56063, Managing Director: Jingwen TAO, Wanzhou MENG, 
>> Lifang CHEN Sitz der Gesellschaft: Düsseldorf, Amtsgericht Düsseldorf, 
>> HRB 56063,
>> Geschäftsführer: Jingwen TAO, Wanzhou MENG, Lifang CHEN
>>
>> This e-mail and its attachments contain confidential information from 
>> HUAWEI, which is intended only for the person or entity whose address 
>> is listed above. Any use of the information contained herein in any 
>> way (including, but not limited to, total or partial disclosure, 
>> reproduction, or dissemination) by persons other than the intended
>> recipient(s) is prohibited. If you receive this e-mail in error, 
>> please notify the sender by phone or email immediately and delete it!
>>
>>  
>>
>> *From:*Vieru, Mihail [mailto:mihail.vieru@zalando.de 
>> <ma...@zalando.de>]
>> *Sent:* Tuesday, December 01, 2015 4:55 PM
>> *To:* user@flink.apache.org <ma...@flink.apache.org>
>> *Subject:* NPE with Flink Streaming from Kafka
>>
>>  
>>
>> Hi,
>>
>> we get the following NullPointerException after ~50 minutes when 
>> running a streaming job with windowing and state that reads data from 
>> Kafka and writes the result to local FS.
>>
>> There are around 170 million messages to be processed, Flink 0.10.1 
>> stops at ~8 million.
>>
>> Flink runs locally, started with the "start-cluster-streaming.sh" script.
>>
>>
>> 12/01/2015 15:06:24    Job execution switched to status RUNNING.
>> 12/01/2015 15:06:24    Source: Custom Source -> Map -> Map(1/1) switched
>> to SCHEDULED
>> 12/01/2015 15:06:24    Source: Custom Source -> Map -> Map(1/1) switched
>> to DEPLOYING
>> 12/01/2015 15:06:24    Fast TumblingTimeWindows(5000) of Reduce at
>> main(ItemPriceAvgPerOrder.java:108) -> Sink: Unnamed(1/1) switched to 
>> SCHEDULED
>> 12/01/2015 15:06:24    Fast TumblingTimeWindows(5000) of Reduce at
>> main(ItemPriceAvgPerOrder.java:108) -> Sink: Unnamed(1/1) switched to 
>> DEPLOYING
>> 12/01/2015 15:06:24    Source: Custom Source -> Map -> Map(1/1) switched
>> to RUNNING
>> 12/01/2015 15:06:24    Fast TumblingTimeWindows(5000) of Reduce at
>> main(ItemPriceAvgPerOrder.java:108) -> Sink: Unnamed(1/1) switched to 
>> RUNNING
>> 12/01/2015 15:56:08    Fast TumblingTimeWindows(5000) of Reduce at
>> main(ItemPriceAvgPerOrder.java:108) -> Sink: Unnamed(1/1) switched to 
>> CANCELED
>> 12/01/2015 15:56:08    Source: Custom Source -> Map -> Map(1/1) switched
>> to FAILED
>> java.lang.Exception
>>     at
>> org.apache.flink.streaming.connectors.kafka.internals.LegacyFetcher.run(LegacyFetcher.java:242)
>>     at
>> org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer.run(FlinkKafkaConsumer.java:397)
>>     at
>> org.apache.flink.streaming.api.operators.StreamSource.run(StreamSource.java:58)
>>     at
>> org.apache.flink.streaming.runtime.tasks.SourceStreamTask.run(SourceStreamTask.java:55)
>>     at
>> org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:218)
>>     at org.apache.flink.runtime.taskmanager.Task.run(Task.java:584)
>>     at java.lang.Thread.run(Thread.java:745)
>> Caused by: java.lang.NullPointerException
>>     at
>> org.I0Itec.zkclient.ZkConnection.writeDataReturnStat(ZkConnection.java:115)
>>     at org.I0Itec.zkclient.ZkClient$10.call(ZkClient.java:817)
>>     at org.I0Itec.zkclient.ZkClient.retryUntilConnected(ZkClient.java:675)
>>     at org.I0Itec.zkclient.ZkClient.writeDataReturnStat(ZkClient.java:813)
>>     at org.I0Itec.zkclient.ZkClient.writeData(ZkClient.java:808)
>>     at org.I0Itec.zkclient.ZkClient.writeData(ZkClient.java:777)
>>     at kafka.utils.ZkUtils$.updatePersistentPath(ZkUtils.scala:332)
>>     at kafka.utils.ZkUtils.updatePersistentPath(ZkUtils.scala)
>>     at
>> org.apache.flink.streaming.connectors.kafka.internals.ZookeeperOffsetHandler.setOffsetInZooKeeper(ZookeeperOffsetHandler.java:112)
>>     at
>> org.apache.flink.streaming.connectors.kafka.internals.ZookeeperOffsetHandler.commit(ZookeeperOffsetHandler.java:80)
>>     at
>> org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer$Periodi
>> cOffsetCommitter.run(FlinkKafkaConsumer.java:632)
>>
>> Any ideas on what could cause this behaviour?
>>
>>  
>>
>> Best,
>>
>> Mihail
>>
>>  
>>
> 


RE: Question about DataStream serialization

Posted by Radu Tudoran <ra...@huawei.com>.
Hi,

Thanks for the answer - it is helpful.
The issue that remains is why is the open function not being executed before the flatmap to load the data in the OperatorState. 

I used something like - and I observe that the dataset is not initialized when being used in the flatmap function

env.socketTextStream
.map() -> to transform data to a Tuple1<String>
.keyby(0) -> to enable the usage of the operatorState which I saw requires keyed structured
.flatmap(RichFlatMapFunction<Tuple1<String>, String>   -> the function
{
private OperatorState<String> dataset;
@Override
public void flatMap(
{
Dataset -> use ...is empty
}
@Override
public void open(
{
dataset -> load 
}
})



Dr. Radu Tudoran
Research Engineer
IT R&D Division


HUAWEI TECHNOLOGIES Duesseldorf GmbH
European Research Center
Riesstrasse 25, 80992 München

E-mail: radu.tudoran@huawei.com
Mobile: +49 15209084330
Telephone: +49 891588344173

HUAWEI TECHNOLOGIES Duesseldorf GmbH
Hansaallee 205, 40549 Düsseldorf, Germany, www.huawei.com
Registered Office: Düsseldorf, Register Court Düsseldorf, HRB 56063,
Managing Director: Jingwen TAO, Wanzhou MENG, Lifang CHEN
Sitz der Gesellschaft: Düsseldorf, Amtsgericht Düsseldorf, HRB 56063,
Geschäftsführer: Jingwen TAO, Wanzhou MENG, Lifang CHEN
This e-mail and its attachments contain confidential information from HUAWEI, which is intended only for the person or entity whose address is listed above. Any use of the information contained herein in any way (including, but not limited to, total or partial disclosure, reproduction, or dissemination) by persons other than the intended recipient(s) is prohibited. If you receive this e-mail in error, please notify the sender by phone or email immediately and delete it!

-----Original Message-----
From: Matthias J. Sax [mailto:mjsax@apache.org] 
Sent: Tuesday, December 08, 2015 8:42 AM
To: user@flink.apache.org
Subject: Re: Question about DataStream serialization

Hi Radu,

you are right. The open() method is called for each parallel instance of a rich function. Thus, if all instanced use the same code, you might read the same data multiple times.

The easiest was to distinguish different instanced within open() is to user the RuntimeContext. If offers two methods  "int getNumberOfParallelSubtasks()" and "int getIndexOfThisSubtask()" that you can use to compute your own partitioning within open().

For example (just a sketch):

@Override
public void open(Configuration parameters) throws Exception {
  RuntimeContext context = super.getRuntimeContext();
  int dop = context.getNumberOfParallelSubtasks();
  int idx = context.getIndexOfThisSubtask();

  // open file
  // get size of file in bytes

  // seek to partition #idx:
  long seek = fileSize * idx / dop;

  // read "fileSize/dop" bytes
}

Hope this helps.

-Matthias


On 12/08/2015 04:28 AM, Radu Tudoran wrote:
> Hi,
> 
>  
> 
> Taking the example you mentioned of using RichFlatMapFunction and in 
> the
> open() reading a file.
> 
> Would this open function be executed on each node where the 
> RichFlatMapFunction gets executed? (I have done some tests and I would 
> get the feeling it does – but I wanted to double - check )
> 
> If so, would this mean that the same data will be loaded multiple 
> times on each parallel instance? Is there anyway, this can be 
> prevented and the data to be hashed and partitioned somehow across nodes?
> 
>  
> 
> Would using the operator state help?:
> 
> “
> 
> OperatorState*<*MyList<String>*>*dataset*;*
> 
> ”
> 
> I would be curious in this case how could the open function look like 
> to initialize the data for this operator state:
> 
>  
> 
>  
> 
> I have tried to just read a file and write it into the dataset, but I 
> encountered a strange behavior that would look like the flatmap 
> function gets executed before the open function, which leads to using 
> an empty dataset in the flatmap function while when this finish 
> executing the dataset gets loaded. Is this an error or I am doing something wrong?
> 
>  
> 
>  
> 
>  
> 
> Dr. Radu Tudoran
> 
> Research Engineer
> 
> IT R&D Division
> 
>  
> 
> cid:image007.jpg@01CD52EB.AD060EE0
> 
> HUAWEI TECHNOLOGIES Duesseldorf GmbH
> 
> European Research Center
> 
> Riesstrasse 25, 80992 München
> 
>  
> 
> E-mail: _radu.tudoran@huawei.com_
> 
> Mobile: +49 15209084330
> 
> Telephone: +49 891588344173
> 
>  
> 
> HUAWEI TECHNOLOGIES Duesseldorf GmbH
> Hansaallee 205, 40549 Düsseldorf, Germany, www.huawei.com 
> <http://www.huawei.com/> Registered Office: Düsseldorf, Register Court 
> Düsseldorf, HRB 56063, Managing Director: Jingwen TAO, Wanzhou MENG, 
> Lifang CHEN Sitz der Gesellschaft: Düsseldorf, Amtsgericht Düsseldorf, 
> HRB 56063,
> Geschäftsführer: Jingwen TAO, Wanzhou MENG, Lifang CHEN
> 
> This e-mail and its attachments contain confidential information from 
> HUAWEI, which is intended only for the person or entity whose address 
> is listed above. Any use of the information contained herein in any 
> way (including, but not limited to, total or partial disclosure, 
> reproduction, or dissemination) by persons other than the intended
> recipient(s) is prohibited. If you receive this e-mail in error, 
> please notify the sender by phone or email immediately and delete it!
> 
>  
> 
> *From:*Robert Metzger [mailto:rmetzger@apache.org]
> *Sent:* Tuesday, December 01, 2015 6:21 PM
> *To:* user@flink.apache.org
> *Cc:* Goetz Brasche
> *Subject:* Re: Question about DataStream serialization
> 
>  
> 
> Hi Radu,
> 
>  
> 
> both emails reached the mailing list :)
> 
>  
> 
> You can not reference to DataSets or DataStreams from inside user 
> defined functions. Both are just abstractions for a data set or 
> stream, so the elements are not really inside the set.
> 
>  
> 
> We don't have any support for mixing the DataSet and DataStream API.
> 
>  
> 
> For your use case, I would recommend you to use a RichFlatMapFunction 
> and in the open() call read the text file.
> 
>  
> 
>  
> 
>  
> 
> On Tue, Dec 1, 2015 at 5:03 PM, Radu Tudoran <radu.tudoran@huawei.com 
> <ma...@huawei.com>> wrote:
> 
>  
> 
> Hello,
> 
>  
> 
> I am not sure if this message was received on the user list, if so I 
> apologies for duplicate messages
> 
>  
> 
> I have the following scenario
> 
>  
> 
> ·         Reading a fixed set
> 
> DataStream<String> /fixedset/ = env.readtextFile(…
> 
> ·         Reading a continuous stream of data
> 
> DataStream<String> /stream/ = ….
> 
>  
> 
> I would need that for each event read from the continuous stream to 
> make some operations onit and on the /fixedsettoghether/
> 
>  
> 
>  
> 
> I have tried something like
> 
>  
> 
> final myObject.referenceStaticSet = fixedset;
> 
> stream.map(new MapFunction<String, String>() {
> 
>                      @Override
> 
>                      public String map(String arg0) throws Exception {
> 
>                           
> 
>                            //for example:   final string2add = arg0;
> 
>                                                                 //the 
> goal of below function would be to add the string2add to the fixedset
> 
>                            myObject.referenceStaticSet = 
> myObject.referenceStaticSet.flatMap(new FlatMapFunction<String, 
> String>() {
> 
>                           
> 
>                                   @Override
> 
>                                   public void flatMap(String arg0, 
> Collector<String> arg1)
> 
>                                                                                
> //for example adding to the fixed set also the string2add object:  
> arg1.collect(string2add);
> 
>                                                
>                                 }
> 
> …
> 
> }
> 
>  
> 
> However,  I get an exception (Exception in thread "main"
> _org.apache.flink.api.common.InvalidProgramException_: ) that object 
> is not serializable (Object MyClass$3@a71081 not serializable )
> 
>  
> 
> Looking into this I see that the issues is that the DataStream<> is 
> not serializable. What would be the solution to this issue?
> 
>  
> 
> As I mentioned before, I would like that for each event from the 
> continuous stream to use the initial fixed set, add the event to it 
> and apply an operation.
> 
> Stephan was mentioning at some point some possibility to create a 
> DataSet and launch a batch processing while operating in stream mode– 
> in case this is possible, can you give me a reference for it, because 
> it might be the good solution to  use in case. I am thinking that I 
> could keep the fixed set as a DataSet and as each new event comes, 
> transform it into a dataset and then join with reference set and apply 
> an operation
> 
>  
> 
> Regards,
> 
>  
> 
>  
> 
>  
> 
>  
> 
> Dr. Radu Tudoran
> 
> Research Engineer
> 
> IT R&D Division
> 
>  
> 
> cid:image007.jpg@01CD52EB.AD060EE0
> 
> HUAWEI TECHNOLOGIES Duesseldorf GmbH
> 
> European Research Center
> 
> Riesstrasse 25, 80992 München
> 
>  
> 
> E-mail: _radu.tudoran@huawei.com <ma...@huawei.com>_
> 
> Mobile: +49 15209084330 <tel:%2B49%2015209084330>
> 
> Telephone: +49 891588344173 <tel:%2B49%20891588344173>
> 
>  
> 
> HUAWEI TECHNOLOGIES Duesseldorf GmbH
> Hansaallee 205, 40549 Düsseldorf, Germany, www.huawei.com 
> <http://www.huawei.com/> Registered Office: Düsseldorf, Register Court 
> Düsseldorf, HRB 56063, Managing Director: Jingwen TAO, Wanzhou MENG, 
> Lifang CHEN Sitz der Gesellschaft: Düsseldorf, Amtsgericht Düsseldorf, 
> HRB 56063,
> Geschäftsführer: Jingwen TAO, Wanzhou MENG, Lifang CHEN
> 
> This e-mail and its attachments contain confidential information from 
> HUAWEI, which is intended only for the person or entity whose address 
> is listed above. Any use of the information contained herein in any 
> way (including, but not limited to, total or partial disclosure, 
> reproduction, or dissemination) by persons other than the intended
> recipient(s) is prohibited. If you receive this e-mail in error, 
> please notify the sender by phone or email immediately and delete it!
> 
>  
> 
> *From:*Vieru, Mihail [mailto:mihail.vieru@zalando.de 
> <ma...@zalando.de>]
> *Sent:* Tuesday, December 01, 2015 4:55 PM
> *To:* user@flink.apache.org <ma...@flink.apache.org>
> *Subject:* NPE with Flink Streaming from Kafka
> 
>  
> 
> Hi,
> 
> we get the following NullPointerException after ~50 minutes when 
> running a streaming job with windowing and state that reads data from 
> Kafka and writes the result to local FS.
> 
> There are around 170 million messages to be processed, Flink 0.10.1 
> stops at ~8 million.
> 
> Flink runs locally, started with the "start-cluster-streaming.sh" script.
> 
> 
> 12/01/2015 15:06:24    Job execution switched to status RUNNING.
> 12/01/2015 15:06:24    Source: Custom Source -> Map -> Map(1/1) switched
> to SCHEDULED
> 12/01/2015 15:06:24    Source: Custom Source -> Map -> Map(1/1) switched
> to DEPLOYING
> 12/01/2015 15:06:24    Fast TumblingTimeWindows(5000) of Reduce at
> main(ItemPriceAvgPerOrder.java:108) -> Sink: Unnamed(1/1) switched to 
> SCHEDULED
> 12/01/2015 15:06:24    Fast TumblingTimeWindows(5000) of Reduce at
> main(ItemPriceAvgPerOrder.java:108) -> Sink: Unnamed(1/1) switched to 
> DEPLOYING
> 12/01/2015 15:06:24    Source: Custom Source -> Map -> Map(1/1) switched
> to RUNNING
> 12/01/2015 15:06:24    Fast TumblingTimeWindows(5000) of Reduce at
> main(ItemPriceAvgPerOrder.java:108) -> Sink: Unnamed(1/1) switched to 
> RUNNING
> 12/01/2015 15:56:08    Fast TumblingTimeWindows(5000) of Reduce at
> main(ItemPriceAvgPerOrder.java:108) -> Sink: Unnamed(1/1) switched to 
> CANCELED
> 12/01/2015 15:56:08    Source: Custom Source -> Map -> Map(1/1) switched
> to FAILED
> java.lang.Exception
>     at
> org.apache.flink.streaming.connectors.kafka.internals.LegacyFetcher.run(LegacyFetcher.java:242)
>     at
> org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer.run(FlinkKafkaConsumer.java:397)
>     at
> org.apache.flink.streaming.api.operators.StreamSource.run(StreamSource.java:58)
>     at
> org.apache.flink.streaming.runtime.tasks.SourceStreamTask.run(SourceStreamTask.java:55)
>     at
> org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:218)
>     at org.apache.flink.runtime.taskmanager.Task.run(Task.java:584)
>     at java.lang.Thread.run(Thread.java:745)
> Caused by: java.lang.NullPointerException
>     at
> org.I0Itec.zkclient.ZkConnection.writeDataReturnStat(ZkConnection.java:115)
>     at org.I0Itec.zkclient.ZkClient$10.call(ZkClient.java:817)
>     at org.I0Itec.zkclient.ZkClient.retryUntilConnected(ZkClient.java:675)
>     at org.I0Itec.zkclient.ZkClient.writeDataReturnStat(ZkClient.java:813)
>     at org.I0Itec.zkclient.ZkClient.writeData(ZkClient.java:808)
>     at org.I0Itec.zkclient.ZkClient.writeData(ZkClient.java:777)
>     at kafka.utils.ZkUtils$.updatePersistentPath(ZkUtils.scala:332)
>     at kafka.utils.ZkUtils.updatePersistentPath(ZkUtils.scala)
>     at
> org.apache.flink.streaming.connectors.kafka.internals.ZookeeperOffsetHandler.setOffsetInZooKeeper(ZookeeperOffsetHandler.java:112)
>     at
> org.apache.flink.streaming.connectors.kafka.internals.ZookeeperOffsetHandler.commit(ZookeeperOffsetHandler.java:80)
>     at
> org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer$Periodi
> cOffsetCommitter.run(FlinkKafkaConsumer.java:632)
> 
> Any ideas on what could cause this behaviour?
> 
>  
> 
> Best,
> 
> Mihail
> 
>  
> 


Re: Question about DataStream serialization

Posted by "Matthias J. Sax" <mj...@apache.org>.
Hi Radu,

you are right. The open() method is called for each parallel instance of
a rich function. Thus, if all instanced use the same code, you might
read the same data multiple times.

The easiest was to distinguish different instanced within open() is to
user the RuntimeContext. If offers two methods  "int
getNumberOfParallelSubtasks()" and "int getIndexOfThisSubtask()" that
you can use to compute your own partitioning within open().

For example (just a sketch):

@Override
public void open(Configuration parameters) throws Exception {
  RuntimeContext context = super.getRuntimeContext();
  int dop = context.getNumberOfParallelSubtasks();
  int idx = context.getIndexOfThisSubtask();

  // open file
  // get size of file in bytes

  // seek to partition #idx:
  long seek = fileSize * idx / dop;

  // read "fileSize/dop" bytes
}

Hope this helps.

-Matthias


On 12/08/2015 04:28 AM, Radu Tudoran wrote:
> Hi,
> 
>  
> 
> Taking the example you mentioned of using RichFlatMapFunction and in the
> open() reading a file.
> 
> Would this open function be executed on each node where the
> RichFlatMapFunction gets executed? (I have done some tests and I would
> get the feeling it does – but I wanted to double - check )
> 
> If so, would this mean that the same data will be loaded multiple times
> on each parallel instance? Is there anyway, this can be prevented and
> the data to be hashed and partitioned somehow across nodes?
> 
>  
> 
> Would using the operator state help?:
> 
> “
> 
> OperatorState*<*MyList<String>*>*dataset*;*
> 
> ”
> 
> I would be curious in this case how could the open function look like to
> initialize the data for this operator state:
> 
>  
> 
>  
> 
> I have tried to just read a file and write it into the dataset, but I
> encountered a strange behavior that would look like the flatmap function
> gets executed before the open function, which leads to using an empty
> dataset in the flatmap function while when this finish executing the
> dataset gets loaded. Is this an error or I am doing something wrong?
> 
>  
> 
>  
> 
>  
> 
> Dr. Radu Tudoran
> 
> Research Engineer
> 
> IT R&D Division
> 
>  
> 
> cid:image007.jpg@01CD52EB.AD060EE0
> 
> HUAWEI TECHNOLOGIES Duesseldorf GmbH
> 
> European Research Center
> 
> Riesstrasse 25, 80992 München
> 
>  
> 
> E-mail: _radu.tudoran@huawei.com_
> 
> Mobile: +49 15209084330
> 
> Telephone: +49 891588344173
> 
>  
> 
> HUAWEI TECHNOLOGIES Duesseldorf GmbH
> Hansaallee 205, 40549 Düsseldorf, Germany, www.huawei.com
> <http://www.huawei.com/>
> Registered Office: Düsseldorf, Register Court Düsseldorf, HRB 56063,
> Managing Director: Jingwen TAO, Wanzhou MENG, Lifang CHEN
> Sitz der Gesellschaft: Düsseldorf, Amtsgericht Düsseldorf, HRB 56063,
> Geschäftsführer: Jingwen TAO, Wanzhou MENG, Lifang CHEN
> 
> This e-mail and its attachments contain confidential information from
> HUAWEI, which is intended only for the person or entity whose address is
> listed above. Any use of the information contained herein in any way
> (including, but not limited to, total or partial disclosure,
> reproduction, or dissemination) by persons other than the intended
> recipient(s) is prohibited. If you receive this e-mail in error, please
> notify the sender by phone or email immediately and delete it!
> 
>  
> 
> *From:*Robert Metzger [mailto:rmetzger@apache.org]
> *Sent:* Tuesday, December 01, 2015 6:21 PM
> *To:* user@flink.apache.org
> *Cc:* Goetz Brasche
> *Subject:* Re: Question about DataStream serialization
> 
>  
> 
> Hi Radu,
> 
>  
> 
> both emails reached the mailing list :)
> 
>  
> 
> You can not reference to DataSets or DataStreams from inside user
> defined functions. Both are just abstractions for a data set or stream,
> so the elements are not really inside the set. 
> 
>  
> 
> We don't have any support for mixing the DataSet and DataStream API.
> 
>  
> 
> For your use case, I would recommend you to use a RichFlatMapFunction
> and in the open() call read the text file.
> 
>  
> 
>  
> 
>  
> 
> On Tue, Dec 1, 2015 at 5:03 PM, Radu Tudoran <radu.tudoran@huawei.com
> <ma...@huawei.com>> wrote:
> 
>  
> 
> Hello,
> 
>  
> 
> I am not sure if this message was received on the user list, if so I
> apologies for duplicate messages
> 
>  
> 
> I have the following scenario  
> 
>  
> 
> ·         Reading a fixed set
> 
> DataStream<String> /fixedset/ = env.readtextFile(…
> 
> ·         Reading a continuous stream of data
> 
> DataStream<String> /stream/ = ….
> 
>  
> 
> I would need that for each event read from the continuous stream to make
> some operations onit and on the /fixedsettoghether/
> 
>  
> 
>  
> 
> I have tried something like
> 
>  
> 
> final myObject.referenceStaticSet = fixedset;
> 
> stream.map(new MapFunction<String, String>() {
> 
>                      @Override
> 
>                      public String map(String arg0) throws Exception {
> 
>                           
> 
>                            //for example:   final string2add = arg0;
> 
>                                                                 //the
> goal of below function would be to add the string2add to the fixedset
> 
>                            myObject.referenceStaticSet =
> myObject.referenceStaticSet.flatMap(new FlatMapFunction<String, String>() {
> 
>                           
> 
>                                   @Override
> 
>                                   public void flatMap(String arg0,
> Collector<String> arg1)
> 
>                                                                                
> //for example adding to the fixed set also the string2add object:  
> arg1.collect(string2add);
> 
>                                                
>                                 }
> 
> …
> 
> }
> 
>  
> 
> However,  I get an exception (Exception in thread "main"
> _org.apache.flink.api.common.InvalidProgramException_: ) that object is
> not serializable (Object MyClass$3@a71081 not serializable )
> 
>  
> 
> Looking into this I see that the issues is that the DataStream<> is not
> serializable. What would be the solution to this issue?
> 
>  
> 
> As I mentioned before, I would like that for each event from the
> continuous stream to use the initial fixed set, add the event to it and
> apply an operation.
> 
> Stephan was mentioning at some point some possibility to create a
> DataSet and launch a batch processing while operating in stream mode– in
> case this is possible, can you give me a reference for it, because it
> might be the good solution to  use in case. I am thinking that I could
> keep the fixed set as a DataSet and as each new event comes, transform
> it into a dataset and then join with reference set and apply an operation
> 
>  
> 
> Regards,
> 
>  
> 
>  
> 
>  
> 
>  
> 
> Dr. Radu Tudoran
> 
> Research Engineer
> 
> IT R&D Division
> 
>  
> 
> cid:image007.jpg@01CD52EB.AD060EE0
> 
> HUAWEI TECHNOLOGIES Duesseldorf GmbH
> 
> European Research Center
> 
> Riesstrasse 25, 80992 München
> 
>  
> 
> E-mail: _radu.tudoran@huawei.com <ma...@huawei.com>_
> 
> Mobile: +49 15209084330 <tel:%2B49%2015209084330>
> 
> Telephone: +49 891588344173 <tel:%2B49%20891588344173>
> 
>  
> 
> HUAWEI TECHNOLOGIES Duesseldorf GmbH
> Hansaallee 205, 40549 Düsseldorf, Germany, www.huawei.com
> <http://www.huawei.com/>
> Registered Office: Düsseldorf, Register Court Düsseldorf, HRB 56063,
> Managing Director: Jingwen TAO, Wanzhou MENG, Lifang CHEN
> Sitz der Gesellschaft: Düsseldorf, Amtsgericht Düsseldorf, HRB 56063,
> Geschäftsführer: Jingwen TAO, Wanzhou MENG, Lifang CHEN
> 
> This e-mail and its attachments contain confidential information from
> HUAWEI, which is intended only for the person or entity whose address is
> listed above. Any use of the information contained herein in any way
> (including, but not limited to, total or partial disclosure,
> reproduction, or dissemination) by persons other than the intended
> recipient(s) is prohibited. If you receive this e-mail in error, please
> notify the sender by phone or email immediately and delete it!
> 
>  
> 
> *From:*Vieru, Mihail [mailto:mihail.vieru@zalando.de
> <ma...@zalando.de>]
> *Sent:* Tuesday, December 01, 2015 4:55 PM
> *To:* user@flink.apache.org <ma...@flink.apache.org>
> *Subject:* NPE with Flink Streaming from Kafka
> 
>  
> 
> Hi,
> 
> we get the following NullPointerException after ~50 minutes when running
> a streaming job with windowing and state that reads data from Kafka and
> writes the result to local FS.
> 
> There are around 170 million messages to be processed, Flink 0.10.1
> stops at ~8 million.
> 
> Flink runs locally, started with the "start-cluster-streaming.sh" script.
> 
> 
> 12/01/2015 15:06:24    Job execution switched to status RUNNING.
> 12/01/2015 15:06:24    Source: Custom Source -> Map -> Map(1/1) switched
> to SCHEDULED
> 12/01/2015 15:06:24    Source: Custom Source -> Map -> Map(1/1) switched
> to DEPLOYING
> 12/01/2015 15:06:24    Fast TumblingTimeWindows(5000) of Reduce at
> main(ItemPriceAvgPerOrder.java:108) -> Sink: Unnamed(1/1) switched to
> SCHEDULED
> 12/01/2015 15:06:24    Fast TumblingTimeWindows(5000) of Reduce at
> main(ItemPriceAvgPerOrder.java:108) -> Sink: Unnamed(1/1) switched to
> DEPLOYING
> 12/01/2015 15:06:24    Source: Custom Source -> Map -> Map(1/1) switched
> to RUNNING
> 12/01/2015 15:06:24    Fast TumblingTimeWindows(5000) of Reduce at
> main(ItemPriceAvgPerOrder.java:108) -> Sink: Unnamed(1/1) switched to
> RUNNING
> 12/01/2015 15:56:08    Fast TumblingTimeWindows(5000) of Reduce at
> main(ItemPriceAvgPerOrder.java:108) -> Sink: Unnamed(1/1) switched to
> CANCELED
> 12/01/2015 15:56:08    Source: Custom Source -> Map -> Map(1/1) switched
> to FAILED
> java.lang.Exception
>     at
> org.apache.flink.streaming.connectors.kafka.internals.LegacyFetcher.run(LegacyFetcher.java:242)
>     at
> org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer.run(FlinkKafkaConsumer.java:397)
>     at
> org.apache.flink.streaming.api.operators.StreamSource.run(StreamSource.java:58)
>     at
> org.apache.flink.streaming.runtime.tasks.SourceStreamTask.run(SourceStreamTask.java:55)
>     at
> org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:218)
>     at org.apache.flink.runtime.taskmanager.Task.run(Task.java:584)
>     at java.lang.Thread.run(Thread.java:745)
> Caused by: java.lang.NullPointerException
>     at
> org.I0Itec.zkclient.ZkConnection.writeDataReturnStat(ZkConnection.java:115)
>     at org.I0Itec.zkclient.ZkClient$10.call(ZkClient.java:817)
>     at org.I0Itec.zkclient.ZkClient.retryUntilConnected(ZkClient.java:675)
>     at org.I0Itec.zkclient.ZkClient.writeDataReturnStat(ZkClient.java:813)
>     at org.I0Itec.zkclient.ZkClient.writeData(ZkClient.java:808)
>     at org.I0Itec.zkclient.ZkClient.writeData(ZkClient.java:777)
>     at kafka.utils.ZkUtils$.updatePersistentPath(ZkUtils.scala:332)
>     at kafka.utils.ZkUtils.updatePersistentPath(ZkUtils.scala)
>     at
> org.apache.flink.streaming.connectors.kafka.internals.ZookeeperOffsetHandler.setOffsetInZooKeeper(ZookeeperOffsetHandler.java:112)
>     at
> org.apache.flink.streaming.connectors.kafka.internals.ZookeeperOffsetHandler.commit(ZookeeperOffsetHandler.java:80)
>     at
> org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer$PeriodicOffsetCommitter.run(FlinkKafkaConsumer.java:632)
> 
> Any ideas on what could cause this behaviour?
> 
>  
> 
> Best,
> 
> Mihail
> 
>  
> 


RE: Question about DataStream serialization

Posted by Radu Tudoran <ra...@huawei.com>.
Hi,

Taking the example you mentioned of using RichFlatMapFunction and in the open() reading a file.
Would this open function be executed on each node where the RichFlatMapFunction gets executed? (I have done some tests and I would get the feeling it does – but I wanted to double - check )
If so, would this mean that the same data will be loaded multiple times on each parallel instance? Is there anyway, this can be prevented and the data to be hashed and partitioned somehow across nodes?

Would using the operator state help?:
“
OperatorState<MyList<String>> dataset;
”
I would be curious in this case how could the open function look like to initialize the data for this operator state:


I have tried to just read a file and write it into the dataset, but I encountered a strange behavior that would look like the flatmap function gets executed before the open function, which leads to using an empty dataset in the flatmap function while when this finish executing the dataset gets loaded. Is this an error or I am doing something wrong?



Dr. Radu Tudoran
Research Engineer
IT R&D Division

[cid:image007.jpg@01CD52EB.AD060EE0]
HUAWEI TECHNOLOGIES Duesseldorf GmbH
European Research Center
Riesstrasse 25, 80992 München

E-mail: radu.tudoran@huawei.com
Mobile: +49 15209084330
Telephone: +49 891588344173

HUAWEI TECHNOLOGIES Duesseldorf GmbH
Hansaallee 205, 40549 Düsseldorf, Germany, www.huawei.com<http://www.huawei.com/>
Registered Office: Düsseldorf, Register Court Düsseldorf, HRB 56063,
Managing Director: Jingwen TAO, Wanzhou MENG, Lifang CHEN
Sitz der Gesellschaft: Düsseldorf, Amtsgericht Düsseldorf, HRB 56063,
Geschäftsführer: Jingwen TAO, Wanzhou MENG, Lifang CHEN
This e-mail and its attachments contain confidential information from HUAWEI, which is intended only for the person or entity whose address is listed above. Any use of the information contained herein in any way (including, but not limited to, total or partial disclosure, reproduction, or dissemination) by persons other than the intended recipient(s) is prohibited. If you receive this e-mail in error, please notify the sender by phone or email immediately and delete it!

From: Robert Metzger [mailto:rmetzger@apache.org]
Sent: Tuesday, December 01, 2015 6:21 PM
To: user@flink.apache.org
Cc: Goetz Brasche
Subject: Re: Question about DataStream serialization

Hi Radu,

both emails reached the mailing list :)

You can not reference to DataSets or DataStreams from inside user defined functions. Both are just abstractions for a data set or stream, so the elements are not really inside the set.

We don't have any support for mixing the DataSet and DataStream API.

For your use case, I would recommend you to use a RichFlatMapFunction and in the open() call read the text file.



On Tue, Dec 1, 2015 at 5:03 PM, Radu Tudoran <ra...@huawei.com>> wrote:

Hello,

I am not sure if this message was received on the user list, if so I apologies for duplicate messages

I have the following scenario


•         Reading a fixed set
DataStream<String> fixedset = env.readtextFile(…

•         Reading a continuous stream of data
DataStream<String> stream = ….

I would need that for each event read from the continuous stream to make some operations onit and on the fixedsettoghether


I have tried something like

final myObject.referenceStaticSet = fixedset;
stream.map(new MapFunction<String, String>() {
                     @Override
                     public String map(String arg0) throws Exception {

                           //for example:   final string2add = arg0;
                                                                //the goal of below function would be to add the string2add to the fixedset
                           myObject.referenceStaticSet = myObject.referenceStaticSet.flatMap(new FlatMapFunction<String, String>() {

                                  @Override
                                  public void flatMap(String arg0, Collector<String> arg1)
                                                                                //for example adding to the fixed set also the string2add object:   arg1.collect(string2add);
                                                                                }
…
}

However,  I get an exception (Exception in thread "main" org.apache.flink.api.common.InvalidProgramException: ) that object is not serializable (Object MyClass$3@a71081 not serializable )

Looking into this I see that the issues is that the DataStream<> is not serializable. What would be the solution to this issue?

As I mentioned before, I would like that for each event from the continuous stream to use the initial fixed set, add the event to it and apply an operation.
Stephan was mentioning at some point some possibility to create a DataSet and launch a batch processing while operating in stream mode– in case this is possible, can you give me a reference for it, because it might be the good solution to  use in case. I am thinking that I could keep the fixed set as a DataSet and as each new event comes, transform it into a dataset and then join with reference set and apply an operation

Regards,




Dr. Radu Tudoran
Research Engineer
IT R&D Division

[cid:image007.jpg@01CD52EB.AD060EE0]
HUAWEI TECHNOLOGIES Duesseldorf GmbH
European Research Center
Riesstrasse 25, 80992 München

E-mail: radu.tudoran@huawei.com<ma...@huawei.com>
Mobile: +49 15209084330<tel:%2B49%2015209084330>
Telephone: +49 891588344173<tel:%2B49%20891588344173>

HUAWEI TECHNOLOGIES Duesseldorf GmbH
Hansaallee 205, 40549 Düsseldorf, Germany, www.huawei.com<http://www.huawei.com/>
Registered Office: Düsseldorf, Register Court Düsseldorf, HRB 56063,
Managing Director: Jingwen TAO, Wanzhou MENG, Lifang CHEN
Sitz der Gesellschaft: Düsseldorf, Amtsgericht Düsseldorf, HRB 56063,
Geschäftsführer: Jingwen TAO, Wanzhou MENG, Lifang CHEN
This e-mail and its attachments contain confidential information from HUAWEI, which is intended only for the person or entity whose address is listed above. Any use of the information contained herein in any way (including, but not limited to, total or partial disclosure, reproduction, or dissemination) by persons other than the intended recipient(s) is prohibited. If you receive this e-mail in error, please notify the sender by phone or email immediately and delete it!

From: Vieru, Mihail [mailto:mihail.vieru@zalando.de<ma...@zalando.de>]
Sent: Tuesday, December 01, 2015 4:55 PM
To: user@flink.apache.org<ma...@flink.apache.org>
Subject: NPE with Flink Streaming from Kafka

Hi,
we get the following NullPointerException after ~50 minutes when running a streaming job with windowing and state that reads data from Kafka and writes the result to local FS.
There are around 170 million messages to be processed, Flink 0.10.1 stops at ~8 million.
Flink runs locally, started with the "start-cluster-streaming.sh" script.

12/01/2015 15:06:24    Job execution switched to status RUNNING.
12/01/2015 15:06:24    Source: Custom Source -> Map -> Map(1/1) switched to SCHEDULED
12/01/2015 15:06:24    Source: Custom Source -> Map -> Map(1/1) switched to DEPLOYING
12/01/2015 15:06:24    Fast TumblingTimeWindows(5000) of Reduce at main(ItemPriceAvgPerOrder.java:108) -> Sink: Unnamed(1/1) switched to SCHEDULED
12/01/2015 15:06:24    Fast TumblingTimeWindows(5000) of Reduce at main(ItemPriceAvgPerOrder.java:108) -> Sink: Unnamed(1/1) switched to DEPLOYING
12/01/2015 15:06:24    Source: Custom Source -> Map -> Map(1/1) switched to RUNNING
12/01/2015 15:06:24    Fast TumblingTimeWindows(5000) of Reduce at main(ItemPriceAvgPerOrder.java:108) -> Sink: Unnamed(1/1) switched to RUNNING
12/01/2015 15:56:08    Fast TumblingTimeWindows(5000) of Reduce at main(ItemPriceAvgPerOrder.java:108) -> Sink: Unnamed(1/1) switched to CANCELED
12/01/2015 15:56:08    Source: Custom Source -> Map -> Map(1/1) switched to FAILED
java.lang.Exception
    at org.apache.flink.streaming.connectors.kafka.internals.LegacyFetcher.run(LegacyFetcher.java:242)
    at org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer.run(FlinkKafkaConsumer.java:397)
    at org.apache.flink.streaming.api.operators.StreamSource.run(StreamSource.java:58)
    at org.apache.flink.streaming.runtime.tasks.SourceStreamTask.run(SourceStreamTask.java:55)
    at org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:218)
    at org.apache.flink.runtime.taskmanager.Task.run(Task.java:584)
    at java.lang.Thread.run(Thread.java:745)
Caused by: java.lang.NullPointerException
    at org.I0Itec.zkclient.ZkConnection.writeDataReturnStat(ZkConnection.java:115)
    at org.I0Itec.zkclient.ZkClient$10.call(ZkClient.java:817)
    at org.I0Itec.zkclient.ZkClient.retryUntilConnected(ZkClient.java:675)
    at org.I0Itec.zkclient.ZkClient.writeDataReturnStat(ZkClient.java:813)
    at org.I0Itec.zkclient.ZkClient.writeData(ZkClient.java:808)
    at org.I0Itec.zkclient.ZkClient.writeData(ZkClient.java:777)
    at kafka.utils.ZkUtils$.updatePersistentPath(ZkUtils.scala:332)
    at kafka.utils.ZkUtils.updatePersistentPath(ZkUtils.scala)
    at org.apache.flink.streaming.connectors.kafka.internals.ZookeeperOffsetHandler.setOffsetInZooKeeper(ZookeeperOffsetHandler.java:112)
    at org.apache.flink.streaming.connectors.kafka.internals.ZookeeperOffsetHandler.commit(ZookeeperOffsetHandler.java:80)
    at org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer$PeriodicOffsetCommitter.run(FlinkKafkaConsumer.java:632)
Any ideas on what could cause this behaviour?

Best,
Mihail


Re: Question about DataStream serialization

Posted by Robert Metzger <rm...@apache.org>.
Hi Radu,

both emails reached the mailing list :)

You can not reference to DataSets or DataStreams from inside user defined
functions. Both are just abstractions for a data set or stream, so the
elements are not really inside the set.

We don't have any support for mixing the DataSet and DataStream API.

For your use case, I would recommend you to use a RichFlatMapFunction and
in the open() call read the text file.



On Tue, Dec 1, 2015 at 5:03 PM, Radu Tudoran <ra...@huawei.com>
wrote:

>
>
> Hello,
>
>
>
> I am not sure if this message was received on the user list, if so I
> apologies for duplicate messages
>
>
>
> I have the following scenario
>
>
>
> ·         Reading a fixed set
>
> DataStream<String> *fixedset* = env.readtextFile(…
>
> ·         Reading a continuous stream of data
>
> DataStream<String> *stream* = ….
>
>
>
> I would need that for each event read from the continuous stream to make
> some operations onit and on the *fixedsettoghether*
>
>
>
>
>
> I have tried something like
>
>
>
> final myObject.referenceStaticSet = fixedset;
>
> stream.map(new MapFunction<String, String>() {
>
>                      @Override
>
>                      public String map(String arg0) throws Exception {
>
>
>
>                            //for example:   final string2add = arg0;
>
>                                                                 //the goal
> of below function would be to add the string2add to the fixedset
>
>                            myObject.referenceStaticSet =
> myObject.referenceStaticSet.flatMap(new FlatMapFunction<String, String>() {
>
>
>
>                                   @Override
>
>                                   public void flatMap(String arg0,
> Collector<String> arg1)
>
>
> //for example adding to the fixed set also the string2add object:
> arg1.collect(string2add);
>
>
>                                 }
>
> …
>
> }
>
>
>
> However,  I get an exception (Exception in thread "main"
> *org.apache.flink.api.common.InvalidProgramException*: ) that object is
> not serializable (Object MyClass$3@a71081 not serializable )
>
>
>
> Looking into this I see that the issues is that the DataStream<> is not
> serializable. What would be the solution to this issue?
>
>
>
> As I mentioned before, I would like that for each event from the
> continuous stream to use the initial fixed set, add the event to it and
> apply an operation.
>
> Stephan was mentioning at some point some possibility to create a DataSet
> and launch a batch processing while operating in stream mode– in case this
> is possible, can you give me a reference for it, because it might be the
> good solution to  use in case. I am thinking that I could keep the fixed
> set as a DataSet and as each new event comes, transform it into a dataset
> and then join with reference set and apply an operation
>
>
>
> Regards,
>
>
>
>
>
>
>
>
>
> Dr. Radu Tudoran
>
> Research Engineer
>
> IT R&D Division
>
>
>
> [image: cid:image007.jpg@01CD52EB.AD060EE0]
>
> HUAWEI TECHNOLOGIES Duesseldorf GmbH
>
> European Research Center
>
> Riesstrasse 25, 80992 München
>
>
>
> E-mail: *radu.tudoran@huawei.com <ra...@huawei.com>*
>
> Mobile: +49 15209084330
>
> Telephone: +49 891588344173
>
>
>
> HUAWEI TECHNOLOGIES Duesseldorf GmbH
> Hansaallee 205, 40549 Düsseldorf, Germany, www.huawei.com
> Registered Office: Düsseldorf, Register Court Düsseldorf, HRB 56063,
> Managing Director: Jingwen TAO, Wanzhou MENG, Lifang CHEN
> Sitz der Gesellschaft: Düsseldorf, Amtsgericht Düsseldorf, HRB 56063,
> Geschäftsführer: Jingwen TAO, Wanzhou MENG, Lifang CHEN
>
> This e-mail and its attachments contain confidential information from
> HUAWEI, which is intended only for the person or entity whose address is
> listed above. Any use of the information contained herein in any way
> (including, but not limited to, total or partial disclosure, reproduction,
> or dissemination) by persons other than the intended recipient(s) is
> prohibited. If you receive this e-mail in error, please notify the sender
> by phone or email immediately and delete it!
>
>
>
> *From:* Vieru, Mihail [mailto:mihail.vieru@zalando.de]
> *Sent:* Tuesday, December 01, 2015 4:55 PM
> *To:* user@flink.apache.org
> *Subject:* NPE with Flink Streaming from Kafka
>
>
>
> Hi,
>
> we get the following NullPointerException after ~50 minutes when running a
> streaming job with windowing and state that reads data from Kafka and
> writes the result to local FS.
>
> There are around 170 million messages to be processed, Flink 0.10.1 stops
> at ~8 million.
>
> Flink runs locally, started with the "start-cluster-streaming.sh" script.
>
>
> 12/01/2015 15:06:24    Job execution switched to status RUNNING.
> 12/01/2015 15:06:24    Source: Custom Source -> Map -> Map(1/1) switched
> to SCHEDULED
> 12/01/2015 15:06:24    Source: Custom Source -> Map -> Map(1/1) switched
> to DEPLOYING
> 12/01/2015 15:06:24    Fast TumblingTimeWindows(5000) of Reduce at
> main(ItemPriceAvgPerOrder.java:108) -> Sink: Unnamed(1/1) switched to
> SCHEDULED
> 12/01/2015 15:06:24    Fast TumblingTimeWindows(5000) of Reduce at
> main(ItemPriceAvgPerOrder.java:108) -> Sink: Unnamed(1/1) switched to
> DEPLOYING
> 12/01/2015 15:06:24    Source: Custom Source -> Map -> Map(1/1) switched
> to RUNNING
> 12/01/2015 15:06:24    Fast TumblingTimeWindows(5000) of Reduce at
> main(ItemPriceAvgPerOrder.java:108) -> Sink: Unnamed(1/1) switched to
> RUNNING
> 12/01/2015 15:56:08    Fast TumblingTimeWindows(5000) of Reduce at
> main(ItemPriceAvgPerOrder.java:108) -> Sink: Unnamed(1/1) switched to
> CANCELED
> 12/01/2015 15:56:08    Source: Custom Source -> Map -> Map(1/1) switched
> to FAILED
> java.lang.Exception
>     at
> org.apache.flink.streaming.connectors.kafka.internals.LegacyFetcher.run(LegacyFetcher.java:242)
>     at
> org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer.run(FlinkKafkaConsumer.java:397)
>     at
> org.apache.flink.streaming.api.operators.StreamSource.run(StreamSource.java:58)
>     at
> org.apache.flink.streaming.runtime.tasks.SourceStreamTask.run(SourceStreamTask.java:55)
>     at
> org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:218)
>     at org.apache.flink.runtime.taskmanager.Task.run(Task.java:584)
>     at java.lang.Thread.run(Thread.java:745)
> Caused by: java.lang.NullPointerException
>     at
> org.I0Itec.zkclient.ZkConnection.writeDataReturnStat(ZkConnection.java:115)
>     at org.I0Itec.zkclient.ZkClient$10.call(ZkClient.java:817)
>     at org.I0Itec.zkclient.ZkClient.retryUntilConnected(ZkClient.java:675)
>     at org.I0Itec.zkclient.ZkClient.writeDataReturnStat(ZkClient.java:813)
>     at org.I0Itec.zkclient.ZkClient.writeData(ZkClient.java:808)
>     at org.I0Itec.zkclient.ZkClient.writeData(ZkClient.java:777)
>     at kafka.utils.ZkUtils$.updatePersistentPath(ZkUtils.scala:332)
>     at kafka.utils.ZkUtils.updatePersistentPath(ZkUtils.scala)
>     at
> org.apache.flink.streaming.connectors.kafka.internals.ZookeeperOffsetHandler.setOffsetInZooKeeper(ZookeeperOffsetHandler.java:112)
>     at
> org.apache.flink.streaming.connectors.kafka.internals.ZookeeperOffsetHandler.commit(ZookeeperOffsetHandler.java:80)
>     at
> org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer$PeriodicOffsetCommitter.run(FlinkKafkaConsumer.java:632)
>
> Any ideas on what could cause this behaviour?
>
>
>
> Best,
>
> Mihail
>

Question about DataStream serialization

Posted by Radu Tudoran <ra...@huawei.com>.
Hello,

I am not sure if this message was received on the user list, if so I apologies for duplicate messages

I have the following scenario


·         Reading a fixed set
DataStream<String> fixedset = env.readtextFile(…

·         Reading a continuous stream of data
DataStream<String> stream = ….

I would need that for each event read from the continuous stream to make some operations onit and on the fixedsettoghether


I have tried something like

final myObject.referenceStaticSet = fixedset;
stream.map(new MapFunction<String, String>() {
                     @Override
                     public String map(String arg0) throws Exception {

                           //for example:   final string2add = arg0;
                                                                //the goal of below function would be to add the string2add to the fixedset
                           myObject.referenceStaticSet = myObject.referenceStaticSet.flatMap(new FlatMapFunction<String, String>() {

                                  @Override
                                  public void flatMap(String arg0, Collector<String> arg1)
                                                                                //for example adding to the fixed set also the string2add object:   arg1.collect(string2add);
                                                                                }
…
}

However,  I get an exception (Exception in thread "main" org.apache.flink.api.common.InvalidProgramException: ) that object is not serializable (Object MyClass$3@a71081 not serializable )

Looking into this I see that the issues is that the DataStream<> is not serializable. What would be the solution to this issue?

As I mentioned before, I would like that for each event from the continuous stream to use the initial fixed set, add the event to it and apply an operation.
Stephan was mentioning at some point some possibility to create a DataSet and launch a batch processing while operating in stream mode– in case this is possible, can you give me a reference for it, because it might be the good solution to  use in case. I am thinking that I could keep the fixed set as a DataSet and as each new event comes, transform it into a dataset and then join with reference set and apply an operation

Regards,




Dr. Radu Tudoran
Research Engineer
IT R&D Division

[cid:image007.jpg@01CD52EB.AD060EE0]
HUAWEI TECHNOLOGIES Duesseldorf GmbH
European Research Center
Riesstrasse 25, 80992 München

E-mail: radu.tudoran@huawei.com
Mobile: +49 15209084330
Telephone: +49 891588344173

HUAWEI TECHNOLOGIES Duesseldorf GmbH
Hansaallee 205, 40549 Düsseldorf, Germany, www.huawei.com<http://www.huawei.com/>
Registered Office: Düsseldorf, Register Court Düsseldorf, HRB 56063,
Managing Director: Jingwen TAO, Wanzhou MENG, Lifang CHEN
Sitz der Gesellschaft: Düsseldorf, Amtsgericht Düsseldorf, HRB 56063,
Geschäftsführer: Jingwen TAO, Wanzhou MENG, Lifang CHEN
This e-mail and its attachments contain confidential information from HUAWEI, which is intended only for the person or entity whose address is listed above. Any use of the information contained herein in any way (including, but not limited to, total or partial disclosure, reproduction, or dissemination) by persons other than the intended recipient(s) is prohibited. If you receive this e-mail in error, please notify the sender by phone or email immediately and delete it!

From: Vieru, Mihail [mailto:mihail.vieru@zalando.de]
Sent: Tuesday, December 01, 2015 4:55 PM
To: user@flink.apache.org
Subject: NPE with Flink Streaming from Kafka

Hi,
we get the following NullPointerException after ~50 minutes when running a streaming job with windowing and state that reads data from Kafka and writes the result to local FS.
There are around 170 million messages to be processed, Flink 0.10.1 stops at ~8 million.
Flink runs locally, started with the "start-cluster-streaming.sh" script.

12/01/2015 15:06:24    Job execution switched to status RUNNING.
12/01/2015 15:06:24    Source: Custom Source -> Map -> Map(1/1) switched to SCHEDULED
12/01/2015 15:06:24    Source: Custom Source -> Map -> Map(1/1) switched to DEPLOYING
12/01/2015 15:06:24    Fast TumblingTimeWindows(5000) of Reduce at main(ItemPriceAvgPerOrder.java:108) -> Sink: Unnamed(1/1) switched to SCHEDULED
12/01/2015 15:06:24    Fast TumblingTimeWindows(5000) of Reduce at main(ItemPriceAvgPerOrder.java:108) -> Sink: Unnamed(1/1) switched to DEPLOYING
12/01/2015 15:06:24    Source: Custom Source -> Map -> Map(1/1) switched to RUNNING
12/01/2015 15:06:24    Fast TumblingTimeWindows(5000) of Reduce at main(ItemPriceAvgPerOrder.java:108) -> Sink: Unnamed(1/1) switched to RUNNING
12/01/2015 15:56:08    Fast TumblingTimeWindows(5000) of Reduce at main(ItemPriceAvgPerOrder.java:108) -> Sink: Unnamed(1/1) switched to CANCELED
12/01/2015 15:56:08    Source: Custom Source -> Map -> Map(1/1) switched to FAILED
java.lang.Exception
    at org.apache.flink.streaming.connectors.kafka.internals.LegacyFetcher.run(LegacyFetcher.java:242)
    at org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer.run(FlinkKafkaConsumer.java:397)
    at org.apache.flink.streaming.api.operators.StreamSource.run(StreamSource.java:58)
    at org.apache.flink.streaming.runtime.tasks.SourceStreamTask.run(SourceStreamTask.java:55)
    at org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:218)
    at org.apache.flink.runtime.taskmanager.Task.run(Task.java:584)
    at java.lang.Thread.run(Thread.java:745)
Caused by: java.lang.NullPointerException
    at org.I0Itec.zkclient.ZkConnection.writeDataReturnStat(ZkConnection.java:115)
    at org.I0Itec.zkclient.ZkClient$10.call(ZkClient.java:817)
    at org.I0Itec.zkclient.ZkClient.retryUntilConnected(ZkClient.java:675)
    at org.I0Itec.zkclient.ZkClient.writeDataReturnStat(ZkClient.java:813)
    at org.I0Itec.zkclient.ZkClient.writeData(ZkClient.java:808)
    at org.I0Itec.zkclient.ZkClient.writeData(ZkClient.java:777)
    at kafka.utils.ZkUtils$.updatePersistentPath(ZkUtils.scala:332)
    at kafka.utils.ZkUtils.updatePersistentPath(ZkUtils.scala)
    at org.apache.flink.streaming.connectors.kafka.internals.ZookeeperOffsetHandler.setOffsetInZooKeeper(ZookeeperOffsetHandler.java:112)
    at org.apache.flink.streaming.connectors.kafka.internals.ZookeeperOffsetHandler.commit(ZookeeperOffsetHandler.java:80)
    at org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer$PeriodicOffsetCommitter.run(FlinkKafkaConsumer.java:632)

Any ideas on what could cause this behaviour?

Best,
Mihail

Re: NPE with Flink Streaming from Kafka

Posted by Stephan Ewen <se...@apache.org>.
A bit of extra information on the example where I posted the link:

The example checks whether two events follow each other within a certain
time:
  - The first event in the example is called "compute.instance.create.start"
(in your case, it would be the event that an order was placed)
  - The second event is called "trove.instance.create" - (in your case that
the package was sent)

What the timeout window does is the following:
  - It triggers either on the second event, or after the timeout is expired
  - The window function checks if the last event was the correct second
event. If yes, it sends a Result(OK), if not it sends a Result(TIMEOUT).

Hope that this helps you build your application!



On Wed, Dec 2, 2015 at 6:25 PM, Stephan Ewen <se...@apache.org> wrote:

> Hi Mihail!
>
> Do I understand you correctly that the use case is to raise an alarm if an
> order has not been processed within a certain time period (certain number
> of days) ?
>
> If that is the case, the use case is actually perfect for a special form
> of session windows that monitor such timeouts. I have prototyped a sample
> application for a different use case, but it should fit your use case as
> well:
>
> https://github.com/StephanEwen/flink-demos/blob/master/timeout-monitoring/src/main/java/com/mythingy/streaming/EventStreamAnalysis.java
>
> In that example, the timeout is 5 seconds, but there is no reason why the
> timeout could not be multiple days. Windows may be very long - no problem.
>
> Unlike many other streaming systems, each key has an individual window, so
> one key's session window may start at one point in time, and the other
> key's session window at a very different point. One window may finish
> within in a few hours (fast processed order), one window see the timout
> after three days (order that was not processed in time).
>
> Greetings,
> Stephan
>
>
> On Wed, Dec 2, 2015 at 6:11 PM, Vieru, Mihail <mi...@zalando.de>
> wrote:
>
>> Hi Gyula, Hi Stephan,
>>
>> thank you for your replies.
>>
>> We need a state which grows indefinitely for the following use case. An
>> event is created when a customer places an order. Another event is created
>> when the order is sent. These events typically occur within days. We need
>> to catch the cases when the said events occur over a specified time period
>> to raise an alarm.
>>
>> So having a window of a couple of days is not feasible. Thus we need the
>> state.
>>
>> I believe having a different state backend would circumvent the OOM
>> issue. We were thinking of Redis for performance reasons. MySQL might do as
>> well, if it doesn't slow down the processing too much.
>>
>> Are there limitations for SqlStateBackend when working with state only?
>> When would the window state limitation occur?
>>
>> Cheers,
>> Mihail
>>
>>
>> 2015-12-02 13:38 GMT+01:00 Stephan Ewen <se...@apache.org>:
>>
>>> Mihail!
>>>
>>> The Flink windows are currently in-memory only. There are plans to relax
>>> that, but for the time being, having enough memory in the cluster is
>>> important.
>>>
>>> @Gyula: I think window state is currently also limited when using the
>>> SqlStateBackend, by the size of a row in the database (because windows are
>>> not key/value state currently)
>>>
>>>
>>> Here are some simple rules-of-thumb to work with:
>>>
>>> 1) For windows, the number of expected keys can be without bound. It is
>>> important to have a rough upper bound for the number of "active keys at a
>>> certain time". For example, if you have your time windows (let's say by 10
>>> minutes or so), it only matters how many keys you have within each 10
>>> minute interval. Those define how much memory you need.
>>>
>>> 2) If you work with the "OperatorState" abstraction, then you need to
>>> think about cleanup a bit. The OperatorState keeps state currently for as
>>> long until you set the state for the key to "null". This manual state is
>>> explicitly designed to allow you to keep state across windows and across
>>> very long time. On the flip side, you need to manage the amount of state
>>> you store, by releasing state for keys.
>>>
>>> 3) If a certain key space grows infinite, you should "scope the state by
>>> time". A pragmatic solution for that is to define a session window:
>>>   - The session length defines after what inactivity the state is
>>> cleaned (let's say 1h session length or so)
>>>   - The trigger implements this session (there are a few mails on this
>>> list already that explain how to do this) and take care of evaluating on
>>> every element.
>>>   - A count(1) evictor makes sure only one element is ever stored
>>>
>>> Greetings,
>>> Stephan
>>>
>>>
>>> On Wed, Dec 2, 2015 at 11:37 AM, Gyula Fóra <gy...@apache.org> wrote:
>>>
>>>> Hi,
>>>>
>>>> I am working on a use case that involves storing state for billions of
>>>> keys. For this we use a MySql state backend that will write each key-value
>>>> state to MySql server so it will only hold a limited set of key-value pairs
>>>> on heap while maintaining the processing guarantees.
>>>>
>>>> This will keep our streaming job from running out of memory as most of
>>>> the state is off heap. I am not sure if this is relevant to your use case
>>>> but if the state size grows indefinitely you might want to give it a try.
>>>>
>>>> I will write a detailed guide in some days but if you want to get
>>>> started check this one out:
>>>>
>>>> https://docs.google.com/document/d/1xx3J88ZR_kuYYHAil3HD3YSBLqBTKV4Pu1SLwhiRiGs/edit?usp=sharing
>>>>
>>>> There are some pending improvements that I will commit in the next days
>>>> that will increase the performance of the MySql adapter
>>>>
>>>> Let me know if you are interested in this!
>>>>
>>>> Cheers,
>>>> Gyula
>>>>
>>>>
>>>> Vieru, Mihail <mi...@zalando.de> ezt írta (időpont: 2015. dec.
>>>> 2., Sze, 11:26):
>>>>
>>>>> Hi Aljoscha,
>>>>>
>>>>> we have no upper bound for the number of expected keys. The max size
>>>>> for an element is 1 KB.
>>>>>
>>>>> There are 2 Maps, a KeyBy, a timeWindow, a Reduce and a writeAsText
>>>>> operators in the job. In the first Map we parse the contained JSON object
>>>>> in each element and forward it as a Flink Tuple. In the Reduce we update
>>>>> the state for each key. That's about it.
>>>>>
>>>>> Best,
>>>>> Mihail
>>>>>
>>>>>
>>>>> 2015-12-02 11:09 GMT+01:00 Aljoscha Krettek <al...@apache.org>:
>>>>>
>>>>>> Hi Mihail,
>>>>>> could you please give some information about the number of keys that
>>>>>> you are expecting in the data and how big the elements are that you are
>>>>>> processing in the window.
>>>>>>
>>>>>> Also, are there any other operations that could be taxing on Memory.
>>>>>> I think the different exception you see for 500MB mem size is just because
>>>>>> Java notices that it ran out of memory at a different part in the program.
>>>>>>
>>>>>> Cheers,
>>>>>> Aljoscha
>>>>>> > On 02 Dec 2015, at 10:57, Vieru, Mihail <mi...@zalando.de>
>>>>>> wrote:
>>>>>> >
>>>>>> > Yes, with the "start-cluster-streaming.sh" script.
>>>>>> > If the TaskManager gets 5GB of heap it manages to process ~100
>>>>>> million messages and then throws the above OOM.
>>>>>> > If it gets only 500MB it manages to process ~8 million and a
>>>>>> somewhat misleading exception is thrown:
>>>>>> >
>>>>>> > 12/01/2015 19:14:07    Source: Custom Source -> Map -> Map(1/1)
>>>>>> switched to FAILED
>>>>>> > java.lang.Exception: Java heap space
>>>>>> >     at
>>>>>> org.apache.flink.streaming.connectors.kafka.internals.LegacyFetcher.run(LegacyFetcher.java:242)
>>>>>> >     at
>>>>>> org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer.run(FlinkKafkaConsumer.java:399)
>>>>>> >     at
>>>>>> org.apache.flink.streaming.api.operators.StreamSource.run(StreamSource.java:58)
>>>>>> >     at
>>>>>> org.apache.flink.streaming.runtime.tasks.SourceStreamTask.run(SourceStreamTask.java:55)
>>>>>> >     at
>>>>>> org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:218)
>>>>>> >     at org.apache.flink.runtime.taskmanager.Task.run(Task.java:584)
>>>>>> >     at java.lang.Thread.run(Thread.java:745)
>>>>>> > Caused by: java.lang.OutOfMemoryError: Java heap space
>>>>>> >     at org.json.simple.parser.Yylex.<init>(Yylex.java:231)
>>>>>> >     at org.json.simple.parser.JSONParser.<init>(JSONParser.java:34)
>>>>>> >     at
>>>>>> de.zalando.saiki.gerakanflink.ItemPriceAvgPerOrder$1.map(ItemPriceAvgPerOrder.java:70)
>>>>>> >     at
>>>>>> de.zalando.saiki.gerakanflink.ItemPriceAvgPerOrder$1.map(ItemPriceAvgPerOrder.java:65)
>>>>>> >     at
>>>>>> org.apache.flink.streaming.api.operators.StreamMap.processElement(StreamMap.java:37)
>>>>>> >     at
>>>>>> org.apache.flink.streaming.runtime.tasks.OperatorChain$CopyingChainingOutput.collect(OperatorChain.java:316)
>>>>>> >     at
>>>>>> org.apache.flink.streaming.runtime.tasks.OperatorChain$CopyingChainingOutput.collect(OperatorChain.java:300)
>>>>>> >     at
>>>>>> org.apache.flink.streaming.runtime.io.CollectorWrapper.collect(CollectorWrapper.java:48)
>>>>>> >     at
>>>>>> org.apache.flink.streaming.runtime.io.CollectorWrapper.collect(CollectorWrapper.java:29)
>>>>>> >     at
>>>>>> org.apache.flink.streaming.runtime.tasks.SourceStreamTask$SourceOutput.collect(SourceStreamTask.java:97)
>>>>>> >     at
>>>>>> org.apache.flink.streaming.api.operators.StreamSource$NonTimestampContext.collect(StreamSource.java:92)
>>>>>> >     at
>>>>>> org.apache.flink.streaming.connectors.kafka.internals.LegacyFetcher$SimpleConsumerThread.run(LegacyFetcher.java:450)
>>>>>> >
>>>>>> >
>>>>>> >
>>>>>> >
>>>>>> > 2015-12-02 10:45 GMT+01:00 Robert Metzger <rm...@apache.org>:
>>>>>> > Its good news that the issue has been resolved.
>>>>>> >
>>>>>> > Regarding the OOM, did you start Flink in the streaming mode?
>>>>>> >
>>>>>> > On Wed, Dec 2, 2015 at 10:18 AM, Vieru, Mihail <
>>>>>> mihail.vieru@zalando.de> wrote:
>>>>>> > Thank you, Robert! The issue with Kafka is now solved with the
>>>>>> 0.10-SNAPSHOT dependency.
>>>>>> >
>>>>>> > We have run into an OutOfMemory exception though, which appears to
>>>>>> be related to the state. As my colleague, Javier Lopez, mentioned in a
>>>>>> previous thread, state handling is crucial for our use case. And as the
>>>>>> jobs are intended to run for months, stability plays an important role in
>>>>>> choosing a stream processing framework.
>>>>>> >
>>>>>> > 12/02/2015 10:03:53    Fast TumblingTimeWindows(5000) of Reduce at
>>>>>> main(ItemPriceAvgPerOrder.java:108) -> Sink: Unnamed(1/1) switched to FAILED
>>>>>> > java.lang.OutOfMemoryError: Java heap space
>>>>>> >     at java.util.HashMap.resize(HashMap.java:703)
>>>>>> >     at java.util.HashMap.putVal(HashMap.java:662)
>>>>>> >     at java.util.HashMap.put(HashMap.java:611)
>>>>>> >     at
>>>>>> org.apache.flink.runtime.state.AbstractHeapKvState.update(AbstractHeapKvState.java:98)
>>>>>> >     at
>>>>>> de.zalando.saiki.gerakanflink.ItemPriceAvgPerOrder$3.reduce(ItemPriceAvgPerOrder.java:121)
>>>>>> >     at
>>>>>> de.zalando.saiki.gerakanflink.ItemPriceAvgPerOrder$3.reduce(ItemPriceAvgPerOrder.java:108)
>>>>>> >     at
>>>>>> org.apache.flink.streaming.runtime.operators.windowing.KeyMap.putOrAggregate(KeyMap.java:196)
>>>>>> >     at
>>>>>> org.apache.flink.streaming.runtime.operators.windowing.AggregatingKeyedTimePanes.addElementToLatestPane(AggregatingKeyedTimePanes.java:50)
>>>>>> >     at
>>>>>> org.apache.flink.streaming.runtime.operators.windowing.AbstractAlignedProcessingTimeWindowOperator.processElement(AbstractAlignedProcessingTimeWindowOperator.java:210)
>>>>>> >     at
>>>>>> org.apache.flink.streaming.runtime.io.StreamInputProcessor.processInput(StreamInputProcessor.java:166)
>>>>>> >     at
>>>>>> org.apache.flink.streaming.runtime.tasks.OneInputStreamTask.run(OneInputStreamTask.java:63)
>>>>>> >     at
>>>>>> org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:218)
>>>>>> >     at org.apache.flink.runtime.taskmanager.Task.run(Task.java:584)
>>>>>> >     at java.lang.Thread.run(Thread.java:745)
>>>>>> >
>>>>>> >
>>>>>> >
>>>>>> >
>>>>>> > 2015-12-01 17:42 GMT+01:00 Maximilian Michels <mx...@apache.org>:
>>>>>> > Thanks! I've linked the issue in JIRA.
>>>>>> >
>>>>>> > On Tue, Dec 1, 2015 at 5:39 PM, Robert Metzger <rm...@apache.org>
>>>>>> wrote:
>>>>>> > > I think its this one
>>>>>> https://issues.apache.org/jira/browse/KAFKA-824
>>>>>> > >
>>>>>> > > On Tue, Dec 1, 2015 at 5:37 PM, Maximilian Michels <
>>>>>> mxm@apache.org> wrote:
>>>>>> > >>
>>>>>> > >> I know this has been fixed already but, out of curiosity, could
>>>>>> you
>>>>>> > >> point me to the Kafka JIRA issue for this
>>>>>> > >> bug? From the Flink issue it looks like this is a Zookeeper
>>>>>> version
>>>>>> > >> mismatch.
>>>>>> > >>
>>>>>> > >> On Tue, Dec 1, 2015 at 5:16 PM, Robert Metzger <
>>>>>> rmetzger@apache.org>
>>>>>> > >> wrote:
>>>>>> > >> > Hi Gyula,
>>>>>> > >> >
>>>>>> > >> > no, I didn't ;) We still deploy 0.10-SNAPSHOT versions from the
>>>>>> > >> > "release-0.10" branch to Apache's maven snapshot repository.
>>>>>> > >> >
>>>>>> > >> >
>>>>>> > >> > I don't think Mihail's code will run when he's compiling it
>>>>>> against
>>>>>> > >> > 1.0-SNAPSHOT.
>>>>>> > >> >
>>>>>> > >> >
>>>>>> > >> > On Tue, Dec 1, 2015 at 5:13 PM, Gyula Fóra <
>>>>>> gyula.fora@gmail.com> wrote:
>>>>>> > >> >>
>>>>>> > >> >> Hi,
>>>>>> > >> >>
>>>>>> > >> >> I think Robert meant to write setting the connector
>>>>>> dependency to
>>>>>> > >> >> 1.0-SNAPSHOT.
>>>>>> > >> >>
>>>>>> > >> >> Cheers,
>>>>>> > >> >> Gyula
>>>>>> > >> >>
>>>>>> > >> >> Robert Metzger <rm...@apache.org> ezt írta (időpont:
>>>>>> 2015. dec. 1.,
>>>>>> > >> >> K,
>>>>>> > >> >> 17:10):
>>>>>> > >> >>>
>>>>>> > >> >>> Hi Mihail,
>>>>>> > >> >>>
>>>>>> > >> >>> the issue is actually a bug in Kafka. We have a JIRA in
>>>>>> Flink for this
>>>>>> > >> >>> as
>>>>>> > >> >>> well: https://issues.apache.org/jira/browse/FLINK-3067
>>>>>> > >> >>>
>>>>>> > >> >>> Sadly, we haven't released a fix for it yet. Flink 0.10.2
>>>>>> will contain
>>>>>> > >> >>> a
>>>>>> > >> >>> fix.
>>>>>> > >> >>>
>>>>>> > >> >>> Since the kafka connector is not contained in the flink
>>>>>> binary, you
>>>>>> > >> >>> can
>>>>>> > >> >>> just set the version in your maven pom file to
>>>>>> 0.10-SNAPSHOT. Maven
>>>>>> > >> >>> will
>>>>>> > >> >>> then download the code planned for the 0.10-SNAPSHOT release.
>>>>>> > >> >>>
>>>>>> > >> >>> On Tue, Dec 1, 2015 at 4:54 PM, Vieru, Mihail
>>>>>> > >> >>> <mi...@zalando.de>
>>>>>> > >> >>> wrote:
>>>>>> > >> >>>>
>>>>>> > >> >>>> Hi,
>>>>>> > >> >>>>
>>>>>> > >> >>>> we get the following NullPointerException after ~50 minutes
>>>>>> when
>>>>>> > >> >>>> running
>>>>>> > >> >>>> a streaming job with windowing and state that reads data
>>>>>> from Kafka
>>>>>> > >> >>>> and
>>>>>> > >> >>>> writes the result to local FS.
>>>>>> > >> >>>> There are around 170 million messages to be processed,
>>>>>> Flink 0.10.1
>>>>>> > >> >>>> stops at ~8 million.
>>>>>> > >> >>>> Flink runs locally, started with the
>>>>>> "start-cluster-streaming.sh"
>>>>>> > >> >>>> script.
>>>>>> > >> >>>>
>>>>>> > >> >>>> 12/01/2015 15:06:24    Job execution switched to status
>>>>>> RUNNING.
>>>>>> > >> >>>> 12/01/2015 15:06:24    Source: Custom Source -> Map ->
>>>>>> Map(1/1)
>>>>>> > >> >>>> switched
>>>>>> > >> >>>> to SCHEDULED
>>>>>> > >> >>>> 12/01/2015 15:06:24    Source: Custom Source -> Map ->
>>>>>> Map(1/1)
>>>>>> > >> >>>> switched
>>>>>> > >> >>>> to DEPLOYING
>>>>>> > >> >>>> 12/01/2015 15:06:24    Fast TumblingTimeWindows(5000) of
>>>>>> Reduce at
>>>>>> > >> >>>> main(ItemPriceAvgPerOrder.java:108) -> Sink: Unnamed(1/1)
>>>>>> switched to
>>>>>> > >> >>>> SCHEDULED
>>>>>> > >> >>>> 12/01/2015 15:06:24    Fast TumblingTimeWindows(5000) of
>>>>>> Reduce at
>>>>>> > >> >>>> main(ItemPriceAvgPerOrder.java:108) -> Sink: Unnamed(1/1)
>>>>>> switched to
>>>>>> > >> >>>> DEPLOYING
>>>>>> > >> >>>> 12/01/2015 15:06:24    Source: Custom Source -> Map ->
>>>>>> Map(1/1)
>>>>>> > >> >>>> switched
>>>>>> > >> >>>> to RUNNING
>>>>>> > >> >>>> 12/01/2015 15:06:24    Fast TumblingTimeWindows(5000) of
>>>>>> Reduce at
>>>>>> > >> >>>> main(ItemPriceAvgPerOrder.java:108) -> Sink: Unnamed(1/1)
>>>>>> switched to
>>>>>> > >> >>>> RUNNING
>>>>>> > >> >>>> 12/01/2015 15:56:08    Fast TumblingTimeWindows(5000) of
>>>>>> Reduce at
>>>>>> > >> >>>> main(ItemPriceAvgPerOrder.java:108) -> Sink: Unnamed(1/1)
>>>>>> switched to
>>>>>> > >> >>>> CANCELED
>>>>>> > >> >>>> 12/01/2015 15:56:08    Source: Custom Source -> Map ->
>>>>>> Map(1/1)
>>>>>> > >> >>>> switched
>>>>>> > >> >>>> to FAILED
>>>>>> > >> >>>> java.lang.Exception
>>>>>> > >> >>>>     at
>>>>>> > >> >>>>
>>>>>> > >> >>>>
>>>>>> org.apache.flink.streaming.connectors.kafka.internals.LegacyFetcher.run(LegacyFetcher.java:242)
>>>>>> > >> >>>>     at
>>>>>> > >> >>>>
>>>>>> > >> >>>>
>>>>>> org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer.run(FlinkKafkaConsumer.java:397)
>>>>>> > >> >>>>     at
>>>>>> > >> >>>>
>>>>>> > >> >>>>
>>>>>> org.apache.flink.streaming.api.operators.StreamSource.run(StreamSource.java:58)
>>>>>> > >> >>>>     at
>>>>>> > >> >>>>
>>>>>> > >> >>>>
>>>>>> org.apache.flink.streaming.runtime.tasks.SourceStreamTask.run(SourceStreamTask.java:55)
>>>>>> > >> >>>>     at
>>>>>> > >> >>>>
>>>>>> > >> >>>>
>>>>>> org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:218)
>>>>>> > >> >>>>     at
>>>>>> org.apache.flink.runtime.taskmanager.Task.run(Task.java:584)
>>>>>> > >> >>>>     at java.lang.Thread.run(Thread.java:745)
>>>>>> > >> >>>> Caused by: java.lang.NullPointerException
>>>>>> > >> >>>>     at
>>>>>> > >> >>>>
>>>>>> > >> >>>>
>>>>>> org.I0Itec.zkclient.ZkConnection.writeDataReturnStat(ZkConnection.java:115)
>>>>>> > >> >>>>     at
>>>>>> org.I0Itec.zkclient.ZkClient$10.call(ZkClient.java:817)
>>>>>> > >> >>>>     at
>>>>>> > >> >>>>
>>>>>> org.I0Itec.zkclient.ZkClient.retryUntilConnected(ZkClient.java:675)
>>>>>> > >> >>>>     at
>>>>>> > >> >>>>
>>>>>> org.I0Itec.zkclient.ZkClient.writeDataReturnStat(ZkClient.java:813)
>>>>>> > >> >>>>     at
>>>>>> org.I0Itec.zkclient.ZkClient.writeData(ZkClient.java:808)
>>>>>> > >> >>>>     at
>>>>>> org.I0Itec.zkclient.ZkClient.writeData(ZkClient.java:777)
>>>>>> > >> >>>>     at
>>>>>> kafka.utils.ZkUtils$.updatePersistentPath(ZkUtils.scala:332)
>>>>>> > >> >>>>     at
>>>>>> kafka.utils.ZkUtils.updatePersistentPath(ZkUtils.scala)
>>>>>> > >> >>>>     at
>>>>>> > >> >>>>
>>>>>> > >> >>>>
>>>>>> org.apache.flink.streaming.connectors.kafka.internals.ZookeeperOffsetHandler.setOffsetInZooKeeper(ZookeeperOffsetHandler.java:112)
>>>>>> > >> >>>>     at
>>>>>> > >> >>>>
>>>>>> > >> >>>>
>>>>>> org.apache.flink.streaming.connectors.kafka.internals.ZookeeperOffsetHandler.commit(ZookeeperOffsetHandler.java:80)
>>>>>> > >> >>>>     at
>>>>>> > >> >>>>
>>>>>> > >> >>>>
>>>>>> org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer$PeriodicOffsetCommitter.run(FlinkKafkaConsumer.java:632)
>>>>>> > >> >>>>
>>>>>> > >> >>>>
>>>>>> > >> >>>> Any ideas on what could cause this behaviour?
>>>>>> > >> >>>>
>>>>>> > >> >>>> Best,
>>>>>> > >> >>>> Mihail
>>>>>> > >> >>>
>>>>>> > >> >>>
>>>>>> > >> >
>>>>>> > >
>>>>>> > >
>>>>>> >
>>>>>> >
>>>>>> >
>>>>>>
>>>>>>
>>>>>
>>>
>>
>

Re: NPE with Flink Streaming from Kafka

Posted by Stephan Ewen <se...@apache.org>.
Hi Mihail!

Do I understand you correctly that the use case is to raise an alarm if an
order has not been processed within a certain time period (certain number
of days) ?

If that is the case, the use case is actually perfect for a special form of
session windows that monitor such timeouts. I have prototyped a sample
application for a different use case, but it should fit your use case as
well:
https://github.com/StephanEwen/flink-demos/blob/master/timeout-monitoring/src/main/java/com/mythingy/streaming/EventStreamAnalysis.java

In that example, the timeout is 5 seconds, but there is no reason why the
timeout could not be multiple days. Windows may be very long - no problem.

Unlike many other streaming systems, each key has an individual window, so
one key's session window may start at one point in time, and the other
key's session window at a very different point. One window may finish
within in a few hours (fast processed order), one window see the timout
after three days (order that was not processed in time).

Greetings,
Stephan


On Wed, Dec 2, 2015 at 6:11 PM, Vieru, Mihail <mi...@zalando.de>
wrote:

> Hi Gyula, Hi Stephan,
>
> thank you for your replies.
>
> We need a state which grows indefinitely for the following use case. An
> event is created when a customer places an order. Another event is created
> when the order is sent. These events typically occur within days. We need
> to catch the cases when the said events occur over a specified time period
> to raise an alarm.
>
> So having a window of a couple of days is not feasible. Thus we need the
> state.
>
> I believe having a different state backend would circumvent the OOM issue.
> We were thinking of Redis for performance reasons. MySQL might do as well,
> if it doesn't slow down the processing too much.
>
> Are there limitations for SqlStateBackend when working with state only?
> When would the window state limitation occur?
>
> Cheers,
> Mihail
>
>
> 2015-12-02 13:38 GMT+01:00 Stephan Ewen <se...@apache.org>:
>
>> Mihail!
>>
>> The Flink windows are currently in-memory only. There are plans to relax
>> that, but for the time being, having enough memory in the cluster is
>> important.
>>
>> @Gyula: I think window state is currently also limited when using the
>> SqlStateBackend, by the size of a row in the database (because windows are
>> not key/value state currently)
>>
>>
>> Here are some simple rules-of-thumb to work with:
>>
>> 1) For windows, the number of expected keys can be without bound. It is
>> important to have a rough upper bound for the number of "active keys at a
>> certain time". For example, if you have your time windows (let's say by 10
>> minutes or so), it only matters how many keys you have within each 10
>> minute interval. Those define how much memory you need.
>>
>> 2) If you work with the "OperatorState" abstraction, then you need to
>> think about cleanup a bit. The OperatorState keeps state currently for as
>> long until you set the state for the key to "null". This manual state is
>> explicitly designed to allow you to keep state across windows and across
>> very long time. On the flip side, you need to manage the amount of state
>> you store, by releasing state for keys.
>>
>> 3) If a certain key space grows infinite, you should "scope the state by
>> time". A pragmatic solution for that is to define a session window:
>>   - The session length defines after what inactivity the state is cleaned
>> (let's say 1h session length or so)
>>   - The trigger implements this session (there are a few mails on this
>> list already that explain how to do this) and take care of evaluating on
>> every element.
>>   - A count(1) evictor makes sure only one element is ever stored
>>
>> Greetings,
>> Stephan
>>
>>
>> On Wed, Dec 2, 2015 at 11:37 AM, Gyula Fóra <gy...@apache.org> wrote:
>>
>>> Hi,
>>>
>>> I am working on a use case that involves storing state for billions of
>>> keys. For this we use a MySql state backend that will write each key-value
>>> state to MySql server so it will only hold a limited set of key-value pairs
>>> on heap while maintaining the processing guarantees.
>>>
>>> This will keep our streaming job from running out of memory as most of
>>> the state is off heap. I am not sure if this is relevant to your use case
>>> but if the state size grows indefinitely you might want to give it a try.
>>>
>>> I will write a detailed guide in some days but if you want to get
>>> started check this one out:
>>>
>>> https://docs.google.com/document/d/1xx3J88ZR_kuYYHAil3HD3YSBLqBTKV4Pu1SLwhiRiGs/edit?usp=sharing
>>>
>>> There are some pending improvements that I will commit in the next days
>>> that will increase the performance of the MySql adapter
>>>
>>> Let me know if you are interested in this!
>>>
>>> Cheers,
>>> Gyula
>>>
>>>
>>> Vieru, Mihail <mi...@zalando.de> ezt írta (időpont: 2015. dec.
>>> 2., Sze, 11:26):
>>>
>>>> Hi Aljoscha,
>>>>
>>>> we have no upper bound for the number of expected keys. The max size
>>>> for an element is 1 KB.
>>>>
>>>> There are 2 Maps, a KeyBy, a timeWindow, a Reduce and a writeAsText
>>>> operators in the job. In the first Map we parse the contained JSON object
>>>> in each element and forward it as a Flink Tuple. In the Reduce we update
>>>> the state for each key. That's about it.
>>>>
>>>> Best,
>>>> Mihail
>>>>
>>>>
>>>> 2015-12-02 11:09 GMT+01:00 Aljoscha Krettek <al...@apache.org>:
>>>>
>>>>> Hi Mihail,
>>>>> could you please give some information about the number of keys that
>>>>> you are expecting in the data and how big the elements are that you are
>>>>> processing in the window.
>>>>>
>>>>> Also, are there any other operations that could be taxing on Memory. I
>>>>> think the different exception you see for 500MB mem size is just because
>>>>> Java notices that it ran out of memory at a different part in the program.
>>>>>
>>>>> Cheers,
>>>>> Aljoscha
>>>>> > On 02 Dec 2015, at 10:57, Vieru, Mihail <mi...@zalando.de>
>>>>> wrote:
>>>>> >
>>>>> > Yes, with the "start-cluster-streaming.sh" script.
>>>>> > If the TaskManager gets 5GB of heap it manages to process ~100
>>>>> million messages and then throws the above OOM.
>>>>> > If it gets only 500MB it manages to process ~8 million and a
>>>>> somewhat misleading exception is thrown:
>>>>> >
>>>>> > 12/01/2015 19:14:07    Source: Custom Source -> Map -> Map(1/1)
>>>>> switched to FAILED
>>>>> > java.lang.Exception: Java heap space
>>>>> >     at
>>>>> org.apache.flink.streaming.connectors.kafka.internals.LegacyFetcher.run(LegacyFetcher.java:242)
>>>>> >     at
>>>>> org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer.run(FlinkKafkaConsumer.java:399)
>>>>> >     at
>>>>> org.apache.flink.streaming.api.operators.StreamSource.run(StreamSource.java:58)
>>>>> >     at
>>>>> org.apache.flink.streaming.runtime.tasks.SourceStreamTask.run(SourceStreamTask.java:55)
>>>>> >     at
>>>>> org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:218)
>>>>> >     at org.apache.flink.runtime.taskmanager.Task.run(Task.java:584)
>>>>> >     at java.lang.Thread.run(Thread.java:745)
>>>>> > Caused by: java.lang.OutOfMemoryError: Java heap space
>>>>> >     at org.json.simple.parser.Yylex.<init>(Yylex.java:231)
>>>>> >     at org.json.simple.parser.JSONParser.<init>(JSONParser.java:34)
>>>>> >     at
>>>>> de.zalando.saiki.gerakanflink.ItemPriceAvgPerOrder$1.map(ItemPriceAvgPerOrder.java:70)
>>>>> >     at
>>>>> de.zalando.saiki.gerakanflink.ItemPriceAvgPerOrder$1.map(ItemPriceAvgPerOrder.java:65)
>>>>> >     at
>>>>> org.apache.flink.streaming.api.operators.StreamMap.processElement(StreamMap.java:37)
>>>>> >     at
>>>>> org.apache.flink.streaming.runtime.tasks.OperatorChain$CopyingChainingOutput.collect(OperatorChain.java:316)
>>>>> >     at
>>>>> org.apache.flink.streaming.runtime.tasks.OperatorChain$CopyingChainingOutput.collect(OperatorChain.java:300)
>>>>> >     at
>>>>> org.apache.flink.streaming.runtime.io.CollectorWrapper.collect(CollectorWrapper.java:48)
>>>>> >     at
>>>>> org.apache.flink.streaming.runtime.io.CollectorWrapper.collect(CollectorWrapper.java:29)
>>>>> >     at
>>>>> org.apache.flink.streaming.runtime.tasks.SourceStreamTask$SourceOutput.collect(SourceStreamTask.java:97)
>>>>> >     at
>>>>> org.apache.flink.streaming.api.operators.StreamSource$NonTimestampContext.collect(StreamSource.java:92)
>>>>> >     at
>>>>> org.apache.flink.streaming.connectors.kafka.internals.LegacyFetcher$SimpleConsumerThread.run(LegacyFetcher.java:450)
>>>>> >
>>>>> >
>>>>> >
>>>>> >
>>>>> > 2015-12-02 10:45 GMT+01:00 Robert Metzger <rm...@apache.org>:
>>>>> > Its good news that the issue has been resolved.
>>>>> >
>>>>> > Regarding the OOM, did you start Flink in the streaming mode?
>>>>> >
>>>>> > On Wed, Dec 2, 2015 at 10:18 AM, Vieru, Mihail <
>>>>> mihail.vieru@zalando.de> wrote:
>>>>> > Thank you, Robert! The issue with Kafka is now solved with the
>>>>> 0.10-SNAPSHOT dependency.
>>>>> >
>>>>> > We have run into an OutOfMemory exception though, which appears to
>>>>> be related to the state. As my colleague, Javier Lopez, mentioned in a
>>>>> previous thread, state handling is crucial for our use case. And as the
>>>>> jobs are intended to run for months, stability plays an important role in
>>>>> choosing a stream processing framework.
>>>>> >
>>>>> > 12/02/2015 10:03:53    Fast TumblingTimeWindows(5000) of Reduce at
>>>>> main(ItemPriceAvgPerOrder.java:108) -> Sink: Unnamed(1/1) switched to FAILED
>>>>> > java.lang.OutOfMemoryError: Java heap space
>>>>> >     at java.util.HashMap.resize(HashMap.java:703)
>>>>> >     at java.util.HashMap.putVal(HashMap.java:662)
>>>>> >     at java.util.HashMap.put(HashMap.java:611)
>>>>> >     at
>>>>> org.apache.flink.runtime.state.AbstractHeapKvState.update(AbstractHeapKvState.java:98)
>>>>> >     at
>>>>> de.zalando.saiki.gerakanflink.ItemPriceAvgPerOrder$3.reduce(ItemPriceAvgPerOrder.java:121)
>>>>> >     at
>>>>> de.zalando.saiki.gerakanflink.ItemPriceAvgPerOrder$3.reduce(ItemPriceAvgPerOrder.java:108)
>>>>> >     at
>>>>> org.apache.flink.streaming.runtime.operators.windowing.KeyMap.putOrAggregate(KeyMap.java:196)
>>>>> >     at
>>>>> org.apache.flink.streaming.runtime.operators.windowing.AggregatingKeyedTimePanes.addElementToLatestPane(AggregatingKeyedTimePanes.java:50)
>>>>> >     at
>>>>> org.apache.flink.streaming.runtime.operators.windowing.AbstractAlignedProcessingTimeWindowOperator.processElement(AbstractAlignedProcessingTimeWindowOperator.java:210)
>>>>> >     at
>>>>> org.apache.flink.streaming.runtime.io.StreamInputProcessor.processInput(StreamInputProcessor.java:166)
>>>>> >     at
>>>>> org.apache.flink.streaming.runtime.tasks.OneInputStreamTask.run(OneInputStreamTask.java:63)
>>>>> >     at
>>>>> org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:218)
>>>>> >     at org.apache.flink.runtime.taskmanager.Task.run(Task.java:584)
>>>>> >     at java.lang.Thread.run(Thread.java:745)
>>>>> >
>>>>> >
>>>>> >
>>>>> >
>>>>> > 2015-12-01 17:42 GMT+01:00 Maximilian Michels <mx...@apache.org>:
>>>>> > Thanks! I've linked the issue in JIRA.
>>>>> >
>>>>> > On Tue, Dec 1, 2015 at 5:39 PM, Robert Metzger <rm...@apache.org>
>>>>> wrote:
>>>>> > > I think its this one
>>>>> https://issues.apache.org/jira/browse/KAFKA-824
>>>>> > >
>>>>> > > On Tue, Dec 1, 2015 at 5:37 PM, Maximilian Michels <mx...@apache.org>
>>>>> wrote:
>>>>> > >>
>>>>> > >> I know this has been fixed already but, out of curiosity, could
>>>>> you
>>>>> > >> point me to the Kafka JIRA issue for this
>>>>> > >> bug? From the Flink issue it looks like this is a Zookeeper
>>>>> version
>>>>> > >> mismatch.
>>>>> > >>
>>>>> > >> On Tue, Dec 1, 2015 at 5:16 PM, Robert Metzger <
>>>>> rmetzger@apache.org>
>>>>> > >> wrote:
>>>>> > >> > Hi Gyula,
>>>>> > >> >
>>>>> > >> > no, I didn't ;) We still deploy 0.10-SNAPSHOT versions from the
>>>>> > >> > "release-0.10" branch to Apache's maven snapshot repository.
>>>>> > >> >
>>>>> > >> >
>>>>> > >> > I don't think Mihail's code will run when he's compiling it
>>>>> against
>>>>> > >> > 1.0-SNAPSHOT.
>>>>> > >> >
>>>>> > >> >
>>>>> > >> > On Tue, Dec 1, 2015 at 5:13 PM, Gyula Fóra <
>>>>> gyula.fora@gmail.com> wrote:
>>>>> > >> >>
>>>>> > >> >> Hi,
>>>>> > >> >>
>>>>> > >> >> I think Robert meant to write setting the connector dependency
>>>>> to
>>>>> > >> >> 1.0-SNAPSHOT.
>>>>> > >> >>
>>>>> > >> >> Cheers,
>>>>> > >> >> Gyula
>>>>> > >> >>
>>>>> > >> >> Robert Metzger <rm...@apache.org> ezt írta (időpont: 2015.
>>>>> dec. 1.,
>>>>> > >> >> K,
>>>>> > >> >> 17:10):
>>>>> > >> >>>
>>>>> > >> >>> Hi Mihail,
>>>>> > >> >>>
>>>>> > >> >>> the issue is actually a bug in Kafka. We have a JIRA in Flink
>>>>> for this
>>>>> > >> >>> as
>>>>> > >> >>> well: https://issues.apache.org/jira/browse/FLINK-3067
>>>>> > >> >>>
>>>>> > >> >>> Sadly, we haven't released a fix for it yet. Flink 0.10.2
>>>>> will contain
>>>>> > >> >>> a
>>>>> > >> >>> fix.
>>>>> > >> >>>
>>>>> > >> >>> Since the kafka connector is not contained in the flink
>>>>> binary, you
>>>>> > >> >>> can
>>>>> > >> >>> just set the version in your maven pom file to 0.10-SNAPSHOT.
>>>>> Maven
>>>>> > >> >>> will
>>>>> > >> >>> then download the code planned for the 0.10-SNAPSHOT release.
>>>>> > >> >>>
>>>>> > >> >>> On Tue, Dec 1, 2015 at 4:54 PM, Vieru, Mihail
>>>>> > >> >>> <mi...@zalando.de>
>>>>> > >> >>> wrote:
>>>>> > >> >>>>
>>>>> > >> >>>> Hi,
>>>>> > >> >>>>
>>>>> > >> >>>> we get the following NullPointerException after ~50 minutes
>>>>> when
>>>>> > >> >>>> running
>>>>> > >> >>>> a streaming job with windowing and state that reads data
>>>>> from Kafka
>>>>> > >> >>>> and
>>>>> > >> >>>> writes the result to local FS.
>>>>> > >> >>>> There are around 170 million messages to be processed, Flink
>>>>> 0.10.1
>>>>> > >> >>>> stops at ~8 million.
>>>>> > >> >>>> Flink runs locally, started with the
>>>>> "start-cluster-streaming.sh"
>>>>> > >> >>>> script.
>>>>> > >> >>>>
>>>>> > >> >>>> 12/01/2015 15:06:24    Job execution switched to status
>>>>> RUNNING.
>>>>> > >> >>>> 12/01/2015 15:06:24    Source: Custom Source -> Map ->
>>>>> Map(1/1)
>>>>> > >> >>>> switched
>>>>> > >> >>>> to SCHEDULED
>>>>> > >> >>>> 12/01/2015 15:06:24    Source: Custom Source -> Map ->
>>>>> Map(1/1)
>>>>> > >> >>>> switched
>>>>> > >> >>>> to DEPLOYING
>>>>> > >> >>>> 12/01/2015 15:06:24    Fast TumblingTimeWindows(5000) of
>>>>> Reduce at
>>>>> > >> >>>> main(ItemPriceAvgPerOrder.java:108) -> Sink: Unnamed(1/1)
>>>>> switched to
>>>>> > >> >>>> SCHEDULED
>>>>> > >> >>>> 12/01/2015 15:06:24    Fast TumblingTimeWindows(5000) of
>>>>> Reduce at
>>>>> > >> >>>> main(ItemPriceAvgPerOrder.java:108) -> Sink: Unnamed(1/1)
>>>>> switched to
>>>>> > >> >>>> DEPLOYING
>>>>> > >> >>>> 12/01/2015 15:06:24    Source: Custom Source -> Map ->
>>>>> Map(1/1)
>>>>> > >> >>>> switched
>>>>> > >> >>>> to RUNNING
>>>>> > >> >>>> 12/01/2015 15:06:24    Fast TumblingTimeWindows(5000) of
>>>>> Reduce at
>>>>> > >> >>>> main(ItemPriceAvgPerOrder.java:108) -> Sink: Unnamed(1/1)
>>>>> switched to
>>>>> > >> >>>> RUNNING
>>>>> > >> >>>> 12/01/2015 15:56:08    Fast TumblingTimeWindows(5000) of
>>>>> Reduce at
>>>>> > >> >>>> main(ItemPriceAvgPerOrder.java:108) -> Sink: Unnamed(1/1)
>>>>> switched to
>>>>> > >> >>>> CANCELED
>>>>> > >> >>>> 12/01/2015 15:56:08    Source: Custom Source -> Map ->
>>>>> Map(1/1)
>>>>> > >> >>>> switched
>>>>> > >> >>>> to FAILED
>>>>> > >> >>>> java.lang.Exception
>>>>> > >> >>>>     at
>>>>> > >> >>>>
>>>>> > >> >>>>
>>>>> org.apache.flink.streaming.connectors.kafka.internals.LegacyFetcher.run(LegacyFetcher.java:242)
>>>>> > >> >>>>     at
>>>>> > >> >>>>
>>>>> > >> >>>>
>>>>> org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer.run(FlinkKafkaConsumer.java:397)
>>>>> > >> >>>>     at
>>>>> > >> >>>>
>>>>> > >> >>>>
>>>>> org.apache.flink.streaming.api.operators.StreamSource.run(StreamSource.java:58)
>>>>> > >> >>>>     at
>>>>> > >> >>>>
>>>>> > >> >>>>
>>>>> org.apache.flink.streaming.runtime.tasks.SourceStreamTask.run(SourceStreamTask.java:55)
>>>>> > >> >>>>     at
>>>>> > >> >>>>
>>>>> > >> >>>>
>>>>> org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:218)
>>>>> > >> >>>>     at
>>>>> org.apache.flink.runtime.taskmanager.Task.run(Task.java:584)
>>>>> > >> >>>>     at java.lang.Thread.run(Thread.java:745)
>>>>> > >> >>>> Caused by: java.lang.NullPointerException
>>>>> > >> >>>>     at
>>>>> > >> >>>>
>>>>> > >> >>>>
>>>>> org.I0Itec.zkclient.ZkConnection.writeDataReturnStat(ZkConnection.java:115)
>>>>> > >> >>>>     at
>>>>> org.I0Itec.zkclient.ZkClient$10.call(ZkClient.java:817)
>>>>> > >> >>>>     at
>>>>> > >> >>>>
>>>>> org.I0Itec.zkclient.ZkClient.retryUntilConnected(ZkClient.java:675)
>>>>> > >> >>>>     at
>>>>> > >> >>>>
>>>>> org.I0Itec.zkclient.ZkClient.writeDataReturnStat(ZkClient.java:813)
>>>>> > >> >>>>     at
>>>>> org.I0Itec.zkclient.ZkClient.writeData(ZkClient.java:808)
>>>>> > >> >>>>     at
>>>>> org.I0Itec.zkclient.ZkClient.writeData(ZkClient.java:777)
>>>>> > >> >>>>     at
>>>>> kafka.utils.ZkUtils$.updatePersistentPath(ZkUtils.scala:332)
>>>>> > >> >>>>     at
>>>>> kafka.utils.ZkUtils.updatePersistentPath(ZkUtils.scala)
>>>>> > >> >>>>     at
>>>>> > >> >>>>
>>>>> > >> >>>>
>>>>> org.apache.flink.streaming.connectors.kafka.internals.ZookeeperOffsetHandler.setOffsetInZooKeeper(ZookeeperOffsetHandler.java:112)
>>>>> > >> >>>>     at
>>>>> > >> >>>>
>>>>> > >> >>>>
>>>>> org.apache.flink.streaming.connectors.kafka.internals.ZookeeperOffsetHandler.commit(ZookeeperOffsetHandler.java:80)
>>>>> > >> >>>>     at
>>>>> > >> >>>>
>>>>> > >> >>>>
>>>>> org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer$PeriodicOffsetCommitter.run(FlinkKafkaConsumer.java:632)
>>>>> > >> >>>>
>>>>> > >> >>>>
>>>>> > >> >>>> Any ideas on what could cause this behaviour?
>>>>> > >> >>>>
>>>>> > >> >>>> Best,
>>>>> > >> >>>> Mihail
>>>>> > >> >>>
>>>>> > >> >>>
>>>>> > >> >
>>>>> > >
>>>>> > >
>>>>> >
>>>>> >
>>>>> >
>>>>>
>>>>>
>>>>
>>
>

Re: NPE with Flink Streaming from Kafka

Posted by Fabian Hueske <fh...@gmail.com>.
Hi Mihail,

not sure if I correctly got your requirements, but you can define windows
on a keyed stream. This basically means that you partition the stream, for
example by order-id, and compute windows over the keyed stream. This will
create one (or more, depending on the window type) window for each key.

All windows are independent from each other and can be evaluated and closed
without interfering with other windows.
So in your use case, windows be closed as soon as the order is sent out.
Windows for keys that have been closed do not allocate any memory until
another element with the same key arrives.

Btw. internally, windows are also realized as a special kind of operator
state.

Best, Fabian

2015-12-02 18:11 GMT+01:00 Vieru, Mihail <mi...@zalando.de>:

> Hi Gyula, Hi Stephan,
>
> thank you for your replies.
>
> We need a state which grows indefinitely for the following use case. An
> event is created when a customer places an order. Another event is created
> when the order is sent. These events typically occur within days. We need
> to catch the cases when the said events occur over a specified time period
> to raise an alarm.
>
> So having a window of a couple of days is not feasible. Thus we need the
> state.
>
> I believe having a different state backend would circumvent the OOM issue.
> We were thinking of Redis for performance reasons. MySQL might do as well,
> if it doesn't slow down the processing too much.
>
> Are there limitations for SqlStateBackend when working with state only?
> When would the window state limitation occur?
>
> Cheers,
> Mihail
>
>
> 2015-12-02 13:38 GMT+01:00 Stephan Ewen <se...@apache.org>:
>
>> Mihail!
>>
>> The Flink windows are currently in-memory only. There are plans to relax
>> that, but for the time being, having enough memory in the cluster is
>> important.
>>
>> @Gyula: I think window state is currently also limited when using the
>> SqlStateBackend, by the size of a row in the database (because windows are
>> not key/value state currently)
>>
>>
>> Here are some simple rules-of-thumb to work with:
>>
>> 1) For windows, the number of expected keys can be without bound. It is
>> important to have a rough upper bound for the number of "active keys at a
>> certain time". For example, if you have your time windows (let's say by 10
>> minutes or so), it only matters how many keys you have within each 10
>> minute interval. Those define how much memory you need.
>>
>> 2) If you work with the "OperatorState" abstraction, then you need to
>> think about cleanup a bit. The OperatorState keeps state currently for as
>> long until you set the state for the key to "null". This manual state is
>> explicitly designed to allow you to keep state across windows and across
>> very long time. On the flip side, you need to manage the amount of state
>> you store, by releasing state for keys.
>>
>> 3) If a certain key space grows infinite, you should "scope the state by
>> time". A pragmatic solution for that is to define a session window:
>>   - The session length defines after what inactivity the state is cleaned
>> (let's say 1h session length or so)
>>   - The trigger implements this session (there are a few mails on this
>> list already that explain how to do this) and take care of evaluating on
>> every element.
>>   - A count(1) evictor makes sure only one element is ever stored
>>
>> Greetings,
>> Stephan
>>
>>
>> On Wed, Dec 2, 2015 at 11:37 AM, Gyula Fóra <gy...@apache.org> wrote:
>>
>>> Hi,
>>>
>>> I am working on a use case that involves storing state for billions of
>>> keys. For this we use a MySql state backend that will write each key-value
>>> state to MySql server so it will only hold a limited set of key-value pairs
>>> on heap while maintaining the processing guarantees.
>>>
>>> This will keep our streaming job from running out of memory as most of
>>> the state is off heap. I am not sure if this is relevant to your use case
>>> but if the state size grows indefinitely you might want to give it a try.
>>>
>>> I will write a detailed guide in some days but if you want to get
>>> started check this one out:
>>>
>>> https://docs.google.com/document/d/1xx3J88ZR_kuYYHAil3HD3YSBLqBTKV4Pu1SLwhiRiGs/edit?usp=sharing
>>>
>>> There are some pending improvements that I will commit in the next days
>>> that will increase the performance of the MySql adapter
>>>
>>> Let me know if you are interested in this!
>>>
>>> Cheers,
>>> Gyula
>>>
>>>
>>> Vieru, Mihail <mi...@zalando.de> ezt írta (időpont: 2015. dec.
>>> 2., Sze, 11:26):
>>>
>>>> Hi Aljoscha,
>>>>
>>>> we have no upper bound for the number of expected keys. The max size
>>>> for an element is 1 KB.
>>>>
>>>> There are 2 Maps, a KeyBy, a timeWindow, a Reduce and a writeAsText
>>>> operators in the job. In the first Map we parse the contained JSON object
>>>> in each element and forward it as a Flink Tuple. In the Reduce we update
>>>> the state for each key. That's about it.
>>>>
>>>> Best,
>>>> Mihail
>>>>
>>>>
>>>> 2015-12-02 11:09 GMT+01:00 Aljoscha Krettek <al...@apache.org>:
>>>>
>>>>> Hi Mihail,
>>>>> could you please give some information about the number of keys that
>>>>> you are expecting in the data and how big the elements are that you are
>>>>> processing in the window.
>>>>>
>>>>> Also, are there any other operations that could be taxing on Memory. I
>>>>> think the different exception you see for 500MB mem size is just because
>>>>> Java notices that it ran out of memory at a different part in the program.
>>>>>
>>>>> Cheers,
>>>>> Aljoscha
>>>>> > On 02 Dec 2015, at 10:57, Vieru, Mihail <mi...@zalando.de>
>>>>> wrote:
>>>>> >
>>>>> > Yes, with the "start-cluster-streaming.sh" script.
>>>>> > If the TaskManager gets 5GB of heap it manages to process ~100
>>>>> million messages and then throws the above OOM.
>>>>> > If it gets only 500MB it manages to process ~8 million and a
>>>>> somewhat misleading exception is thrown:
>>>>> >
>>>>> > 12/01/2015 19:14:07    Source: Custom Source -> Map -> Map(1/1)
>>>>> switched to FAILED
>>>>> > java.lang.Exception: Java heap space
>>>>> >     at
>>>>> org.apache.flink.streaming.connectors.kafka.internals.LegacyFetcher.run(LegacyFetcher.java:242)
>>>>> >     at
>>>>> org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer.run(FlinkKafkaConsumer.java:399)
>>>>> >     at
>>>>> org.apache.flink.streaming.api.operators.StreamSource.run(StreamSource.java:58)
>>>>> >     at
>>>>> org.apache.flink.streaming.runtime.tasks.SourceStreamTask.run(SourceStreamTask.java:55)
>>>>> >     at
>>>>> org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:218)
>>>>> >     at org.apache.flink.runtime.taskmanager.Task.run(Task.java:584)
>>>>> >     at java.lang.Thread.run(Thread.java:745)
>>>>> > Caused by: java.lang.OutOfMemoryError: Java heap space
>>>>> >     at org.json.simple.parser.Yylex.<init>(Yylex.java:231)
>>>>> >     at org.json.simple.parser.JSONParser.<init>(JSONParser.java:34)
>>>>> >     at
>>>>> de.zalando.saiki.gerakanflink.ItemPriceAvgPerOrder$1.map(ItemPriceAvgPerOrder.java:70)
>>>>> >     at
>>>>> de.zalando.saiki.gerakanflink.ItemPriceAvgPerOrder$1.map(ItemPriceAvgPerOrder.java:65)
>>>>> >     at
>>>>> org.apache.flink.streaming.api.operators.StreamMap.processElement(StreamMap.java:37)
>>>>> >     at
>>>>> org.apache.flink.streaming.runtime.tasks.OperatorChain$CopyingChainingOutput.collect(OperatorChain.java:316)
>>>>> >     at
>>>>> org.apache.flink.streaming.runtime.tasks.OperatorChain$CopyingChainingOutput.collect(OperatorChain.java:300)
>>>>> >     at
>>>>> org.apache.flink.streaming.runtime.io.CollectorWrapper.collect(CollectorWrapper.java:48)
>>>>> >     at
>>>>> org.apache.flink.streaming.runtime.io.CollectorWrapper.collect(CollectorWrapper.java:29)
>>>>> >     at
>>>>> org.apache.flink.streaming.runtime.tasks.SourceStreamTask$SourceOutput.collect(SourceStreamTask.java:97)
>>>>> >     at
>>>>> org.apache.flink.streaming.api.operators.StreamSource$NonTimestampContext.collect(StreamSource.java:92)
>>>>> >     at
>>>>> org.apache.flink.streaming.connectors.kafka.internals.LegacyFetcher$SimpleConsumerThread.run(LegacyFetcher.java:450)
>>>>> >
>>>>> >
>>>>> >
>>>>> >
>>>>> > 2015-12-02 10:45 GMT+01:00 Robert Metzger <rm...@apache.org>:
>>>>> > Its good news that the issue has been resolved.
>>>>> >
>>>>> > Regarding the OOM, did you start Flink in the streaming mode?
>>>>> >
>>>>> > On Wed, Dec 2, 2015 at 10:18 AM, Vieru, Mihail <
>>>>> mihail.vieru@zalando.de> wrote:
>>>>> > Thank you, Robert! The issue with Kafka is now solved with the
>>>>> 0.10-SNAPSHOT dependency.
>>>>> >
>>>>> > We have run into an OutOfMemory exception though, which appears to
>>>>> be related to the state. As my colleague, Javier Lopez, mentioned in a
>>>>> previous thread, state handling is crucial for our use case. And as the
>>>>> jobs are intended to run for months, stability plays an important role in
>>>>> choosing a stream processing framework.
>>>>> >
>>>>> > 12/02/2015 10:03:53    Fast TumblingTimeWindows(5000) of Reduce at
>>>>> main(ItemPriceAvgPerOrder.java:108) -> Sink: Unnamed(1/1) switched to FAILED
>>>>> > java.lang.OutOfMemoryError: Java heap space
>>>>> >     at java.util.HashMap.resize(HashMap.java:703)
>>>>> >     at java.util.HashMap.putVal(HashMap.java:662)
>>>>> >     at java.util.HashMap.put(HashMap.java:611)
>>>>> >     at
>>>>> org.apache.flink.runtime.state.AbstractHeapKvState.update(AbstractHeapKvState.java:98)
>>>>> >     at
>>>>> de.zalando.saiki.gerakanflink.ItemPriceAvgPerOrder$3.reduce(ItemPriceAvgPerOrder.java:121)
>>>>> >     at
>>>>> de.zalando.saiki.gerakanflink.ItemPriceAvgPerOrder$3.reduce(ItemPriceAvgPerOrder.java:108)
>>>>> >     at
>>>>> org.apache.flink.streaming.runtime.operators.windowing.KeyMap.putOrAggregate(KeyMap.java:196)
>>>>> >     at
>>>>> org.apache.flink.streaming.runtime.operators.windowing.AggregatingKeyedTimePanes.addElementToLatestPane(AggregatingKeyedTimePanes.java:50)
>>>>> >     at
>>>>> org.apache.flink.streaming.runtime.operators.windowing.AbstractAlignedProcessingTimeWindowOperator.processElement(AbstractAlignedProcessingTimeWindowOperator.java:210)
>>>>> >     at
>>>>> org.apache.flink.streaming.runtime.io.StreamInputProcessor.processInput(StreamInputProcessor.java:166)
>>>>> >     at
>>>>> org.apache.flink.streaming.runtime.tasks.OneInputStreamTask.run(OneInputStreamTask.java:63)
>>>>> >     at
>>>>> org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:218)
>>>>> >     at org.apache.flink.runtime.taskmanager.Task.run(Task.java:584)
>>>>> >     at java.lang.Thread.run(Thread.java:745)
>>>>> >
>>>>> >
>>>>> >
>>>>> >
>>>>> > 2015-12-01 17:42 GMT+01:00 Maximilian Michels <mx...@apache.org>:
>>>>> > Thanks! I've linked the issue in JIRA.
>>>>> >
>>>>> > On Tue, Dec 1, 2015 at 5:39 PM, Robert Metzger <rm...@apache.org>
>>>>> wrote:
>>>>> > > I think its this one
>>>>> https://issues.apache.org/jira/browse/KAFKA-824
>>>>> > >
>>>>> > > On Tue, Dec 1, 2015 at 5:37 PM, Maximilian Michels <mx...@apache.org>
>>>>> wrote:
>>>>> > >>
>>>>> > >> I know this has been fixed already but, out of curiosity, could
>>>>> you
>>>>> > >> point me to the Kafka JIRA issue for this
>>>>> > >> bug? From the Flink issue it looks like this is a Zookeeper
>>>>> version
>>>>> > >> mismatch.
>>>>> > >>
>>>>> > >> On Tue, Dec 1, 2015 at 5:16 PM, Robert Metzger <
>>>>> rmetzger@apache.org>
>>>>> > >> wrote:
>>>>> > >> > Hi Gyula,
>>>>> > >> >
>>>>> > >> > no, I didn't ;) We still deploy 0.10-SNAPSHOT versions from the
>>>>> > >> > "release-0.10" branch to Apache's maven snapshot repository.
>>>>> > >> >
>>>>> > >> >
>>>>> > >> > I don't think Mihail's code will run when he's compiling it
>>>>> against
>>>>> > >> > 1.0-SNAPSHOT.
>>>>> > >> >
>>>>> > >> >
>>>>> > >> > On Tue, Dec 1, 2015 at 5:13 PM, Gyula Fóra <
>>>>> gyula.fora@gmail.com> wrote:
>>>>> > >> >>
>>>>> > >> >> Hi,
>>>>> > >> >>
>>>>> > >> >> I think Robert meant to write setting the connector dependency
>>>>> to
>>>>> > >> >> 1.0-SNAPSHOT.
>>>>> > >> >>
>>>>> > >> >> Cheers,
>>>>> > >> >> Gyula
>>>>> > >> >>
>>>>> > >> >> Robert Metzger <rm...@apache.org> ezt írta (időpont: 2015.
>>>>> dec. 1.,
>>>>> > >> >> K,
>>>>> > >> >> 17:10):
>>>>> > >> >>>
>>>>> > >> >>> Hi Mihail,
>>>>> > >> >>>
>>>>> > >> >>> the issue is actually a bug in Kafka. We have a JIRA in Flink
>>>>> for this
>>>>> > >> >>> as
>>>>> > >> >>> well: https://issues.apache.org/jira/browse/FLINK-3067
>>>>> > >> >>>
>>>>> > >> >>> Sadly, we haven't released a fix for it yet. Flink 0.10.2
>>>>> will contain
>>>>> > >> >>> a
>>>>> > >> >>> fix.
>>>>> > >> >>>
>>>>> > >> >>> Since the kafka connector is not contained in the flink
>>>>> binary, you
>>>>> > >> >>> can
>>>>> > >> >>> just set the version in your maven pom file to 0.10-SNAPSHOT.
>>>>> Maven
>>>>> > >> >>> will
>>>>> > >> >>> then download the code planned for the 0.10-SNAPSHOT release.
>>>>> > >> >>>
>>>>> > >> >>> On Tue, Dec 1, 2015 at 4:54 PM, Vieru, Mihail
>>>>> > >> >>> <mi...@zalando.de>
>>>>> > >> >>> wrote:
>>>>> > >> >>>>
>>>>> > >> >>>> Hi,
>>>>> > >> >>>>
>>>>> > >> >>>> we get the following NullPointerException after ~50 minutes
>>>>> when
>>>>> > >> >>>> running
>>>>> > >> >>>> a streaming job with windowing and state that reads data
>>>>> from Kafka
>>>>> > >> >>>> and
>>>>> > >> >>>> writes the result to local FS.
>>>>> > >> >>>> There are around 170 million messages to be processed, Flink
>>>>> 0.10.1
>>>>> > >> >>>> stops at ~8 million.
>>>>> > >> >>>> Flink runs locally, started with the
>>>>> "start-cluster-streaming.sh"
>>>>> > >> >>>> script.
>>>>> > >> >>>>
>>>>> > >> >>>> 12/01/2015 15:06:24    Job execution switched to status
>>>>> RUNNING.
>>>>> > >> >>>> 12/01/2015 15:06:24    Source: Custom Source -> Map ->
>>>>> Map(1/1)
>>>>> > >> >>>> switched
>>>>> > >> >>>> to SCHEDULED
>>>>> > >> >>>> 12/01/2015 15:06:24    Source: Custom Source -> Map ->
>>>>> Map(1/1)
>>>>> > >> >>>> switched
>>>>> > >> >>>> to DEPLOYING
>>>>> > >> >>>> 12/01/2015 15:06:24    Fast TumblingTimeWindows(5000) of
>>>>> Reduce at
>>>>> > >> >>>> main(ItemPriceAvgPerOrder.java:108) -> Sink: Unnamed(1/1)
>>>>> switched to
>>>>> > >> >>>> SCHEDULED
>>>>> > >> >>>> 12/01/2015 15:06:24    Fast TumblingTimeWindows(5000) of
>>>>> Reduce at
>>>>> > >> >>>> main(ItemPriceAvgPerOrder.java:108) -> Sink: Unnamed(1/1)
>>>>> switched to
>>>>> > >> >>>> DEPLOYING
>>>>> > >> >>>> 12/01/2015 15:06:24    Source: Custom Source -> Map ->
>>>>> Map(1/1)
>>>>> > >> >>>> switched
>>>>> > >> >>>> to RUNNING
>>>>> > >> >>>> 12/01/2015 15:06:24    Fast TumblingTimeWindows(5000) of
>>>>> Reduce at
>>>>> > >> >>>> main(ItemPriceAvgPerOrder.java:108) -> Sink: Unnamed(1/1)
>>>>> switched to
>>>>> > >> >>>> RUNNING
>>>>> > >> >>>> 12/01/2015 15:56:08    Fast TumblingTimeWindows(5000) of
>>>>> Reduce at
>>>>> > >> >>>> main(ItemPriceAvgPerOrder.java:108) -> Sink: Unnamed(1/1)
>>>>> switched to
>>>>> > >> >>>> CANCELED
>>>>> > >> >>>> 12/01/2015 15:56:08    Source: Custom Source -> Map ->
>>>>> Map(1/1)
>>>>> > >> >>>> switched
>>>>> > >> >>>> to FAILED
>>>>> > >> >>>> java.lang.Exception
>>>>> > >> >>>>     at
>>>>> > >> >>>>
>>>>> > >> >>>>
>>>>> org.apache.flink.streaming.connectors.kafka.internals.LegacyFetcher.run(LegacyFetcher.java:242)
>>>>> > >> >>>>     at
>>>>> > >> >>>>
>>>>> > >> >>>>
>>>>> org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer.run(FlinkKafkaConsumer.java:397)
>>>>> > >> >>>>     at
>>>>> > >> >>>>
>>>>> > >> >>>>
>>>>> org.apache.flink.streaming.api.operators.StreamSource.run(StreamSource.java:58)
>>>>> > >> >>>>     at
>>>>> > >> >>>>
>>>>> > >> >>>>
>>>>> org.apache.flink.streaming.runtime.tasks.SourceStreamTask.run(SourceStreamTask.java:55)
>>>>> > >> >>>>     at
>>>>> > >> >>>>
>>>>> > >> >>>>
>>>>> org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:218)
>>>>> > >> >>>>     at
>>>>> org.apache.flink.runtime.taskmanager.Task.run(Task.java:584)
>>>>> > >> >>>>     at java.lang.Thread.run(Thread.java:745)
>>>>> > >> >>>> Caused by: java.lang.NullPointerException
>>>>> > >> >>>>     at
>>>>> > >> >>>>
>>>>> > >> >>>>
>>>>> org.I0Itec.zkclient.ZkConnection.writeDataReturnStat(ZkConnection.java:115)
>>>>> > >> >>>>     at
>>>>> org.I0Itec.zkclient.ZkClient$10.call(ZkClient.java:817)
>>>>> > >> >>>>     at
>>>>> > >> >>>>
>>>>> org.I0Itec.zkclient.ZkClient.retryUntilConnected(ZkClient.java:675)
>>>>> > >> >>>>     at
>>>>> > >> >>>>
>>>>> org.I0Itec.zkclient.ZkClient.writeDataReturnStat(ZkClient.java:813)
>>>>> > >> >>>>     at
>>>>> org.I0Itec.zkclient.ZkClient.writeData(ZkClient.java:808)
>>>>> > >> >>>>     at
>>>>> org.I0Itec.zkclient.ZkClient.writeData(ZkClient.java:777)
>>>>> > >> >>>>     at
>>>>> kafka.utils.ZkUtils$.updatePersistentPath(ZkUtils.scala:332)
>>>>> > >> >>>>     at
>>>>> kafka.utils.ZkUtils.updatePersistentPath(ZkUtils.scala)
>>>>> > >> >>>>     at
>>>>> > >> >>>>
>>>>> > >> >>>>
>>>>> org.apache.flink.streaming.connectors.kafka.internals.ZookeeperOffsetHandler.setOffsetInZooKeeper(ZookeeperOffsetHandler.java:112)
>>>>> > >> >>>>     at
>>>>> > >> >>>>
>>>>> > >> >>>>
>>>>> org.apache.flink.streaming.connectors.kafka.internals.ZookeeperOffsetHandler.commit(ZookeeperOffsetHandler.java:80)
>>>>> > >> >>>>     at
>>>>> > >> >>>>
>>>>> > >> >>>>
>>>>> org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer$PeriodicOffsetCommitter.run(FlinkKafkaConsumer.java:632)
>>>>> > >> >>>>
>>>>> > >> >>>>
>>>>> > >> >>>> Any ideas on what could cause this behaviour?
>>>>> > >> >>>>
>>>>> > >> >>>> Best,
>>>>> > >> >>>> Mihail
>>>>> > >> >>>
>>>>> > >> >>>
>>>>> > >> >
>>>>> > >
>>>>> > >
>>>>> >
>>>>> >
>>>>> >
>>>>>
>>>>>
>>>>
>>
>

Re: NPE with Flink Streaming from Kafka

Posted by "Vieru, Mihail" <mi...@zalando.de>.
Hi Gyula, Hi Stephan,

thank you for your replies.

We need a state which grows indefinitely for the following use case. An
event is created when a customer places an order. Another event is created
when the order is sent. These events typically occur within days. We need
to catch the cases when the said events occur over a specified time period
to raise an alarm.

So having a window of a couple of days is not feasible. Thus we need the
state.

I believe having a different state backend would circumvent the OOM issue.
We were thinking of Redis for performance reasons. MySQL might do as well,
if it doesn't slow down the processing too much.

Are there limitations for SqlStateBackend when working with state only?
When would the window state limitation occur?

Cheers,
Mihail


2015-12-02 13:38 GMT+01:00 Stephan Ewen <se...@apache.org>:

> Mihail!
>
> The Flink windows are currently in-memory only. There are plans to relax
> that, but for the time being, having enough memory in the cluster is
> important.
>
> @Gyula: I think window state is currently also limited when using the
> SqlStateBackend, by the size of a row in the database (because windows are
> not key/value state currently)
>
>
> Here are some simple rules-of-thumb to work with:
>
> 1) For windows, the number of expected keys can be without bound. It is
> important to have a rough upper bound for the number of "active keys at a
> certain time". For example, if you have your time windows (let's say by 10
> minutes or so), it only matters how many keys you have within each 10
> minute interval. Those define how much memory you need.
>
> 2) If you work with the "OperatorState" abstraction, then you need to
> think about cleanup a bit. The OperatorState keeps state currently for as
> long until you set the state for the key to "null". This manual state is
> explicitly designed to allow you to keep state across windows and across
> very long time. On the flip side, you need to manage the amount of state
> you store, by releasing state for keys.
>
> 3) If a certain key space grows infinite, you should "scope the state by
> time". A pragmatic solution for that is to define a session window:
>   - The session length defines after what inactivity the state is cleaned
> (let's say 1h session length or so)
>   - The trigger implements this session (there are a few mails on this
> list already that explain how to do this) and take care of evaluating on
> every element.
>   - A count(1) evictor makes sure only one element is ever stored
>
> Greetings,
> Stephan
>
>
> On Wed, Dec 2, 2015 at 11:37 AM, Gyula Fóra <gy...@apache.org> wrote:
>
>> Hi,
>>
>> I am working on a use case that involves storing state for billions of
>> keys. For this we use a MySql state backend that will write each key-value
>> state to MySql server so it will only hold a limited set of key-value pairs
>> on heap while maintaining the processing guarantees.
>>
>> This will keep our streaming job from running out of memory as most of
>> the state is off heap. I am not sure if this is relevant to your use case
>> but if the state size grows indefinitely you might want to give it a try.
>>
>> I will write a detailed guide in some days but if you want to get started
>> check this one out:
>>
>> https://docs.google.com/document/d/1xx3J88ZR_kuYYHAil3HD3YSBLqBTKV4Pu1SLwhiRiGs/edit?usp=sharing
>>
>> There are some pending improvements that I will commit in the next days
>> that will increase the performance of the MySql adapter
>>
>> Let me know if you are interested in this!
>>
>> Cheers,
>> Gyula
>>
>>
>> Vieru, Mihail <mi...@zalando.de> ezt írta (időpont: 2015. dec.
>> 2., Sze, 11:26):
>>
>>> Hi Aljoscha,
>>>
>>> we have no upper bound for the number of expected keys. The max size for
>>> an element is 1 KB.
>>>
>>> There are 2 Maps, a KeyBy, a timeWindow, a Reduce and a writeAsText
>>> operators in the job. In the first Map we parse the contained JSON object
>>> in each element and forward it as a Flink Tuple. In the Reduce we update
>>> the state for each key. That's about it.
>>>
>>> Best,
>>> Mihail
>>>
>>>
>>> 2015-12-02 11:09 GMT+01:00 Aljoscha Krettek <al...@apache.org>:
>>>
>>>> Hi Mihail,
>>>> could you please give some information about the number of keys that
>>>> you are expecting in the data and how big the elements are that you are
>>>> processing in the window.
>>>>
>>>> Also, are there any other operations that could be taxing on Memory. I
>>>> think the different exception you see for 500MB mem size is just because
>>>> Java notices that it ran out of memory at a different part in the program.
>>>>
>>>> Cheers,
>>>> Aljoscha
>>>> > On 02 Dec 2015, at 10:57, Vieru, Mihail <mi...@zalando.de>
>>>> wrote:
>>>> >
>>>> > Yes, with the "start-cluster-streaming.sh" script.
>>>> > If the TaskManager gets 5GB of heap it manages to process ~100
>>>> million messages and then throws the above OOM.
>>>> > If it gets only 500MB it manages to process ~8 million and a somewhat
>>>> misleading exception is thrown:
>>>> >
>>>> > 12/01/2015 19:14:07    Source: Custom Source -> Map -> Map(1/1)
>>>> switched to FAILED
>>>> > java.lang.Exception: Java heap space
>>>> >     at
>>>> org.apache.flink.streaming.connectors.kafka.internals.LegacyFetcher.run(LegacyFetcher.java:242)
>>>> >     at
>>>> org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer.run(FlinkKafkaConsumer.java:399)
>>>> >     at
>>>> org.apache.flink.streaming.api.operators.StreamSource.run(StreamSource.java:58)
>>>> >     at
>>>> org.apache.flink.streaming.runtime.tasks.SourceStreamTask.run(SourceStreamTask.java:55)
>>>> >     at
>>>> org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:218)
>>>> >     at org.apache.flink.runtime.taskmanager.Task.run(Task.java:584)
>>>> >     at java.lang.Thread.run(Thread.java:745)
>>>> > Caused by: java.lang.OutOfMemoryError: Java heap space
>>>> >     at org.json.simple.parser.Yylex.<init>(Yylex.java:231)
>>>> >     at org.json.simple.parser.JSONParser.<init>(JSONParser.java:34)
>>>> >     at
>>>> de.zalando.saiki.gerakanflink.ItemPriceAvgPerOrder$1.map(ItemPriceAvgPerOrder.java:70)
>>>> >     at
>>>> de.zalando.saiki.gerakanflink.ItemPriceAvgPerOrder$1.map(ItemPriceAvgPerOrder.java:65)
>>>> >     at
>>>> org.apache.flink.streaming.api.operators.StreamMap.processElement(StreamMap.java:37)
>>>> >     at
>>>> org.apache.flink.streaming.runtime.tasks.OperatorChain$CopyingChainingOutput.collect(OperatorChain.java:316)
>>>> >     at
>>>> org.apache.flink.streaming.runtime.tasks.OperatorChain$CopyingChainingOutput.collect(OperatorChain.java:300)
>>>> >     at
>>>> org.apache.flink.streaming.runtime.io.CollectorWrapper.collect(CollectorWrapper.java:48)
>>>> >     at
>>>> org.apache.flink.streaming.runtime.io.CollectorWrapper.collect(CollectorWrapper.java:29)
>>>> >     at
>>>> org.apache.flink.streaming.runtime.tasks.SourceStreamTask$SourceOutput.collect(SourceStreamTask.java:97)
>>>> >     at
>>>> org.apache.flink.streaming.api.operators.StreamSource$NonTimestampContext.collect(StreamSource.java:92)
>>>> >     at
>>>> org.apache.flink.streaming.connectors.kafka.internals.LegacyFetcher$SimpleConsumerThread.run(LegacyFetcher.java:450)
>>>> >
>>>> >
>>>> >
>>>> >
>>>> > 2015-12-02 10:45 GMT+01:00 Robert Metzger <rm...@apache.org>:
>>>> > Its good news that the issue has been resolved.
>>>> >
>>>> > Regarding the OOM, did you start Flink in the streaming mode?
>>>> >
>>>> > On Wed, Dec 2, 2015 at 10:18 AM, Vieru, Mihail <
>>>> mihail.vieru@zalando.de> wrote:
>>>> > Thank you, Robert! The issue with Kafka is now solved with the
>>>> 0.10-SNAPSHOT dependency.
>>>> >
>>>> > We have run into an OutOfMemory exception though, which appears to be
>>>> related to the state. As my colleague, Javier Lopez, mentioned in a
>>>> previous thread, state handling is crucial for our use case. And as the
>>>> jobs are intended to run for months, stability plays an important role in
>>>> choosing a stream processing framework.
>>>> >
>>>> > 12/02/2015 10:03:53    Fast TumblingTimeWindows(5000) of Reduce at
>>>> main(ItemPriceAvgPerOrder.java:108) -> Sink: Unnamed(1/1) switched to FAILED
>>>> > java.lang.OutOfMemoryError: Java heap space
>>>> >     at java.util.HashMap.resize(HashMap.java:703)
>>>> >     at java.util.HashMap.putVal(HashMap.java:662)
>>>> >     at java.util.HashMap.put(HashMap.java:611)
>>>> >     at
>>>> org.apache.flink.runtime.state.AbstractHeapKvState.update(AbstractHeapKvState.java:98)
>>>> >     at
>>>> de.zalando.saiki.gerakanflink.ItemPriceAvgPerOrder$3.reduce(ItemPriceAvgPerOrder.java:121)
>>>> >     at
>>>> de.zalando.saiki.gerakanflink.ItemPriceAvgPerOrder$3.reduce(ItemPriceAvgPerOrder.java:108)
>>>> >     at
>>>> org.apache.flink.streaming.runtime.operators.windowing.KeyMap.putOrAggregate(KeyMap.java:196)
>>>> >     at
>>>> org.apache.flink.streaming.runtime.operators.windowing.AggregatingKeyedTimePanes.addElementToLatestPane(AggregatingKeyedTimePanes.java:50)
>>>> >     at
>>>> org.apache.flink.streaming.runtime.operators.windowing.AbstractAlignedProcessingTimeWindowOperator.processElement(AbstractAlignedProcessingTimeWindowOperator.java:210)
>>>> >     at
>>>> org.apache.flink.streaming.runtime.io.StreamInputProcessor.processInput(StreamInputProcessor.java:166)
>>>> >     at
>>>> org.apache.flink.streaming.runtime.tasks.OneInputStreamTask.run(OneInputStreamTask.java:63)
>>>> >     at
>>>> org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:218)
>>>> >     at org.apache.flink.runtime.taskmanager.Task.run(Task.java:584)
>>>> >     at java.lang.Thread.run(Thread.java:745)
>>>> >
>>>> >
>>>> >
>>>> >
>>>> > 2015-12-01 17:42 GMT+01:00 Maximilian Michels <mx...@apache.org>:
>>>> > Thanks! I've linked the issue in JIRA.
>>>> >
>>>> > On Tue, Dec 1, 2015 at 5:39 PM, Robert Metzger <rm...@apache.org>
>>>> wrote:
>>>> > > I think its this one
>>>> https://issues.apache.org/jira/browse/KAFKA-824
>>>> > >
>>>> > > On Tue, Dec 1, 2015 at 5:37 PM, Maximilian Michels <mx...@apache.org>
>>>> wrote:
>>>> > >>
>>>> > >> I know this has been fixed already but, out of curiosity, could you
>>>> > >> point me to the Kafka JIRA issue for this
>>>> > >> bug? From the Flink issue it looks like this is a Zookeeper version
>>>> > >> mismatch.
>>>> > >>
>>>> > >> On Tue, Dec 1, 2015 at 5:16 PM, Robert Metzger <
>>>> rmetzger@apache.org>
>>>> > >> wrote:
>>>> > >> > Hi Gyula,
>>>> > >> >
>>>> > >> > no, I didn't ;) We still deploy 0.10-SNAPSHOT versions from the
>>>> > >> > "release-0.10" branch to Apache's maven snapshot repository.
>>>> > >> >
>>>> > >> >
>>>> > >> > I don't think Mihail's code will run when he's compiling it
>>>> against
>>>> > >> > 1.0-SNAPSHOT.
>>>> > >> >
>>>> > >> >
>>>> > >> > On Tue, Dec 1, 2015 at 5:13 PM, Gyula Fóra <gy...@gmail.com>
>>>> wrote:
>>>> > >> >>
>>>> > >> >> Hi,
>>>> > >> >>
>>>> > >> >> I think Robert meant to write setting the connector dependency
>>>> to
>>>> > >> >> 1.0-SNAPSHOT.
>>>> > >> >>
>>>> > >> >> Cheers,
>>>> > >> >> Gyula
>>>> > >> >>
>>>> > >> >> Robert Metzger <rm...@apache.org> ezt írta (időpont: 2015.
>>>> dec. 1.,
>>>> > >> >> K,
>>>> > >> >> 17:10):
>>>> > >> >>>
>>>> > >> >>> Hi Mihail,
>>>> > >> >>>
>>>> > >> >>> the issue is actually a bug in Kafka. We have a JIRA in Flink
>>>> for this
>>>> > >> >>> as
>>>> > >> >>> well: https://issues.apache.org/jira/browse/FLINK-3067
>>>> > >> >>>
>>>> > >> >>> Sadly, we haven't released a fix for it yet. Flink 0.10.2 will
>>>> contain
>>>> > >> >>> a
>>>> > >> >>> fix.
>>>> > >> >>>
>>>> > >> >>> Since the kafka connector is not contained in the flink
>>>> binary, you
>>>> > >> >>> can
>>>> > >> >>> just set the version in your maven pom file to 0.10-SNAPSHOT.
>>>> Maven
>>>> > >> >>> will
>>>> > >> >>> then download the code planned for the 0.10-SNAPSHOT release.
>>>> > >> >>>
>>>> > >> >>> On Tue, Dec 1, 2015 at 4:54 PM, Vieru, Mihail
>>>> > >> >>> <mi...@zalando.de>
>>>> > >> >>> wrote:
>>>> > >> >>>>
>>>> > >> >>>> Hi,
>>>> > >> >>>>
>>>> > >> >>>> we get the following NullPointerException after ~50 minutes
>>>> when
>>>> > >> >>>> running
>>>> > >> >>>> a streaming job with windowing and state that reads data from
>>>> Kafka
>>>> > >> >>>> and
>>>> > >> >>>> writes the result to local FS.
>>>> > >> >>>> There are around 170 million messages to be processed, Flink
>>>> 0.10.1
>>>> > >> >>>> stops at ~8 million.
>>>> > >> >>>> Flink runs locally, started with the
>>>> "start-cluster-streaming.sh"
>>>> > >> >>>> script.
>>>> > >> >>>>
>>>> > >> >>>> 12/01/2015 15:06:24    Job execution switched to status
>>>> RUNNING.
>>>> > >> >>>> 12/01/2015 15:06:24    Source: Custom Source -> Map ->
>>>> Map(1/1)
>>>> > >> >>>> switched
>>>> > >> >>>> to SCHEDULED
>>>> > >> >>>> 12/01/2015 15:06:24    Source: Custom Source -> Map ->
>>>> Map(1/1)
>>>> > >> >>>> switched
>>>> > >> >>>> to DEPLOYING
>>>> > >> >>>> 12/01/2015 15:06:24    Fast TumblingTimeWindows(5000) of
>>>> Reduce at
>>>> > >> >>>> main(ItemPriceAvgPerOrder.java:108) -> Sink: Unnamed(1/1)
>>>> switched to
>>>> > >> >>>> SCHEDULED
>>>> > >> >>>> 12/01/2015 15:06:24    Fast TumblingTimeWindows(5000) of
>>>> Reduce at
>>>> > >> >>>> main(ItemPriceAvgPerOrder.java:108) -> Sink: Unnamed(1/1)
>>>> switched to
>>>> > >> >>>> DEPLOYING
>>>> > >> >>>> 12/01/2015 15:06:24    Source: Custom Source -> Map ->
>>>> Map(1/1)
>>>> > >> >>>> switched
>>>> > >> >>>> to RUNNING
>>>> > >> >>>> 12/01/2015 15:06:24    Fast TumblingTimeWindows(5000) of
>>>> Reduce at
>>>> > >> >>>> main(ItemPriceAvgPerOrder.java:108) -> Sink: Unnamed(1/1)
>>>> switched to
>>>> > >> >>>> RUNNING
>>>> > >> >>>> 12/01/2015 15:56:08    Fast TumblingTimeWindows(5000) of
>>>> Reduce at
>>>> > >> >>>> main(ItemPriceAvgPerOrder.java:108) -> Sink: Unnamed(1/1)
>>>> switched to
>>>> > >> >>>> CANCELED
>>>> > >> >>>> 12/01/2015 15:56:08    Source: Custom Source -> Map ->
>>>> Map(1/1)
>>>> > >> >>>> switched
>>>> > >> >>>> to FAILED
>>>> > >> >>>> java.lang.Exception
>>>> > >> >>>>     at
>>>> > >> >>>>
>>>> > >> >>>>
>>>> org.apache.flink.streaming.connectors.kafka.internals.LegacyFetcher.run(LegacyFetcher.java:242)
>>>> > >> >>>>     at
>>>> > >> >>>>
>>>> > >> >>>>
>>>> org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer.run(FlinkKafkaConsumer.java:397)
>>>> > >> >>>>     at
>>>> > >> >>>>
>>>> > >> >>>>
>>>> org.apache.flink.streaming.api.operators.StreamSource.run(StreamSource.java:58)
>>>> > >> >>>>     at
>>>> > >> >>>>
>>>> > >> >>>>
>>>> org.apache.flink.streaming.runtime.tasks.SourceStreamTask.run(SourceStreamTask.java:55)
>>>> > >> >>>>     at
>>>> > >> >>>>
>>>> > >> >>>>
>>>> org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:218)
>>>> > >> >>>>     at
>>>> org.apache.flink.runtime.taskmanager.Task.run(Task.java:584)
>>>> > >> >>>>     at java.lang.Thread.run(Thread.java:745)
>>>> > >> >>>> Caused by: java.lang.NullPointerException
>>>> > >> >>>>     at
>>>> > >> >>>>
>>>> > >> >>>>
>>>> org.I0Itec.zkclient.ZkConnection.writeDataReturnStat(ZkConnection.java:115)
>>>> > >> >>>>     at org.I0Itec.zkclient.ZkClient$10.call(ZkClient.java:817)
>>>> > >> >>>>     at
>>>> > >> >>>>
>>>> org.I0Itec.zkclient.ZkClient.retryUntilConnected(ZkClient.java:675)
>>>> > >> >>>>     at
>>>> > >> >>>>
>>>> org.I0Itec.zkclient.ZkClient.writeDataReturnStat(ZkClient.java:813)
>>>> > >> >>>>     at
>>>> org.I0Itec.zkclient.ZkClient.writeData(ZkClient.java:808)
>>>> > >> >>>>     at
>>>> org.I0Itec.zkclient.ZkClient.writeData(ZkClient.java:777)
>>>> > >> >>>>     at
>>>> kafka.utils.ZkUtils$.updatePersistentPath(ZkUtils.scala:332)
>>>> > >> >>>>     at kafka.utils.ZkUtils.updatePersistentPath(ZkUtils.scala)
>>>> > >> >>>>     at
>>>> > >> >>>>
>>>> > >> >>>>
>>>> org.apache.flink.streaming.connectors.kafka.internals.ZookeeperOffsetHandler.setOffsetInZooKeeper(ZookeeperOffsetHandler.java:112)
>>>> > >> >>>>     at
>>>> > >> >>>>
>>>> > >> >>>>
>>>> org.apache.flink.streaming.connectors.kafka.internals.ZookeeperOffsetHandler.commit(ZookeeperOffsetHandler.java:80)
>>>> > >> >>>>     at
>>>> > >> >>>>
>>>> > >> >>>>
>>>> org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer$PeriodicOffsetCommitter.run(FlinkKafkaConsumer.java:632)
>>>> > >> >>>>
>>>> > >> >>>>
>>>> > >> >>>> Any ideas on what could cause this behaviour?
>>>> > >> >>>>
>>>> > >> >>>> Best,
>>>> > >> >>>> Mihail
>>>> > >> >>>
>>>> > >> >>>
>>>> > >> >
>>>> > >
>>>> > >
>>>> >
>>>> >
>>>> >
>>>>
>>>>
>>>
>

Re: NPE with Flink Streaming from Kafka

Posted by Stephan Ewen <se...@apache.org>.
Mihail!

The Flink windows are currently in-memory only. There are plans to relax
that, but for the time being, having enough memory in the cluster is
important.

@Gyula: I think window state is currently also limited when using the
SqlStateBackend, by the size of a row in the database (because windows are
not key/value state currently)


Here are some simple rules-of-thumb to work with:

1) For windows, the number of expected keys can be without bound. It is
important to have a rough upper bound for the number of "active keys at a
certain time". For example, if you have your time windows (let's say by 10
minutes or so), it only matters how many keys you have within each 10
minute interval. Those define how much memory you need.

2) If you work with the "OperatorState" abstraction, then you need to think
about cleanup a bit. The OperatorState keeps state currently for as long
until you set the state for the key to "null". This manual state is
explicitly designed to allow you to keep state across windows and across
very long time. On the flip side, you need to manage the amount of state
you store, by releasing state for keys.

3) If a certain key space grows infinite, you should "scope the state by
time". A pragmatic solution for that is to define a session window:
  - The session length defines after what inactivity the state is cleaned
(let's say 1h session length or so)
  - The trigger implements this session (there are a few mails on this list
already that explain how to do this) and take care of evaluating on every
element.
  - A count(1) evictor makes sure only one element is ever stored

Greetings,
Stephan


On Wed, Dec 2, 2015 at 11:37 AM, Gyula Fóra <gy...@apache.org> wrote:

> Hi,
>
> I am working on a use case that involves storing state for billions of
> keys. For this we use a MySql state backend that will write each key-value
> state to MySql server so it will only hold a limited set of key-value pairs
> on heap while maintaining the processing guarantees.
>
> This will keep our streaming job from running out of memory as most of the
> state is off heap. I am not sure if this is relevant to your use case but
> if the state size grows indefinitely you might want to give it a try.
>
> I will write a detailed guide in some days but if you want to get started
> check this one out:
>
> https://docs.google.com/document/d/1xx3J88ZR_kuYYHAil3HD3YSBLqBTKV4Pu1SLwhiRiGs/edit?usp=sharing
>
> There are some pending improvements that I will commit in the next days
> that will increase the performance of the MySql adapter
>
> Let me know if you are interested in this!
>
> Cheers,
> Gyula
>
>
> Vieru, Mihail <mi...@zalando.de> ezt írta (időpont: 2015. dec. 2.,
> Sze, 11:26):
>
>> Hi Aljoscha,
>>
>> we have no upper bound for the number of expected keys. The max size for
>> an element is 1 KB.
>>
>> There are 2 Maps, a KeyBy, a timeWindow, a Reduce and a writeAsText
>> operators in the job. In the first Map we parse the contained JSON object
>> in each element and forward it as a Flink Tuple. In the Reduce we update
>> the state for each key. That's about it.
>>
>> Best,
>> Mihail
>>
>>
>> 2015-12-02 11:09 GMT+01:00 Aljoscha Krettek <al...@apache.org>:
>>
>>> Hi Mihail,
>>> could you please give some information about the number of keys that you
>>> are expecting in the data and how big the elements are that you are
>>> processing in the window.
>>>
>>> Also, are there any other operations that could be taxing on Memory. I
>>> think the different exception you see for 500MB mem size is just because
>>> Java notices that it ran out of memory at a different part in the program.
>>>
>>> Cheers,
>>> Aljoscha
>>> > On 02 Dec 2015, at 10:57, Vieru, Mihail <mi...@zalando.de>
>>> wrote:
>>> >
>>> > Yes, with the "start-cluster-streaming.sh" script.
>>> > If the TaskManager gets 5GB of heap it manages to process ~100 million
>>> messages and then throws the above OOM.
>>> > If it gets only 500MB it manages to process ~8 million and a somewhat
>>> misleading exception is thrown:
>>> >
>>> > 12/01/2015 19:14:07    Source: Custom Source -> Map -> Map(1/1)
>>> switched to FAILED
>>> > java.lang.Exception: Java heap space
>>> >     at
>>> org.apache.flink.streaming.connectors.kafka.internals.LegacyFetcher.run(LegacyFetcher.java:242)
>>> >     at
>>> org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer.run(FlinkKafkaConsumer.java:399)
>>> >     at
>>> org.apache.flink.streaming.api.operators.StreamSource.run(StreamSource.java:58)
>>> >     at
>>> org.apache.flink.streaming.runtime.tasks.SourceStreamTask.run(SourceStreamTask.java:55)
>>> >     at
>>> org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:218)
>>> >     at org.apache.flink.runtime.taskmanager.Task.run(Task.java:584)
>>> >     at java.lang.Thread.run(Thread.java:745)
>>> > Caused by: java.lang.OutOfMemoryError: Java heap space
>>> >     at org.json.simple.parser.Yylex.<init>(Yylex.java:231)
>>> >     at org.json.simple.parser.JSONParser.<init>(JSONParser.java:34)
>>> >     at
>>> de.zalando.saiki.gerakanflink.ItemPriceAvgPerOrder$1.map(ItemPriceAvgPerOrder.java:70)
>>> >     at
>>> de.zalando.saiki.gerakanflink.ItemPriceAvgPerOrder$1.map(ItemPriceAvgPerOrder.java:65)
>>> >     at
>>> org.apache.flink.streaming.api.operators.StreamMap.processElement(StreamMap.java:37)
>>> >     at
>>> org.apache.flink.streaming.runtime.tasks.OperatorChain$CopyingChainingOutput.collect(OperatorChain.java:316)
>>> >     at
>>> org.apache.flink.streaming.runtime.tasks.OperatorChain$CopyingChainingOutput.collect(OperatorChain.java:300)
>>> >     at
>>> org.apache.flink.streaming.runtime.io.CollectorWrapper.collect(CollectorWrapper.java:48)
>>> >     at
>>> org.apache.flink.streaming.runtime.io.CollectorWrapper.collect(CollectorWrapper.java:29)
>>> >     at
>>> org.apache.flink.streaming.runtime.tasks.SourceStreamTask$SourceOutput.collect(SourceStreamTask.java:97)
>>> >     at
>>> org.apache.flink.streaming.api.operators.StreamSource$NonTimestampContext.collect(StreamSource.java:92)
>>> >     at
>>> org.apache.flink.streaming.connectors.kafka.internals.LegacyFetcher$SimpleConsumerThread.run(LegacyFetcher.java:450)
>>> >
>>> >
>>> >
>>> >
>>> > 2015-12-02 10:45 GMT+01:00 Robert Metzger <rm...@apache.org>:
>>> > Its good news that the issue has been resolved.
>>> >
>>> > Regarding the OOM, did you start Flink in the streaming mode?
>>> >
>>> > On Wed, Dec 2, 2015 at 10:18 AM, Vieru, Mihail <
>>> mihail.vieru@zalando.de> wrote:
>>> > Thank you, Robert! The issue with Kafka is now solved with the
>>> 0.10-SNAPSHOT dependency.
>>> >
>>> > We have run into an OutOfMemory exception though, which appears to be
>>> related to the state. As my colleague, Javier Lopez, mentioned in a
>>> previous thread, state handling is crucial for our use case. And as the
>>> jobs are intended to run for months, stability plays an important role in
>>> choosing a stream processing framework.
>>> >
>>> > 12/02/2015 10:03:53    Fast TumblingTimeWindows(5000) of Reduce at
>>> main(ItemPriceAvgPerOrder.java:108) -> Sink: Unnamed(1/1) switched to FAILED
>>> > java.lang.OutOfMemoryError: Java heap space
>>> >     at java.util.HashMap.resize(HashMap.java:703)
>>> >     at java.util.HashMap.putVal(HashMap.java:662)
>>> >     at java.util.HashMap.put(HashMap.java:611)
>>> >     at
>>> org.apache.flink.runtime.state.AbstractHeapKvState.update(AbstractHeapKvState.java:98)
>>> >     at
>>> de.zalando.saiki.gerakanflink.ItemPriceAvgPerOrder$3.reduce(ItemPriceAvgPerOrder.java:121)
>>> >     at
>>> de.zalando.saiki.gerakanflink.ItemPriceAvgPerOrder$3.reduce(ItemPriceAvgPerOrder.java:108)
>>> >     at
>>> org.apache.flink.streaming.runtime.operators.windowing.KeyMap.putOrAggregate(KeyMap.java:196)
>>> >     at
>>> org.apache.flink.streaming.runtime.operators.windowing.AggregatingKeyedTimePanes.addElementToLatestPane(AggregatingKeyedTimePanes.java:50)
>>> >     at
>>> org.apache.flink.streaming.runtime.operators.windowing.AbstractAlignedProcessingTimeWindowOperator.processElement(AbstractAlignedProcessingTimeWindowOperator.java:210)
>>> >     at
>>> org.apache.flink.streaming.runtime.io.StreamInputProcessor.processInput(StreamInputProcessor.java:166)
>>> >     at
>>> org.apache.flink.streaming.runtime.tasks.OneInputStreamTask.run(OneInputStreamTask.java:63)
>>> >     at
>>> org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:218)
>>> >     at org.apache.flink.runtime.taskmanager.Task.run(Task.java:584)
>>> >     at java.lang.Thread.run(Thread.java:745)
>>> >
>>> >
>>> >
>>> >
>>> > 2015-12-01 17:42 GMT+01:00 Maximilian Michels <mx...@apache.org>:
>>> > Thanks! I've linked the issue in JIRA.
>>> >
>>> > On Tue, Dec 1, 2015 at 5:39 PM, Robert Metzger <rm...@apache.org>
>>> wrote:
>>> > > I think its this one https://issues.apache.org/jira/browse/KAFKA-824
>>> > >
>>> > > On Tue, Dec 1, 2015 at 5:37 PM, Maximilian Michels <mx...@apache.org>
>>> wrote:
>>> > >>
>>> > >> I know this has been fixed already but, out of curiosity, could you
>>> > >> point me to the Kafka JIRA issue for this
>>> > >> bug? From the Flink issue it looks like this is a Zookeeper version
>>> > >> mismatch.
>>> > >>
>>> > >> On Tue, Dec 1, 2015 at 5:16 PM, Robert Metzger <rmetzger@apache.org
>>> >
>>> > >> wrote:
>>> > >> > Hi Gyula,
>>> > >> >
>>> > >> > no, I didn't ;) We still deploy 0.10-SNAPSHOT versions from the
>>> > >> > "release-0.10" branch to Apache's maven snapshot repository.
>>> > >> >
>>> > >> >
>>> > >> > I don't think Mihail's code will run when he's compiling it
>>> against
>>> > >> > 1.0-SNAPSHOT.
>>> > >> >
>>> > >> >
>>> > >> > On Tue, Dec 1, 2015 at 5:13 PM, Gyula Fóra <gy...@gmail.com>
>>> wrote:
>>> > >> >>
>>> > >> >> Hi,
>>> > >> >>
>>> > >> >> I think Robert meant to write setting the connector dependency to
>>> > >> >> 1.0-SNAPSHOT.
>>> > >> >>
>>> > >> >> Cheers,
>>> > >> >> Gyula
>>> > >> >>
>>> > >> >> Robert Metzger <rm...@apache.org> ezt írta (időpont: 2015.
>>> dec. 1.,
>>> > >> >> K,
>>> > >> >> 17:10):
>>> > >> >>>
>>> > >> >>> Hi Mihail,
>>> > >> >>>
>>> > >> >>> the issue is actually a bug in Kafka. We have a JIRA in Flink
>>> for this
>>> > >> >>> as
>>> > >> >>> well: https://issues.apache.org/jira/browse/FLINK-3067
>>> > >> >>>
>>> > >> >>> Sadly, we haven't released a fix for it yet. Flink 0.10.2 will
>>> contain
>>> > >> >>> a
>>> > >> >>> fix.
>>> > >> >>>
>>> > >> >>> Since the kafka connector is not contained in the flink binary,
>>> you
>>> > >> >>> can
>>> > >> >>> just set the version in your maven pom file to 0.10-SNAPSHOT.
>>> Maven
>>> > >> >>> will
>>> > >> >>> then download the code planned for the 0.10-SNAPSHOT release.
>>> > >> >>>
>>> > >> >>> On Tue, Dec 1, 2015 at 4:54 PM, Vieru, Mihail
>>> > >> >>> <mi...@zalando.de>
>>> > >> >>> wrote:
>>> > >> >>>>
>>> > >> >>>> Hi,
>>> > >> >>>>
>>> > >> >>>> we get the following NullPointerException after ~50 minutes
>>> when
>>> > >> >>>> running
>>> > >> >>>> a streaming job with windowing and state that reads data from
>>> Kafka
>>> > >> >>>> and
>>> > >> >>>> writes the result to local FS.
>>> > >> >>>> There are around 170 million messages to be processed, Flink
>>> 0.10.1
>>> > >> >>>> stops at ~8 million.
>>> > >> >>>> Flink runs locally, started with the
>>> "start-cluster-streaming.sh"
>>> > >> >>>> script.
>>> > >> >>>>
>>> > >> >>>> 12/01/2015 15:06:24    Job execution switched to status
>>> RUNNING.
>>> > >> >>>> 12/01/2015 15:06:24    Source: Custom Source -> Map -> Map(1/1)
>>> > >> >>>> switched
>>> > >> >>>> to SCHEDULED
>>> > >> >>>> 12/01/2015 15:06:24    Source: Custom Source -> Map -> Map(1/1)
>>> > >> >>>> switched
>>> > >> >>>> to DEPLOYING
>>> > >> >>>> 12/01/2015 15:06:24    Fast TumblingTimeWindows(5000) of
>>> Reduce at
>>> > >> >>>> main(ItemPriceAvgPerOrder.java:108) -> Sink: Unnamed(1/1)
>>> switched to
>>> > >> >>>> SCHEDULED
>>> > >> >>>> 12/01/2015 15:06:24    Fast TumblingTimeWindows(5000) of
>>> Reduce at
>>> > >> >>>> main(ItemPriceAvgPerOrder.java:108) -> Sink: Unnamed(1/1)
>>> switched to
>>> > >> >>>> DEPLOYING
>>> > >> >>>> 12/01/2015 15:06:24    Source: Custom Source -> Map -> Map(1/1)
>>> > >> >>>> switched
>>> > >> >>>> to RUNNING
>>> > >> >>>> 12/01/2015 15:06:24    Fast TumblingTimeWindows(5000) of
>>> Reduce at
>>> > >> >>>> main(ItemPriceAvgPerOrder.java:108) -> Sink: Unnamed(1/1)
>>> switched to
>>> > >> >>>> RUNNING
>>> > >> >>>> 12/01/2015 15:56:08    Fast TumblingTimeWindows(5000) of
>>> Reduce at
>>> > >> >>>> main(ItemPriceAvgPerOrder.java:108) -> Sink: Unnamed(1/1)
>>> switched to
>>> > >> >>>> CANCELED
>>> > >> >>>> 12/01/2015 15:56:08    Source: Custom Source -> Map -> Map(1/1)
>>> > >> >>>> switched
>>> > >> >>>> to FAILED
>>> > >> >>>> java.lang.Exception
>>> > >> >>>>     at
>>> > >> >>>>
>>> > >> >>>>
>>> org.apache.flink.streaming.connectors.kafka.internals.LegacyFetcher.run(LegacyFetcher.java:242)
>>> > >> >>>>     at
>>> > >> >>>>
>>> > >> >>>>
>>> org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer.run(FlinkKafkaConsumer.java:397)
>>> > >> >>>>     at
>>> > >> >>>>
>>> > >> >>>>
>>> org.apache.flink.streaming.api.operators.StreamSource.run(StreamSource.java:58)
>>> > >> >>>>     at
>>> > >> >>>>
>>> > >> >>>>
>>> org.apache.flink.streaming.runtime.tasks.SourceStreamTask.run(SourceStreamTask.java:55)
>>> > >> >>>>     at
>>> > >> >>>>
>>> > >> >>>>
>>> org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:218)
>>> > >> >>>>     at
>>> org.apache.flink.runtime.taskmanager.Task.run(Task.java:584)
>>> > >> >>>>     at java.lang.Thread.run(Thread.java:745)
>>> > >> >>>> Caused by: java.lang.NullPointerException
>>> > >> >>>>     at
>>> > >> >>>>
>>> > >> >>>>
>>> org.I0Itec.zkclient.ZkConnection.writeDataReturnStat(ZkConnection.java:115)
>>> > >> >>>>     at org.I0Itec.zkclient.ZkClient$10.call(ZkClient.java:817)
>>> > >> >>>>     at
>>> > >> >>>>
>>> org.I0Itec.zkclient.ZkClient.retryUntilConnected(ZkClient.java:675)
>>> > >> >>>>     at
>>> > >> >>>>
>>> org.I0Itec.zkclient.ZkClient.writeDataReturnStat(ZkClient.java:813)
>>> > >> >>>>     at
>>> org.I0Itec.zkclient.ZkClient.writeData(ZkClient.java:808)
>>> > >> >>>>     at
>>> org.I0Itec.zkclient.ZkClient.writeData(ZkClient.java:777)
>>> > >> >>>>     at
>>> kafka.utils.ZkUtils$.updatePersistentPath(ZkUtils.scala:332)
>>> > >> >>>>     at kafka.utils.ZkUtils.updatePersistentPath(ZkUtils.scala)
>>> > >> >>>>     at
>>> > >> >>>>
>>> > >> >>>>
>>> org.apache.flink.streaming.connectors.kafka.internals.ZookeeperOffsetHandler.setOffsetInZooKeeper(ZookeeperOffsetHandler.java:112)
>>> > >> >>>>     at
>>> > >> >>>>
>>> > >> >>>>
>>> org.apache.flink.streaming.connectors.kafka.internals.ZookeeperOffsetHandler.commit(ZookeeperOffsetHandler.java:80)
>>> > >> >>>>     at
>>> > >> >>>>
>>> > >> >>>>
>>> org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer$PeriodicOffsetCommitter.run(FlinkKafkaConsumer.java:632)
>>> > >> >>>>
>>> > >> >>>>
>>> > >> >>>> Any ideas on what could cause this behaviour?
>>> > >> >>>>
>>> > >> >>>> Best,
>>> > >> >>>> Mihail
>>> > >> >>>
>>> > >> >>>
>>> > >> >
>>> > >
>>> > >
>>> >
>>> >
>>> >
>>>
>>>
>>

Re: NPE with Flink Streaming from Kafka

Posted by Gyula Fóra <gy...@apache.org>.
Hi,

I am working on a use case that involves storing state for billions of
keys. For this we use a MySql state backend that will write each key-value
state to MySql server so it will only hold a limited set of key-value pairs
on heap while maintaining the processing guarantees.

This will keep our streaming job from running out of memory as most of the
state is off heap. I am not sure if this is relevant to your use case but
if the state size grows indefinitely you might want to give it a try.

I will write a detailed guide in some days but if you want to get started
check this one out:
https://docs.google.com/document/d/1xx3J88ZR_kuYYHAil3HD3YSBLqBTKV4Pu1SLwhiRiGs/edit?usp=sharing

There are some pending improvements that I will commit in the next days
that will increase the performance of the MySql adapter

Let me know if you are interested in this!

Cheers,
Gyula


Vieru, Mihail <mi...@zalando.de> ezt írta (időpont: 2015. dec. 2.,
Sze, 11:26):

> Hi Aljoscha,
>
> we have no upper bound for the number of expected keys. The max size for
> an element is 1 KB.
>
> There are 2 Maps, a KeyBy, a timeWindow, a Reduce and a writeAsText
> operators in the job. In the first Map we parse the contained JSON object
> in each element and forward it as a Flink Tuple. In the Reduce we update
> the state for each key. That's about it.
>
> Best,
> Mihail
>
>
> 2015-12-02 11:09 GMT+01:00 Aljoscha Krettek <al...@apache.org>:
>
>> Hi Mihail,
>> could you please give some information about the number of keys that you
>> are expecting in the data and how big the elements are that you are
>> processing in the window.
>>
>> Also, are there any other operations that could be taxing on Memory. I
>> think the different exception you see for 500MB mem size is just because
>> Java notices that it ran out of memory at a different part in the program.
>>
>> Cheers,
>> Aljoscha
>> > On 02 Dec 2015, at 10:57, Vieru, Mihail <mi...@zalando.de>
>> wrote:
>> >
>> > Yes, with the "start-cluster-streaming.sh" script.
>> > If the TaskManager gets 5GB of heap it manages to process ~100 million
>> messages and then throws the above OOM.
>> > If it gets only 500MB it manages to process ~8 million and a somewhat
>> misleading exception is thrown:
>> >
>> > 12/01/2015 19:14:07    Source: Custom Source -> Map -> Map(1/1)
>> switched to FAILED
>> > java.lang.Exception: Java heap space
>> >     at
>> org.apache.flink.streaming.connectors.kafka.internals.LegacyFetcher.run(LegacyFetcher.java:242)
>> >     at
>> org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer.run(FlinkKafkaConsumer.java:399)
>> >     at
>> org.apache.flink.streaming.api.operators.StreamSource.run(StreamSource.java:58)
>> >     at
>> org.apache.flink.streaming.runtime.tasks.SourceStreamTask.run(SourceStreamTask.java:55)
>> >     at
>> org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:218)
>> >     at org.apache.flink.runtime.taskmanager.Task.run(Task.java:584)
>> >     at java.lang.Thread.run(Thread.java:745)
>> > Caused by: java.lang.OutOfMemoryError: Java heap space
>> >     at org.json.simple.parser.Yylex.<init>(Yylex.java:231)
>> >     at org.json.simple.parser.JSONParser.<init>(JSONParser.java:34)
>> >     at
>> de.zalando.saiki.gerakanflink.ItemPriceAvgPerOrder$1.map(ItemPriceAvgPerOrder.java:70)
>> >     at
>> de.zalando.saiki.gerakanflink.ItemPriceAvgPerOrder$1.map(ItemPriceAvgPerOrder.java:65)
>> >     at
>> org.apache.flink.streaming.api.operators.StreamMap.processElement(StreamMap.java:37)
>> >     at
>> org.apache.flink.streaming.runtime.tasks.OperatorChain$CopyingChainingOutput.collect(OperatorChain.java:316)
>> >     at
>> org.apache.flink.streaming.runtime.tasks.OperatorChain$CopyingChainingOutput.collect(OperatorChain.java:300)
>> >     at
>> org.apache.flink.streaming.runtime.io.CollectorWrapper.collect(CollectorWrapper.java:48)
>> >     at
>> org.apache.flink.streaming.runtime.io.CollectorWrapper.collect(CollectorWrapper.java:29)
>> >     at
>> org.apache.flink.streaming.runtime.tasks.SourceStreamTask$SourceOutput.collect(SourceStreamTask.java:97)
>> >     at
>> org.apache.flink.streaming.api.operators.StreamSource$NonTimestampContext.collect(StreamSource.java:92)
>> >     at
>> org.apache.flink.streaming.connectors.kafka.internals.LegacyFetcher$SimpleConsumerThread.run(LegacyFetcher.java:450)
>> >
>> >
>> >
>> >
>> > 2015-12-02 10:45 GMT+01:00 Robert Metzger <rm...@apache.org>:
>> > Its good news that the issue has been resolved.
>> >
>> > Regarding the OOM, did you start Flink in the streaming mode?
>> >
>> > On Wed, Dec 2, 2015 at 10:18 AM, Vieru, Mihail <mi...@zalando.de>
>> wrote:
>> > Thank you, Robert! The issue with Kafka is now solved with the
>> 0.10-SNAPSHOT dependency.
>> >
>> > We have run into an OutOfMemory exception though, which appears to be
>> related to the state. As my colleague, Javier Lopez, mentioned in a
>> previous thread, state handling is crucial for our use case. And as the
>> jobs are intended to run for months, stability plays an important role in
>> choosing a stream processing framework.
>> >
>> > 12/02/2015 10:03:53    Fast TumblingTimeWindows(5000) of Reduce at
>> main(ItemPriceAvgPerOrder.java:108) -> Sink: Unnamed(1/1) switched to FAILED
>> > java.lang.OutOfMemoryError: Java heap space
>> >     at java.util.HashMap.resize(HashMap.java:703)
>> >     at java.util.HashMap.putVal(HashMap.java:662)
>> >     at java.util.HashMap.put(HashMap.java:611)
>> >     at
>> org.apache.flink.runtime.state.AbstractHeapKvState.update(AbstractHeapKvState.java:98)
>> >     at
>> de.zalando.saiki.gerakanflink.ItemPriceAvgPerOrder$3.reduce(ItemPriceAvgPerOrder.java:121)
>> >     at
>> de.zalando.saiki.gerakanflink.ItemPriceAvgPerOrder$3.reduce(ItemPriceAvgPerOrder.java:108)
>> >     at
>> org.apache.flink.streaming.runtime.operators.windowing.KeyMap.putOrAggregate(KeyMap.java:196)
>> >     at
>> org.apache.flink.streaming.runtime.operators.windowing.AggregatingKeyedTimePanes.addElementToLatestPane(AggregatingKeyedTimePanes.java:50)
>> >     at
>> org.apache.flink.streaming.runtime.operators.windowing.AbstractAlignedProcessingTimeWindowOperator.processElement(AbstractAlignedProcessingTimeWindowOperator.java:210)
>> >     at
>> org.apache.flink.streaming.runtime.io.StreamInputProcessor.processInput(StreamInputProcessor.java:166)
>> >     at
>> org.apache.flink.streaming.runtime.tasks.OneInputStreamTask.run(OneInputStreamTask.java:63)
>> >     at
>> org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:218)
>> >     at org.apache.flink.runtime.taskmanager.Task.run(Task.java:584)
>> >     at java.lang.Thread.run(Thread.java:745)
>> >
>> >
>> >
>> >
>> > 2015-12-01 17:42 GMT+01:00 Maximilian Michels <mx...@apache.org>:
>> > Thanks! I've linked the issue in JIRA.
>> >
>> > On Tue, Dec 1, 2015 at 5:39 PM, Robert Metzger <rm...@apache.org>
>> wrote:
>> > > I think its this one https://issues.apache.org/jira/browse/KAFKA-824
>> > >
>> > > On Tue, Dec 1, 2015 at 5:37 PM, Maximilian Michels <mx...@apache.org>
>> wrote:
>> > >>
>> > >> I know this has been fixed already but, out of curiosity, could you
>> > >> point me to the Kafka JIRA issue for this
>> > >> bug? From the Flink issue it looks like this is a Zookeeper version
>> > >> mismatch.
>> > >>
>> > >> On Tue, Dec 1, 2015 at 5:16 PM, Robert Metzger <rm...@apache.org>
>> > >> wrote:
>> > >> > Hi Gyula,
>> > >> >
>> > >> > no, I didn't ;) We still deploy 0.10-SNAPSHOT versions from the
>> > >> > "release-0.10" branch to Apache's maven snapshot repository.
>> > >> >
>> > >> >
>> > >> > I don't think Mihail's code will run when he's compiling it against
>> > >> > 1.0-SNAPSHOT.
>> > >> >
>> > >> >
>> > >> > On Tue, Dec 1, 2015 at 5:13 PM, Gyula Fóra <gy...@gmail.com>
>> wrote:
>> > >> >>
>> > >> >> Hi,
>> > >> >>
>> > >> >> I think Robert meant to write setting the connector dependency to
>> > >> >> 1.0-SNAPSHOT.
>> > >> >>
>> > >> >> Cheers,
>> > >> >> Gyula
>> > >> >>
>> > >> >> Robert Metzger <rm...@apache.org> ezt írta (időpont: 2015.
>> dec. 1.,
>> > >> >> K,
>> > >> >> 17:10):
>> > >> >>>
>> > >> >>> Hi Mihail,
>> > >> >>>
>> > >> >>> the issue is actually a bug in Kafka. We have a JIRA in Flink
>> for this
>> > >> >>> as
>> > >> >>> well: https://issues.apache.org/jira/browse/FLINK-3067
>> > >> >>>
>> > >> >>> Sadly, we haven't released a fix for it yet. Flink 0.10.2 will
>> contain
>> > >> >>> a
>> > >> >>> fix.
>> > >> >>>
>> > >> >>> Since the kafka connector is not contained in the flink binary,
>> you
>> > >> >>> can
>> > >> >>> just set the version in your maven pom file to 0.10-SNAPSHOT.
>> Maven
>> > >> >>> will
>> > >> >>> then download the code planned for the 0.10-SNAPSHOT release.
>> > >> >>>
>> > >> >>> On Tue, Dec 1, 2015 at 4:54 PM, Vieru, Mihail
>> > >> >>> <mi...@zalando.de>
>> > >> >>> wrote:
>> > >> >>>>
>> > >> >>>> Hi,
>> > >> >>>>
>> > >> >>>> we get the following NullPointerException after ~50 minutes when
>> > >> >>>> running
>> > >> >>>> a streaming job with windowing and state that reads data from
>> Kafka
>> > >> >>>> and
>> > >> >>>> writes the result to local FS.
>> > >> >>>> There are around 170 million messages to be processed, Flink
>> 0.10.1
>> > >> >>>> stops at ~8 million.
>> > >> >>>> Flink runs locally, started with the
>> "start-cluster-streaming.sh"
>> > >> >>>> script.
>> > >> >>>>
>> > >> >>>> 12/01/2015 15:06:24    Job execution switched to status RUNNING.
>> > >> >>>> 12/01/2015 15:06:24    Source: Custom Source -> Map -> Map(1/1)
>> > >> >>>> switched
>> > >> >>>> to SCHEDULED
>> > >> >>>> 12/01/2015 15:06:24    Source: Custom Source -> Map -> Map(1/1)
>> > >> >>>> switched
>> > >> >>>> to DEPLOYING
>> > >> >>>> 12/01/2015 15:06:24    Fast TumblingTimeWindows(5000) of Reduce
>> at
>> > >> >>>> main(ItemPriceAvgPerOrder.java:108) -> Sink: Unnamed(1/1)
>> switched to
>> > >> >>>> SCHEDULED
>> > >> >>>> 12/01/2015 15:06:24    Fast TumblingTimeWindows(5000) of Reduce
>> at
>> > >> >>>> main(ItemPriceAvgPerOrder.java:108) -> Sink: Unnamed(1/1)
>> switched to
>> > >> >>>> DEPLOYING
>> > >> >>>> 12/01/2015 15:06:24    Source: Custom Source -> Map -> Map(1/1)
>> > >> >>>> switched
>> > >> >>>> to RUNNING
>> > >> >>>> 12/01/2015 15:06:24    Fast TumblingTimeWindows(5000) of Reduce
>> at
>> > >> >>>> main(ItemPriceAvgPerOrder.java:108) -> Sink: Unnamed(1/1)
>> switched to
>> > >> >>>> RUNNING
>> > >> >>>> 12/01/2015 15:56:08    Fast TumblingTimeWindows(5000) of Reduce
>> at
>> > >> >>>> main(ItemPriceAvgPerOrder.java:108) -> Sink: Unnamed(1/1)
>> switched to
>> > >> >>>> CANCELED
>> > >> >>>> 12/01/2015 15:56:08    Source: Custom Source -> Map -> Map(1/1)
>> > >> >>>> switched
>> > >> >>>> to FAILED
>> > >> >>>> java.lang.Exception
>> > >> >>>>     at
>> > >> >>>>
>> > >> >>>>
>> org.apache.flink.streaming.connectors.kafka.internals.LegacyFetcher.run(LegacyFetcher.java:242)
>> > >> >>>>     at
>> > >> >>>>
>> > >> >>>>
>> org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer.run(FlinkKafkaConsumer.java:397)
>> > >> >>>>     at
>> > >> >>>>
>> > >> >>>>
>> org.apache.flink.streaming.api.operators.StreamSource.run(StreamSource.java:58)
>> > >> >>>>     at
>> > >> >>>>
>> > >> >>>>
>> org.apache.flink.streaming.runtime.tasks.SourceStreamTask.run(SourceStreamTask.java:55)
>> > >> >>>>     at
>> > >> >>>>
>> > >> >>>>
>> org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:218)
>> > >> >>>>     at
>> org.apache.flink.runtime.taskmanager.Task.run(Task.java:584)
>> > >> >>>>     at java.lang.Thread.run(Thread.java:745)
>> > >> >>>> Caused by: java.lang.NullPointerException
>> > >> >>>>     at
>> > >> >>>>
>> > >> >>>>
>> org.I0Itec.zkclient.ZkConnection.writeDataReturnStat(ZkConnection.java:115)
>> > >> >>>>     at org.I0Itec.zkclient.ZkClient$10.call(ZkClient.java:817)
>> > >> >>>>     at
>> > >> >>>>
>> org.I0Itec.zkclient.ZkClient.retryUntilConnected(ZkClient.java:675)
>> > >> >>>>     at
>> > >> >>>>
>> org.I0Itec.zkclient.ZkClient.writeDataReturnStat(ZkClient.java:813)
>> > >> >>>>     at org.I0Itec.zkclient.ZkClient.writeData(ZkClient.java:808)
>> > >> >>>>     at org.I0Itec.zkclient.ZkClient.writeData(ZkClient.java:777)
>> > >> >>>>     at
>> kafka.utils.ZkUtils$.updatePersistentPath(ZkUtils.scala:332)
>> > >> >>>>     at kafka.utils.ZkUtils.updatePersistentPath(ZkUtils.scala)
>> > >> >>>>     at
>> > >> >>>>
>> > >> >>>>
>> org.apache.flink.streaming.connectors.kafka.internals.ZookeeperOffsetHandler.setOffsetInZooKeeper(ZookeeperOffsetHandler.java:112)
>> > >> >>>>     at
>> > >> >>>>
>> > >> >>>>
>> org.apache.flink.streaming.connectors.kafka.internals.ZookeeperOffsetHandler.commit(ZookeeperOffsetHandler.java:80)
>> > >> >>>>     at
>> > >> >>>>
>> > >> >>>>
>> org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer$PeriodicOffsetCommitter.run(FlinkKafkaConsumer.java:632)
>> > >> >>>>
>> > >> >>>>
>> > >> >>>> Any ideas on what could cause this behaviour?
>> > >> >>>>
>> > >> >>>> Best,
>> > >> >>>> Mihail
>> > >> >>>
>> > >> >>>
>> > >> >
>> > >
>> > >
>> >
>> >
>> >
>>
>>
>

Re: NPE with Flink Streaming from Kafka

Posted by "Vieru, Mihail" <mi...@zalando.de>.
Hi Aljoscha,

we have no upper bound for the number of expected keys. The max size for an
element is 1 KB.

There are 2 Maps, a KeyBy, a timeWindow, a Reduce and a writeAsText
operators in the job. In the first Map we parse the contained JSON object
in each element and forward it as a Flink Tuple. In the Reduce we update
the state for each key. That's about it.

Best,
Mihail


2015-12-02 11:09 GMT+01:00 Aljoscha Krettek <al...@apache.org>:

> Hi Mihail,
> could you please give some information about the number of keys that you
> are expecting in the data and how big the elements are that you are
> processing in the window.
>
> Also, are there any other operations that could be taxing on Memory. I
> think the different exception you see for 500MB mem size is just because
> Java notices that it ran out of memory at a different part in the program.
>
> Cheers,
> Aljoscha
> > On 02 Dec 2015, at 10:57, Vieru, Mihail <mi...@zalando.de> wrote:
> >
> > Yes, with the "start-cluster-streaming.sh" script.
> > If the TaskManager gets 5GB of heap it manages to process ~100 million
> messages and then throws the above OOM.
> > If it gets only 500MB it manages to process ~8 million and a somewhat
> misleading exception is thrown:
> >
> > 12/01/2015 19:14:07    Source: Custom Source -> Map -> Map(1/1) switched
> to FAILED
> > java.lang.Exception: Java heap space
> >     at
> org.apache.flink.streaming.connectors.kafka.internals.LegacyFetcher.run(LegacyFetcher.java:242)
> >     at
> org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer.run(FlinkKafkaConsumer.java:399)
> >     at
> org.apache.flink.streaming.api.operators.StreamSource.run(StreamSource.java:58)
> >     at
> org.apache.flink.streaming.runtime.tasks.SourceStreamTask.run(SourceStreamTask.java:55)
> >     at
> org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:218)
> >     at org.apache.flink.runtime.taskmanager.Task.run(Task.java:584)
> >     at java.lang.Thread.run(Thread.java:745)
> > Caused by: java.lang.OutOfMemoryError: Java heap space
> >     at org.json.simple.parser.Yylex.<init>(Yylex.java:231)
> >     at org.json.simple.parser.JSONParser.<init>(JSONParser.java:34)
> >     at
> de.zalando.saiki.gerakanflink.ItemPriceAvgPerOrder$1.map(ItemPriceAvgPerOrder.java:70)
> >     at
> de.zalando.saiki.gerakanflink.ItemPriceAvgPerOrder$1.map(ItemPriceAvgPerOrder.java:65)
> >     at
> org.apache.flink.streaming.api.operators.StreamMap.processElement(StreamMap.java:37)
> >     at
> org.apache.flink.streaming.runtime.tasks.OperatorChain$CopyingChainingOutput.collect(OperatorChain.java:316)
> >     at
> org.apache.flink.streaming.runtime.tasks.OperatorChain$CopyingChainingOutput.collect(OperatorChain.java:300)
> >     at
> org.apache.flink.streaming.runtime.io.CollectorWrapper.collect(CollectorWrapper.java:48)
> >     at
> org.apache.flink.streaming.runtime.io.CollectorWrapper.collect(CollectorWrapper.java:29)
> >     at
> org.apache.flink.streaming.runtime.tasks.SourceStreamTask$SourceOutput.collect(SourceStreamTask.java:97)
> >     at
> org.apache.flink.streaming.api.operators.StreamSource$NonTimestampContext.collect(StreamSource.java:92)
> >     at
> org.apache.flink.streaming.connectors.kafka.internals.LegacyFetcher$SimpleConsumerThread.run(LegacyFetcher.java:450)
> >
> >
> >
> >
> > 2015-12-02 10:45 GMT+01:00 Robert Metzger <rm...@apache.org>:
> > Its good news that the issue has been resolved.
> >
> > Regarding the OOM, did you start Flink in the streaming mode?
> >
> > On Wed, Dec 2, 2015 at 10:18 AM, Vieru, Mihail <mi...@zalando.de>
> wrote:
> > Thank you, Robert! The issue with Kafka is now solved with the
> 0.10-SNAPSHOT dependency.
> >
> > We have run into an OutOfMemory exception though, which appears to be
> related to the state. As my colleague, Javier Lopez, mentioned in a
> previous thread, state handling is crucial for our use case. And as the
> jobs are intended to run for months, stability plays an important role in
> choosing a stream processing framework.
> >
> > 12/02/2015 10:03:53    Fast TumblingTimeWindows(5000) of Reduce at
> main(ItemPriceAvgPerOrder.java:108) -> Sink: Unnamed(1/1) switched to FAILED
> > java.lang.OutOfMemoryError: Java heap space
> >     at java.util.HashMap.resize(HashMap.java:703)
> >     at java.util.HashMap.putVal(HashMap.java:662)
> >     at java.util.HashMap.put(HashMap.java:611)
> >     at
> org.apache.flink.runtime.state.AbstractHeapKvState.update(AbstractHeapKvState.java:98)
> >     at
> de.zalando.saiki.gerakanflink.ItemPriceAvgPerOrder$3.reduce(ItemPriceAvgPerOrder.java:121)
> >     at
> de.zalando.saiki.gerakanflink.ItemPriceAvgPerOrder$3.reduce(ItemPriceAvgPerOrder.java:108)
> >     at
> org.apache.flink.streaming.runtime.operators.windowing.KeyMap.putOrAggregate(KeyMap.java:196)
> >     at
> org.apache.flink.streaming.runtime.operators.windowing.AggregatingKeyedTimePanes.addElementToLatestPane(AggregatingKeyedTimePanes.java:50)
> >     at
> org.apache.flink.streaming.runtime.operators.windowing.AbstractAlignedProcessingTimeWindowOperator.processElement(AbstractAlignedProcessingTimeWindowOperator.java:210)
> >     at
> org.apache.flink.streaming.runtime.io.StreamInputProcessor.processInput(StreamInputProcessor.java:166)
> >     at
> org.apache.flink.streaming.runtime.tasks.OneInputStreamTask.run(OneInputStreamTask.java:63)
> >     at
> org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:218)
> >     at org.apache.flink.runtime.taskmanager.Task.run(Task.java:584)
> >     at java.lang.Thread.run(Thread.java:745)
> >
> >
> >
> >
> > 2015-12-01 17:42 GMT+01:00 Maximilian Michels <mx...@apache.org>:
> > Thanks! I've linked the issue in JIRA.
> >
> > On Tue, Dec 1, 2015 at 5:39 PM, Robert Metzger <rm...@apache.org>
> wrote:
> > > I think its this one https://issues.apache.org/jira/browse/KAFKA-824
> > >
> > > On Tue, Dec 1, 2015 at 5:37 PM, Maximilian Michels <mx...@apache.org>
> wrote:
> > >>
> > >> I know this has been fixed already but, out of curiosity, could you
> > >> point me to the Kafka JIRA issue for this
> > >> bug? From the Flink issue it looks like this is a Zookeeper version
> > >> mismatch.
> > >>
> > >> On Tue, Dec 1, 2015 at 5:16 PM, Robert Metzger <rm...@apache.org>
> > >> wrote:
> > >> > Hi Gyula,
> > >> >
> > >> > no, I didn't ;) We still deploy 0.10-SNAPSHOT versions from the
> > >> > "release-0.10" branch to Apache's maven snapshot repository.
> > >> >
> > >> >
> > >> > I don't think Mihail's code will run when he's compiling it against
> > >> > 1.0-SNAPSHOT.
> > >> >
> > >> >
> > >> > On Tue, Dec 1, 2015 at 5:13 PM, Gyula Fóra <gy...@gmail.com>
> wrote:
> > >> >>
> > >> >> Hi,
> > >> >>
> > >> >> I think Robert meant to write setting the connector dependency to
> > >> >> 1.0-SNAPSHOT.
> > >> >>
> > >> >> Cheers,
> > >> >> Gyula
> > >> >>
> > >> >> Robert Metzger <rm...@apache.org> ezt írta (időpont: 2015.
> dec. 1.,
> > >> >> K,
> > >> >> 17:10):
> > >> >>>
> > >> >>> Hi Mihail,
> > >> >>>
> > >> >>> the issue is actually a bug in Kafka. We have a JIRA in Flink for
> this
> > >> >>> as
> > >> >>> well: https://issues.apache.org/jira/browse/FLINK-3067
> > >> >>>
> > >> >>> Sadly, we haven't released a fix for it yet. Flink 0.10.2 will
> contain
> > >> >>> a
> > >> >>> fix.
> > >> >>>
> > >> >>> Since the kafka connector is not contained in the flink binary,
> you
> > >> >>> can
> > >> >>> just set the version in your maven pom file to 0.10-SNAPSHOT.
> Maven
> > >> >>> will
> > >> >>> then download the code planned for the 0.10-SNAPSHOT release.
> > >> >>>
> > >> >>> On Tue, Dec 1, 2015 at 4:54 PM, Vieru, Mihail
> > >> >>> <mi...@zalando.de>
> > >> >>> wrote:
> > >> >>>>
> > >> >>>> Hi,
> > >> >>>>
> > >> >>>> we get the following NullPointerException after ~50 minutes when
> > >> >>>> running
> > >> >>>> a streaming job with windowing and state that reads data from
> Kafka
> > >> >>>> and
> > >> >>>> writes the result to local FS.
> > >> >>>> There are around 170 million messages to be processed, Flink
> 0.10.1
> > >> >>>> stops at ~8 million.
> > >> >>>> Flink runs locally, started with the "start-cluster-streaming.sh"
> > >> >>>> script.
> > >> >>>>
> > >> >>>> 12/01/2015 15:06:24    Job execution switched to status RUNNING.
> > >> >>>> 12/01/2015 15:06:24    Source: Custom Source -> Map -> Map(1/1)
> > >> >>>> switched
> > >> >>>> to SCHEDULED
> > >> >>>> 12/01/2015 15:06:24    Source: Custom Source -> Map -> Map(1/1)
> > >> >>>> switched
> > >> >>>> to DEPLOYING
> > >> >>>> 12/01/2015 15:06:24    Fast TumblingTimeWindows(5000) of Reduce
> at
> > >> >>>> main(ItemPriceAvgPerOrder.java:108) -> Sink: Unnamed(1/1)
> switched to
> > >> >>>> SCHEDULED
> > >> >>>> 12/01/2015 15:06:24    Fast TumblingTimeWindows(5000) of Reduce
> at
> > >> >>>> main(ItemPriceAvgPerOrder.java:108) -> Sink: Unnamed(1/1)
> switched to
> > >> >>>> DEPLOYING
> > >> >>>> 12/01/2015 15:06:24    Source: Custom Source -> Map -> Map(1/1)
> > >> >>>> switched
> > >> >>>> to RUNNING
> > >> >>>> 12/01/2015 15:06:24    Fast TumblingTimeWindows(5000) of Reduce
> at
> > >> >>>> main(ItemPriceAvgPerOrder.java:108) -> Sink: Unnamed(1/1)
> switched to
> > >> >>>> RUNNING
> > >> >>>> 12/01/2015 15:56:08    Fast TumblingTimeWindows(5000) of Reduce
> at
> > >> >>>> main(ItemPriceAvgPerOrder.java:108) -> Sink: Unnamed(1/1)
> switched to
> > >> >>>> CANCELED
> > >> >>>> 12/01/2015 15:56:08    Source: Custom Source -> Map -> Map(1/1)
> > >> >>>> switched
> > >> >>>> to FAILED
> > >> >>>> java.lang.Exception
> > >> >>>>     at
> > >> >>>>
> > >> >>>>
> org.apache.flink.streaming.connectors.kafka.internals.LegacyFetcher.run(LegacyFetcher.java:242)
> > >> >>>>     at
> > >> >>>>
> > >> >>>>
> org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer.run(FlinkKafkaConsumer.java:397)
> > >> >>>>     at
> > >> >>>>
> > >> >>>>
> org.apache.flink.streaming.api.operators.StreamSource.run(StreamSource.java:58)
> > >> >>>>     at
> > >> >>>>
> > >> >>>>
> org.apache.flink.streaming.runtime.tasks.SourceStreamTask.run(SourceStreamTask.java:55)
> > >> >>>>     at
> > >> >>>>
> > >> >>>>
> org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:218)
> > >> >>>>     at
> org.apache.flink.runtime.taskmanager.Task.run(Task.java:584)
> > >> >>>>     at java.lang.Thread.run(Thread.java:745)
> > >> >>>> Caused by: java.lang.NullPointerException
> > >> >>>>     at
> > >> >>>>
> > >> >>>>
> org.I0Itec.zkclient.ZkConnection.writeDataReturnStat(ZkConnection.java:115)
> > >> >>>>     at org.I0Itec.zkclient.ZkClient$10.call(ZkClient.java:817)
> > >> >>>>     at
> > >> >>>>
> org.I0Itec.zkclient.ZkClient.retryUntilConnected(ZkClient.java:675)
> > >> >>>>     at
> > >> >>>>
> org.I0Itec.zkclient.ZkClient.writeDataReturnStat(ZkClient.java:813)
> > >> >>>>     at org.I0Itec.zkclient.ZkClient.writeData(ZkClient.java:808)
> > >> >>>>     at org.I0Itec.zkclient.ZkClient.writeData(ZkClient.java:777)
> > >> >>>>     at
> kafka.utils.ZkUtils$.updatePersistentPath(ZkUtils.scala:332)
> > >> >>>>     at kafka.utils.ZkUtils.updatePersistentPath(ZkUtils.scala)
> > >> >>>>     at
> > >> >>>>
> > >> >>>>
> org.apache.flink.streaming.connectors.kafka.internals.ZookeeperOffsetHandler.setOffsetInZooKeeper(ZookeeperOffsetHandler.java:112)
> > >> >>>>     at
> > >> >>>>
> > >> >>>>
> org.apache.flink.streaming.connectors.kafka.internals.ZookeeperOffsetHandler.commit(ZookeeperOffsetHandler.java:80)
> > >> >>>>     at
> > >> >>>>
> > >> >>>>
> org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer$PeriodicOffsetCommitter.run(FlinkKafkaConsumer.java:632)
> > >> >>>>
> > >> >>>>
> > >> >>>> Any ideas on what could cause this behaviour?
> > >> >>>>
> > >> >>>> Best,
> > >> >>>> Mihail
> > >> >>>
> > >> >>>
> > >> >
> > >
> > >
> >
> >
> >
>
>

Re: NPE with Flink Streaming from Kafka

Posted by Aljoscha Krettek <al...@apache.org>.
Hi Mihail,
could you please give some information about the number of keys that you are expecting in the data and how big the elements are that you are processing in the window.

Also, are there any other operations that could be taxing on Memory. I think the different exception you see for 500MB mem size is just because Java notices that it ran out of memory at a different part in the program.

Cheers,
Aljoscha
> On 02 Dec 2015, at 10:57, Vieru, Mihail <mi...@zalando.de> wrote:
> 
> Yes, with the "start-cluster-streaming.sh" script.
> If the TaskManager gets 5GB of heap it manages to process ~100 million messages and then throws the above OOM.
> If it gets only 500MB it manages to process ~8 million and a somewhat misleading exception is thrown:
> 
> 12/01/2015 19:14:07    Source: Custom Source -> Map -> Map(1/1) switched to FAILED 
> java.lang.Exception: Java heap space
>     at org.apache.flink.streaming.connectors.kafka.internals.LegacyFetcher.run(LegacyFetcher.java:242)
>     at org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer.run(FlinkKafkaConsumer.java:399)
>     at org.apache.flink.streaming.api.operators.StreamSource.run(StreamSource.java:58)
>     at org.apache.flink.streaming.runtime.tasks.SourceStreamTask.run(SourceStreamTask.java:55)
>     at org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:218)
>     at org.apache.flink.runtime.taskmanager.Task.run(Task.java:584)
>     at java.lang.Thread.run(Thread.java:745)
> Caused by: java.lang.OutOfMemoryError: Java heap space
>     at org.json.simple.parser.Yylex.<init>(Yylex.java:231)
>     at org.json.simple.parser.JSONParser.<init>(JSONParser.java:34)
>     at de.zalando.saiki.gerakanflink.ItemPriceAvgPerOrder$1.map(ItemPriceAvgPerOrder.java:70)
>     at de.zalando.saiki.gerakanflink.ItemPriceAvgPerOrder$1.map(ItemPriceAvgPerOrder.java:65)
>     at org.apache.flink.streaming.api.operators.StreamMap.processElement(StreamMap.java:37)
>     at org.apache.flink.streaming.runtime.tasks.OperatorChain$CopyingChainingOutput.collect(OperatorChain.java:316)
>     at org.apache.flink.streaming.runtime.tasks.OperatorChain$CopyingChainingOutput.collect(OperatorChain.java:300)
>     at org.apache.flink.streaming.runtime.io.CollectorWrapper.collect(CollectorWrapper.java:48)
>     at org.apache.flink.streaming.runtime.io.CollectorWrapper.collect(CollectorWrapper.java:29)
>     at org.apache.flink.streaming.runtime.tasks.SourceStreamTask$SourceOutput.collect(SourceStreamTask.java:97)
>     at org.apache.flink.streaming.api.operators.StreamSource$NonTimestampContext.collect(StreamSource.java:92)
>     at org.apache.flink.streaming.connectors.kafka.internals.LegacyFetcher$SimpleConsumerThread.run(LegacyFetcher.java:450)
> 
> 
> 
> 
> 2015-12-02 10:45 GMT+01:00 Robert Metzger <rm...@apache.org>:
> Its good news that the issue has been resolved.
> 
> Regarding the OOM, did you start Flink in the streaming mode?
> 
> On Wed, Dec 2, 2015 at 10:18 AM, Vieru, Mihail <mi...@zalando.de> wrote:
> Thank you, Robert! The issue with Kafka is now solved with the 0.10-SNAPSHOT dependency.
> 
> We have run into an OutOfMemory exception though, which appears to be related to the state. As my colleague, Javier Lopez, mentioned in a previous thread, state handling is crucial for our use case. And as the jobs are intended to run for months, stability plays an important role in choosing a stream processing framework.
> 
> 12/02/2015 10:03:53    Fast TumblingTimeWindows(5000) of Reduce at main(ItemPriceAvgPerOrder.java:108) -> Sink: Unnamed(1/1) switched to FAILED 
> java.lang.OutOfMemoryError: Java heap space
>     at java.util.HashMap.resize(HashMap.java:703)
>     at java.util.HashMap.putVal(HashMap.java:662)
>     at java.util.HashMap.put(HashMap.java:611)
>     at org.apache.flink.runtime.state.AbstractHeapKvState.update(AbstractHeapKvState.java:98)
>     at de.zalando.saiki.gerakanflink.ItemPriceAvgPerOrder$3.reduce(ItemPriceAvgPerOrder.java:121)
>     at de.zalando.saiki.gerakanflink.ItemPriceAvgPerOrder$3.reduce(ItemPriceAvgPerOrder.java:108)
>     at org.apache.flink.streaming.runtime.operators.windowing.KeyMap.putOrAggregate(KeyMap.java:196)
>     at org.apache.flink.streaming.runtime.operators.windowing.AggregatingKeyedTimePanes.addElementToLatestPane(AggregatingKeyedTimePanes.java:50)
>     at org.apache.flink.streaming.runtime.operators.windowing.AbstractAlignedProcessingTimeWindowOperator.processElement(AbstractAlignedProcessingTimeWindowOperator.java:210)
>     at org.apache.flink.streaming.runtime.io.StreamInputProcessor.processInput(StreamInputProcessor.java:166)
>     at org.apache.flink.streaming.runtime.tasks.OneInputStreamTask.run(OneInputStreamTask.java:63)
>     at org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:218)
>     at org.apache.flink.runtime.taskmanager.Task.run(Task.java:584)
>     at java.lang.Thread.run(Thread.java:745)
> 
> 
> 
> 
> 2015-12-01 17:42 GMT+01:00 Maximilian Michels <mx...@apache.org>:
> Thanks! I've linked the issue in JIRA.
> 
> On Tue, Dec 1, 2015 at 5:39 PM, Robert Metzger <rm...@apache.org> wrote:
> > I think its this one https://issues.apache.org/jira/browse/KAFKA-824
> >
> > On Tue, Dec 1, 2015 at 5:37 PM, Maximilian Michels <mx...@apache.org> wrote:
> >>
> >> I know this has been fixed already but, out of curiosity, could you
> >> point me to the Kafka JIRA issue for this
> >> bug? From the Flink issue it looks like this is a Zookeeper version
> >> mismatch.
> >>
> >> On Tue, Dec 1, 2015 at 5:16 PM, Robert Metzger <rm...@apache.org>
> >> wrote:
> >> > Hi Gyula,
> >> >
> >> > no, I didn't ;) We still deploy 0.10-SNAPSHOT versions from the
> >> > "release-0.10" branch to Apache's maven snapshot repository.
> >> >
> >> >
> >> > I don't think Mihail's code will run when he's compiling it against
> >> > 1.0-SNAPSHOT.
> >> >
> >> >
> >> > On Tue, Dec 1, 2015 at 5:13 PM, Gyula Fóra <gy...@gmail.com> wrote:
> >> >>
> >> >> Hi,
> >> >>
> >> >> I think Robert meant to write setting the connector dependency to
> >> >> 1.0-SNAPSHOT.
> >> >>
> >> >> Cheers,
> >> >> Gyula
> >> >>
> >> >> Robert Metzger <rm...@apache.org> ezt írta (időpont: 2015. dec. 1.,
> >> >> K,
> >> >> 17:10):
> >> >>>
> >> >>> Hi Mihail,
> >> >>>
> >> >>> the issue is actually a bug in Kafka. We have a JIRA in Flink for this
> >> >>> as
> >> >>> well: https://issues.apache.org/jira/browse/FLINK-3067
> >> >>>
> >> >>> Sadly, we haven't released a fix for it yet. Flink 0.10.2 will contain
> >> >>> a
> >> >>> fix.
> >> >>>
> >> >>> Since the kafka connector is not contained in the flink binary, you
> >> >>> can
> >> >>> just set the version in your maven pom file to 0.10-SNAPSHOT. Maven
> >> >>> will
> >> >>> then download the code planned for the 0.10-SNAPSHOT release.
> >> >>>
> >> >>> On Tue, Dec 1, 2015 at 4:54 PM, Vieru, Mihail
> >> >>> <mi...@zalando.de>
> >> >>> wrote:
> >> >>>>
> >> >>>> Hi,
> >> >>>>
> >> >>>> we get the following NullPointerException after ~50 minutes when
> >> >>>> running
> >> >>>> a streaming job with windowing and state that reads data from Kafka
> >> >>>> and
> >> >>>> writes the result to local FS.
> >> >>>> There are around 170 million messages to be processed, Flink 0.10.1
> >> >>>> stops at ~8 million.
> >> >>>> Flink runs locally, started with the "start-cluster-streaming.sh"
> >> >>>> script.
> >> >>>>
> >> >>>> 12/01/2015 15:06:24    Job execution switched to status RUNNING.
> >> >>>> 12/01/2015 15:06:24    Source: Custom Source -> Map -> Map(1/1)
> >> >>>> switched
> >> >>>> to SCHEDULED
> >> >>>> 12/01/2015 15:06:24    Source: Custom Source -> Map -> Map(1/1)
> >> >>>> switched
> >> >>>> to DEPLOYING
> >> >>>> 12/01/2015 15:06:24    Fast TumblingTimeWindows(5000) of Reduce at
> >> >>>> main(ItemPriceAvgPerOrder.java:108) -> Sink: Unnamed(1/1) switched to
> >> >>>> SCHEDULED
> >> >>>> 12/01/2015 15:06:24    Fast TumblingTimeWindows(5000) of Reduce at
> >> >>>> main(ItemPriceAvgPerOrder.java:108) -> Sink: Unnamed(1/1) switched to
> >> >>>> DEPLOYING
> >> >>>> 12/01/2015 15:06:24    Source: Custom Source -> Map -> Map(1/1)
> >> >>>> switched
> >> >>>> to RUNNING
> >> >>>> 12/01/2015 15:06:24    Fast TumblingTimeWindows(5000) of Reduce at
> >> >>>> main(ItemPriceAvgPerOrder.java:108) -> Sink: Unnamed(1/1) switched to
> >> >>>> RUNNING
> >> >>>> 12/01/2015 15:56:08    Fast TumblingTimeWindows(5000) of Reduce at
> >> >>>> main(ItemPriceAvgPerOrder.java:108) -> Sink: Unnamed(1/1) switched to
> >> >>>> CANCELED
> >> >>>> 12/01/2015 15:56:08    Source: Custom Source -> Map -> Map(1/1)
> >> >>>> switched
> >> >>>> to FAILED
> >> >>>> java.lang.Exception
> >> >>>>     at
> >> >>>>
> >> >>>> org.apache.flink.streaming.connectors.kafka.internals.LegacyFetcher.run(LegacyFetcher.java:242)
> >> >>>>     at
> >> >>>>
> >> >>>> org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer.run(FlinkKafkaConsumer.java:397)
> >> >>>>     at
> >> >>>>
> >> >>>> org.apache.flink.streaming.api.operators.StreamSource.run(StreamSource.java:58)
> >> >>>>     at
> >> >>>>
> >> >>>> org.apache.flink.streaming.runtime.tasks.SourceStreamTask.run(SourceStreamTask.java:55)
> >> >>>>     at
> >> >>>>
> >> >>>> org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:218)
> >> >>>>     at org.apache.flink.runtime.taskmanager.Task.run(Task.java:584)
> >> >>>>     at java.lang.Thread.run(Thread.java:745)
> >> >>>> Caused by: java.lang.NullPointerException
> >> >>>>     at
> >> >>>>
> >> >>>> org.I0Itec.zkclient.ZkConnection.writeDataReturnStat(ZkConnection.java:115)
> >> >>>>     at org.I0Itec.zkclient.ZkClient$10.call(ZkClient.java:817)
> >> >>>>     at
> >> >>>> org.I0Itec.zkclient.ZkClient.retryUntilConnected(ZkClient.java:675)
> >> >>>>     at
> >> >>>> org.I0Itec.zkclient.ZkClient.writeDataReturnStat(ZkClient.java:813)
> >> >>>>     at org.I0Itec.zkclient.ZkClient.writeData(ZkClient.java:808)
> >> >>>>     at org.I0Itec.zkclient.ZkClient.writeData(ZkClient.java:777)
> >> >>>>     at kafka.utils.ZkUtils$.updatePersistentPath(ZkUtils.scala:332)
> >> >>>>     at kafka.utils.ZkUtils.updatePersistentPath(ZkUtils.scala)
> >> >>>>     at
> >> >>>>
> >> >>>> org.apache.flink.streaming.connectors.kafka.internals.ZookeeperOffsetHandler.setOffsetInZooKeeper(ZookeeperOffsetHandler.java:112)
> >> >>>>     at
> >> >>>>
> >> >>>> org.apache.flink.streaming.connectors.kafka.internals.ZookeeperOffsetHandler.commit(ZookeeperOffsetHandler.java:80)
> >> >>>>     at
> >> >>>>
> >> >>>> org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer$PeriodicOffsetCommitter.run(FlinkKafkaConsumer.java:632)
> >> >>>>
> >> >>>>
> >> >>>> Any ideas on what could cause this behaviour?
> >> >>>>
> >> >>>> Best,
> >> >>>> Mihail
> >> >>>
> >> >>>
> >> >
> >
> >
> 
> 
> 


Re: NPE with Flink Streaming from Kafka

Posted by "Vieru, Mihail" <mi...@zalando.de>.
Yes, with the "start-cluster-streaming.sh" script.
If the TaskManager gets 5GB of heap it manages to process ~100 million
messages and then throws the above OOM.
If it gets only 500MB it manages to process ~8 million and a somewhat
misleading exception is thrown:

12/01/2015 19:14:07    Source: Custom Source -> Map -> Map(1/1) switched to
FAILED
java.lang.Exception: Java heap space
    at
org.apache.flink.streaming.connectors.kafka.internals.LegacyFetcher.run(LegacyFetcher.java:242)
    at
org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer.run(FlinkKafkaConsumer.java:399)
    at
org.apache.flink.streaming.api.operators.StreamSource.run(StreamSource.java:58)
    at
org.apache.flink.streaming.runtime.tasks.SourceStreamTask.run(SourceStreamTask.java:55)
    at
org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:218)
    at org.apache.flink.runtime.taskmanager.Task.run(Task.java:584)
    at java.lang.Thread.run(Thread.java:745)
Caused by: java.lang.OutOfMemoryError: Java heap space
    at org.json.simple.parser.Yylex.<init>(Yylex.java:231)
    at org.json.simple.parser.JSONParser.<init>(JSONParser.java:34)
    at
de.zalando.saiki.gerakanflink.ItemPriceAvgPerOrder$1.map(ItemPriceAvgPerOrder.java:70)
    at
de.zalando.saiki.gerakanflink.ItemPriceAvgPerOrder$1.map(ItemPriceAvgPerOrder.java:65)
    at
org.apache.flink.streaming.api.operators.StreamMap.processElement(StreamMap.java:37)
    at
org.apache.flink.streaming.runtime.tasks.OperatorChain$CopyingChainingOutput.collect(OperatorChain.java:316)
    at
org.apache.flink.streaming.runtime.tasks.OperatorChain$CopyingChainingOutput.collect(OperatorChain.java:300)
    at
org.apache.flink.streaming.runtime.io.CollectorWrapper.collect(CollectorWrapper.java:48)
    at
org.apache.flink.streaming.runtime.io.CollectorWrapper.collect(CollectorWrapper.java:29)
    at
org.apache.flink.streaming.runtime.tasks.SourceStreamTask$SourceOutput.collect(SourceStreamTask.java:97)
    at
org.apache.flink.streaming.api.operators.StreamSource$NonTimestampContext.collect(StreamSource.java:92)
    at
org.apache.flink.streaming.connectors.kafka.internals.LegacyFetcher$SimpleConsumerThread.run(LegacyFetcher.java:450)




2015-12-02 10:45 GMT+01:00 Robert Metzger <rm...@apache.org>:

> Its good news that the issue has been resolved.
>
> Regarding the OOM, did you start Flink in the streaming mode?
>
> On Wed, Dec 2, 2015 at 10:18 AM, Vieru, Mihail <mi...@zalando.de>
> wrote:
>
>> Thank you, Robert! The issue with Kafka is now solved with the
>> 0.10-SNAPSHOT dependency.
>>
>> We have run into an OutOfMemory exception though, which appears to be
>> related to the state. As my colleague, Javier Lopez, mentioned in a
>> previous thread, state handling is crucial for our use case. And as the
>> jobs are intended to run for months, stability plays an important role in
>> choosing a stream processing framework.
>>
>> 12/02/2015 10:03:53    Fast TumblingTimeWindows(5000) of Reduce at
>> main(ItemPriceAvgPerOrder.java:108) -> Sink: Unnamed(1/1) switched to
>> FAILED
>> java.lang.OutOfMemoryError: Java heap space
>>     at java.util.HashMap.resize(HashMap.java:703)
>>     at java.util.HashMap.putVal(HashMap.java:662)
>>     at java.util.HashMap.put(HashMap.java:611)
>>     at
>> org.apache.flink.runtime.state.AbstractHeapKvState.update(AbstractHeapKvState.java:98)
>>     at
>> de.zalando.saiki.gerakanflink.ItemPriceAvgPerOrder$3.reduce(ItemPriceAvgPerOrder.java:121)
>>     at
>> de.zalando.saiki.gerakanflink.ItemPriceAvgPerOrder$3.reduce(ItemPriceAvgPerOrder.java:108)
>>     at
>> org.apache.flink.streaming.runtime.operators.windowing.KeyMap.putOrAggregate(KeyMap.java:196)
>>     at
>> org.apache.flink.streaming.runtime.operators.windowing.AggregatingKeyedTimePanes.addElementToLatestPane(AggregatingKeyedTimePanes.java:50)
>>     at
>> org.apache.flink.streaming.runtime.operators.windowing.AbstractAlignedProcessingTimeWindowOperator.processElement(AbstractAlignedProcessingTimeWindowOperator.java:210)
>>     at
>> org.apache.flink.streaming.runtime.io.StreamInputProcessor.processInput(StreamInputProcessor.java:166)
>>     at
>> org.apache.flink.streaming.runtime.tasks.OneInputStreamTask.run(OneInputStreamTask.java:63)
>>     at
>> org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:218)
>>     at org.apache.flink.runtime.taskmanager.Task.run(Task.java:584)
>>     at java.lang.Thread.run(Thread.java:745)
>>
>>
>>
>>
>> 2015-12-01 17:42 GMT+01:00 Maximilian Michels <mx...@apache.org>:
>>
>>> Thanks! I've linked the issue in JIRA.
>>>
>>> On Tue, Dec 1, 2015 at 5:39 PM, Robert Metzger <rm...@apache.org>
>>> wrote:
>>> > I think its this one https://issues.apache.org/jira/browse/KAFKA-824
>>> >
>>> > On Tue, Dec 1, 2015 at 5:37 PM, Maximilian Michels <mx...@apache.org>
>>> wrote:
>>> >>
>>> >> I know this has been fixed already but, out of curiosity, could you
>>> >> point me to the Kafka JIRA issue for this
>>> >> bug? From the Flink issue it looks like this is a Zookeeper version
>>> >> mismatch.
>>> >>
>>> >> On Tue, Dec 1, 2015 at 5:16 PM, Robert Metzger <rm...@apache.org>
>>> >> wrote:
>>> >> > Hi Gyula,
>>> >> >
>>> >> > no, I didn't ;) We still deploy 0.10-SNAPSHOT versions from the
>>> >> > "release-0.10" branch to Apache's maven snapshot repository.
>>> >> >
>>> >> >
>>> >> > I don't think Mihail's code will run when he's compiling it against
>>> >> > 1.0-SNAPSHOT.
>>> >> >
>>> >> >
>>> >> > On Tue, Dec 1, 2015 at 5:13 PM, Gyula Fóra <gy...@gmail.com>
>>> wrote:
>>> >> >>
>>> >> >> Hi,
>>> >> >>
>>> >> >> I think Robert meant to write setting the connector dependency to
>>> >> >> 1.0-SNAPSHOT.
>>> >> >>
>>> >> >> Cheers,
>>> >> >> Gyula
>>> >> >>
>>> >> >> Robert Metzger <rm...@apache.org> ezt írta (időpont: 2015.
>>> dec. 1.,
>>> >> >> K,
>>> >> >> 17:10):
>>> >> >>>
>>> >> >>> Hi Mihail,
>>> >> >>>
>>> >> >>> the issue is actually a bug in Kafka. We have a JIRA in Flink for
>>> this
>>> >> >>> as
>>> >> >>> well: https://issues.apache.org/jira/browse/FLINK-3067
>>> >> >>>
>>> >> >>> Sadly, we haven't released a fix for it yet. Flink 0.10.2 will
>>> contain
>>> >> >>> a
>>> >> >>> fix.
>>> >> >>>
>>> >> >>> Since the kafka connector is not contained in the flink binary,
>>> you
>>> >> >>> can
>>> >> >>> just set the version in your maven pom file to 0.10-SNAPSHOT.
>>> Maven
>>> >> >>> will
>>> >> >>> then download the code planned for the 0.10-SNAPSHOT release.
>>> >> >>>
>>> >> >>> On Tue, Dec 1, 2015 at 4:54 PM, Vieru, Mihail
>>> >> >>> <mi...@zalando.de>
>>> >> >>> wrote:
>>> >> >>>>
>>> >> >>>> Hi,
>>> >> >>>>
>>> >> >>>> we get the following NullPointerException after ~50 minutes when
>>> >> >>>> running
>>> >> >>>> a streaming job with windowing and state that reads data from
>>> Kafka
>>> >> >>>> and
>>> >> >>>> writes the result to local FS.
>>> >> >>>> There are around 170 million messages to be processed, Flink
>>> 0.10.1
>>> >> >>>> stops at ~8 million.
>>> >> >>>> Flink runs locally, started with the "start-cluster-streaming.sh"
>>> >> >>>> script.
>>> >> >>>>
>>> >> >>>> 12/01/2015 15:06:24    Job execution switched to status RUNNING.
>>> >> >>>> 12/01/2015 15:06:24    Source: Custom Source -> Map -> Map(1/1)
>>> >> >>>> switched
>>> >> >>>> to SCHEDULED
>>> >> >>>> 12/01/2015 15:06:24    Source: Custom Source -> Map -> Map(1/1)
>>> >> >>>> switched
>>> >> >>>> to DEPLOYING
>>> >> >>>> 12/01/2015 15:06:24    Fast TumblingTimeWindows(5000) of Reduce
>>> at
>>> >> >>>> main(ItemPriceAvgPerOrder.java:108) -> Sink: Unnamed(1/1)
>>> switched to
>>> >> >>>> SCHEDULED
>>> >> >>>> 12/01/2015 15:06:24    Fast TumblingTimeWindows(5000) of Reduce
>>> at
>>> >> >>>> main(ItemPriceAvgPerOrder.java:108) -> Sink: Unnamed(1/1)
>>> switched to
>>> >> >>>> DEPLOYING
>>> >> >>>> 12/01/2015 15:06:24    Source: Custom Source -> Map -> Map(1/1)
>>> >> >>>> switched
>>> >> >>>> to RUNNING
>>> >> >>>> 12/01/2015 15:06:24    Fast TumblingTimeWindows(5000) of Reduce
>>> at
>>> >> >>>> main(ItemPriceAvgPerOrder.java:108) -> Sink: Unnamed(1/1)
>>> switched to
>>> >> >>>> RUNNING
>>> >> >>>> 12/01/2015 15:56:08    Fast TumblingTimeWindows(5000) of Reduce
>>> at
>>> >> >>>> main(ItemPriceAvgPerOrder.java:108) -> Sink: Unnamed(1/1)
>>> switched to
>>> >> >>>> CANCELED
>>> >> >>>> 12/01/2015 15:56:08    Source: Custom Source -> Map -> Map(1/1)
>>> >> >>>> switched
>>> >> >>>> to FAILED
>>> >> >>>> java.lang.Exception
>>> >> >>>>     at
>>> >> >>>>
>>> >> >>>>
>>> org.apache.flink.streaming.connectors.kafka.internals.LegacyFetcher.run(LegacyFetcher.java:242)
>>> >> >>>>     at
>>> >> >>>>
>>> >> >>>>
>>> org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer.run(FlinkKafkaConsumer.java:397)
>>> >> >>>>     at
>>> >> >>>>
>>> >> >>>>
>>> org.apache.flink.streaming.api.operators.StreamSource.run(StreamSource.java:58)
>>> >> >>>>     at
>>> >> >>>>
>>> >> >>>>
>>> org.apache.flink.streaming.runtime.tasks.SourceStreamTask.run(SourceStreamTask.java:55)
>>> >> >>>>     at
>>> >> >>>>
>>> >> >>>>
>>> org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:218)
>>> >> >>>>     at
>>> org.apache.flink.runtime.taskmanager.Task.run(Task.java:584)
>>> >> >>>>     at java.lang.Thread.run(Thread.java:745)
>>> >> >>>> Caused by: java.lang.NullPointerException
>>> >> >>>>     at
>>> >> >>>>
>>> >> >>>>
>>> org.I0Itec.zkclient.ZkConnection.writeDataReturnStat(ZkConnection.java:115)
>>> >> >>>>     at org.I0Itec.zkclient.ZkClient$10.call(ZkClient.java:817)
>>> >> >>>>     at
>>> >> >>>>
>>> org.I0Itec.zkclient.ZkClient.retryUntilConnected(ZkClient.java:675)
>>> >> >>>>     at
>>> >> >>>>
>>> org.I0Itec.zkclient.ZkClient.writeDataReturnStat(ZkClient.java:813)
>>> >> >>>>     at org.I0Itec.zkclient.ZkClient.writeData(ZkClient.java:808)
>>> >> >>>>     at org.I0Itec.zkclient.ZkClient.writeData(ZkClient.java:777)
>>> >> >>>>     at
>>> kafka.utils.ZkUtils$.updatePersistentPath(ZkUtils.scala:332)
>>> >> >>>>     at kafka.utils.ZkUtils.updatePersistentPath(ZkUtils.scala)
>>> >> >>>>     at
>>> >> >>>>
>>> >> >>>>
>>> org.apache.flink.streaming.connectors.kafka.internals.ZookeeperOffsetHandler.setOffsetInZooKeeper(ZookeeperOffsetHandler.java:112)
>>> >> >>>>     at
>>> >> >>>>
>>> >> >>>>
>>> org.apache.flink.streaming.connectors.kafka.internals.ZookeeperOffsetHandler.commit(ZookeeperOffsetHandler.java:80)
>>> >> >>>>     at
>>> >> >>>>
>>> >> >>>>
>>> org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer$PeriodicOffsetCommitter.run(FlinkKafkaConsumer.java:632)
>>> >> >>>>
>>> >> >>>>
>>> >> >>>> Any ideas on what could cause this behaviour?
>>> >> >>>>
>>> >> >>>> Best,
>>> >> >>>> Mihail
>>> >> >>>
>>> >> >>>
>>> >> >
>>> >
>>> >
>>>
>>
>>
>

Re: NPE with Flink Streaming from Kafka

Posted by Robert Metzger <rm...@apache.org>.
Its good news that the issue has been resolved.

Regarding the OOM, did you start Flink in the streaming mode?

On Wed, Dec 2, 2015 at 10:18 AM, Vieru, Mihail <mi...@zalando.de>
wrote:

> Thank you, Robert! The issue with Kafka is now solved with the
> 0.10-SNAPSHOT dependency.
>
> We have run into an OutOfMemory exception though, which appears to be
> related to the state. As my colleague, Javier Lopez, mentioned in a
> previous thread, state handling is crucial for our use case. And as the
> jobs are intended to run for months, stability plays an important role in
> choosing a stream processing framework.
>
> 12/02/2015 10:03:53    Fast TumblingTimeWindows(5000) of Reduce at
> main(ItemPriceAvgPerOrder.java:108) -> Sink: Unnamed(1/1) switched to
> FAILED
> java.lang.OutOfMemoryError: Java heap space
>     at java.util.HashMap.resize(HashMap.java:703)
>     at java.util.HashMap.putVal(HashMap.java:662)
>     at java.util.HashMap.put(HashMap.java:611)
>     at
> org.apache.flink.runtime.state.AbstractHeapKvState.update(AbstractHeapKvState.java:98)
>     at
> de.zalando.saiki.gerakanflink.ItemPriceAvgPerOrder$3.reduce(ItemPriceAvgPerOrder.java:121)
>     at
> de.zalando.saiki.gerakanflink.ItemPriceAvgPerOrder$3.reduce(ItemPriceAvgPerOrder.java:108)
>     at
> org.apache.flink.streaming.runtime.operators.windowing.KeyMap.putOrAggregate(KeyMap.java:196)
>     at
> org.apache.flink.streaming.runtime.operators.windowing.AggregatingKeyedTimePanes.addElementToLatestPane(AggregatingKeyedTimePanes.java:50)
>     at
> org.apache.flink.streaming.runtime.operators.windowing.AbstractAlignedProcessingTimeWindowOperator.processElement(AbstractAlignedProcessingTimeWindowOperator.java:210)
>     at
> org.apache.flink.streaming.runtime.io.StreamInputProcessor.processInput(StreamInputProcessor.java:166)
>     at
> org.apache.flink.streaming.runtime.tasks.OneInputStreamTask.run(OneInputStreamTask.java:63)
>     at
> org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:218)
>     at org.apache.flink.runtime.taskmanager.Task.run(Task.java:584)
>     at java.lang.Thread.run(Thread.java:745)
>
>
>
>
> 2015-12-01 17:42 GMT+01:00 Maximilian Michels <mx...@apache.org>:
>
>> Thanks! I've linked the issue in JIRA.
>>
>> On Tue, Dec 1, 2015 at 5:39 PM, Robert Metzger <rm...@apache.org>
>> wrote:
>> > I think its this one https://issues.apache.org/jira/browse/KAFKA-824
>> >
>> > On Tue, Dec 1, 2015 at 5:37 PM, Maximilian Michels <mx...@apache.org>
>> wrote:
>> >>
>> >> I know this has been fixed already but, out of curiosity, could you
>> >> point me to the Kafka JIRA issue for this
>> >> bug? From the Flink issue it looks like this is a Zookeeper version
>> >> mismatch.
>> >>
>> >> On Tue, Dec 1, 2015 at 5:16 PM, Robert Metzger <rm...@apache.org>
>> >> wrote:
>> >> > Hi Gyula,
>> >> >
>> >> > no, I didn't ;) We still deploy 0.10-SNAPSHOT versions from the
>> >> > "release-0.10" branch to Apache's maven snapshot repository.
>> >> >
>> >> >
>> >> > I don't think Mihail's code will run when he's compiling it against
>> >> > 1.0-SNAPSHOT.
>> >> >
>> >> >
>> >> > On Tue, Dec 1, 2015 at 5:13 PM, Gyula Fóra <gy...@gmail.com>
>> wrote:
>> >> >>
>> >> >> Hi,
>> >> >>
>> >> >> I think Robert meant to write setting the connector dependency to
>> >> >> 1.0-SNAPSHOT.
>> >> >>
>> >> >> Cheers,
>> >> >> Gyula
>> >> >>
>> >> >> Robert Metzger <rm...@apache.org> ezt írta (időpont: 2015. dec.
>> 1.,
>> >> >> K,
>> >> >> 17:10):
>> >> >>>
>> >> >>> Hi Mihail,
>> >> >>>
>> >> >>> the issue is actually a bug in Kafka. We have a JIRA in Flink for
>> this
>> >> >>> as
>> >> >>> well: https://issues.apache.org/jira/browse/FLINK-3067
>> >> >>>
>> >> >>> Sadly, we haven't released a fix for it yet. Flink 0.10.2 will
>> contain
>> >> >>> a
>> >> >>> fix.
>> >> >>>
>> >> >>> Since the kafka connector is not contained in the flink binary, you
>> >> >>> can
>> >> >>> just set the version in your maven pom file to 0.10-SNAPSHOT. Maven
>> >> >>> will
>> >> >>> then download the code planned for the 0.10-SNAPSHOT release.
>> >> >>>
>> >> >>> On Tue, Dec 1, 2015 at 4:54 PM, Vieru, Mihail
>> >> >>> <mi...@zalando.de>
>> >> >>> wrote:
>> >> >>>>
>> >> >>>> Hi,
>> >> >>>>
>> >> >>>> we get the following NullPointerException after ~50 minutes when
>> >> >>>> running
>> >> >>>> a streaming job with windowing and state that reads data from
>> Kafka
>> >> >>>> and
>> >> >>>> writes the result to local FS.
>> >> >>>> There are around 170 million messages to be processed, Flink
>> 0.10.1
>> >> >>>> stops at ~8 million.
>> >> >>>> Flink runs locally, started with the "start-cluster-streaming.sh"
>> >> >>>> script.
>> >> >>>>
>> >> >>>> 12/01/2015 15:06:24    Job execution switched to status RUNNING.
>> >> >>>> 12/01/2015 15:06:24    Source: Custom Source -> Map -> Map(1/1)
>> >> >>>> switched
>> >> >>>> to SCHEDULED
>> >> >>>> 12/01/2015 15:06:24    Source: Custom Source -> Map -> Map(1/1)
>> >> >>>> switched
>> >> >>>> to DEPLOYING
>> >> >>>> 12/01/2015 15:06:24    Fast TumblingTimeWindows(5000) of Reduce at
>> >> >>>> main(ItemPriceAvgPerOrder.java:108) -> Sink: Unnamed(1/1)
>> switched to
>> >> >>>> SCHEDULED
>> >> >>>> 12/01/2015 15:06:24    Fast TumblingTimeWindows(5000) of Reduce at
>> >> >>>> main(ItemPriceAvgPerOrder.java:108) -> Sink: Unnamed(1/1)
>> switched to
>> >> >>>> DEPLOYING
>> >> >>>> 12/01/2015 15:06:24    Source: Custom Source -> Map -> Map(1/1)
>> >> >>>> switched
>> >> >>>> to RUNNING
>> >> >>>> 12/01/2015 15:06:24    Fast TumblingTimeWindows(5000) of Reduce at
>> >> >>>> main(ItemPriceAvgPerOrder.java:108) -> Sink: Unnamed(1/1)
>> switched to
>> >> >>>> RUNNING
>> >> >>>> 12/01/2015 15:56:08    Fast TumblingTimeWindows(5000) of Reduce at
>> >> >>>> main(ItemPriceAvgPerOrder.java:108) -> Sink: Unnamed(1/1)
>> switched to
>> >> >>>> CANCELED
>> >> >>>> 12/01/2015 15:56:08    Source: Custom Source -> Map -> Map(1/1)
>> >> >>>> switched
>> >> >>>> to FAILED
>> >> >>>> java.lang.Exception
>> >> >>>>     at
>> >> >>>>
>> >> >>>>
>> org.apache.flink.streaming.connectors.kafka.internals.LegacyFetcher.run(LegacyFetcher.java:242)
>> >> >>>>     at
>> >> >>>>
>> >> >>>>
>> org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer.run(FlinkKafkaConsumer.java:397)
>> >> >>>>     at
>> >> >>>>
>> >> >>>>
>> org.apache.flink.streaming.api.operators.StreamSource.run(StreamSource.java:58)
>> >> >>>>     at
>> >> >>>>
>> >> >>>>
>> org.apache.flink.streaming.runtime.tasks.SourceStreamTask.run(SourceStreamTask.java:55)
>> >> >>>>     at
>> >> >>>>
>> >> >>>>
>> org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:218)
>> >> >>>>     at
>> org.apache.flink.runtime.taskmanager.Task.run(Task.java:584)
>> >> >>>>     at java.lang.Thread.run(Thread.java:745)
>> >> >>>> Caused by: java.lang.NullPointerException
>> >> >>>>     at
>> >> >>>>
>> >> >>>>
>> org.I0Itec.zkclient.ZkConnection.writeDataReturnStat(ZkConnection.java:115)
>> >> >>>>     at org.I0Itec.zkclient.ZkClient$10.call(ZkClient.java:817)
>> >> >>>>     at
>> >> >>>>
>> org.I0Itec.zkclient.ZkClient.retryUntilConnected(ZkClient.java:675)
>> >> >>>>     at
>> >> >>>>
>> org.I0Itec.zkclient.ZkClient.writeDataReturnStat(ZkClient.java:813)
>> >> >>>>     at org.I0Itec.zkclient.ZkClient.writeData(ZkClient.java:808)
>> >> >>>>     at org.I0Itec.zkclient.ZkClient.writeData(ZkClient.java:777)
>> >> >>>>     at
>> kafka.utils.ZkUtils$.updatePersistentPath(ZkUtils.scala:332)
>> >> >>>>     at kafka.utils.ZkUtils.updatePersistentPath(ZkUtils.scala)
>> >> >>>>     at
>> >> >>>>
>> >> >>>>
>> org.apache.flink.streaming.connectors.kafka.internals.ZookeeperOffsetHandler.setOffsetInZooKeeper(ZookeeperOffsetHandler.java:112)
>> >> >>>>     at
>> >> >>>>
>> >> >>>>
>> org.apache.flink.streaming.connectors.kafka.internals.ZookeeperOffsetHandler.commit(ZookeeperOffsetHandler.java:80)
>> >> >>>>     at
>> >> >>>>
>> >> >>>>
>> org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer$PeriodicOffsetCommitter.run(FlinkKafkaConsumer.java:632)
>> >> >>>>
>> >> >>>>
>> >> >>>> Any ideas on what could cause this behaviour?
>> >> >>>>
>> >> >>>> Best,
>> >> >>>> Mihail
>> >> >>>
>> >> >>>
>> >> >
>> >
>> >
>>
>
>

Re: NPE with Flink Streaming from Kafka

Posted by "Vieru, Mihail" <mi...@zalando.de>.
Thank you, Robert! The issue with Kafka is now solved with the
0.10-SNAPSHOT dependency.

We have run into an OutOfMemory exception though, which appears to be
related to the state. As my colleague, Javier Lopez, mentioned in a
previous thread, state handling is crucial for our use case. And as the
jobs are intended to run for months, stability plays an important role in
choosing a stream processing framework.

12/02/2015 10:03:53    Fast TumblingTimeWindows(5000) of Reduce at
main(ItemPriceAvgPerOrder.java:108) -> Sink: Unnamed(1/1) switched to
FAILED
java.lang.OutOfMemoryError: Java heap space
    at java.util.HashMap.resize(HashMap.java:703)
    at java.util.HashMap.putVal(HashMap.java:662)
    at java.util.HashMap.put(HashMap.java:611)
    at
org.apache.flink.runtime.state.AbstractHeapKvState.update(AbstractHeapKvState.java:98)
    at
de.zalando.saiki.gerakanflink.ItemPriceAvgPerOrder$3.reduce(ItemPriceAvgPerOrder.java:121)
    at
de.zalando.saiki.gerakanflink.ItemPriceAvgPerOrder$3.reduce(ItemPriceAvgPerOrder.java:108)
    at
org.apache.flink.streaming.runtime.operators.windowing.KeyMap.putOrAggregate(KeyMap.java:196)
    at
org.apache.flink.streaming.runtime.operators.windowing.AggregatingKeyedTimePanes.addElementToLatestPane(AggregatingKeyedTimePanes.java:50)
    at
org.apache.flink.streaming.runtime.operators.windowing.AbstractAlignedProcessingTimeWindowOperator.processElement(AbstractAlignedProcessingTimeWindowOperator.java:210)
    at
org.apache.flink.streaming.runtime.io.StreamInputProcessor.processInput(StreamInputProcessor.java:166)
    at
org.apache.flink.streaming.runtime.tasks.OneInputStreamTask.run(OneInputStreamTask.java:63)
    at
org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:218)
    at org.apache.flink.runtime.taskmanager.Task.run(Task.java:584)
    at java.lang.Thread.run(Thread.java:745)




2015-12-01 17:42 GMT+01:00 Maximilian Michels <mx...@apache.org>:

> Thanks! I've linked the issue in JIRA.
>
> On Tue, Dec 1, 2015 at 5:39 PM, Robert Metzger <rm...@apache.org>
> wrote:
> > I think its this one https://issues.apache.org/jira/browse/KAFKA-824
> >
> > On Tue, Dec 1, 2015 at 5:37 PM, Maximilian Michels <mx...@apache.org>
> wrote:
> >>
> >> I know this has been fixed already but, out of curiosity, could you
> >> point me to the Kafka JIRA issue for this
> >> bug? From the Flink issue it looks like this is a Zookeeper version
> >> mismatch.
> >>
> >> On Tue, Dec 1, 2015 at 5:16 PM, Robert Metzger <rm...@apache.org>
> >> wrote:
> >> > Hi Gyula,
> >> >
> >> > no, I didn't ;) We still deploy 0.10-SNAPSHOT versions from the
> >> > "release-0.10" branch to Apache's maven snapshot repository.
> >> >
> >> >
> >> > I don't think Mihail's code will run when he's compiling it against
> >> > 1.0-SNAPSHOT.
> >> >
> >> >
> >> > On Tue, Dec 1, 2015 at 5:13 PM, Gyula Fóra <gy...@gmail.com>
> wrote:
> >> >>
> >> >> Hi,
> >> >>
> >> >> I think Robert meant to write setting the connector dependency to
> >> >> 1.0-SNAPSHOT.
> >> >>
> >> >> Cheers,
> >> >> Gyula
> >> >>
> >> >> Robert Metzger <rm...@apache.org> ezt írta (időpont: 2015. dec.
> 1.,
> >> >> K,
> >> >> 17:10):
> >> >>>
> >> >>> Hi Mihail,
> >> >>>
> >> >>> the issue is actually a bug in Kafka. We have a JIRA in Flink for
> this
> >> >>> as
> >> >>> well: https://issues.apache.org/jira/browse/FLINK-3067
> >> >>>
> >> >>> Sadly, we haven't released a fix for it yet. Flink 0.10.2 will
> contain
> >> >>> a
> >> >>> fix.
> >> >>>
> >> >>> Since the kafka connector is not contained in the flink binary, you
> >> >>> can
> >> >>> just set the version in your maven pom file to 0.10-SNAPSHOT. Maven
> >> >>> will
> >> >>> then download the code planned for the 0.10-SNAPSHOT release.
> >> >>>
> >> >>> On Tue, Dec 1, 2015 at 4:54 PM, Vieru, Mihail
> >> >>> <mi...@zalando.de>
> >> >>> wrote:
> >> >>>>
> >> >>>> Hi,
> >> >>>>
> >> >>>> we get the following NullPointerException after ~50 minutes when
> >> >>>> running
> >> >>>> a streaming job with windowing and state that reads data from Kafka
> >> >>>> and
> >> >>>> writes the result to local FS.
> >> >>>> There are around 170 million messages to be processed, Flink 0.10.1
> >> >>>> stops at ~8 million.
> >> >>>> Flink runs locally, started with the "start-cluster-streaming.sh"
> >> >>>> script.
> >> >>>>
> >> >>>> 12/01/2015 15:06:24    Job execution switched to status RUNNING.
> >> >>>> 12/01/2015 15:06:24    Source: Custom Source -> Map -> Map(1/1)
> >> >>>> switched
> >> >>>> to SCHEDULED
> >> >>>> 12/01/2015 15:06:24    Source: Custom Source -> Map -> Map(1/1)
> >> >>>> switched
> >> >>>> to DEPLOYING
> >> >>>> 12/01/2015 15:06:24    Fast TumblingTimeWindows(5000) of Reduce at
> >> >>>> main(ItemPriceAvgPerOrder.java:108) -> Sink: Unnamed(1/1) switched
> to
> >> >>>> SCHEDULED
> >> >>>> 12/01/2015 15:06:24    Fast TumblingTimeWindows(5000) of Reduce at
> >> >>>> main(ItemPriceAvgPerOrder.java:108) -> Sink: Unnamed(1/1) switched
> to
> >> >>>> DEPLOYING
> >> >>>> 12/01/2015 15:06:24    Source: Custom Source -> Map -> Map(1/1)
> >> >>>> switched
> >> >>>> to RUNNING
> >> >>>> 12/01/2015 15:06:24    Fast TumblingTimeWindows(5000) of Reduce at
> >> >>>> main(ItemPriceAvgPerOrder.java:108) -> Sink: Unnamed(1/1) switched
> to
> >> >>>> RUNNING
> >> >>>> 12/01/2015 15:56:08    Fast TumblingTimeWindows(5000) of Reduce at
> >> >>>> main(ItemPriceAvgPerOrder.java:108) -> Sink: Unnamed(1/1) switched
> to
> >> >>>> CANCELED
> >> >>>> 12/01/2015 15:56:08    Source: Custom Source -> Map -> Map(1/1)
> >> >>>> switched
> >> >>>> to FAILED
> >> >>>> java.lang.Exception
> >> >>>>     at
> >> >>>>
> >> >>>>
> org.apache.flink.streaming.connectors.kafka.internals.LegacyFetcher.run(LegacyFetcher.java:242)
> >> >>>>     at
> >> >>>>
> >> >>>>
> org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer.run(FlinkKafkaConsumer.java:397)
> >> >>>>     at
> >> >>>>
> >> >>>>
> org.apache.flink.streaming.api.operators.StreamSource.run(StreamSource.java:58)
> >> >>>>     at
> >> >>>>
> >> >>>>
> org.apache.flink.streaming.runtime.tasks.SourceStreamTask.run(SourceStreamTask.java:55)
> >> >>>>     at
> >> >>>>
> >> >>>>
> org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:218)
> >> >>>>     at org.apache.flink.runtime.taskmanager.Task.run(Task.java:584)
> >> >>>>     at java.lang.Thread.run(Thread.java:745)
> >> >>>> Caused by: java.lang.NullPointerException
> >> >>>>     at
> >> >>>>
> >> >>>>
> org.I0Itec.zkclient.ZkConnection.writeDataReturnStat(ZkConnection.java:115)
> >> >>>>     at org.I0Itec.zkclient.ZkClient$10.call(ZkClient.java:817)
> >> >>>>     at
> >> >>>> org.I0Itec.zkclient.ZkClient.retryUntilConnected(ZkClient.java:675)
> >> >>>>     at
> >> >>>> org.I0Itec.zkclient.ZkClient.writeDataReturnStat(ZkClient.java:813)
> >> >>>>     at org.I0Itec.zkclient.ZkClient.writeData(ZkClient.java:808)
> >> >>>>     at org.I0Itec.zkclient.ZkClient.writeData(ZkClient.java:777)
> >> >>>>     at kafka.utils.ZkUtils$.updatePersistentPath(ZkUtils.scala:332)
> >> >>>>     at kafka.utils.ZkUtils.updatePersistentPath(ZkUtils.scala)
> >> >>>>     at
> >> >>>>
> >> >>>>
> org.apache.flink.streaming.connectors.kafka.internals.ZookeeperOffsetHandler.setOffsetInZooKeeper(ZookeeperOffsetHandler.java:112)
> >> >>>>     at
> >> >>>>
> >> >>>>
> org.apache.flink.streaming.connectors.kafka.internals.ZookeeperOffsetHandler.commit(ZookeeperOffsetHandler.java:80)
> >> >>>>     at
> >> >>>>
> >> >>>>
> org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer$PeriodicOffsetCommitter.run(FlinkKafkaConsumer.java:632)
> >> >>>>
> >> >>>>
> >> >>>> Any ideas on what could cause this behaviour?
> >> >>>>
> >> >>>> Best,
> >> >>>> Mihail
> >> >>>
> >> >>>
> >> >
> >
> >
>

Re: NPE with Flink Streaming from Kafka

Posted by Maximilian Michels <mx...@apache.org>.
Thanks! I've linked the issue in JIRA.

On Tue, Dec 1, 2015 at 5:39 PM, Robert Metzger <rm...@apache.org> wrote:
> I think its this one https://issues.apache.org/jira/browse/KAFKA-824
>
> On Tue, Dec 1, 2015 at 5:37 PM, Maximilian Michels <mx...@apache.org> wrote:
>>
>> I know this has been fixed already but, out of curiosity, could you
>> point me to the Kafka JIRA issue for this
>> bug? From the Flink issue it looks like this is a Zookeeper version
>> mismatch.
>>
>> On Tue, Dec 1, 2015 at 5:16 PM, Robert Metzger <rm...@apache.org>
>> wrote:
>> > Hi Gyula,
>> >
>> > no, I didn't ;) We still deploy 0.10-SNAPSHOT versions from the
>> > "release-0.10" branch to Apache's maven snapshot repository.
>> >
>> >
>> > I don't think Mihail's code will run when he's compiling it against
>> > 1.0-SNAPSHOT.
>> >
>> >
>> > On Tue, Dec 1, 2015 at 5:13 PM, Gyula Fóra <gy...@gmail.com> wrote:
>> >>
>> >> Hi,
>> >>
>> >> I think Robert meant to write setting the connector dependency to
>> >> 1.0-SNAPSHOT.
>> >>
>> >> Cheers,
>> >> Gyula
>> >>
>> >> Robert Metzger <rm...@apache.org> ezt írta (időpont: 2015. dec. 1.,
>> >> K,
>> >> 17:10):
>> >>>
>> >>> Hi Mihail,
>> >>>
>> >>> the issue is actually a bug in Kafka. We have a JIRA in Flink for this
>> >>> as
>> >>> well: https://issues.apache.org/jira/browse/FLINK-3067
>> >>>
>> >>> Sadly, we haven't released a fix for it yet. Flink 0.10.2 will contain
>> >>> a
>> >>> fix.
>> >>>
>> >>> Since the kafka connector is not contained in the flink binary, you
>> >>> can
>> >>> just set the version in your maven pom file to 0.10-SNAPSHOT. Maven
>> >>> will
>> >>> then download the code planned for the 0.10-SNAPSHOT release.
>> >>>
>> >>> On Tue, Dec 1, 2015 at 4:54 PM, Vieru, Mihail
>> >>> <mi...@zalando.de>
>> >>> wrote:
>> >>>>
>> >>>> Hi,
>> >>>>
>> >>>> we get the following NullPointerException after ~50 minutes when
>> >>>> running
>> >>>> a streaming job with windowing and state that reads data from Kafka
>> >>>> and
>> >>>> writes the result to local FS.
>> >>>> There are around 170 million messages to be processed, Flink 0.10.1
>> >>>> stops at ~8 million.
>> >>>> Flink runs locally, started with the "start-cluster-streaming.sh"
>> >>>> script.
>> >>>>
>> >>>> 12/01/2015 15:06:24    Job execution switched to status RUNNING.
>> >>>> 12/01/2015 15:06:24    Source: Custom Source -> Map -> Map(1/1)
>> >>>> switched
>> >>>> to SCHEDULED
>> >>>> 12/01/2015 15:06:24    Source: Custom Source -> Map -> Map(1/1)
>> >>>> switched
>> >>>> to DEPLOYING
>> >>>> 12/01/2015 15:06:24    Fast TumblingTimeWindows(5000) of Reduce at
>> >>>> main(ItemPriceAvgPerOrder.java:108) -> Sink: Unnamed(1/1) switched to
>> >>>> SCHEDULED
>> >>>> 12/01/2015 15:06:24    Fast TumblingTimeWindows(5000) of Reduce at
>> >>>> main(ItemPriceAvgPerOrder.java:108) -> Sink: Unnamed(1/1) switched to
>> >>>> DEPLOYING
>> >>>> 12/01/2015 15:06:24    Source: Custom Source -> Map -> Map(1/1)
>> >>>> switched
>> >>>> to RUNNING
>> >>>> 12/01/2015 15:06:24    Fast TumblingTimeWindows(5000) of Reduce at
>> >>>> main(ItemPriceAvgPerOrder.java:108) -> Sink: Unnamed(1/1) switched to
>> >>>> RUNNING
>> >>>> 12/01/2015 15:56:08    Fast TumblingTimeWindows(5000) of Reduce at
>> >>>> main(ItemPriceAvgPerOrder.java:108) -> Sink: Unnamed(1/1) switched to
>> >>>> CANCELED
>> >>>> 12/01/2015 15:56:08    Source: Custom Source -> Map -> Map(1/1)
>> >>>> switched
>> >>>> to FAILED
>> >>>> java.lang.Exception
>> >>>>     at
>> >>>>
>> >>>> org.apache.flink.streaming.connectors.kafka.internals.LegacyFetcher.run(LegacyFetcher.java:242)
>> >>>>     at
>> >>>>
>> >>>> org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer.run(FlinkKafkaConsumer.java:397)
>> >>>>     at
>> >>>>
>> >>>> org.apache.flink.streaming.api.operators.StreamSource.run(StreamSource.java:58)
>> >>>>     at
>> >>>>
>> >>>> org.apache.flink.streaming.runtime.tasks.SourceStreamTask.run(SourceStreamTask.java:55)
>> >>>>     at
>> >>>>
>> >>>> org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:218)
>> >>>>     at org.apache.flink.runtime.taskmanager.Task.run(Task.java:584)
>> >>>>     at java.lang.Thread.run(Thread.java:745)
>> >>>> Caused by: java.lang.NullPointerException
>> >>>>     at
>> >>>>
>> >>>> org.I0Itec.zkclient.ZkConnection.writeDataReturnStat(ZkConnection.java:115)
>> >>>>     at org.I0Itec.zkclient.ZkClient$10.call(ZkClient.java:817)
>> >>>>     at
>> >>>> org.I0Itec.zkclient.ZkClient.retryUntilConnected(ZkClient.java:675)
>> >>>>     at
>> >>>> org.I0Itec.zkclient.ZkClient.writeDataReturnStat(ZkClient.java:813)
>> >>>>     at org.I0Itec.zkclient.ZkClient.writeData(ZkClient.java:808)
>> >>>>     at org.I0Itec.zkclient.ZkClient.writeData(ZkClient.java:777)
>> >>>>     at kafka.utils.ZkUtils$.updatePersistentPath(ZkUtils.scala:332)
>> >>>>     at kafka.utils.ZkUtils.updatePersistentPath(ZkUtils.scala)
>> >>>>     at
>> >>>>
>> >>>> org.apache.flink.streaming.connectors.kafka.internals.ZookeeperOffsetHandler.setOffsetInZooKeeper(ZookeeperOffsetHandler.java:112)
>> >>>>     at
>> >>>>
>> >>>> org.apache.flink.streaming.connectors.kafka.internals.ZookeeperOffsetHandler.commit(ZookeeperOffsetHandler.java:80)
>> >>>>     at
>> >>>>
>> >>>> org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer$PeriodicOffsetCommitter.run(FlinkKafkaConsumer.java:632)
>> >>>>
>> >>>>
>> >>>> Any ideas on what could cause this behaviour?
>> >>>>
>> >>>> Best,
>> >>>> Mihail
>> >>>
>> >>>
>> >
>
>

Re: NPE with Flink Streaming from Kafka

Posted by Robert Metzger <rm...@apache.org>.
I think its this one https://issues.apache.org/jira/browse/KAFKA-824

On Tue, Dec 1, 2015 at 5:37 PM, Maximilian Michels <mx...@apache.org> wrote:

> I know this has been fixed already but, out of curiosity, could you
> point me to the Kafka JIRA issue for this
> bug? From the Flink issue it looks like this is a Zookeeper version
> mismatch.
>
> On Tue, Dec 1, 2015 at 5:16 PM, Robert Metzger <rm...@apache.org>
> wrote:
> > Hi Gyula,
> >
> > no, I didn't ;) We still deploy 0.10-SNAPSHOT versions from the
> > "release-0.10" branch to Apache's maven snapshot repository.
> >
> >
> > I don't think Mihail's code will run when he's compiling it against
> > 1.0-SNAPSHOT.
> >
> >
> > On Tue, Dec 1, 2015 at 5:13 PM, Gyula Fóra <gy...@gmail.com> wrote:
> >>
> >> Hi,
> >>
> >> I think Robert meant to write setting the connector dependency to
> >> 1.0-SNAPSHOT.
> >>
> >> Cheers,
> >> Gyula
> >>
> >> Robert Metzger <rm...@apache.org> ezt írta (időpont: 2015. dec. 1.,
> K,
> >> 17:10):
> >>>
> >>> Hi Mihail,
> >>>
> >>> the issue is actually a bug in Kafka. We have a JIRA in Flink for this
> as
> >>> well: https://issues.apache.org/jira/browse/FLINK-3067
> >>>
> >>> Sadly, we haven't released a fix for it yet. Flink 0.10.2 will contain
> a
> >>> fix.
> >>>
> >>> Since the kafka connector is not contained in the flink binary, you can
> >>> just set the version in your maven pom file to 0.10-SNAPSHOT. Maven
> will
> >>> then download the code planned for the 0.10-SNAPSHOT release.
> >>>
> >>> On Tue, Dec 1, 2015 at 4:54 PM, Vieru, Mihail <mihail.vieru@zalando.de
> >
> >>> wrote:
> >>>>
> >>>> Hi,
> >>>>
> >>>> we get the following NullPointerException after ~50 minutes when
> running
> >>>> a streaming job with windowing and state that reads data from Kafka
> and
> >>>> writes the result to local FS.
> >>>> There are around 170 million messages to be processed, Flink 0.10.1
> >>>> stops at ~8 million.
> >>>> Flink runs locally, started with the "start-cluster-streaming.sh"
> >>>> script.
> >>>>
> >>>> 12/01/2015 15:06:24    Job execution switched to status RUNNING.
> >>>> 12/01/2015 15:06:24    Source: Custom Source -> Map -> Map(1/1)
> switched
> >>>> to SCHEDULED
> >>>> 12/01/2015 15:06:24    Source: Custom Source -> Map -> Map(1/1)
> switched
> >>>> to DEPLOYING
> >>>> 12/01/2015 15:06:24    Fast TumblingTimeWindows(5000) of Reduce at
> >>>> main(ItemPriceAvgPerOrder.java:108) -> Sink: Unnamed(1/1) switched to
> >>>> SCHEDULED
> >>>> 12/01/2015 15:06:24    Fast TumblingTimeWindows(5000) of Reduce at
> >>>> main(ItemPriceAvgPerOrder.java:108) -> Sink: Unnamed(1/1) switched to
> >>>> DEPLOYING
> >>>> 12/01/2015 15:06:24    Source: Custom Source -> Map -> Map(1/1)
> switched
> >>>> to RUNNING
> >>>> 12/01/2015 15:06:24    Fast TumblingTimeWindows(5000) of Reduce at
> >>>> main(ItemPriceAvgPerOrder.java:108) -> Sink: Unnamed(1/1) switched to
> >>>> RUNNING
> >>>> 12/01/2015 15:56:08    Fast TumblingTimeWindows(5000) of Reduce at
> >>>> main(ItemPriceAvgPerOrder.java:108) -> Sink: Unnamed(1/1) switched to
> >>>> CANCELED
> >>>> 12/01/2015 15:56:08    Source: Custom Source -> Map -> Map(1/1)
> switched
> >>>> to FAILED
> >>>> java.lang.Exception
> >>>>     at
> >>>>
> org.apache.flink.streaming.connectors.kafka.internals.LegacyFetcher.run(LegacyFetcher.java:242)
> >>>>     at
> >>>>
> org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer.run(FlinkKafkaConsumer.java:397)
> >>>>     at
> >>>>
> org.apache.flink.streaming.api.operators.StreamSource.run(StreamSource.java:58)
> >>>>     at
> >>>>
> org.apache.flink.streaming.runtime.tasks.SourceStreamTask.run(SourceStreamTask.java:55)
> >>>>     at
> >>>>
> org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:218)
> >>>>     at org.apache.flink.runtime.taskmanager.Task.run(Task.java:584)
> >>>>     at java.lang.Thread.run(Thread.java:745)
> >>>> Caused by: java.lang.NullPointerException
> >>>>     at
> >>>>
> org.I0Itec.zkclient.ZkConnection.writeDataReturnStat(ZkConnection.java:115)
> >>>>     at org.I0Itec.zkclient.ZkClient$10.call(ZkClient.java:817)
> >>>>     at
> >>>> org.I0Itec.zkclient.ZkClient.retryUntilConnected(ZkClient.java:675)
> >>>>     at
> >>>> org.I0Itec.zkclient.ZkClient.writeDataReturnStat(ZkClient.java:813)
> >>>>     at org.I0Itec.zkclient.ZkClient.writeData(ZkClient.java:808)
> >>>>     at org.I0Itec.zkclient.ZkClient.writeData(ZkClient.java:777)
> >>>>     at kafka.utils.ZkUtils$.updatePersistentPath(ZkUtils.scala:332)
> >>>>     at kafka.utils.ZkUtils.updatePersistentPath(ZkUtils.scala)
> >>>>     at
> >>>>
> org.apache.flink.streaming.connectors.kafka.internals.ZookeeperOffsetHandler.setOffsetInZooKeeper(ZookeeperOffsetHandler.java:112)
> >>>>     at
> >>>>
> org.apache.flink.streaming.connectors.kafka.internals.ZookeeperOffsetHandler.commit(ZookeeperOffsetHandler.java:80)
> >>>>     at
> >>>>
> org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer$PeriodicOffsetCommitter.run(FlinkKafkaConsumer.java:632)
> >>>>
> >>>>
> >>>> Any ideas on what could cause this behaviour?
> >>>>
> >>>> Best,
> >>>> Mihail
> >>>
> >>>
> >
>

Re: NPE with Flink Streaming from Kafka

Posted by Maximilian Michels <mx...@apache.org>.
I know this has been fixed already but, out of curiosity, could you
point me to the Kafka JIRA issue for this
bug? From the Flink issue it looks like this is a Zookeeper version mismatch.

On Tue, Dec 1, 2015 at 5:16 PM, Robert Metzger <rm...@apache.org> wrote:
> Hi Gyula,
>
> no, I didn't ;) We still deploy 0.10-SNAPSHOT versions from the
> "release-0.10" branch to Apache's maven snapshot repository.
>
>
> I don't think Mihail's code will run when he's compiling it against
> 1.0-SNAPSHOT.
>
>
> On Tue, Dec 1, 2015 at 5:13 PM, Gyula Fóra <gy...@gmail.com> wrote:
>>
>> Hi,
>>
>> I think Robert meant to write setting the connector dependency to
>> 1.0-SNAPSHOT.
>>
>> Cheers,
>> Gyula
>>
>> Robert Metzger <rm...@apache.org> ezt írta (időpont: 2015. dec. 1., K,
>> 17:10):
>>>
>>> Hi Mihail,
>>>
>>> the issue is actually a bug in Kafka. We have a JIRA in Flink for this as
>>> well: https://issues.apache.org/jira/browse/FLINK-3067
>>>
>>> Sadly, we haven't released a fix for it yet. Flink 0.10.2 will contain a
>>> fix.
>>>
>>> Since the kafka connector is not contained in the flink binary, you can
>>> just set the version in your maven pom file to 0.10-SNAPSHOT. Maven will
>>> then download the code planned for the 0.10-SNAPSHOT release.
>>>
>>> On Tue, Dec 1, 2015 at 4:54 PM, Vieru, Mihail <mi...@zalando.de>
>>> wrote:
>>>>
>>>> Hi,
>>>>
>>>> we get the following NullPointerException after ~50 minutes when running
>>>> a streaming job with windowing and state that reads data from Kafka and
>>>> writes the result to local FS.
>>>> There are around 170 million messages to be processed, Flink 0.10.1
>>>> stops at ~8 million.
>>>> Flink runs locally, started with the "start-cluster-streaming.sh"
>>>> script.
>>>>
>>>> 12/01/2015 15:06:24    Job execution switched to status RUNNING.
>>>> 12/01/2015 15:06:24    Source: Custom Source -> Map -> Map(1/1) switched
>>>> to SCHEDULED
>>>> 12/01/2015 15:06:24    Source: Custom Source -> Map -> Map(1/1) switched
>>>> to DEPLOYING
>>>> 12/01/2015 15:06:24    Fast TumblingTimeWindows(5000) of Reduce at
>>>> main(ItemPriceAvgPerOrder.java:108) -> Sink: Unnamed(1/1) switched to
>>>> SCHEDULED
>>>> 12/01/2015 15:06:24    Fast TumblingTimeWindows(5000) of Reduce at
>>>> main(ItemPriceAvgPerOrder.java:108) -> Sink: Unnamed(1/1) switched to
>>>> DEPLOYING
>>>> 12/01/2015 15:06:24    Source: Custom Source -> Map -> Map(1/1) switched
>>>> to RUNNING
>>>> 12/01/2015 15:06:24    Fast TumblingTimeWindows(5000) of Reduce at
>>>> main(ItemPriceAvgPerOrder.java:108) -> Sink: Unnamed(1/1) switched to
>>>> RUNNING
>>>> 12/01/2015 15:56:08    Fast TumblingTimeWindows(5000) of Reduce at
>>>> main(ItemPriceAvgPerOrder.java:108) -> Sink: Unnamed(1/1) switched to
>>>> CANCELED
>>>> 12/01/2015 15:56:08    Source: Custom Source -> Map -> Map(1/1) switched
>>>> to FAILED
>>>> java.lang.Exception
>>>>     at
>>>> org.apache.flink.streaming.connectors.kafka.internals.LegacyFetcher.run(LegacyFetcher.java:242)
>>>>     at
>>>> org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer.run(FlinkKafkaConsumer.java:397)
>>>>     at
>>>> org.apache.flink.streaming.api.operators.StreamSource.run(StreamSource.java:58)
>>>>     at
>>>> org.apache.flink.streaming.runtime.tasks.SourceStreamTask.run(SourceStreamTask.java:55)
>>>>     at
>>>> org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:218)
>>>>     at org.apache.flink.runtime.taskmanager.Task.run(Task.java:584)
>>>>     at java.lang.Thread.run(Thread.java:745)
>>>> Caused by: java.lang.NullPointerException
>>>>     at
>>>> org.I0Itec.zkclient.ZkConnection.writeDataReturnStat(ZkConnection.java:115)
>>>>     at org.I0Itec.zkclient.ZkClient$10.call(ZkClient.java:817)
>>>>     at
>>>> org.I0Itec.zkclient.ZkClient.retryUntilConnected(ZkClient.java:675)
>>>>     at
>>>> org.I0Itec.zkclient.ZkClient.writeDataReturnStat(ZkClient.java:813)
>>>>     at org.I0Itec.zkclient.ZkClient.writeData(ZkClient.java:808)
>>>>     at org.I0Itec.zkclient.ZkClient.writeData(ZkClient.java:777)
>>>>     at kafka.utils.ZkUtils$.updatePersistentPath(ZkUtils.scala:332)
>>>>     at kafka.utils.ZkUtils.updatePersistentPath(ZkUtils.scala)
>>>>     at
>>>> org.apache.flink.streaming.connectors.kafka.internals.ZookeeperOffsetHandler.setOffsetInZooKeeper(ZookeeperOffsetHandler.java:112)
>>>>     at
>>>> org.apache.flink.streaming.connectors.kafka.internals.ZookeeperOffsetHandler.commit(ZookeeperOffsetHandler.java:80)
>>>>     at
>>>> org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer$PeriodicOffsetCommitter.run(FlinkKafkaConsumer.java:632)
>>>>
>>>>
>>>> Any ideas on what could cause this behaviour?
>>>>
>>>> Best,
>>>> Mihail
>>>
>>>
>

Re: NPE with Flink Streaming from Kafka

Posted by Robert Metzger <rm...@apache.org>.
Hi Gyula,

no, I didn't ;) We still deploy 0.10-SNAPSHOT versions from the
"release-0.10" branch to Apache's maven snapshot repository.


I don't think Mihail's code will run when he's compiling it against
1.0-SNAPSHOT.


On Tue, Dec 1, 2015 at 5:13 PM, Gyula Fóra <gy...@gmail.com> wrote:

> Hi,
>
> I think Robert meant to write setting the connector dependency to
> 1.0-SNAPSHOT.
>
> Cheers,
> Gyula
>
> Robert Metzger <rm...@apache.org> ezt írta (időpont: 2015. dec. 1., K,
> 17:10):
>
>> Hi Mihail,
>>
>> the issue is actually a bug in Kafka. We have a JIRA in Flink for this as
>> well: https://issues.apache.org/jira/browse/FLINK-3067
>>
>> Sadly, we haven't released a fix for it yet. Flink 0.10.2 will contain a
>> fix.
>>
>> Since the kafka connector is not contained in the flink binary, you can
>> just set the version in your maven pom file to 0.10-SNAPSHOT. Maven will
>> then download the code planned for the 0.10-SNAPSHOT release.
>>
>> On Tue, Dec 1, 2015 at 4:54 PM, Vieru, Mihail <mi...@zalando.de>
>> wrote:
>>
>>> Hi,
>>>
>>> we get the following NullPointerException after ~50 minutes when running
>>> a streaming job with windowing and state that reads data from Kafka and
>>> writes the result to local FS.
>>> There are around 170 million messages to be processed, Flink 0.10.1
>>> stops at ~8 million.
>>> Flink runs locally, started with the "start-cluster-streaming.sh" script.
>>>
>>> 12/01/2015 15:06:24    Job execution switched to status RUNNING.
>>> 12/01/2015 15:06:24    Source: Custom Source -> Map -> Map(1/1) switched
>>> to SCHEDULED
>>> 12/01/2015 15:06:24    Source: Custom Source -> Map -> Map(1/1) switched
>>> to DEPLOYING
>>> 12/01/2015 15:06:24    Fast TumblingTimeWindows(5000) of Reduce at
>>> main(ItemPriceAvgPerOrder.java:108) -> Sink: Unnamed(1/1) switched to
>>> SCHEDULED
>>> 12/01/2015 15:06:24    Fast TumblingTimeWindows(5000) of Reduce at
>>> main(ItemPriceAvgPerOrder.java:108) -> Sink: Unnamed(1/1) switched to
>>> DEPLOYING
>>> 12/01/2015 15:06:24    Source: Custom Source -> Map -> Map(1/1) switched
>>> to RUNNING
>>> 12/01/2015 15:06:24    Fast TumblingTimeWindows(5000) of Reduce at
>>> main(ItemPriceAvgPerOrder.java:108) -> Sink: Unnamed(1/1) switched to
>>> RUNNING
>>> 12/01/2015 15:56:08    Fast TumblingTimeWindows(5000) of Reduce at
>>> main(ItemPriceAvgPerOrder.java:108) -> Sink: Unnamed(1/1) switched to
>>> CANCELED
>>> 12/01/2015 15:56:08    Source: Custom Source -> Map -> Map(1/1) switched
>>> to FAILED
>>> java.lang.Exception
>>>     at
>>> org.apache.flink.streaming.connectors.kafka.internals.LegacyFetcher.run(LegacyFetcher.java:242)
>>>     at
>>> org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer.run(FlinkKafkaConsumer.java:397)
>>>     at
>>> org.apache.flink.streaming.api.operators.StreamSource.run(StreamSource.java:58)
>>>     at
>>> org.apache.flink.streaming.runtime.tasks.SourceStreamTask.run(SourceStreamTask.java:55)
>>>     at
>>> org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:218)
>>>     at org.apache.flink.runtime.taskmanager.Task.run(Task.java:584)
>>>     at java.lang.Thread.run(Thread.java:745)
>>> Caused by: java.lang.NullPointerException
>>>     at
>>> org.I0Itec.zkclient.ZkConnection.writeDataReturnStat(ZkConnection.java:115)
>>>     at org.I0Itec.zkclient.ZkClient$10.call(ZkClient.java:817)
>>>     at
>>> org.I0Itec.zkclient.ZkClient.retryUntilConnected(ZkClient.java:675)
>>>     at
>>> org.I0Itec.zkclient.ZkClient.writeDataReturnStat(ZkClient.java:813)
>>>     at org.I0Itec.zkclient.ZkClient.writeData(ZkClient.java:808)
>>>     at org.I0Itec.zkclient.ZkClient.writeData(ZkClient.java:777)
>>>     at kafka.utils.ZkUtils$.updatePersistentPath(ZkUtils.scala:332)
>>>     at kafka.utils.ZkUtils.updatePersistentPath(ZkUtils.scala)
>>>     at
>>> org.apache.flink.streaming.connectors.kafka.internals.ZookeeperOffsetHandler.setOffsetInZooKeeper(ZookeeperOffsetHandler.java:112)
>>>     at
>>> org.apache.flink.streaming.connectors.kafka.internals.ZookeeperOffsetHandler.commit(ZookeeperOffsetHandler.java:80)
>>>     at
>>> org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer$PeriodicOffsetCommitter.run(FlinkKafkaConsumer.java:632)
>>>
>>>
>>> Any ideas on what could cause this behaviour?
>>>
>>> Best,
>>> Mihail
>>>
>>
>>

Re: NPE with Flink Streaming from Kafka

Posted by Gyula Fóra <gy...@gmail.com>.
Hi,

I think Robert meant to write setting the connector dependency to
1.0-SNAPSHOT.

Cheers,
Gyula

Robert Metzger <rm...@apache.org> ezt írta (időpont: 2015. dec. 1., K,
17:10):

> Hi Mihail,
>
> the issue is actually a bug in Kafka. We have a JIRA in Flink for this as
> well: https://issues.apache.org/jira/browse/FLINK-3067
>
> Sadly, we haven't released a fix for it yet. Flink 0.10.2 will contain a
> fix.
>
> Since the kafka connector is not contained in the flink binary, you can
> just set the version in your maven pom file to 0.10-SNAPSHOT. Maven will
> then download the code planned for the 0.10-SNAPSHOT release.
>
> On Tue, Dec 1, 2015 at 4:54 PM, Vieru, Mihail <mi...@zalando.de>
> wrote:
>
>> Hi,
>>
>> we get the following NullPointerException after ~50 minutes when running
>> a streaming job with windowing and state that reads data from Kafka and
>> writes the result to local FS.
>> There are around 170 million messages to be processed, Flink 0.10.1 stops
>> at ~8 million.
>> Flink runs locally, started with the "start-cluster-streaming.sh" script.
>>
>> 12/01/2015 15:06:24    Job execution switched to status RUNNING.
>> 12/01/2015 15:06:24    Source: Custom Source -> Map -> Map(1/1) switched
>> to SCHEDULED
>> 12/01/2015 15:06:24    Source: Custom Source -> Map -> Map(1/1) switched
>> to DEPLOYING
>> 12/01/2015 15:06:24    Fast TumblingTimeWindows(5000) of Reduce at
>> main(ItemPriceAvgPerOrder.java:108) -> Sink: Unnamed(1/1) switched to
>> SCHEDULED
>> 12/01/2015 15:06:24    Fast TumblingTimeWindows(5000) of Reduce at
>> main(ItemPriceAvgPerOrder.java:108) -> Sink: Unnamed(1/1) switched to
>> DEPLOYING
>> 12/01/2015 15:06:24    Source: Custom Source -> Map -> Map(1/1) switched
>> to RUNNING
>> 12/01/2015 15:06:24    Fast TumblingTimeWindows(5000) of Reduce at
>> main(ItemPriceAvgPerOrder.java:108) -> Sink: Unnamed(1/1) switched to
>> RUNNING
>> 12/01/2015 15:56:08    Fast TumblingTimeWindows(5000) of Reduce at
>> main(ItemPriceAvgPerOrder.java:108) -> Sink: Unnamed(1/1) switched to
>> CANCELED
>> 12/01/2015 15:56:08    Source: Custom Source -> Map -> Map(1/1) switched
>> to FAILED
>> java.lang.Exception
>>     at
>> org.apache.flink.streaming.connectors.kafka.internals.LegacyFetcher.run(LegacyFetcher.java:242)
>>     at
>> org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer.run(FlinkKafkaConsumer.java:397)
>>     at
>> org.apache.flink.streaming.api.operators.StreamSource.run(StreamSource.java:58)
>>     at
>> org.apache.flink.streaming.runtime.tasks.SourceStreamTask.run(SourceStreamTask.java:55)
>>     at
>> org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:218)
>>     at org.apache.flink.runtime.taskmanager.Task.run(Task.java:584)
>>     at java.lang.Thread.run(Thread.java:745)
>> Caused by: java.lang.NullPointerException
>>     at
>> org.I0Itec.zkclient.ZkConnection.writeDataReturnStat(ZkConnection.java:115)
>>     at org.I0Itec.zkclient.ZkClient$10.call(ZkClient.java:817)
>>     at org.I0Itec.zkclient.ZkClient.retryUntilConnected(ZkClient.java:675)
>>     at org.I0Itec.zkclient.ZkClient.writeDataReturnStat(ZkClient.java:813)
>>     at org.I0Itec.zkclient.ZkClient.writeData(ZkClient.java:808)
>>     at org.I0Itec.zkclient.ZkClient.writeData(ZkClient.java:777)
>>     at kafka.utils.ZkUtils$.updatePersistentPath(ZkUtils.scala:332)
>>     at kafka.utils.ZkUtils.updatePersistentPath(ZkUtils.scala)
>>     at
>> org.apache.flink.streaming.connectors.kafka.internals.ZookeeperOffsetHandler.setOffsetInZooKeeper(ZookeeperOffsetHandler.java:112)
>>     at
>> org.apache.flink.streaming.connectors.kafka.internals.ZookeeperOffsetHandler.commit(ZookeeperOffsetHandler.java:80)
>>     at
>> org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer$PeriodicOffsetCommitter.run(FlinkKafkaConsumer.java:632)
>>
>>
>> Any ideas on what could cause this behaviour?
>>
>> Best,
>> Mihail
>>
>
>

Re: NPE with Flink Streaming from Kafka

Posted by Robert Metzger <rm...@apache.org>.
Hi Mihail,

the issue is actually a bug in Kafka. We have a JIRA in Flink for this as
well: https://issues.apache.org/jira/browse/FLINK-3067

Sadly, we haven't released a fix for it yet. Flink 0.10.2 will contain a
fix.

Since the kafka connector is not contained in the flink binary, you can
just set the version in your maven pom file to 0.10-SNAPSHOT. Maven will
then download the code planned for the 0.10-SNAPSHOT release.

On Tue, Dec 1, 2015 at 4:54 PM, Vieru, Mihail <mi...@zalando.de>
wrote:

> Hi,
>
> we get the following NullPointerException after ~50 minutes when running a
> streaming job with windowing and state that reads data from Kafka and
> writes the result to local FS.
> There are around 170 million messages to be processed, Flink 0.10.1 stops
> at ~8 million.
> Flink runs locally, started with the "start-cluster-streaming.sh" script.
>
> 12/01/2015 15:06:24    Job execution switched to status RUNNING.
> 12/01/2015 15:06:24    Source: Custom Source -> Map -> Map(1/1) switched
> to SCHEDULED
> 12/01/2015 15:06:24    Source: Custom Source -> Map -> Map(1/1) switched
> to DEPLOYING
> 12/01/2015 15:06:24    Fast TumblingTimeWindows(5000) of Reduce at
> main(ItemPriceAvgPerOrder.java:108) -> Sink: Unnamed(1/1) switched to
> SCHEDULED
> 12/01/2015 15:06:24    Fast TumblingTimeWindows(5000) of Reduce at
> main(ItemPriceAvgPerOrder.java:108) -> Sink: Unnamed(1/1) switched to
> DEPLOYING
> 12/01/2015 15:06:24    Source: Custom Source -> Map -> Map(1/1) switched
> to RUNNING
> 12/01/2015 15:06:24    Fast TumblingTimeWindows(5000) of Reduce at
> main(ItemPriceAvgPerOrder.java:108) -> Sink: Unnamed(1/1) switched to
> RUNNING
> 12/01/2015 15:56:08    Fast TumblingTimeWindows(5000) of Reduce at
> main(ItemPriceAvgPerOrder.java:108) -> Sink: Unnamed(1/1) switched to
> CANCELED
> 12/01/2015 15:56:08    Source: Custom Source -> Map -> Map(1/1) switched
> to FAILED
> java.lang.Exception
>     at
> org.apache.flink.streaming.connectors.kafka.internals.LegacyFetcher.run(LegacyFetcher.java:242)
>     at
> org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer.run(FlinkKafkaConsumer.java:397)
>     at
> org.apache.flink.streaming.api.operators.StreamSource.run(StreamSource.java:58)
>     at
> org.apache.flink.streaming.runtime.tasks.SourceStreamTask.run(SourceStreamTask.java:55)
>     at
> org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:218)
>     at org.apache.flink.runtime.taskmanager.Task.run(Task.java:584)
>     at java.lang.Thread.run(Thread.java:745)
> Caused by: java.lang.NullPointerException
>     at
> org.I0Itec.zkclient.ZkConnection.writeDataReturnStat(ZkConnection.java:115)
>     at org.I0Itec.zkclient.ZkClient$10.call(ZkClient.java:817)
>     at org.I0Itec.zkclient.ZkClient.retryUntilConnected(ZkClient.java:675)
>     at org.I0Itec.zkclient.ZkClient.writeDataReturnStat(ZkClient.java:813)
>     at org.I0Itec.zkclient.ZkClient.writeData(ZkClient.java:808)
>     at org.I0Itec.zkclient.ZkClient.writeData(ZkClient.java:777)
>     at kafka.utils.ZkUtils$.updatePersistentPath(ZkUtils.scala:332)
>     at kafka.utils.ZkUtils.updatePersistentPath(ZkUtils.scala)
>     at
> org.apache.flink.streaming.connectors.kafka.internals.ZookeeperOffsetHandler.setOffsetInZooKeeper(ZookeeperOffsetHandler.java:112)
>     at
> org.apache.flink.streaming.connectors.kafka.internals.ZookeeperOffsetHandler.commit(ZookeeperOffsetHandler.java:80)
>     at
> org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer$PeriodicOffsetCommitter.run(FlinkKafkaConsumer.java:632)
>
>
> Any ideas on what could cause this behaviour?
>
> Best,
> Mihail
>