Go Library that uses writer and reader schema

2020-09-10 Thread Scott Reynolds
Hi List, TL;DR Looking for a Go library that can Decode binary data written with a different Schema then the Reader. We have a system in place that allows for multiple Avro schemas to be live for data streams. There can be producers producing an earlier version of the schema (version 1) onto the

Re: Kafka Streams, WindowBy, Grace Period, Late events, Suppres operation

2020-05-11 Thread Scott Reynolds
Baki, You can get this message "o.a.k.s.state.internals.WindowKeySchema : Warning: window end time was truncated to Long.MAX"" when your TimeWindowDeserializer is created without a windowSize. There are two constructors for a TimeWindowDeserializer, are you using the one with WindowSize?

Re: Kafka streams in Kubernetes

2019-06-10 Thread Scott Reynolds
We have been giving this a bunch of thought lately. We attempted to replace PARTITION_ASSIGNMENT_STRATEGY_CONFIG with our implementation that hooks into our deployment service. The idea is simple, the new deployment gets *Standby tasks assigned to them until they are caught up*. Once they are

Re: Default commit interval for Kafka Streams

2019-04-16 Thread Scott Reynolds
: > The commit interval is small to keep end-to-end processing latency > small. For example, if data is repartitioned, a downstream task can only > read the data after the upstream tasks commit its transaction. > > -Matthias > > On 4/16/19 9:56 AM, Scott Reynolds wrote: > > Hi,

Default commit interval for Kafka Streams

2019-04-16 Thread Scott Reynolds
Hi, I have been unable to determine why the default commit interval for Exactly Once Streams application is 100L. This seems really aggressive and does produce a large amount of offsets to our broker. I have changed this in our application but I am worried I have now introduced a bug in the

Re: GDPR appliance

2017-11-23 Thread Scott Reynolds
g > > the key when data has to deleted. > > Sadly, our legal department after some checkins has conclude that this > > approach is "to block" data but not deleting it, as a consequence it can > > take us problems. If my guess about your solution is right, you could &g

Re: GDPR appliance

2017-11-22 Thread Scott Reynolds
; delete some data contained in the messages. So not deleting the message, > but editing it. > For doing that, my intention is to replicate the topic and apply a > transformation over it. > I think that frameworks like Kafka Streams or Apache Storm. > > Did anybody had to solve this problem

Re: Data loss while upgrading confluent 3.0.0 kafka cluster to confluent 3.2.2

2017-09-18 Thread Scott Reynolds
is older version leader, will not be synced with > other > >>replicas and if this leader(older version) goes down for an > upgrade, other > >>updated replicas will be shown in in-sync column and become leader, > but > >>they lag in offset with old versio

Re: moving brokers to different servers

2017-08-21 Thread Scott Reynolds
; Thanks > > > > Printing e-mails wastes valuable natural resources. Please don't print > this message unless it is absolutely necessary. Thank you for thinking > green! > > Sent from my iPhone > -- Scott Reynolds Principal Engineer [image: twilio] <http://www.twilio.com/?utm_source=email_signature> MOBILE (630) 254-2474 EMAIL sreyno...@twilio.com

Re: Kafka 10 Consumer Reading from Kafka 8 Cluster?

2016-10-06 Thread Scott Reynolds
you cannot use a k10 client with a k8 cluster. The protocol changed You CAN use a k8 client with a k10 cluster. On Thu, Oct 6, 2016 at 12:00 PM Craig Swift wrote: > We're doing some fairly intensive data transformations in the current > workers so it's not

Struggling to understand Group Coordinator election (0.9)

2016-09-12 Thread Scott Reynolds
List, Documentation about Group Coordinator election seems to reference zookeeper, but I am unable to find anything in zookeeper. Is it using kafka to store this information like consumer offsets ? Can someone explain how Group Coordinator election works ? We had an incident during a

Re: Kafka Connect misconfiguration. Need some help

2016-05-09 Thread Scott Reynolds
ion happens nor where the heartbeat response is generated. Anyone have any guidance on where to look or how to debug ? Grasping at straws at this moment. On Fri, Apr 15, 2016 at 10:36 AM Scott Reynolds <sreyno...@twilio.com> wrote: > Awesome that is what I thought. Answer seems simple, speed

Re: Kafka Connect misconfiguration. Need some help

2016-04-15 Thread Scott Reynolds
onsumer.session.timeout.ms or > have some timeout mechanism in the implementation of the flush method to > return the control back to the framework so that it can send heartbeat to > the coordinator. > > Thanks, > Liquan > > On Fri, Apr 15, 2016 at 9:56 AM, Scott Rey

Kafka Connect misconfiguration. Need some help

2016-04-15 Thread Scott Reynolds
List, We are struggling with Kafka Connect settings. The process start up and handle a bunch of messages and flush. Then slowly the Group coordinator removes them. This is has to be a interplay between Connect's flush interval and the call to poll for each of these tasks. Here is my current

Help understanding what happened

2016-03-20 Thread Scott Reynolds
In a test in staging environment, we kill -9 the broker. It was started back up by runit and started recovering. We are seeing errors like this: WARN Found an corrupted index file, /mnt/services/kafka/data/TOPIC-17/16763460.index, deleting and rebuilding index... (kafka.log.Log) The

Re: Zookeeper DNS TTL

2016-03-19 Thread Scott Reynolds
Ah I have been down this path. It is the zookeeper client. It resolves and caches the ip addresses: https://github.com/apache/zookeeper/blob/bd9a1448f9b29859092e6bdca93da121ec166b7a/src/java/main/org/apache/zookeeper/client/StaticHostProvider.java#L108 I believe they are cached forever. We have

Re: Deletion of topic on 0.9.0.0 spams this exception

2016-03-14 Thread Scott Reynolds
Yep that is it. Thanks. I will watch the issue. On Mon, Mar 14, 2016 at 1:13 PM Stevo Slavić <ssla...@gmail.com> wrote: > I've recently created related ticket > https://issues.apache.org/jira/browse/KAFKA-3390 > > On Mon, Mar 14, 2016, 20:54 Scott Reynolds <sreyno

Re: Controlled shutdown not relinquishing leadership of all partitions

2016-01-12 Thread Scott Reynolds
Luke, We practice the same immutable pattern on AWS. To decommission a broker, we use partition reassignment first to move the partitions off of the node and preferred leadership election. To do this with a web ui, so that you can handle it on lizard brain at 3 am, we have the Yahoo Kafka Manager

Re: Controlled shutdown not relinquishing leadership of all partitions

2016-01-12 Thread Scott Reynolds
< luke.steen...@braintreepayments.com> wrote: > Ah, that's a good idea. Do you know if kafka-manager works with kafka 0.9 > by chance? That would be a nice improvement of the cli tools. > > Thanks, > Luke > > > On Tue, Jan 12, 2016 at 4:53 PM, Scott Reynolds <sreyno..

Re: consumer offset tool and JMX metrics do not match

2015-11-16 Thread Scott Reynolds
On Mon, Nov 16, 2015 at 8:27 AM, Abu-Obeid, Osama < osama.abu-ob...@morganstanley.com> wrote: > I can observe the same thing: > > - Lag values read through the Kafka consumer JMX is 0 > This metric includes *uncommitted* offsets - Lag values read through kafka-run-class.sh >

Re: Consumer lag lies - orphaned offsets?

2015-06-04 Thread Scott Reynolds
On Thu, Jun 4, 2015 at 1:55 PM, Otis Gospodnetić otis.gospodne...@gmail.com wrote: Hi, On Thu, Jun 4, 2015 at 4:26 PM, Scott Reynolds sreyno...@twilio.com wrote: I believe the JMX metrics reflect the consumer PRIOR to committing offsets to Kafka / Zookeeper. But when you query from

Consumers closing sockets abruptly?

2015-01-27 Thread Scott Reynolds
On my brokers I am seeing this error log message: Closing socket for /X because of error (X is the ip address of a consumer) 2015-01-27_17:32:58.29890 java.io.IOException: Connection reset by peer 2015-01-27_17:32:58.29890 at sun.nio.ch.FileChannelImpl.transferTo0(Native Method)

Re: Achieving Consistency and Durability

2014-10-14 Thread Scott Reynolds
A question about 0.8.1.1 and acks. I was under the impression that setting acks to 2 will not throw an exception when there is only one node in ISR. Am I incorrect ? Thus the need for min_isr. On Tue, Oct 14, 2014 at 11:50 AM, Kyle Banker kyleban...@gmail.com wrote: It's quite difficult to