Re: [DISCUSS] KIP-113: Support replicas movement between log directories

2017-08-11 Thread Jun Rao
Hi, Tom, The inter-broker-with-log-dirs case can be split into inter-broker-w/o-log-dirs and log-dirs change per broker. KIP-113 proposes to do the split in the tool. I am not sure if we really need to persist log-dirs changes in ZK. During the discussion in KIP-113, we realized that there is

Re: [DISCUSS] KIP-185: Make exactly once in order delivery per partition the default producer setting

2017-08-11 Thread Becket Qin
Hi Apurva, I agree that most changes we are talking about here are for default values of the configurations and users can always override them. So I think the question to ask is more about the out of the box experience. If the change makes strict improvement compared with the current settings,

Re: [DISCUSS] KIP-183 - Change PreferredReplicaLeaderElectionCommand to use AdminClient

2017-08-11 Thread Jun Rao
Hi, Tom, 2. Yes, that's how manual preferred leader election currently works. I think we could use this opportunity to simplify the process a bit though. Doing preferred leader election is a lot cheaper than partition reassignment since it only involves writing the new leader to ZK. So, we

Re: [VOTE] KIP-163: Lower the Minimum Required ACL Permission of OffsetFetch

2017-08-11 Thread Vahid S Hashemian
Thank you all for your votes. The KIP has passed with 3 binding and 2 non-binding +1's: Binding +1's: * Jason * Ewen * Jun Non-binding +1's: * Mickael * Manikumar --Vahid From: "Vahid S Hashemian" To: dev Date: 08/02/2017 02:40 PM

Using Java Kafka Producer API for nearly Real Time Applications

2017-08-11 Thread Pavel Moukhataev
Hi Sometimes kafka is used in nearly real-time java applications that has low latency requirements. In that case it is very important to minify latency. In kafka producer API there are two things that are done synchronously and can be optimized: - cluster metadata fetch - wait for free memory

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

2017-08-11 Thread Apache Jenkins Server
See Changes: [damian.guy] MINOR: First cut at porting State Store docs to AK -- [...truncated 867.60 KB...] kafka.utils.json.JsonValueTest > testDecodeInt STARTED

[GitHub] kafka pull request #3656: MINOR: More failure testing

2017-08-11 Thread enothereska
GitHub user enothereska opened a pull request: https://github.com/apache/kafka/pull/3656 MINOR: More failure testing You can merge this pull request into a Git repository by running: $ git pull https://github.com/enothereska/kafka minor-add-more-tests Alternatively you can

Re: [VOTE] KIP-163: Lower the Minimum Required ACL Permission of OffsetFetch

2017-08-11 Thread Manikumar
+1 (non-binding) On Fri, Aug 11, 2017 at 5:31 AM, Jun Rao wrote: > Hi, Vahid, > > Thanks for the KIP. +1 from me. > > Jun > > On Wed, Aug 2, 2017 at 2:40 PM, Vahid S Hashemian < > vahidhashem...@us.ibm.com > > wrote: > > > Hi all, > > > > Thanks to everyone who participated

[GitHub] kafka pull request #3655: KAFKA-5724: AbstractPartitionAssignor should suppo...

2017-08-11 Thread huxihx
GitHub user huxihx opened a pull request: https://github.com/apache/kafka/pull/3655 KAFKA-5724: AbstractPartitionAssignor should support assignment for topics with non-consecutive partitions Current design does consider the siutation when user creates a topic via KafkaAdminClient

[GitHub] kafka pull request #3629: MINOR: First cut at porting State Store docs to AK

2017-08-11 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/3629 --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is

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

2017-08-11 Thread Apache Jenkins Server
See Changes: [damian.guy] MINOR: First cut at porting State Store docs to AK -- [...truncated 2.00 MB...] org.apache.kafka.common.security.scram.ScramMessagesTest >

Re: [DISCUSS] KIP-113: Support replicas movement between log directories

2017-08-11 Thread Tom Bentley
Hi Jun and Dong, Thanks for your replies... On 10 August 2017 at 20:43, Dong Lin wrote: > This is a very good idea. I have updated the KIP-113 so that > DescribeDirResponse returns lag instead of LEO. Excellent! On Thu, Aug 10, 2017 at 10:21 AM, Jun Rao

[GitHub] kafka pull request #3624: KAFKA-5702: extract refactor StreamThread

2017-08-11 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/3624 --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is

[GitHub] kafka pull request #3658: MINOR: AdminClient should register with `AppInfoPa...

2017-08-11 Thread ijuma
GitHub user ijuma opened a pull request: https://github.com/apache/kafka/pull/3658 MINOR: AdminClient should register with `AppInfoParser` Also make "created" message more consistent across clients. You can merge this pull request into a Git repository by running: $ git pull

[GitHub] kafka pull request #3657: Kafka-5676

2017-08-11 Thread DevelopersWithPassion
GitHub user DevelopersWithPassion opened a pull request: https://github.com/apache/kafka/pull/3657 Kafka-5676 1. Change package for MockStreamsMetrics class 2. Removed metric object from MockStreamsMetrics constcruor You can merge this pull request into a Git repository by

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

2017-08-11 Thread Apache Jenkins Server
See Changes: [damian.guy] KAFKA-5702; extract refactor StreamThread -- Started by an SCM change [EnvInject] - Loading node environment variables. Building remotely on

Re: [DISCUSS] KIP-183 - Change PreferredReplicaLeaderElectionCommand to use AdminClient

2017-08-11 Thread Tom Bentley
Hi Jun, Thanks for your reply, I've got a few comment inline... On 11 August 2017 at 01:51, Jun Rao wrote: > Hi, Tom, > > Thanks for the KIP. Looks good overall. A few minor comments. > > 1. In most requests with topic partitions, we nest partitions inside topic. > So,

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

2017-08-11 Thread Apache Jenkins Server
See Changes: [damian.guy] KAFKA-5702; extract refactor StreamThread -- [...truncated 932.49 KB...] kafka.producer.AsyncProducerTest > testInvalidConfiguration STARTED

Re: [DISCUSS] KIP-91 Provide Intuitive User Timeouts in The Producer

2017-08-11 Thread Sumant Tambe
Hi Jun, Thanks for looking into it. Yes, we did consider this message-level timeout approach and expiring batches selectively in a request but rejected it due to the reasons of added complexity without a strong benefit to counter-weigh that. Your proposal is a slight variation so I'll mention

[jira] [Created] (KAFKA-5727) Add the archetype project along with "write applications" web docs.

2017-08-11 Thread Guozhang Wang (JIRA)
Guozhang Wang created KAFKA-5727: Summary: Add the archetype project along with "write applications" web docs. Key: KAFKA-5727 URL: https://issues.apache.org/jira/browse/KAFKA-5727 Project: Kafka

[VOTE] KIP-180: Adding a new metric for brokers specifying the number of consumer group rebalances in progress

2017-08-11 Thread Colin McCabe
Hi all, I think it's a good time to vote on KIP-180. It adds a helpful new metric that shows consumer group states. The full proposal, and links to more discussion, are here:

[GitHub] kafka pull request #3661: KAFKA-4585: Lower the Minimum Required ACL Permiss...

2017-08-11 Thread vahidhashemian
GitHub user vahidhashemian opened a pull request: https://github.com/apache/kafka/pull/3661 KAFKA-4585: Lower the Minimum Required ACL Permission of OffsetFetch (KIP-163) Details can be found in the

Re: [DISCUSS] KIP-185: Make exactly once in order delivery per partition the default producer setting

2017-08-11 Thread Apurva Mehta
Thanks for your email Becket. I would be interested in hearing others opinions on which should be a better default between acks=1 and acks=all. One important point on which I disagree is your statement that 'users need to do a lot of work to get exactly-once with acks=all'. This is debatable. If

[jira] [Resolved] (KAFKA-1379) Partition reassignment resets clock for time-based retention

2017-08-11 Thread Andrew Olson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1379?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Andrew Olson resolved KAFKA-1379. - Resolution: Fixed Assignee: Jiangjie Qin Fix Version/s: 0.10.1.0 Marking this bug

[GitHub] kafka pull request #3630: KAFKA-5727: Add Streams quickstart tutorial as an ...

2017-08-11 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/3630 --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is

[jira] [Resolved] (KAFKA-5727) Add the archetype project along with "write applications" web docs.

2017-08-11 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5727?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang resolved KAFKA-5727. -- Resolution: Fixed Fix Version/s: 1.0.0 Issue resolved by pull request 3630

[GitHub] kafka pull request #3660: KAFKA-5727: Archetype project for Streams quicksta...

2017-08-11 Thread guozhangwang
GitHub user guozhangwang opened a pull request: https://github.com/apache/kafka/pull/3660 KAFKA-5727: Archetype project for Streams quickstart and tutorial web docs You can merge this pull request into a Git repository by running: $ git pull

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

2017-08-11 Thread Apache Jenkins Server
See Changes: [wangguoz] KAFKA-5727: Add Streams quickstart tutorial as an archetype project -- [...truncated 911.74 KB...] kafka.integration.TopicMetadataTest >

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

2017-08-11 Thread Apache Jenkins Server
See Changes: [wangguoz] HOTFIX: missing imports and version on web docs -- Started by an SCM change [EnvInject] - Loading node environment variables. Building remotely on

[jira] [Created] (KAFKA-5728) Stopping consumer thread cause loosing message in the partition

2017-08-11 Thread Vasudevan Karnan (JIRA)
Vasudevan Karnan created KAFKA-5728: --- Summary: Stopping consumer thread cause loosing message in the partition Key: KAFKA-5728 URL: https://issues.apache.org/jira/browse/KAFKA-5728 Project: Kafka

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

2017-08-11 Thread Apache Jenkins Server
See Changes: [wangguoz] HOTFIX: missing imports and version on web docs -- [...truncated 2.00 MB...] org.apache.kafka.common.security.scram.ScramMessagesTest >

Re: [VOTE] KIP-180: Adding a new metric for brokers specifying the number of consumer group rebalances in progress

2017-08-11 Thread Bill Bejeck
+1 Thanks, Bill On Fri, Aug 11, 2017 at 6:00 PM, Colin McCabe wrote: > Hi all, > > I think it's a good time to vote on KIP-180. It adds a helpful new > metric that shows consumer group states. > > The full proposal, and links to more discussion, are here: >

Jenkins build is back to normal : kafka-trunk-jdk7 #2633

2017-08-11 Thread Apache Jenkins Server
See

[jira] [Created] (KAFKA-5731) Connect WorkerSinkTask out of order offset commit can lead to inconsistent state

2017-08-11 Thread Jason Gustafson (JIRA)
Jason Gustafson created KAFKA-5731: -- Summary: Connect WorkerSinkTask out of order offset commit can lead to inconsistent state Key: KAFKA-5731 URL: https://issues.apache.org/jira/browse/KAFKA-5731

Re: [DISCUSS] KIP-185: Make exactly once in order delivery per partition the default producer setting

2017-08-11 Thread Becket Qin
Hi Apurva, Thanks for the reply. When I was thinking of exactly once I am thinking of "exactly once with availability", Users probably wouldn't want to sacrifice availability for exactly once. To achieve exactly once with same availability and acks=all, users actually need to pay more cost. To

[jira] [Created] (KAFKA-5729) Consumer should verify offset commits are from assigned partitions

2017-08-11 Thread Jason Gustafson (JIRA)
Jason Gustafson created KAFKA-5729: -- Summary: Consumer should verify offset commits are from assigned partitions Key: KAFKA-5729 URL: https://issues.apache.org/jira/browse/KAFKA-5729 Project: Kafka

[jira] [Created] (KAFKA-5730) Consumer should ensure consistent ordering between sync and async offset commits

2017-08-11 Thread Jason Gustafson (JIRA)
Jason Gustafson created KAFKA-5730: -- Summary: Consumer should ensure consistent ordering between sync and async offset commits Key: KAFKA-5730 URL: https://issues.apache.org/jira/browse/KAFKA-5730

Re: Using Java Kafka Producer API for nearly Real Time Applications

2017-08-11 Thread Pavel Moukhataev
Imagine I have application with very strong requirements about latency. And I want to save my data to kafka for each event (many times per second). And my application can't wait for metadata fetch. So I need to do something quickly from my main thread. If there is kafka problem then I can

Re: [DISCUSS] KIP-185: Make exactly once in order delivery per partition the default producer setting

2017-08-11 Thread Becket Qin
BTW, I feel that the configurations we have around those guarantees have become too complicated for the users. Not sure if this is considered before but Maybe we can have some helper functions provided to the users. For example: Properties TopicConfig.forSemantc(Semantic semantic); Properties

Re: Using Java Kafka Producer API for nearly Real Time Applications

2017-08-11 Thread Apurva Mehta
Thanks for clarifying. There are no plans to implement a 100% async kafka client which you decribe. However, if you file a KIP with a detailed proposal for how you want to implement it, we can then have a discussion about the tradeoffs in terms of additional code complexity and also with other use

Re: [DISCUSS] KIP-91 Provide Intuitive User Timeouts in The Producer

2017-08-11 Thread Jun Rao
Hi, Sumant, 1. Yes, it's probably reasonable to require max.message.delivery.wait.ms > linger.ms. As for retries, perhaps we can set the default retries to infinite or just ignore it. Then the latency will be bounded by max.message.delivery.wait.ms. request.timeout.ms is the max time the request

Re: [VOTE] KIP-180: Adding a new metric for brokers specifying the number of consumer group rebalances in progress

2017-08-11 Thread Apurva Mehta
+1, Thanks, Apurva On Fri, Aug 11, 2017 at 3:02 PM, Bill Bejeck wrote: > +1 > > Thanks, > Bill > > On Fri, Aug 11, 2017 at 6:00 PM, Colin McCabe wrote: > > > Hi all, > > > > I think it's a good time to vote on KIP-180. It adds a helpful new > > metric

[jira] [Created] (KAFKA-5725) Additional failure testing for streams with bouncing brokers

2017-08-11 Thread Eno Thereska (JIRA)
Eno Thereska created KAFKA-5725: --- Summary: Additional failure testing for streams with bouncing brokers Key: KAFKA-5725 URL: https://issues.apache.org/jira/browse/KAFKA-5725 Project: Kafka

Re: [VOTE] KIP-175: Additional '--describe' views for ConsumerGroupCommand

2017-08-11 Thread Mickael Maison
+1 non-binding, thanks Vahid On Wed, Aug 9, 2017 at 9:31 PM, Jason Gustafson wrote: > Thanks for the KIP. +1 > > On Thu, Jul 27, 2017 at 2:04 PM, Vahid S Hashemian < > vahidhashem...@us.ibm.com> wrote: > >> Hi all, >> >> Thanks to everyone who participated in the discussion

[GitHub] kafka pull request #3659: KAFKA-4643: Improve test coverage of StreamsKafkaC...

2017-08-11 Thread adyach
GitHub user adyach opened a pull request: https://github.com/apache/kafka/pull/3659 KAFKA-4643: Improve test coverage of StreamsKafkaClient The commit brings improved test coverage for StreamsKafkaClientTest.java You can merge this pull request into a Git repository by running:

Re: Using Java Kafka Producer API for nearly Real Time Applications

2017-08-11 Thread Apurva Mehta
What precise use case do you have in mind? If you don't have cluster metadata, you can't send the requests anyway. If you want to bound your memory and run out of it, that means that you are not able to send data for some reason. The best you can do in both cases is to drop old messages from the

Re: [DISCUSS] KIP-113: Support replicas movement between log directories

2017-08-11 Thread Jun Rao
Hi, Tom, One approach is to have a PartitionReassignmentRequest that only deals with inter broker data movement (i.e, w/o any log dirs in the request). The request is directed to any broker, which then just writes the reassignment json to ZK. There is a separate AlterReplicaDirRequest that only

[jira] [Created] (KAFKA-5726) KafkaConsumer.subscribe() override that takes just Pattern without ConsumerRebalanceListener

2017-08-11 Thread Yeva Byzek (JIRA)
Yeva Byzek created KAFKA-5726: - Summary: KafkaConsumer.subscribe() override that takes just Pattern without ConsumerRebalanceListener Key: KAFKA-5726 URL: https://issues.apache.org/jira/browse/KAFKA-5726

[GitHub] kafka pull request #3654: KAFKA-5562: Do streams state directory cleanup on ...

2017-08-11 Thread dguy
Github user dguy closed the pull request at: https://github.com/apache/kafka/pull/3654 --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is

Re: [DISCUSS] KIP-113: Support replicas movement between log directories

2017-08-11 Thread Tom Bentley
Hi Jun, The inter-broker movement case has two subcases: 1. Where no log dir is supplied. This corresponds to the existing kafka-reassign-partitions script. This just needs the appropriate JSON to be written to the reassignment znode. 2. Where the log dir is supplied. This is covered in KIP-113

Re: [DISCUSS] KIP-91 Provide Intuitive User Timeouts in The Producer

2017-08-11 Thread Sumant Tambe
> Thanks for the KIP. Nice documentation on all current issues with the > timeout. For the KIP writeup, all credit goes to Joel Koshy. I'll follow up on your comments a little later. > > You also brought up a good use case for timing out a message. For > applications that collect and send