Re: Database Replication Question

2015-03-26 Thread Guozhang Wang
Hello Josh, We do not have a ticket open for idempotent producer as it is still in the discussion process, but here is the wiki: https://cwiki.apache.org/confluence/display/KAFKA/Idempotent+Producer As for transactional messaging, we have a prototype implementation at LinkedIn which is not yet i

Re: Database Replication Question

2015-03-16 Thread Josh Rader
Thanks Guozhang. Are there JIRAs created for tracking idempotent producer or transactional messaging features? Maybe we can pick up some of the tasks to expedite the release? On Fri, Mar 6, 2015 at 1:53 AM, Guozhang Wang wrote: > Josh, > > Dedupping on the consumer side may be tricky as it req

Re: Database Replication Question

2015-03-12 Thread Guozhang Wang
Jay, I think what Xiao was proposing is allowing synchronous fsync, i.e. calling fsync after appending messages to the log, for applications that require disk data persistency. Maybe we can add another ack condition in addition to "#.isr replicated" that requires the data to be persisted to disk b

Re: Database Replication Question

2015-03-12 Thread Jay Kreps
Xiao, Not sure about AIX or HP-UX. There are some people running on Windows, though we don't do real systemic testing against that. I would be surprised if z/os worked, someone would have to try. The existing fsync policy already works at the batch level, and Kafka already does batching quite agg

Re: Database Replication Question

2015-03-12 Thread Xiao
Hi, Jay and Guozhang, In the long term, I think Kafka can be evolved to replace general-purpose messaging systems. For example, if I were a customer who is using IBM MQ, could we use Kafka instead with very minor code changes? http://www-01.ibm.com/support/knowledgecenter/?lang=en#!/SSFKSJ_7.1

Re: Database Replication Question

2015-03-11 Thread Xiao
Hi, Pete, Thank you for sharing your experience with me! sendfile and mmap are common system calls, but it sounds like we still need to consider at least the file-system differences when deploying Kafka. Cross-platform supports are a headache. : ) Best wishes, Xiao Li On Mar 10, 2015, at

Re: Database Replication Question

2015-03-11 Thread Xiao
Hi Jiangjie and Guozhang, Native z/OS support is what I need. I wrote a few native z/OS applications using C/C++ before. Based on my current understanding, I have two alternatives: 1) Write native z/OS producers in C/C++ to feed data to Kafka clusters that are running on LUW servers. I

Re: Database Replication Question

2015-03-10 Thread Jiangjie Qin
Hi Xiao, For z/OS, do you mean z/VM or native z/OS? For z/VM, it probably will work fine, but for z/OS, I would be surprised if Kafka can run directly on it. I think Guozhang¹s approach for cross colo replication is worth trying. One thing might need to be aware of for deploying Kafka cluster cro

Re: Database Replication Question

2015-03-10 Thread Pete Wright
On 03/09/15 22:56, Xiao wrote: > Hi, Jay, > > Thank you! > > The Kafka document shows “Kafka should run well on any unix system". I assume > it includes the major two Unix versions, IBM AIX and HP-UX. Right? FWIW I have had good success running Kafka under load on FreeBSD. I was using OpenJD

Re: Database Replication Question

2015-03-10 Thread Guozhang Wang
Hello Xiao, The proposed transactional messaging at Kafka is aimed at improving the at-least-once semantics of delivery to exactly-once, i.e. to avoid duplicates. It is not aimed for grouping fsync of multiple messages into one, as for avoiding data loss it is still dependent on data replication.

Re: Database Replication Question

2015-03-09 Thread Xiao
Hi, Jay, Thank you! The Kafka document shows “Kafka should run well on any unix system". I assume it includes the major two Unix versions, IBM AIX and HP-UX. Right? 1. Unfortunately, we aims at supporting all the platforms, Linux, Unix, Windows and especially z/OS. I know z/OS is not easy to

Re: Database Replication Question

2015-03-07 Thread Jay Kreps
Xiao, FileChannel.force is fsync on unix. To force fsync on every message: log.flush.interval.messages=1 You are looking at the time based fsync, which, naturally, as you say, is time-based. -Jay On Fri, Mar 6, 2015 at 11:35 PM, Xiao wrote: > Hi, Jay, > > Thank you for your answer. > > Sorry

Re: Database Replication Question

2015-03-06 Thread Xiao
Hi, Jay, Thank you for your answer. Sorry, I still do not understand your meaning. I guess the two parameters you mentioned are log.flush.interval and log.default.flush.interval.ms. However, these two parameters only control when Kafka issues a flush (i.e., calling FileChannel.force()). Fs

Re: Database Replication Question

2015-03-06 Thread Xiao
Hi, James, You also mentioned you want to implement another critical component for monitoring the data consistency between your sources and targets and correcting the inconsistent data. Unfortunately, data compare is not easy when some applications/replications still change your source and t

Re: Database Replication Question

2015-03-06 Thread Xiao
Hi, James, uh… iOS Gmail app crashed. Let me resend the email to answer your concern. First, I am not a Kafka user. Like you, I am trying to see if Kafka can be used for replication-related tasks. If Kafka can provide "unit of work", the design will be much simpler. As Guozhang said, Kafka h

Re: Database Replication Question

2015-03-05 Thread Roger Hoover
Hi Jonathan, TCP will take care of re-ordering the packets. On Wed, Mar 4, 2015 at 6:05 PM, Jonathan Hodges wrote: > Thanks James. This is really helpful. Another extreme edge case might be > that the single producer is sending the database log changes and the > network causes them to reach K

Re: Database Replication Question

2015-03-05 Thread Jay Kreps
Hey Xiao, That's not quite right. Fsync is controlled by either a time based criteria (flush every 30 seconds) or a number of messages criteria. So if you set the number of messages to 1 the flush is synchronous with the write, which I think is what you are looking for. -Jay On Thu, Mar 5, 2015

Re: Database Replication Question

2015-03-05 Thread James Cheng
On Mar 5, 2015, at 12:59 AM, Xiao wrote: > Hi, James, > > This design regarding the restart point has a few potential issues, I think. > > - The restart point is based on the messages that you last published. The > message could be pruned. How large is your log.retention.hours? That's a go

Re: Database Replication Question

2015-03-05 Thread Guozhang Wang
Josh, Dedupping on the consumer side may be tricky as it requires some sequence number on the messages in order to achieve idempotency. On the other hand, we are planning to add idempotent producer or transactional messaging https://cwiki.apache.org/confluence/display/KAFKA/Idempotent+Producer h

Re: Database Replication Question

2015-03-05 Thread Xiao
Hey, Jay, Thank you for your answer! Based on my understanding, Kafka fsync is regularly issued by a dedicated helper thread. It is not issued based on the semantics. The producers are unable to issue a COMMIT to trigger fsync. Not sure if this requirement is highly desirable to the others

Re: Database Replication Question

2015-03-05 Thread Xiao
Hi, James, This design regarding the restart point has a few potential issues, I think. - The restart point is based on the messages that you last published. The message could be pruned. How large is your log.retention.hours? - If the Kafka message order is different from your log sequence, yo

Re: Database Replication Question

2015-03-04 Thread James Cheng
> On Mar 3, 2015, at 4:18 PM, Guozhang Wang wrote: > > Additionally to Jay's recommendation, you also need to have some special > cares in error handling of the producer in order to preserve ordering since > producer uses batching and async sending. That is, if you already sent > messages 1,2,3,

Re: Database Replication Question

2015-03-04 Thread Jonathan Hodges
Thanks James. This is really helpful. Another extreme edge case might be that the single producer is sending the database log changes and the network causes them to reach Kafka out of order. How do you prevent something like this, I guess relying on the scn on the consumer side? On Wed, Mar 4,

Re: Database Replication Question

2015-03-04 Thread James Cheng
Another thing to think about is delivery guarantees. Exactly once, at least once, etc. If you have a publisher that consumes from the database log and pushes out to Kafka, and then the publisher crashes, what happens when it starts back up? Depending on how you keep track of the database's tran

Re: Database Replication Question

2015-03-04 Thread Jonathan Hodges
Yes you are right on the oplog per partition as well as that mapping well to the Kafka partitions. I think we are making this harder than it is based on previous attempts and trying to leverage something like Databus for propagating log changes from MongoDB and Cassandra since it requires a scn.

Re: Database Replication Question

2015-03-04 Thread Jay Kreps
Hey Xiao, Yeah I agree that without fsync you will not get durability in the case of a power outage or other correlated failure, and likewise without replication you won't get durability in the case of disk failure. If each batch is fsync'd it will definitely be slower, depending on the capabilit

Re: Database Replication Question

2015-03-04 Thread Xiao
Hey Jay, Yeah. I understood the advantage of Kafka is one to many. That is why I am reading the source codes of Kafka. Your guys did a good product! : ) Our major concern is its message persistency. Zero data loss is a must in our applications. Below is what I copied from the Kafka document.

Re: Database Replication Question

2015-03-04 Thread Jay Kreps
Hey Xiao, 1. Nothing prevents applying transactions transactionally on the destination side, though that is obviously more work. But I think the key point here is that much of the time the replication is not Oracle=>Oracle, but Oracle=>{W, X, Y, Z} where W/X/Y/Z are totally heterogenous systems th

Re: Database Replication Question

2015-03-04 Thread Jay Kreps
Hey Josh, NoSQL DBs may actually be easier because they themselves generally don't have a global order. I.e. I believe Mongo has a per-partition oplog, is that right? Their partitions would match our partitions. -Jay On Wed, Mar 4, 2015 at 5:18 AM, Josh Rader wrote: > Thanks everyone for your

Re: Database Replication Question

2015-03-04 Thread Xiao
Hi, Josh, That depends on how you implemented it. Basically, Kafka can provide a good throughput only when you have multiple partitions. - If you have multiple consumers and multiple partitions, each of which has a dedicated partition. That means, you need a coordinator to ensure all the c

Re: Database Replication Question

2015-03-04 Thread Josh Rader
Thanks everyone for your responses! These are great. It seems our cases matches closest to Jay's recommendations. The one part that sounds a little tricky is point #5 'Include in each message the database's transaction id, scn, or other identifier '. This is pretty straightforward with the RDBM

Re: Database Replication Question

2015-03-03 Thread Xiao
Hey Josh, Sorry, after reading codes, Kafka did fsync the data using a separate thread. The recovery point (oldest transaction timestamp) can be got from the file recovery-point-offset-checkpoint. You can adjust the value config.logFlushOffsetCheckpointIntervalMs, if you think the speed is n

Re: Database Replication Question

2015-03-03 Thread Xiao
Hey Josh, If you put different tables into different partitions or topics, it might break transaction ACID at the target side. This is risky for some use cases. Besides unit of work issues, you also need to think about the load balancing too. For failover, you have to find the timestamp for p

Re: Database Replication Question

2015-03-03 Thread Guozhang Wang
Additionally to Jay's recommendation, you also need to have some special cares in error handling of the producer in order to preserve ordering since producer uses batching and async sending. That is, if you already sent messages 1,2,3,4,5 to producer but later on be notified that message 3 failed t

Re: Database Replication Question

2015-03-03 Thread Xiao
Hey Josh, Transactions can be applied in parallel in the consumer side based on transaction dependency checking. http://www.google.com.ar/patents/US20080163222 This patent documents how it work. It is easy to understand, however, you also need to consider the hash collision issues. This has b

Re: Database Replication Question

2015-03-03 Thread Jay Kreps
Hey Josh, As you say, ordering is per partition. Technically it is generally possible to publish all changes to a database to a single partition--generally the kafka partition should be high throughput enough to keep up. However there are a couple of downsides to this: 1. Consumer parallelism is l

Re: Database Replication Question

2015-03-03 Thread Stevo Slavić
Have you considered including order information in messages that are sent to Kafka, and then restoring order in logic that is processing messages consumed from Kafka? http://www.enterpriseintegrationpatterns.com/Resequencer.html Kind regards, Stevo Slavic. On Wed, Mar 4, 2015 at 12:15 AM, Josh Ra