Re: RecordCollectorImpl: task [1_1] Error sending records

2019-06-04 Thread Bruno Cadonna
Hi Mohan, Could you post the log messages you see and you think you should not see? It is hard to help you without any actual logs. Best, Bruno On Wed, Jun 5, 2019 at 6:52 AM Parthasarathy, Mohan wrote: > > Hi, > > As mentioned here > > https://issues.apache.org/jira/browse/KAFKA-7510 > > I do

Re: [kafka-clients] [VOTE] 2.3.0 RC1

2019-06-04 Thread Colin McCabe
On Tue, Jun 4, 2019, at 23:17, Colin McCabe wrote: > Hi all, > > This is the first candidate for the release of Apache Kafka 2.3.0. > > This release includes many new features, including: > * Support for incremental cooperative rebalancing > * An in-memory session store and window store for Kafka

Re: unsub help

2019-06-04 Thread Matthias J. Sax
If you send an email to users-unsubscr...@kafka.apache.org, you should get a reply from a bot, and need to verify that you want to unsubscribe. Maybe check your spam folder if the bot reply ended up there? -Matthias On 6/4/19 6:17 PM, Adaryl Wakefield wrote: > I'm sorry, I know you guys hate em

[VOTE] 2.3.0 RC1

2019-06-04 Thread Colin McCabe
Hi all, This is the first candidate for the release of Apache Kafka 2.3.0. This release includes many new features, including: * Support for incremental cooperative rebalancing * An in-memory session store and window store for Kafka Streams * An API for allowing users to determine what operations

RecordCollectorImpl: task [1_1] Error sending records

2019-06-04 Thread Parthasarathy, Mohan
Hi, As mentioned here https://issues.apache.org/jira/browse/KAFKA-7510 I do see these logs turned on by default. We are running 2.2 and I still see the error. It looks like the fix is in 2.0. What am I missing ? It seems to be logged in “WARN” level instead of “debug” level. It spews a huge a

unsub help

2019-06-04 Thread Adaryl Wakefield
I'm sorry, I know you guys hate emails like this. I'm having an issue unsubscribing. I sent an email (numerous actually) to users-unsubscr...@kafka.apache.org, but it doesn’t seem to be taking. Is it a batch process and I have to wait a day? Adaryl "Bob" Wakefield, MBA President Extensive Ente

Repeating UNKNOWN_PRODUCER_ID errors for Kafka streams applications

2019-06-04 Thread Pieter Hameete
Hello, Our Kafka streams applications are showing the following warning every few seconds (on each of our 3 brokers, and on each of the 2 instances of the streams application): [Producer clientId=event-rule-engine-dd71ae9b-523c-425d-a7c0-c62993315b30-StreamThread-1-1_24-producer, transaction

Error building

2019-06-04 Thread Landon Kuhn
Hello, I am unable to build the project and am looking for help. "./gradlew --console=plain installAll" results in the following output: > Configure project : Building project 'core' with Scala version 2.12.8 Building project 'streams-scala' with Scala version 2.12.8 > Task :generator:compileJav

interpretation of ErrorsPerSec JMX metric

2019-06-04 Thread Daniel LaFlamme
I started examining metrics on one of the nodes in my Kafka cluster and I am confused about one of the metrics that is exposed. Specifically, this metric: kafka.network:name=ErrorsPerSec,error=NONE,request=Fetch,type=RequestMetrics The name is ErrorsPerSec but error is set to "NONE". What is the c

Re: Repeating UNKNOWN_PRODUCER_ID errors for Kafka streams applications

2019-06-04 Thread Guozhang Wang
Hello Pieter, If you only have one record every few seconds that may be too small given you have at least 25 partitions (as I saw you have a xxx--repartition-24 partition), which means that for a single partition, it may not see any records for a long time, and in this case you may need to overrid

Re: What happens in Rebalancing state ?

2019-06-04 Thread Guozhang Wang
Hello Mohan, That is right, note though that it is transiting to RUNNING after all assigned tasks to its threads has completed restoration and starting running. This means even under REBALANCING state some tasks that finished restoration early may still be executed as well. Guozhang On Tue, Jun

What happens in Rebalancing state ?

2019-06-04 Thread Parthasarathy, Mohan
Hi, When applications are restarted, I see it go into rebalancing state. Is it trying to recreate state from the state store files ? I am seeing an application stuck for a long time and wondering how to debug the application in this state ? Thanks Mohan

Re: Unable to build

2019-06-04 Thread Martin Gainty
in build.gradle assuming sourceSets is defined as sourceSets { main { java { srcDirs = ['src/java'] } resources { srcDirs = ['src/resources'] } } } add missing source files: org.apache.kafka.common.security.authenticator.SaslServerCa

Unable to build

2019-06-04 Thread Landon Kuhn
Hello, I am unable to build the project and am looking for help. "./gradlew --console=plain installAll" results in the following output: > Configure project : Building project 'core' with Scala version 2.12.8 Building project 'streams-scala' with Scala version 2.12.8 > Task :generator:compileJav

Re: MirrorMaker 2.0 XDCR / KIP-382

2019-06-04 Thread Ryanne Dolan
Jeremy, please see relevant changes documented here: https://github.com/apache/kafka/blob/cae2a5e1f0779a0889f6cb43b523ebc8a812f4c2/connect/mirror/README.md#multicluster-environments I've added a --clusters argument which makes XDCR a lot easier to manage, obviating the configuration race issue.

Java consumer error handling on DNS lookup failure

2019-06-04 Thread Tim Ward
I have a Kafka client written in Java running in Kubernetes, and Kafka running in Kubernetes. When the client is running but no Kafka nodes are running it appears from the exception below that the DNS lookup fails, then something catches the exception, logs it, and reties. Apparently without re

Re: Using Kafka on Windows (Production environment)

2019-06-04 Thread M. Manna
Kafka cannot be run on Windows in production. There are problems with memory map allocation/releases which results into fatal shutdown. On Linux it’s allowed, but on Windows it’s prevented. You can reproduce this by setting a small log retention period on your windows and test with a QuickStart.

RE: Live data streaming from Oracle to oracle using Kafka

2019-06-04 Thread Kailash Kota
Thanks for the info Joe ! Appreciate it Thanks & Regards, Kailash Kota Product Development | JDA Software Pvt Ltd. Ph: +91 80 6101 8649 -Original Message- From: Joe Ammann Sent: 04 June 2019 18:23 To: users@kafka.apache.org Subject: Re: Live data streaming from Oracle to oracle using K

Re: Live data streaming from Oracle to oracle using Kafka

2019-06-04 Thread Joe Ammann
Hi Kailash On 6/4/19 12:35 PM, Kailash Kota wrote: > We want to live stream data from Oracle DB and to Oracle DB as a target using > Kafka and we do not want to use Oracle Golden Gate because of the extra > license cost. > Is there a way we can read the redo logs and achieve this ? Debezium htt

Using Kafka on Windows (Production environment)

2019-06-04 Thread Thiago Sebestyen Varea
Hi everyone! I know it is not a common scenario, but I would like to know whether Kafka is ready for running on Windows in a production environment? If yes, does someone have any experience with this scenario to tell me if it is good or not? Thanks in advance! Best Regards, Thiago Sebestyen V

Re: KafkaConnect not consuming from SSL/SASL cluster

2019-06-04 Thread Ivan Yurchenko
Hi Javier, As far as I understand, the reason for this separation is that Connect has separate producers and consumers for its internal mechanics (like Kafka-backed configuration/status/offset storage) and for tasks. It will use some non-prefixed configurations for both tasks' consumers and produc

Live data streaming from Oracle to oracle using Kafka

2019-06-04 Thread Kailash Kota
Hi Users, Here is our requirement : We want to live stream data from Oracle DB and to Oracle DB as a target using Kafka and we do not want to use Oracle Golden Gate because of the extra license cost. Is there a way we can read the redo logs and achieve this ? If not, can you help us on the othe

Re: KafkaConnect not consuming from SSL/SASL cluster

2019-06-04 Thread Javier Arias Losada
Hello Andrew, yes it fixed it! Thank you! I was missing the consumer. configuration... so here we seem to be configuring different Kafka clients... what is the reason for this? The configs prefixed with consumer. are for the consumers I'm trying to connect with sinks? and the others are for interna

Re: KafkaConnect not consuming from SSL/SASL cluster

2019-06-04 Thread Andrew Schofield
Hi, The thing that always seem to catch people out with this is that it’s necessary to repeat the SSL/SASL configuration. For a sink connector, you need something like: security.protocol=SASL_SSL ssl.protocol=TLSv1.2 sasl.mechanism=PLAIN sasl.jaas.config=... And you also need the same wi

KafkaConnect not consuming from SSL/SASL cluster

2019-06-04 Thread Javier Arias Losada
Hello there, we are trying to use KafkaConnect, but it isn't consuming any messages after changing to a SSL and authenticated Kafka cluster. With a cluster without SSL or authentication it's working perfectly fine with the same configuration except for the sasl/ssl settings. I think probably is

Repeating UNKNOWN_PRODUCER_ID errors for Kafka streams applications

2019-06-04 Thread Pieter Hameete
Hello, Our Kafka streams applications are showing the following warning every few seconds (on each of our 3 brokers, and on each of the 2 instances of the streams application): [Producer clientId=event-rule-engine-dd71ae9b-523c-425d-a7c0-c62993315b30-StreamThread-1-1_24-producer, transaction

Re: kafka-topics.sh in ssl environment

2019-06-04 Thread j . lammerts
Never mind, I figured it out. Use of the --command-config option really helps ;=)) On 2019/06/04 08:39:40, j.lamme...@chello.nl wrote: > Hi, > > I'm using kafka 2.11-2.2.1. > The use of --zookeeper when running kafka-topics.sh is still working, but > deprecated. > If I try to use --bootst

Re: Can't see topics anymore after upgrade from 1.1 to 2.2.1

2019-06-04 Thread j . lammerts
Thanks, Issue is solved. I forgot to copy over all files from the zookeeper dataDir directory to the new 2.2.1 environment. Cheers ! On 2019/06/03 16:54:47, "M. Manna" wrote: > Zookeeper hosted topics are often not displayed if you use > bootstrap-server; aka new configs. > > Try using zooke

kafka-topics.sh in ssl environment

2019-06-04 Thread j . lammerts
Hi, I'm using kafka 2.11-2.2.1. The use of --zookeeper when running kafka-topics.sh is still working, but deprecated. If I try to use --bootstrap-server in my SSL enabled Kafka environment, I get a nice OutOfMemory error: [root@tlkfk001 bin]# ./kafka-topics.sh --bootstrap-server tlkfk001:9093 -