Meaning of request.timeout.ms

2016-01-29 Thread tao xiao
Hi team, I want to understanding the meaning of request.timeout.ms that is used in producer. As per the doc this property is used to expire records that have been waiting for response from server for more than request.timeout.ms which also means the records have been sitting in InFlightRequests fo

Re: Accumulating data in Kafka Connect source tasks

2016-01-29 Thread Randall Hauch
On January 28, 2016 at 7:07:02 PM, Ewen Cheslack-Postava (e...@confluent.io) wrote: Randall,  Great question. Ideally you wouldn't need this type of state since it  should really be available in the source system. In your case, it might  actually make sense to be able to grab that information fro

Re: MongoDB Kafka Connect driver

2016-01-29 Thread Ewen Cheslack-Postava
Sunny, As I said on Twitter, I'm stoked to hear you're working on a Mongo connector! It struck me as a pretty natural source to tackle since it does such a nice job of cleanly exposing the op log. Regarding the problem of only getting deltas, unfortunately there is not a trivial solution here --

Re: MongoDB Kafka Connect driver

2016-01-29 Thread Jay Kreps
Hey Ewen, how come you need to get it all in memory for approach (1)? I guess the obvious thing to do would just be to query for the record after-image when you get the diff--e.g. just read a batch of changes and multi-get the final values. I don't know how bad the overhead of this would be...batch

Re: MongoDB Kafka Connect driver

2016-01-29 Thread Ewen Cheslack-Postava
Jay, You can query after the fact, but you're not necessarily going to get the same value back. There could easily be dozens of changes to the document in the oplog so the delta you see may not even make sense given the current state of the document. Even if you can apply it the delta, you'd still

Re: MongoDB Kafka Connect driver

2016-01-29 Thread Jay Kreps
Ah, agreed. This approach is actually quite common in change capture, though. For many use cases getting the final value is actually preferable to getting intermediates. The exception is usually if you want to do analytics on something like number of changes. On Fri, Jan 29, 2016 at 9:35 AM, Ewen

Re: MongoDB Kafka Connect driver

2016-01-29 Thread Jay Kreps
Also, most database provide a "full logging" option that let's you capture the whole row in the log (I know Oracle and MySQL have this) but it sounds like Mongo doesn't yet. That would be the ideal solution. -Jay On Fri, Jan 29, 2016 at 9:38 AM, Jay Kreps wrote: > Ah, agreed. This approach is a

Re: Accumulating data in Kafka Connect source tasks

2016-01-29 Thread Ewen Cheslack-Postava
On Fri, Jan 29, 2016 at 7:06 AM, Randall Hauch wrote: > On January 28, 2016 at 7:07:02 PM, Ewen Cheslack-Postava ( > e...@confluent.io) wrote: > > Randall, > > Great question. Ideally you wouldn't need this type of state since it > should really be available in the source system. In your case, it

Re: How to bind all Kafka tcp port to private net address

2016-01-29 Thread Stephen Powis
Pretty sure you want to set this option in your server.properties file: # Hostname the broker will bind to. If not set, the server will bind to all > interfaces > #host.name=localhost > On Thu, Jan 28, 2016 at 10:58 PM, costa xu wrote: > My version is kafka_2.11-0.9.0.0. I find that the kafka l

Re: Meaning of request.timeout.ms

2016-01-29 Thread Jason Gustafson
That is correct. KIP-19 has the details: https://cwiki.apache.org/confluence/display/KAFKA/KIP-19+-+Add+a+request+timeout+to+NetworkClient . -Jason On Fri, Jan 29, 2016 at 3:08 AM, tao xiao wrote: > Hi team, > > I want to understanding the meaning of request.timeout.ms that is used in > produce

Re: Accumulating data in Kafka Connect source tasks

2016-01-29 Thread James Cheng
> On Jan 29, 2016, at 7:06 AM, Randall Hauch wrote: > > On January 28, 2016 at 7:07:02 PM, Ewen Cheslack-Postava (e...@confluent.io) > wrote: > Randall, > > Great question. Ideally you wouldn't need this type of state since it > should really be available in the source system. In your case, it m

Re: MongoDB Kafka Connect driver

2016-01-29 Thread James Cheng
Not sure if this will help anything, but just throwing it out there. The Maxwell and mypipe projects both do CDC from MySQL and support bootstrapping. The way they do it is kind of "eventually consistent". 1) At time T1, record coordinates of the end of the binlog as of T1. 2) At time T2, do a f

Re: Accumulating data in Kafka Connect source tasks

2016-01-29 Thread Randall Hauch
On January 29, 2016 at 11:59:28 AM, Ewen Cheslack-Postava (e...@confluent.io) wrote: On Fri, Jan 29, 2016 at 7:06 AM, Randall Hauch  wrote: Luckily, MySQL includes the DDL statements in the log, so my connector parses these as part of processing and builds up the schema state as it goes along.

Kafka Latest offsets information

2016-01-29 Thread Kashif Usmani
Hello Community, When I login to my Kafka Manager(version 0.8.2.0), I can see the latest offset information for each topic and partition. I want to be able to get latest offsets programmatically. So I followed the example on https://cwiki.apache.org/confluence/display/KAFKA/Committing+and+fetching

Unable to send message

2016-01-29 Thread Jatin D Patel
Error logs from zookeeper [2016-01-29 06:28:20,100] INFO Accepted socket connection from / 172.31.28.145:35607 (org.apache.zookeeper.server.NIOServerCnxnFactory) [2016-01-29 06:28:20,124] INFO Client attempting to establish new session at /172.31.28.145:35607 (org.apache.zookeeper.server.ZooKeep

Questions from new user

2016-01-29 Thread allen chan
Use case: We are using kafka as broker in one of our elasticsearch clusters. Kafka caches the logs if elasticsearch has any performance issues. I have Kafka set to delete logs pretty quickly to keep things in the file cache to limit IO. Questions: 1. in 0.9 it seems like consumer offers are store