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 assume the mainframe and Kafka clusters are running in the same DC. 2) Through additional component (like IBM MQ) to send data from z/OS to LUW, implement JAVA producers on LUW servers by reading MQ messages to feed Kafka clusters. Do you think they can work? Any potential issue based on your experiences? Before starting implementation of prototypes, I have to understand if Kafka is a right solution for enterprise customers, if we can ignore the security issues in the initial stage. (I already saw the security-related features will be released next month.) Thank you for sharing the background with me, Guozhang! It helps me understand the reason why asynchronous fsync is introduced. However, I still think the original design can be kept. Users can choose a better solution based on their use cases. Replication is good for throughput and performance but it will increase the total operation costs and complicate the design of some Kafka applications. Occasional latency spikes might be ok for some use cases and some use cases do not have very high throughput requirements. Disk persistence is a fundamental requirement for us. : ( Zero data loss is always assumed; otherwise, it requires a very complicated recovery procedure. Cross center replication is not acceptable to most customers. It will reduce both performance and usability. The simplest solution is using dual power supplies in Kafka deployment. I am not sure if this is enough. BTW, the design proposal of “transactional messaging” misses a design change in the Log recovery? Recovery checkpoints might be in the middle of multiple in-flight transactions. Thank you very much! Xiao Li On Mar 10, 2015, at 1:01 PM, Jiangjie Qin <j...@linkedin.com.INVALID> wrote: > 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 cross colo > is that Kafka is not designed to tolerate network partition - which is > more likely to occur when you deploy Kafka cross multiple DC. Another > option is to run separate Kafka clusters in different data center and do > double writes to two separate clusters. But that definitely needs more > work to ensure consistence. > > Thanks. > > Jiangjie (Becket) Qin > > > > On 3/10/15, 8:27 AM, "Guozhang Wang" <wangg...@gmail.com> wrote: > >> 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. >> >> As I understands your situation, you are mainly concerned with data center >> power outage tolerance via disk persistence, which cannot be easily >> supported via asynchronous time-based fsync. We used to do synchronized >> fsync before data replication is introduced, and later switched to async >> fsync for mainly for performance (KAFKA-615 >> <https://issues.apache.org/jira/browse/KAFKA-615>), since we see the disk >> IO spikes can some times cause the whole broker to slow down quite a lot >> due to processor thread pool sharing. On the other hand, even with fsync, >> data loss may still happen upon power outage depending on your file system >> journaling settings. Another alternative to achieve data center failures >> is >> geo-replication: you can try to put your replicas in different data >> centers, more specifically: >> >> 1) you can use one producer to send normal data with ack mode = 1 for not >> waiting cross-colo replication latency. >> 2) at each batch boundary, you can use another producer to send a "commit" >> message with ack mode = -1, which requires all data before this message to >> be already replicated, to mimic the "group committing" behavior. >> >> This will of course still increase the write latency and require a >> cross-colo ZK setup. >> >> Guozhang >> >> On Mon, Mar 9, 2015 at 10:56 PM, Xiao <lixiao1...@gmail.com> 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? >>> >>> 1. Unfortunately, we aims at supporting all the platforms, Linux, Unix, >>> Windows and especially z/OS. I know z/OS is not easy to support. >>> >>> 2. Fsync per message is very expensive and Fsync per batch will break >>> the >>> transaction atomicity. We are looking for transaction-level fsync, >>> which is >>> more efficient. Then, our producers can easily combine multiple small >>> transactions into a single bigger batch transaction. I hope the >>> implementation of the ongoing Kafka feature ³Transactional Messaging in >>> Kafka² already considered all these issues, although the following link >>> does not mention it: >>> >>> https://cwiki.apache.org/confluence/display/KAFKA/Transactional+Messaging >>> +in+Kafka >>> >>> 3. After a quick reading of the design of ³Transactional Messaging in >>> Kafka², I have a doubt if it can scale especially when most transactions >>> are very short (e.g., containing a single message). >>> - In many use cases, we do not need global transactions, which >>> might be too expensive. The partition-specific transaction granularity >>> might be fine. >>> - To reduce the overhead of transaction-level fsync + network or >>> address-space roundtrips, Kafka might need two extra parameters for >>> batching the small transactions into a single one. The performance >>> benefits >>> of batching are huge, as shown in the MRI (multi-row insert) feature in >>> Oracle and DB2 z/OS. >>> >>> I believe transactional messaging is a critical feature. The design >>> document is not very clear. Do you have more materials or links about >>> it? >>> >>> Thanks, >>> >>> Xiao Li >>> >>> >>> On Mar 7, 2015, at 9:33 AM, Jay Kreps <jay.kr...@gmail.com> wrote: >>> >>>> 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 <lixiao1...@gmail.com> wrote: >>>> >>>>> 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()). >>>>> >>>>> Fsync (fileOutputStream.getFD().sync()) is controlled by another >>> parameter >>>>> log.default.flush.scheduler.interval.ms. >>>>> >>>>> scheduler.schedule("kafka-recovery-point-checkpoint", >>>>> checkpointRecoveryPointOffsets, >>>>> delay = InitialTaskDelayMs, >>>>> period = flushCheckpointMs, >>>>> TimeUnit.MILLISECONDS) >>>>> >>>>> This thread is only time-controlled. It does not check the number of >>>>> messages. >>>>> >>>>> Thank you, >>>>> >>>>> Xiao Li >>>>> >>>>> >>>>> On Mar 5, 2015, at 11:59 AM, Jay Kreps <jay.kr...@gmail.com> wrote: >>>>> >>>>>> 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 >>>>> >>>>> >>> >>> >> >> >> -- >> -- Guozhang >