Re: Samza and sliding window

2015-07-23 Thread Yi Pan
Yeah, that's why I added some test code in the window() to call store.all() and iterate through. I traced into it in my local environment and verified that the iterator is functioning with store.all(). -Yi On Thu, Jul 23, 2015 at 4:26 PM, Shekar Tippur ctip...@gmail.com wrote: Yi, In my

Re: Kafka broker error from samza producer

2015-07-23 Thread Roger Hoover
I forgot to mention that for me the error always happened after restarting a broker. Sent from my iPhone On Jul 23, 2015, at 4:25 PM, Jordan Shaw jor...@pubnub.com wrote: Hey Roger, I restarted the producer and the error went away on the broker. If it comes back I'll switch over to lz4.

Re: Samza and sliding window

2015-07-23 Thread Shekar Tippur
Yi, In my case, I am able to append to the key but I am not able to get the store and iterate through. If you look at http://pastebin.com/fKGpHwW6, line 146, I am able to get the store value. but in window routine - line 187, I am unable to get the values from store. - Shekar

Re: Kafka broker error from samza producer

2015-07-23 Thread Jordan Shaw
Hey Roger, I restarted the producer and the error went away on the broker. If it comes back I'll switch over to lz4. Thanks for the reply. -Jordan On Thu, Jul 23, 2015 at 9:32 AM, Roger Hoover roger.hoo...@gmail.com wrote: Hi Jordan, I ran into a similiar issue when using snappy compression

Re: Review Request 36545: SAMZA-682 Refactor Coordinator stream messages

2015-07-23 Thread Yan Fang
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/36545/#review92825 ---

Re: Review Request 36692: SAMZA-717 Expose the TaskNameGrouper API

2015-07-23 Thread József Márton Jung
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/36692/ --- (Updated July 23, 2015, 11:23 a.m.) Review request for samza. Repository:

kafka producer failed

2015-07-23 Thread Job-Selina Wu
Hi, When the messages was send to Kafka by KafkaProducer, It always failed when the message more than 3000 - 4000 messages. The error is shown below. I am wondering if any topic size I need to set at Samza configuration? [2015-07-23 17:30:03,792] WARN

Samza: can not produce new data to kafka

2015-07-23 Thread Job-Selina Wu
Hi, All I am trying to write my first StreamTask class. I have a topic at Kafka called http-demo. I like to read the topic and write it to another topic called demo-duplicate Howeven there is not topic written to Kafka. My properties file and StreamTask are below. Can anyone told

Re: kafka producer failed

2015-07-23 Thread Job-Selina Wu
Hi, After I got the error in previous email. I try to check the content of the topic. It show the error below. However when I stop Samza and re-run it again, it will be fine. Does anyone know What was the problem? [2015-07-23 17:50:08,391] WARN

Re: Kafka broker error from samza producer

2015-07-23 Thread Roger Hoover
Hi Jordan, I ran into a similiar issue when using snappy compression and the new producer. If you disable compression or switch to lz4 or gzip, does the issue go away? Cheers, Roger On Wed, Jul 22, 2015 at 11:54 PM, Jordan Shaw jor...@pubnub.com wrote: Hey Everyone, I'm getting an:

Review Request 36728: HELLO-SAMZA-563 Upgrade Samza to YARN 2.6.0

2015-07-23 Thread Aleksandar Pejakovic
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/36728/ --- Review request for samza. Repository: samza-hello-samza Description ---

How to map document version to the Elasticsearch System Producer?

2015-07-23 Thread Roger Hoover
Hi Dan and Samza devs, I have a use case for which I need to set an external version on Elasticsearch documents. Versioning ( https://www.elastic.co/guide/en/elasticsearch/reference/current/docs-index_.html#index-versioning) lets you prevent duplicate messages from temporarily overwriting new

Kafka broker error from samza producer

2015-07-23 Thread Jordan Shaw
Hey Everyone, I'm getting an: kafka.message.InvalidMessageException: Message found with corrupt size (0) in my kafka server.log here is the full stack trace: https://gist.github.com/jshaw86/516cf47b6fd7559e7dc1. It indicates that the error is caused by a bad produce call from the samza producer