Re: [DISCUSS] KIP-253: Support in-order message delivery with partition expansion

2018-03-27 Thread Dong Lin
Hey Jan, Thanks for the enthusiasm in improving Kafka's design. Now that I have read through your discussion with Jun, here are my thoughts: - The latest proposal should with log compacted topics by properly deleting old messages after a new message with the same key is produced. So it is probabl

Re: [DISCUSS] KIP-253: Support in-order message delivery with partition expansion

2018-03-27 Thread Dong Lin
On Tue, Mar 27, 2018 at 12:04 AM, Dong Lin wrote: > Hey Jan, > > Thanks for the enthusiasm in improving Kafka's design. Now that I have > read through your discussion with Jun, here are my thoughts: > > - The latest proposal should with log compacted topics by properly > deleting old messages aft

[jira] [Created] (KAFKA-6717) TopicPartition Assined twice to a consumer group for different consumers

2018-03-27 Thread Yuancheng PENG (JIRA)
Yuancheng PENG created KAFKA-6717: - Summary: TopicPartition Assined twice to a consumer group for different consumers Key: KAFKA-6717 URL: https://issues.apache.org/jira/browse/KAFKA-6717 Project: Ka

Producer timeout and retry behavior

2018-03-27 Thread Sandor Murakozi
Hi All, I have a question about intended producer behavior if the broker is lost - do I see a bug or the code works as specified? What I do and see using trunk: *a) No message send timeout at all if there is no available broker* - no broker is started - consoleproducer is started using --broker-

Re: [VOTE] KIP-268: Simplify Kafka Streams Rebalance Metadata Upgrade

2018-03-27 Thread zhenya Sun
I'm terribly sorry that I made such a mistake. Thank you for your suggestion! | | zhenya Sun 邮箱:toke...@126.com | 签名由 网易邮箱大师 定制 On 03/27/2018 04:38, Matthias J. Sax wrote: Thanks Ted for pointing out. Sorry for this mistake Zhenya. You should not have voted twice though ;) (Even if it wa

Re: Producer timeout and retry behavior

2018-03-27 Thread Manikumar
Hi, a) If there are no brokers available, producer send() will block up to max_block_ms (default: 6) after which it will throw a TimeoutException. No retries in this case. by-default console producer uses Callback and just logs the error after max_block_ms. b) Current behavior is, If

[jira] [Created] (KAFKA-6718) Rack Aware Replica Task Assignment for Kafka Streams

2018-03-27 Thread Deepak Goyal (JIRA)
Deepak Goyal created KAFKA-6718: --- Summary: Rack Aware Replica Task Assignment for Kafka Streams Key: KAFKA-6718 URL: https://issues.apache.org/jira/browse/KAFKA-6718 Project: Kafka Issue Type:

Re: [VOTE] 1.1.0 RC4

2018-03-27 Thread Rajini Sivaram
Can we get some more votes for this RC so that the release can be rolled out soon? Many thanks, Rajini On Sat, Mar 24, 2018 at 6:54 PM, Ted Yu wrote: > I wasn't able to reproduce the test failure when it is run alone. > > This seems to be flaky test. > > +1 from me. > > On Sat, Mar 24, 2018 at

Re: [kafka-clients] Re: [VOTE] 1.1.0 RC4

2018-03-27 Thread Manikumar
+1 (non-binding) - Verified src, binary artifacts and basic quick start - Verified delegation token operations and docs - Verified dynamic broker configuration and docs. On Tue, Mar 27, 2018 at 6:52 PM, Rajini Sivaram wrote: > Can we get some more votes for this RC so that the release can be r

Re: [kafka-clients] Re: [VOTE] 1.1.0 RC4

2018-03-27 Thread Jason Gustafson
+1 Went through the quickstart, checked upgrade documentation. Thanks Rajini! On Tue, Mar 27, 2018 at 6:28 AM, Manikumar wrote: > +1 (non-binding) > > - Verified src, binary artifacts and basic quick start > - Verified delegation token operations and docs > - Verified dynamic broker configuratio

Re: [DISCUSS] KIP-262 Metadata should include the number of state stores for task

2018-03-27 Thread Richard Yu
Hi Matthias, Sorry for taking so long to get back to you. I will change my KIP to also include the old format. Thanks, Richard On Thu, Mar 22, 2018 at 9:15 PM, Matthias J. Sax wrote: > Hi Richard, > > with KIP-268 in place (should be accepted soon) the upgrade path is > covered. Thus, you can

[jira] [Resolved] (KAFKA-6446) KafkaProducer with transactionId endless waits when bootstrap server is down

2018-03-27 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-6446?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson resolved KAFKA-6446. Resolution: Fixed Fix Version/s: 1.2.0 > KafkaProducer with transactionId endless wai

Build failed in Jenkins: kafka-trunk-jdk7 #3289

2018-03-27 Thread Apache Jenkins Server
See Changes: [jason] KAFKA-6446; KafkaProducer initTransactions() should timeout after -- Started by an SCM change [EnvInject] - Loading node environment variables. Buildi

Permission to start KIP on 'Rack-Aware Replica Task Assignment' for Kafka Streams

2018-03-27 Thread Deepak Goyal
Hi, Feature: Rack Aware Replica Task Assignment for Kafka Streams Following the motivation to KIP-36 Rack aware replica assignment, we have a similar requirement for assigning replica tasks to as many differ

Build failed in Jenkins: kafka-trunk-jdk8 #2504

2018-03-27 Thread Apache Jenkins Server
See Changes: [jason] KAFKA-6446; KafkaProducer initTransactions() should timeout after -- Started by an SCM change [EnvInject] - Loading node environment variables. Buildi

Re: Permission to start KIP on 'Rack-Aware Replica Task Assignment' for Kafka Streams

2018-03-27 Thread Guozhang Wang
Hello Deepak, We've granted you to the permission, please feel free to go to the KIP wiki page and follow the steps to create a new KIP. https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Improvement+Proposals Guozhang On Tue, Mar 27, 2018 at 6:32 AM, Deepak Goyal wrote: > Hi, > > Featur

Re: [kafka-clients] Re: [VOTE] 1.1.0 RC4

2018-03-27 Thread Jeff Chao
Hello, +1 (non-binding). Ran through our regression and performance suite. Looks good, thanks. Jeff Chao Heroku On Tue, Mar 27, 2018 at 8:44 AM, Jason Gustafson wrote: > +1 Went through the quickstart, checked upgrade documentation. Thanks > Rajini! > > On Tue, Mar 27, 2018 at 6:28 AM, Manikum

Re: [kafka-clients] Re: [VOTE] 1.1.0 RC4

2018-03-27 Thread Vahid S Hashemian
Hi Rajini, +1 (non-binding) Built from source on Linux and Windows (Java 8), and tested quickstart on both platforms. Connect quickstart on Windows is not working as per my note on RC3: https://www.mail-archive.com/dev@kafka.apache.org/msg86138.html No other issues detected. Thanks! --Vahid

[jira] [Created] (KAFKA-6719) Kafka Reassign Partitions Failure

2018-03-27 Thread Srinivas Dhruvakumar (JIRA)
Srinivas Dhruvakumar created KAFKA-6719: --- Summary: Kafka Reassign Partitions Failure Key: KAFKA-6719 URL: https://issues.apache.org/jira/browse/KAFKA-6719 Project: Kafka Issue Type: Bu

Re: [DISCUSS] KIP-253: Support in-order message delivery with partition expansion

2018-03-27 Thread John Roesler
Hey Dong and Jun, Thanks for the thoughtful responses. If you don't mind, I'll mix my replies together to try for a coherent response. I'm not too familiar with mailing-list etiquette, though. I'm going to keep numbering my points because it makes it easy for you all to respond. Point 1: As I re

[jira] [Resolved] (KAFKA-6719) Kafka Reassign Partitions Failure

2018-03-27 Thread Srinivas Dhruvakumar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-6719?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Srinivas Dhruvakumar resolved KAFKA-6719. - Resolution: Invalid > Kafka Reassign Partitions Failure > ---

Re: [DISCUSS] KIP-253: Support in-order message delivery with partition expansion

2018-03-27 Thread Dong Lin
Hey John, Thanks much for the detailed comments. Here are my thoughts: - The need to delete messages from log compacted topics is mainly for performance (e.g. storage space) optimization than for correctness for this KIP. I agree that we probably don't need to focus on this in our discussion sinc

Re: [DISCUSS] KIP-253: Support in-order message delivery with partition expansion

2018-03-27 Thread Dong Lin
BTW, here is my understanding of the scope of this KIP. We want to allow consumers to always consume messages with the same key from the same producer in the order they are produced. And we need to provide a way for stream use-case to be able to flush/load state when messages with the same key are

Re: [kafka-clients] Re: [VOTE] 1.1.0 RC4

2018-03-27 Thread Jun Rao
Hi, Rajini, Thanks for running the release. Verified quickstart on scala 2.11 binary. +1 Jun On Tue, Mar 27, 2018 at 6:22 AM, Rajini Sivaram wrote: > Can we get some more votes for this RC so that the release can be rolled > out soon? > > Many thanks, > > Rajini > > On Sat, Mar 24, 2018 at 6:

Re: [VOTE] KIP-257 - Configurable Quota Management

2018-03-27 Thread Jun Rao
Hi, Rajini, Thanks for the KIP. +1 Jun On Thu, Mar 22, 2018 at 2:56 PM, Rajini Sivaram wrote: > Hi all, > > I would like to start vote on KIP-257 to enable customisation of client > quota computation: > > https://cwiki.apache.org/confluence/display/KAFKA/KIP- > 257+-+Configurable+Quota+Managem

Build failed in Jenkins: kafka-trunk-jdk7 #3290

2018-03-27 Thread Apache Jenkins Server
See Changes: [jason] MINOR: Fix -- Started by an SCM change [EnvInject] - Loading node environment variables. Building remotely on H24 (ubuntu xenial) in workspace

Build failed in Jenkins: kafka-0.10.2-jdk7 #207

2018-03-27 Thread Apache Jenkins Server
See Changes: [github] MINOR: Fix typos in docs and address some missed review comments (#4778) -- Started by an SCM change [EnvInject] - Loading node environment variable

[jira] [Resolved] (KAFKA-6473) Add MockProcessorContext to public test-utils

2018-03-27 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-6473?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax resolved KAFKA-6473. Resolution: Fixed Fix Version/s: 1.2.0 > Add MockProcessorContext to public test-util

Build failed in Jenkins: kafka-trunk-jdk8 #2505

2018-03-27 Thread Apache Jenkins Server
See Changes: [jason] MINOR: Fix [mjsax] KAFKA-6473: Add MockProcessorContext to public test-utils (#4736) -- Started by an SCM change [EnvInject] - Loading node environme

Build failed in Jenkins: kafka-trunk-jdk7 #3291

2018-03-27 Thread Apache Jenkins Server
See Changes: [mjsax] KAFKA-6473: Add MockProcessorContext to public test-utils (#4736) -- Started by an SCM change [EnvInject] - Loading node environment variables. Buildi

[jira] [Created] (KAFKA-6720) Inconsistent Kafka Streams behaviour when topic does not exist

2018-03-27 Thread Daniel Wojda (JIRA)
Daniel Wojda created KAFKA-6720: --- Summary: Inconsistent Kafka Streams behaviour when topic does not exist Key: KAFKA-6720 URL: https://issues.apache.org/jira/browse/KAFKA-6720 Project: Kafka I

[jira] [Resolved] (KAFKA-6720) Inconsistent Kafka Streams behaviour when topic does not exist

2018-03-27 Thread Mariam John (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-6720?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Mariam John resolved KAFKA-6720. Resolution: Duplicate This is similar to KAFKA-6437. > Inconsistent Kafka Streams behaviour when to

[jira] [Created] (KAFKA-6721) Consolidate state store management for global stores and normal stores

2018-03-27 Thread Guozhang Wang (JIRA)
Guozhang Wang created KAFKA-6721: Summary: Consolidate state store management for global stores and normal stores Key: KAFKA-6721 URL: https://issues.apache.org/jira/browse/KAFKA-6721 Project: Kafka

Build failed in Jenkins: kafka-trunk-jdk8 #2506

2018-03-27 Thread Apache Jenkins Server
See Changes: [wangguoz] KAFKA-6386: Use Properties instead of StreamsConfig in KafkaStreams -- Started by an SCM change [EnvInject] - Loading node environment variables. B

Re: [DISCUSS] KIP-253: Support in-order message delivery with partition expansion

2018-03-27 Thread Jun Rao
Hi, John, Thanks for the reply. I agree that the backfill approach works cleaner for newly started consumers. I am just not sure if it's a good primitive to support for existing consumers. One of the challenges that I see is the remapping of the offsets. In your approach, we need to copy the exist

Re: [DISCUSS] KIP-253: Support in-order message delivery with partition expansion

2018-03-27 Thread John Roesler
Thanks for the response, Dong. Here are my answers to your questions: - "Asking producers and consumers, or even two different producers, to > share code like the partition function is a pretty huge ask. What if they > are using different languages?". It seems that today we already require > diff

Build failed in Jenkins: kafka-trunk-jdk7 #3292

2018-03-27 Thread Apache Jenkins Server
See Changes: [wangguoz] KAFKA-6386: Use Properties instead of StreamsConfig in KafkaStreams -- Started by an SCM change [EnvInject] - Loading node environment variables. B

Re: [DISCUSS] KIP-253: Support in-order message delivery with partition expansion

2018-03-27 Thread John Roesler
Hi Jun, That's a good point. Yeah, I don't think it would work too well for existing consumers in the middle of gen 0 to try and switch to a newly backfilled prefix of gen 1. They probably just need to finish up until they get to the end of gen 0 and transition just as if there were no backfill a

Re: Gradle strategy for exposing and using public test-utils modules

2018-03-27 Thread John Roesler
Hi again everyone, Just for the sake of closure, I think everyone is generally in agreement with this approach. If concerns arise later on, please let me know! Thanks, -John On Fri, Mar 23, 2018 at 12:41 AM, zhenya Sun wrote: > +1 > > 在 2018年3月23日,下午12:20,Ted Yu 写道: > > > > +1 > > Or

[jira] [Created] (KAFKA-6722) SensorAccess.getOrCreate should be more efficient

2018-03-27 Thread wade wu (JIRA)
wade wu created KAFKA-6722: -- Summary: SensorAccess.getOrCreate should be more efficient Key: KAFKA-6722 URL: https://issues.apache.org/jira/browse/KAFKA-6722 Project: Kafka Issue Type: Improvement

Re: [VOTE] 1.1.0 RC4

2018-03-27 Thread Brett Rann
+1 non binding. downloaded kafka_2.11-1.1.0.tgz artifact and upgraded in place including message format / broker protocol in our smallest non-production environment. Upgraded path has been 1.0.1->1.1.0-rc3->1.1.0-rc4 and it has run fine on rc4 for 2 days. We have a mixture of 0.9 through to 1.0.1

[jira] [Created] (KAFKA-6723) Separate "max.poll.record" for restore consumer and common consumer

2018-03-27 Thread Boyang Chen (JIRA)
Boyang Chen created KAFKA-6723: -- Summary: Separate "max.poll.record" for restore consumer and common consumer Key: KAFKA-6723 URL: https://issues.apache.org/jira/browse/KAFKA-6723 Project: Kafka

Build failed in Jenkins: kafka-trunk-jdk8 #2507

2018-03-27 Thread Apache Jenkins Server
See Changes: [github] MINOR: LogCleaner.validateReconfiguration fixes (#4770) -- Started by an SCM change [EnvInject] - Loading node environment variables. Building remote

Build failed in Jenkins: kafka-trunk-jdk7 #3293

2018-03-27 Thread Apache Jenkins Server
See Changes: [github] MINOR: LogCleaner.validateReconfiguration fixes (#4770) -- Started by an SCM change [EnvInject] - Loading node environment variables. Building remote

Re: [VOTE] 1.1.0 RC4

2018-03-27 Thread Gwen Shapira
+1 Checked keys, built, ran quickstart. LGTM. On Fri, Mar 23, 2018 at 4:37 PM, Rajini Sivaram wrote: > Hello Kafka users, developers and client-developers, > > This is the fifth candidate for release of Apache Kafka 1.1.0. > > https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=759

Re: [DISCUSS] KIP-253: Support in-order message delivery with partition expansion

2018-03-27 Thread Jan Filipiak
Thank you so much John Roesler! <3 Thank you for also seeing the core strengths of apache kafka! We just cannot make sacrifices like this to the architecture for the benefits of streams. Streams somehow got lost in the whole "Interactive Query" idea that is spreading like cancer across the p

Build failed in Jenkins: kafka-trunk-jdk8 #2508

2018-03-27 Thread Apache Jenkins Server
See Changes: [jason] MINOR: Update Jackson to 2.9.5 (#4776) -- Started by an SCM change [EnvInject] - Loading node environment variables. Building remotely on H32 (ubuntu

Re: [DISCUSS] KIP-253: Support in-order message delivery with partition expansion

2018-03-27 Thread Dong Lin
Hey John, Great! Thanks for all the comment. It seems that we agree that the current KIP is in good shape for core Kafka. IMO, what we have been discussing in the recent email exchanges is mostly about the second step, i.e. how to address problem for the stream use-case (or stateful processing in

Build failed in Jenkins: kafka-trunk-jdk7 #3294

2018-03-27 Thread Apache Jenkins Server
See Changes: [jason] MINOR: Update Jackson to 2.9.5 (#4776) -- Started by an SCM change [EnvInject] - Loading node environment variables. Building remotely on H20 (ubuntu