Re: Samza 0.14.1 not correctly handling OffsetOutOfRangeException exception?

2018-08-24 Thread Gaurav Agarwal
, but just trying to ask the next thing that naturally comes to mind :-) -- thanks, gaurav On Thu, Aug 23, 2018 at 7:06 PM Gaurav Agarwal wrote: > Few more notes (based on reading a similar thread from few days ago): > - this exception is while initializing offset for the data topic par

Re: Samza 0.14.1 not correctly handling OffsetOutOfRangeException exception?

2018-08-23 Thread Gaurav Agarwal
lable offset is greater than what samza has in its checkpoint - and hence when samza is querying kafka with the offset it checkpointed last, it is seeing this error. Please let me know if more logs are required. On Thu, Aug 23, 2018 at 4:30 PM Gaurav Agarwal wrote: > Hi All, > >

Samza 0.14.1 not correctly handling OffsetOutOfRangeException exception?

2018-08-23 Thread Gaurav Agarwal
Hi All, We are facing identical problem as described in thread https://www.mail-archive.com/dev@samza.apache.org/msg06740.html Here - Samza is requesting for an Kafka partition offset that is too old (i.e Kafka log has moved ahead). We are setting the property *consumer.auto.offset.reset to small

Re: Per task/topic checkpoint?

2017-11-01 Thread Gaurav Agarwal
explicitly set the checkpoint offset on a per SSP basis, > the checkpoints are actually recorded that way. And if your job consumes > multiple topics, the offsets will be granular enough, because they're > per-SSP. > > More details on checkpointing here: > > http://samza.apache.org

Re: Per task/topic checkpoint?

2017-11-01 Thread Gaurav Agarwal
. > GroupBySystemStreamPartitionFactory. > > Config reference: https://samza.apache.org/learn/documentation/0.10/jobs/ > configuration-table.html > > What are you trying to do? Maybe, there's a simpler way to achieve it? > > > > On Sat, Oct 28, 2017 at 4:09 AM, Gaurav Aga

Per task/topic checkpoint?

2017-10-28 Thread Gaurav Agarwal
Hi All, If I had Samza Tasks that were consuming message from multiple topics, how would checkpoint/commit work in that case? On calling taskCordinator.commit(), would current offset of all topics be saved for the caller task (only the partitions assigned to the caller task)? Is there a way to co

Re: Possible bug causing reset in offsets [Version: 0.12.0].

2017-05-02 Thread Gaurav Agarwal
May 2, 2017 at 11:35 PM, Gaurav Agarwal wrote: > Looking further, the reason for this "jump back" seems not so straight > forward: > In Kafka's Simple Consumer code: > > private def sendRequest(request: RequestOrResponse): NetworkReceive = { > lock synchronized { >

Re: Possible bug causing reset in offsets [Version: 0.12.0].

2017-05-02 Thread Gaurav Agarwal
sed Samza to thing that the offset is invalid. However, I am unable to understand what could have caused this? Did Kafka return this exception accidentally or did Samza ask for the offset that was beyond what was present in kafka queue? On Tue, May 2, 2017 at 9:31 AM, Gaurav Agarwal wrote: > This

Re: Possible bug causing reset in offsets [Version: 0.12.0].

2017-05-01 Thread Gaurav Agarwal
ay 2, 2017 at 9:17 AM, Gaurav Agarwal wrote: > Hi All, > > We recently observed an issue in our Samza application (version > 0.12.0) where we found that that the message offsets "jumped back" causing > many of the messages to be re-processed. > > On digging deeper -

Possible bug causing reset in offsets [Version: 0.12.0].

2017-05-01 Thread Gaurav Agarwal
Hi All, We recently observed an issue in our Samza application (version 0.12.0) where we found that that the message offsets "jumped back" causing many of the messages to be re-processed. On digging deeper - here is what we found: - There was a some network related issues at that time causing t

Re: Samza 0.12.0 + synchronous KafkaProducer ?

2017-03-09 Thread Gaurav Agarwal
ily mean there are multiple produce requests in-flight on the wire. We will have only one un-acknowledged produce request on a single connection ensuring that we are immune to potential re-orderings (occurring due to retries). On Thu, Mar 9, 2017 at 4:49 PM,

Re: Samza 0.12.0 + synchronous KafkaProducer ?

2017-03-09 Thread Gaurav Agarwal
wise) > > Do you want ordering multiple send calls across partitions? (We have not > had a scenario for that yet). > > Thanks, > Jagadish > > > > > > On Thu, Mar 9, 2017 at 10:19 AM, Gaurav Agarwal > wrote: > > > There is one more case to consid

Re: Samza 0.12.0 + synchronous KafkaProducer ?

2017-03-09 Thread Gaurav Agarwal
subsequent sends() from same process call. Thanks for being patient with my questions! From: Gaurav Agarwal Sent: Thursday, March 9, 2017 11:27:17 PM To: dev@samza.apache.org Cc: Mukul Gupta; Kshitij Gupta Subject: Re: Samza 0.12.0 + synchronous KafkaProducer ? Hi

Re: Samza 0.12.0 + synchronous KafkaProducer ?

2017-03-09 Thread Gaurav Agarwal
ot;batch" up some amount of work and then increment the offset once that batch is processed (which I believe is the original intent of checkpointing). If we used up "checkpointing" for guaranteeing ordered delivery, we would loose the capability of batching. > >

Re: Samza 0.12.0 + synchronous KafkaProducer ?

2017-03-08 Thread Gaurav Agarwal
ia the > *taskCoordinator > *abstraction). Invoking *taskCoordinator.commit* will wait on pending > futures, flush buffers, flush state stores and checkpoint offsets. > > Please let us know if we can be of more help! > > Thanks, > Jagadish > > > > > > On

Re: Samza 0.12.0 + synchronous KafkaProducer ?

2017-03-08 Thread Gaurav Agarwal
(correcting recipient address) On Thu, Mar 9, 2017 at 10:39 AM, Gaurav Agarwal wrote: > Hi All, > > We are trying to upgrade to Kafka 0.12.0. In the process we noticed that > the Kafka 0.10.0 KafkaProducer client api does not provide any > configuration to send() the messages syn

Samza 0.12.0 + synchronous KafkaProducer ?

2017-03-08 Thread Gaurav Agarwal
Hi All, We are trying to upgrade to Kafka 0.12.0. In the process we noticed that the Kafka 0.10.0 KafkaProducer client api does not provide any configuration to send() the messages synchronously. One needs to wait on the returned Future for synchronous guarantees. I am trying to figure out, how t

Re: kafka dependency version

2016-08-23 Thread Gaurav Agarwal
-Yi > > On Mon, Aug 22, 2016 at 1:11 AM, Gaurav Agarwal > wrote: > > > My initial attempt to build against kafka 0.9.0 or 0.10.0 stopped short > due > > to some non-trivial interface changes and refactorings in Kafka; and I > did > > not want to make those changes

Re: kafka dependency version

2016-08-22 Thread Gaurav Agarwal
dependency? Should I attempt to make the build/tests pass against kafka 0.10 and submit a PR? Or would someone from samza dev team prefer to do this? On Mon, Aug 22, 2016 at 12:36 PM, Gaurav Agarwal wrote: > Hi All, > > What kafka version is supported by Samza release 0.10.1? > >

kafka dependency version

2016-08-22 Thread Gaurav Agarwal
Hi All, What kafka version is supported by Samza release 0.10.1? In the gradle build file (dependency-versions.gradle) I see it as kafkaVersion = "0.8.2.1". This is very old kafka version and lot of very serious bugs have been fixed since then (including the one we are hitting where kafka client

Re: Samza retries on kafka exception

2016-07-29 Thread Gaurav Agarwal
or the diligent observation and walk through. We ran into a similar > issue at LinkedIn and addressed it in SAMZA-911. > > I think the 0.10.1 release had this fix. > > Thanks > Jagdish > > On Friday, July 29, 2016, Gaurav Agarwal wrote: > > > More debugging notes an

Re: Samza retries on kafka exception

2016-07-29 Thread Gaurav Agarwal
e an infinite attempt loop. Have I understood this correctly? and if so, should user be given a hook point to control the behavior on underlying failures? On Fri, Jul 29, 2016 at 5:07 PM, Gaurav Agarwal wrote: > (correction: we are using samza 0.9.0) > > On Fri, Jul 29, 2016 at 12:09 PM,

Re: Samza retries on kafka exception

2016-07-29 Thread Gaurav Agarwal
(correction: we are using samza 0.9.0) On Fri, Jul 29, 2016 at 12:09 PM, Gaurav Agarwal wrote: > Hi All, > > We are using Samza (0.10.0) in our system and recently ran into a problem > where due to Kafka broker being unstable for few moments, our samza tasks > while trying to w

Samza retries on kafka exception

2016-07-28 Thread Gaurav Agarwal
Hi All, We are using Samza (0.10.0) in our system and recently ran into a problem where due to Kafka broker being unstable for few moments, our samza tasks while trying to write message to kafka got exceptions. After that moment, they went into a very long retry loop (Integer.MAX times). The repe