[jira] [Work stopped] (KAFKA-4938) Creating a connector with missing name parameter throws a NullPointerException

2017-04-03 Thread Balint Molnar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4938?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Work on KAFKA-4938 stopped by Balint Molnar. > Creating a connector with missing name parameter throws a NullPointerException

[jira] [Commented] (KAFKA-4997) Issue with running kafka-acls.sh when using SASL between Kafka and ZK

2017-04-03 Thread Rajini Sivaram (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4997?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15953144#comment-15953144 ] Rajini Sivaram commented on KAFKA-4997: --- The user principal (by default for Kerberos

[jira] [Commented] (KAFKA-4984) Unable to produce or consume when enabling authentication SASL/Kerberos

2017-04-03 Thread Ait haj Slimane (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4984?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15953162#comment-15953162 ] Ait haj Slimane commented on KAFKA-4984: Thank you LAkshmi for your quick reply ,

[jira] [Updated] (KAFKA-4984) Unable to produce or consume when enabling authentication SASL/Kerberos

2017-04-03 Thread Ait haj Slimane (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4984?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ait haj Slimane updated KAFKA-4984: --- Attachment: logKafka.txt logZookeeper.txt > Unable to produce or consume when

[DISCUSS] KIP-138: Change punctuate semantics

2017-04-03 Thread Michal Borowiecki
Hi all, I have created a draft for KIP-138: Change punctuate semantics . Appreciating there can be different views on system-time vs event-time semantics for punctuation depending on use-case and the importa

[jira] [Commented] (KAFKA-4984) Unable to produce or consume when enabling authentication SASL/Kerberos

2017-04-03 Thread lakshminarayanasyamala (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4984?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15953247#comment-15953247 ] lakshminarayanasyamala commented on KAFKA-4984: --- Have you tried restarting t

[GitHub] kafka pull request #2792: MINOR: Fix potential deadlock in consumer close te...

2017-04-03 Thread rajinisivaram
GitHub user rajinisivaram opened a pull request: https://github.com/apache/kafka/pull/2792 MINOR: Fix potential deadlock in consumer close test You can merge this pull request into a Git repository by running: $ git pull https://github.com/rajinisivaram/kafka MINOR-closetest-d

[jira] [Commented] (KAFKA-4984) Unable to produce or consume when enabling authentication SASL/Kerberos

2017-04-03 Thread lakshminarayanasyamala (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4984?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15953253#comment-15953253 ] lakshminarayanasyamala commented on KAFKA-4984: --- and how about a normal prod

[GitHub] kafka pull request #2793: KAFKA-4916: test streams with brokers failing (for...

2017-04-03 Thread enothereska
GitHub user enothereska opened a pull request: https://github.com/apache/kafka/pull/2793 KAFKA-4916: test streams with brokers failing (for 0.10.2) You can merge this pull request into a Git repository by running: $ git pull https://github.com/enothereska/kafka KAFKA-4916-0.10

[jira] [Commented] (KAFKA-4916) Add streams tests with brokers failing

2017-04-03 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4916?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15953272#comment-15953272 ] ASF GitHub Bot commented on KAFKA-4916: --- GitHub user enothereska opened a pull reque

[jira] [Commented] (KAFKA-4984) Unable to produce or consume when enabling authentication SASL/Kerberos

2017-04-03 Thread Ait haj Slimane (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4984?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15953285#comment-15953285 ] Ait haj Slimane commented on KAFKA-4984: Yes i already restart the broker but the

Re: [VOTE] KIP-81: Bound Fetch memory usage in the consumer

2017-04-03 Thread Rajini Sivaram
+1 (non-binding) On Fri, Mar 31, 2017 at 5:36 PM, radai wrote: > possible priorities: > > 1. keepalives/coordination > 2. inter-broker-traffic > 3. produce traffic > 4. consume traffic > > (dont want to start a debate, just to illustrate there may be >2 of them so > int is better than bool) > >

[jira] [Created] (KAFKA-5002) Stream does't seem to consider partitions for processing which are being consumed

2017-04-03 Thread Mustak (JIRA)
Mustak created KAFKA-5002: - Summary: Stream does't seem to consider partitions for processing which are being consumed Key: KAFKA-5002 URL: https://issues.apache.org/jira/browse/KAFKA-5002 Project: Kafka

[GitHub] kafka pull request #2791: MINOR: Fix Deadlock in StreamThread

2017-04-03 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2791 --- 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 enab

Re: [DISCUSS] KIP-86: Configurable SASL callback handlers

2017-04-03 Thread Rajini Sivaram
If there are no other concerns or suggestions on this KIP, I will start vote later this week. Thank you... Regards, Rajini On Thu, Mar 30, 2017 at 9:42 PM, Rajini Sivaram wrote: > I have made a minor change to the callback handler interface to pass in > the JAAS configuration entries in *conf

[GitHub] kafka pull request #2792: MINOR: Fix potential deadlock in consumer close te...

2017-04-03 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2792 --- 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 enab

[jira] [Updated] (KAFKA-1211) Hold the produce request with ack > 1 in purgatory until replicas' HW has larger than the produce offset (KIP-101)

2017-04-03 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1211?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma updated KAFKA-1211: --- Assignee: Ben Stopford Status: Patch Available (was: Open) > Hold the produce request with ack

[jira] [Updated] (KAFKA-4586) Add purgeDataBefore() API in AdminClient

2017-04-03 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4586?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma updated KAFKA-4586: --- Resolution: Fixed Status: Resolved (was: Patch Available) > Add purgeDataBefore() API in Admi

[jira] [Updated] (KAFKA-4208) Add Record Headers

2017-04-03 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4208?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma updated KAFKA-4208: --- Status: Patch Available (was: Open) > Add Record Headers > -- > > Key

[jira] [Commented] (KAFKA-4814) ZookeeperLeaderElector not respecting zookeeper.set.acl

2017-04-03 Thread Balint Molnar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4814?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15953384#comment-15953384 ] Balint Molnar commented on KAFKA-4814: -- [~ijuma] Something odd happening here, or I d

[jira] [Commented] (KAFKA-4814) ZookeeperLeaderElector not respecting zookeeper.set.acl

2017-04-03 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4814?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15953428#comment-15953428 ] Ismael Juma commented on KAFKA-4814: For commands, it's enough to check `JaasUtils.isZ

[GitHub] kafka pull request #2794: HOTFIX: Set `baseOffset` and `writeLimit` correctl...

2017-04-03 Thread ijuma
GitHub user ijuma opened a pull request: https://github.com/apache/kafka/pull/2794 HOTFIX: Set `baseOffset` and `writeLimit` correctly in `RecordAccumulator` You can merge this pull request into a Git repository by running: $ git pull https://github.com/ijuma/kafka fix-records

Re: [VOTE] KIP-134: Delay initial consumer group rebalance

2017-04-03 Thread Mathieu Fenniak
+1 (non-binding) This will be very helpful for me, looking forward to it! :-) On Thu, Mar 30, 2017 at 4:46 AM, Damian Guy wrote: > Hi All, > > I'd like to start the voting thread on KIP-134: > https://cwiki.apache.org/confluence/display/KAFKA/KIP- > 134%3A+Delay+initial+consumer+group+rebalance

[jira] [Updated] (KAFKA-4971) Why is there no difference between kafka benchmark tests on SSD and HDD?

2017-04-03 Thread Dasol Kim (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4971?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Dasol Kim updated KAFKA-4971: - Description: I installed OS and kafka in the two SSD and two HDDs to perform the kafka benchmark test ba

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

2017-04-03 Thread Apache Jenkins Server
See Changes: [ismael] MINOR: Fix deadlock between StreamThread and KafkaStreams -- [...truncated 1.49 MB...] kafka.security.auth.SimpleAclAuthorizerTest > testHighConcurr

[jira] [Commented] (KAFKA-4971) Why is there no difference between kafka benchmark tests on SSD and HDD?

2017-04-03 Thread Michal Borowiecki (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4971?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15953608#comment-15953608 ] Michal Borowiecki commented on KAFKA-4971: -- I think your question would be easier

[jira] [Comment Edited] (KAFKA-4971) Why is there no difference between kafka benchmark tests on SSD and HDD?

2017-04-03 Thread Michal Borowiecki (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4971?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15953608#comment-15953608 ] Michal Borowiecki edited comment on KAFKA-4971 at 4/3/17 2:54 PM: --

Re: [VOTE] KIP-134: Delay initial consumer group rebalance

2017-04-03 Thread Bill Bejeck
+1 (non-binding) On Mon, Apr 3, 2017 at 9:53 AM, Mathieu Fenniak < mathieu.fenn...@replicon.com> wrote: > +1 (non-binding) > > This will be very helpful for me, looking forward to it! :-) > > On Thu, Mar 30, 2017 at 4:46 AM, Damian Guy wrote: > > > Hi All, > > > > I'd like to start the voting th

Re: [VOTE] KIP-134: Delay initial consumer group rebalance

2017-04-03 Thread Onur Karaman
Hi Damian. After reading the discussion thread again, it still doesn't seem like the thread discussed the option I mentioned earlier. >From what I had understood from the broker-side vs. client-side config debate was that the client-side config from the discussion would cause a wire format change

[jira] [Updated] (KAFKA-4916) Add streams tests with brokers failing

2017-04-03 Thread Eno Thereska (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4916?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Eno Thereska updated KAFKA-4916: Priority: Blocker (was: Critical) > Add streams tests with brokers failing > --

[jira] [Updated] (KAFKA-4916) Add streams tests with brokers failing

2017-04-03 Thread Eno Thereska (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4916?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Eno Thereska updated KAFKA-4916: Fix Version/s: (was: 0.11.0.0) 0.10.2.1 > Add streams tests with brokers fail

[jira] [Updated] (KAFKA-4913) creating a window store with one segment throws division by zero error

2017-04-03 Thread Eno Thereska (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4913?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Eno Thereska updated KAFKA-4913: Fix Version/s: 0.10.2.1 > creating a window store with one segment throws division by zero error > -

[jira] [Created] (KAFKA-5003) StreamThread should catch InvalidTopicException

2017-04-03 Thread Eno Thereska (JIRA)
Eno Thereska created KAFKA-5003: --- Summary: StreamThread should catch InvalidTopicException Key: KAFKA-5003 URL: https://issues.apache.org/jira/browse/KAFKA-5003 Project: Kafka Issue Type: Bug

[jira] [Commented] (KAFKA-5002) Stream does't seem to consider partitions for processing which are being consumed

2017-04-03 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5002?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15953717#comment-15953717 ] Matthias J. Sax commented on KAFKA-5002: "if i start consumer which consumes data

Re: [DISCUSS] Time for 0.10.2.1 bugfix release

2017-04-03 Thread Gwen Shapira
Reminder that: 1. If you have a bug fix that you want to see in 0.10.2.1, now is the right time to mark the JIRA as "fixVersion=0.10.2.1", so I can track it, and get a committer to cherrypick the fix into 0.10.2 branch. Only JIRAs with PRs please, unless it is an absolute blocker. 2. If you have

[jira] [Commented] (KAFKA-4916) Add streams tests with brokers failing

2017-04-03 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4916?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15953722#comment-15953722 ] Gwen Shapira commented on KAFKA-4916: - This doesn't block the release, right? It doesn

[jira] [Updated] (KAFKA-4848) Stream thread getting into deadlock state while trying to get rocksdb lock in retryWithBackoff

2017-04-03 Thread Eno Thereska (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4848?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Eno Thereska updated KAFKA-4848: Fix Version/s: 0.10.2.1 > Stream thread getting into deadlock state while trying to get rocksdb lock

[jira] [Commented] (KAFKA-4916) Add streams tests with brokers failing

2017-04-03 Thread Eno Thereska (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4916?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15953723#comment-15953723 ] Eno Thereska commented on KAFKA-4916: - There is a critical bug fix as part of the test

[jira] [Comment Edited] (KAFKA-4916) Add streams tests with brokers failing

2017-04-03 Thread Eno Thereska (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4916?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15953723#comment-15953723 ] Eno Thereska edited comment on KAFKA-4916 at 4/3/17 4:19 PM: -

[jira] [Updated] (KAFKA-4916) Add streams tests with brokers failing

2017-04-03 Thread Eno Thereska (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4916?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Eno Thereska updated KAFKA-4916: Description: We need to add either integration or system tests with streams and have Kafka brokers

[jira] [Commented] (KAFKA-4848) Stream thread getting into deadlock state while trying to get rocksdb lock in retryWithBackoff

2017-04-03 Thread Eno Thereska (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4848?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15953734#comment-15953734 ] Eno Thereska commented on KAFKA-4848: - Needs to go to 0.10.2.1 too. Reopening to track

[jira] [Commented] (KAFKA-5003) StreamThread should catch InvalidTopicException

2017-04-03 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5003?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15953736#comment-15953736 ] Matthias J. Sax commented on KAFKA-5003: The PR from about is for {{trunk}} -- the

[jira] [Commented] (KAFKA-3795) Transient system test failure upgrade_test.TestUpgrade

2017-04-03 Thread Roger Hoover (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3795?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15953759#comment-15953759 ] Roger Hoover commented on KAFKA-3795: - Happened again: http://confluent-systest.s3-we

[jira] [Updated] (KAFKA-5003) StreamThread should catch InvalidTopicException

2017-04-03 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5003?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax updated KAFKA-5003: --- Fix Version/s: 0.11.0.0 Status: Patch Available (was: Open) > StreamThread should

[jira] [Updated] (KAFKA-3795) Transient system test failure upgrade_test.TestUpgrade

2017-04-03 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3795?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma updated KAFKA-3795: --- Labels: kip-101 reliability (was: reliability) > Transient system test failure upgrade_test.TestUpgra

[GitHub] kafka pull request #2794: HOTFIX: Set `baseOffset` correctly in `RecordAccum...

2017-04-03 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2794 --- 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 enab

[jira] [Commented] (KAFKA-4848) Stream thread getting into deadlock state while trying to get rocksdb lock in retryWithBackoff

2017-04-03 Thread Sachin Mittal (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4848?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15953777#comment-15953777 ] Sachin Mittal commented on KAFKA-4848: -- Please let me know if this will be done in 0.

[GitHub] kafka pull request #2795: MINOR: Add a release script that helps generate re...

2017-04-03 Thread ewencp
GitHub user ewencp opened a pull request: https://github.com/apache/kafka/pull/2795 MINOR: Add a release script that helps generate release candidates. You can merge this pull request into a Git repository by running: $ git pull https://github.com/ewencp/kafka release-script

[VOTE] KIP-112 - Handle disk failure for JBOD

2017-04-03 Thread Dong Lin
Hi all, It seems that there is no further concern with the KIP-112. We would like to start the voting process. The KIP can be found at *https://cwiki.apache.org/confluence/display/KAFKA/KIP-112%3A+Handle+disk+failure+for+JBOD

[VOTE] KIP-113 - Support replicas movement between log directories

2017-04-03 Thread Dong Lin
Hi all, It seems that there is no further concern with the KIP-113. We would like to start the voting process. The KIP can be found at *https://cwiki.apache.org/confluence/display/KAFKA/KIP-113%3A+Support+replicas+movement+between+log+directories

[jira] [Comment Edited] (KAFKA-4848) Stream thread getting into deadlock state while trying to get rocksdb lock in retryWithBackoff

2017-04-03 Thread Eno Thereska (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4848?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15953804#comment-15953804 ] Eno Thereska edited comment on KAFKA-4848 at 4/3/17 4:54 PM: -

[jira] [Commented] (KAFKA-4848) Stream thread getting into deadlock state while trying to get rocksdb lock in retryWithBackoff

2017-04-03 Thread Eno Thereska (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4848?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15953804#comment-15953804 ] Eno Thereska commented on KAFKA-4848: - Saching, stay tuned, the committers are looking

[GitHub] kafka pull request #2778: MINOR: fix cleanup phase for KStreamWindowAggregat...

2017-04-03 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2778 --- 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 enab

Re: [VOTE] KIP-81: Bound Fetch memory usage in the consumer

2017-04-03 Thread Becket Qin
+1. Thanks for the KIP. On Mon, Apr 3, 2017 at 4:29 AM, Rajini Sivaram wrote: > +1 (non-binding) > > On Fri, Mar 31, 2017 at 5:36 PM, radai wrote: > > > possible priorities: > > > > 1. keepalives/coordination > > 2. inter-broker-traffic > > 3. produce traffic > > 4. consume traffic > > > > (don

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

2017-04-03 Thread Apache Jenkins Server
See Changes: [ismael] MINOR: Fix potential deadlock in consumer close test -- [...truncated 1.43 MB...] kafka.api.PlaintextConsumerTest > testEarliestOrLatestOffsets START

[GitHub] kafka pull request #2796: Close the producer batch data stream when the batc...

2017-04-03 Thread apurvam
GitHub user apurvam opened a pull request: https://github.com/apache/kafka/pull/2796 Close the producer batch data stream when the batch gets full to free up compression buffers, etc. You can merge this pull request into a Git repository by running: $ git pull https://github.

Re: [DISCUSS] KIP-138: Change punctuate semantics

2017-04-03 Thread Thomas Becker
Although I fully agree we need a way to trigger periodic processing that is independent from whether and when messages arrive, I'm not sure I like the idea of changing the existing semantics across the board. What if we added an additional callback to Processor that can be scheduled similarly to pu

Re: [VOTE] KIP-134: Delay initial consumer group rebalance

2017-04-03 Thread Becket Qin
Hey Onur, Are you suggesting letting the consumers to hold back on sending SyncGroupRequest on the first rebalance? I am not sure how exactly that works. But it looks that having the group coordinator to control the rebalance progress would be clearer and probably safer than letting the group memb

[jira] [Created] (KAFKA-5004) poll() timeout not enforced when connecting to 0.10.0 broker

2017-04-03 Thread Matthias J. Sax (JIRA)
Matthias J. Sax created KAFKA-5004: -- Summary: poll() timeout not enforced when connecting to 0.10.0 broker Key: KAFKA-5004 URL: https://issues.apache.org/jira/browse/KAFKA-5004 Project: Kafka

Re: [DISCUSS] KIP-138: Change punctuate semantics

2017-04-03 Thread Michal Borowiecki
Thanks Thomas, I'm also wary of changing the existing semantics of punctuate, for backward compatibility reasons, although I like the conceptual simplicity of that option. Adding a new method to me feels safer but, in a way, uglier. I added this to the KIP now as option (C). The TimestampE

Jenkins build is back to normal : kafka-trunk-jdk8 #1395

2017-04-03 Thread Apache Jenkins Server
See

[jira] [Resolved] (KAFKA-4222) Transient failure in QueryableStateIntegrationTest.queryOnRebalance

2017-04-03 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4222?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax resolved KAFKA-4222. Resolution: Fixed > Transient failure in QueryableStateIntegrationTest.queryOnRebalance > --

[jira] [Reopened] (KAFKA-4222) Transient failure in QueryableStateIntegrationTest.queryOnRebalance

2017-04-03 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4222?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax reopened KAFKA-4222: > Transient failure in QueryableStateIntegrationTest.queryOnRebalance >

[jira] [Created] (KAFKA-5005) JoinIntegrationTest.testLeftKStreamKStream() fails occasionally

2017-04-03 Thread Matthias J. Sax (JIRA)
Matthias J. Sax created KAFKA-5005: -- Summary: JoinIntegrationTest.testLeftKStreamKStream() fails occasionally Key: KAFKA-5005 URL: https://issues.apache.org/jira/browse/KAFKA-5005 Project: Kafka

[GitHub] kafka pull request #2722: KAFKA-4878: Improved Invalid Connect Config Error ...

2017-04-03 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2722 --- 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 enab

[jira] [Resolved] (KAFKA-4878) Kafka Connect does not log connector configuration errors

2017-04-03 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4878?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gwen Shapira resolved KAFKA-4878. - Resolution: Fixed Fix Version/s: 0.11.0.0 Issue resolved by pull request 2722 [https://gith

[jira] [Commented] (KAFKA-4878) Kafka Connect does not log connector configuration errors

2017-04-03 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4878?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15954085#comment-15954085 ] ASF GitHub Bot commented on KAFKA-4878: --- Github user asfgit closed the pull request

[GitHub] kafka pull request #2797: KAFKA-4990: Add API stubs, config parameters, and ...

2017-04-03 Thread mjsax
GitHub user mjsax opened a pull request: https://github.com/apache/kafka/pull/2797 KAFKA-4990: Add API stubs, config parameters, and request types You can merge this pull request into a Git repository by running: $ git pull https://github.com/mjsax/kafka exactly-once-transact

[jira] [Commented] (KAFKA-4990) Add API stubs, config parameters, and request types

2017-04-03 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4990?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15954131#comment-15954131 ] ASF GitHub Bot commented on KAFKA-4990: --- GitHub user mjsax opened a pull request:

[GitHub] kafka pull request #2797: KAFKA-4990: Add API stubs, config parameters, and ...

2017-04-03 Thread mjsax
Github user mjsax closed the pull request at: https://github.com/apache/kafka/pull/2797 --- 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 enabl

[jira] [Commented] (KAFKA-4990) Add API stubs, config parameters, and request types

2017-04-03 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4990?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15954134#comment-15954134 ] ASF GitHub Bot commented on KAFKA-4990: --- Github user mjsax closed the pull request a

Re: [VOTE] KIP-134: Delay initial consumer group rebalance

2017-04-03 Thread Onur Karaman
Delaying the SyncGroupRequest is not what I had in mind. What I was thinking was essentially a client-side stabilization window where the client does nothing other than participate in the group membership protocol and wait a bit for the group to stabilize. During this window, several rounds of re

[GitHub] kafka pull request #2798: KAFKA-4837: Fix class name comparison in connector...

2017-04-03 Thread kkonstantine
GitHub user kkonstantine opened a pull request: https://github.com/apache/kafka/pull/2798 KAFKA-4837: Fix class name comparison in connector-plugins REST endpoint You can merge this pull request into a Git repository by running: $ git pull https://github.com/kkonstantine/kafka

[jira] [Commented] (KAFKA-4837) Config validation in Connector plugins need to compare against both canonical and simple class names

2017-04-03 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4837?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15954144#comment-15954144 ] ASF GitHub Bot commented on KAFKA-4837: --- GitHub user kkonstantine opened a pull requ

[jira] [Created] (KAFKA-5006) KeyValueStore.put may throw exception unrelated to the current put attempt

2017-04-03 Thread JIRA
Xavier Léauté created KAFKA-5006: Summary: KeyValueStore.put may throw exception unrelated to the current put attempt Key: KAFKA-5006 URL: https://issues.apache.org/jira/browse/KAFKA-5006 Project: Kaf

[jira] [Updated] (KAFKA-5006) KeyValueStore.put may throw exception unrelated to the current put attempt

2017-04-03 Thread Eno Thereska (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5006?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Eno Thereska updated KAFKA-5006: Fix Version/s: 0.11.0.0 > KeyValueStore.put may throw exception unrelated to the current put attempt

Re: [VOTE] KIP-112 - Handle disk failure for JBOD

2017-04-03 Thread radai
+1, LGTM On Mon, Apr 3, 2017 at 9:49 AM, Dong Lin wrote: > Hi all, > > It seems that there is no further concern with the KIP-112. We would like > to start the voting process. The KIP can be found at > *https://cwiki.apache.org/confluence/display/KAFKA/KIP- > 112%3A+Handle+disk+failure+for+JBOD

[GitHub] kafka pull request #2799: Kafka-4990: Add API stubs, config parameters, and ...

2017-04-03 Thread mjsax
GitHub user mjsax opened a pull request: https://github.com/apache/kafka/pull/2799 Kafka-4990: Add API stubs, config parameters, and request types You can merge this pull request into a Git repository by running: $ git pull https://github.com/mjsax/kafka kafka-4990-add-api-st

[jira] [Created] (KAFKA-5007) Kafka Replica Fetcher Thread- Resource Leak

2017-04-03 Thread Joseph Aliase (JIRA)
Joseph Aliase created KAFKA-5007: Summary: Kafka Replica Fetcher Thread- Resource Leak Key: KAFKA-5007 URL: https://issues.apache.org/jira/browse/KAFKA-5007 Project: Kafka Issue Type: Bug

[jira] [Created] (KAFKA-5008) Kafka-Clients not OSGi ready

2017-04-03 Thread Marc (JIRA)
Marc created KAFKA-5008: --- Summary: Kafka-Clients not OSGi ready Key: KAFKA-5008 URL: https://issues.apache.org/jira/browse/KAFKA-5008 Project: Kafka Issue Type: Bug Components: clients Af

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

2017-04-03 Thread Apache Jenkins Server
See Changes: [cshapi] KAFKA-4878: Improved Invalid Connect Config Error Message -- [...truncated 808.45 KB...] org.apache.kafka.streams.integration.KStreamKTableJoinInteg

[jira] [Updated] (KAFKA-4990) Add API stubs, config parameters, and request types

2017-04-03 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4990?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax updated KAFKA-4990: --- Status: Patch Available (was: Open) > Add API stubs, config parameters, and request types > -

[GitHub] kafka pull request #2796: MINOR: Close the producer batch data stream when t...

2017-04-03 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2796 --- 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 enab

[GitHub] kafka pull request #2676: MINOR: Suppress an inappropriate warning in Mirror...

2017-04-03 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2676 --- 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 enab

[VOTE] KIP-135 : Send of null key to a compacted topic should throw non-retriable error back to user

2017-04-03 Thread Mayuresh Gharat
Hi All, It seems that there is no further concern with the KIP-135. At this point we would like to start the voting process. The KIP can be found at https://cwiki.apache.org/confluence/display/KAFKA/KIP-135+%3A+Send+of+null+key+to+a+compacted+topic+should+throw+non-retriable+error+back+to+user

[jira] [Updated] (KAFKA-5004) poll() timeout not enforced when connecting to 0.10.0 broker

2017-04-03 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5004?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ewen Cheslack-Postava updated KAFKA-5004: - Fix Version/s: (was: 0.10.2.0) Component/s: consumer [~mjsax] Moved this

[jira] [Updated] (KAFKA-4972) Kafka 0.10.0 Found a corrupted index file during Kafka broker startup

2017-04-03 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4972?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ewen Cheslack-Postava updated KAFKA-4972: - Fix Version/s: (was: 0.10.2.0) (was: 0.10.1.1)

[jira] [Updated] (KAFKA-4927) KStreamsTestDriver fails with NPE when KStream.to() sinks are used

2017-04-03 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4927?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ewen Cheslack-Postava updated KAFKA-4927: - Fix Version/s: (was: 0.10.2.0) Removed already released version from fix versi

[jira] [Resolved] (KAFKA-4766) Document lz4 and lz4hc in confluence

2017-04-03 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4766?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ewen Cheslack-Postava resolved KAFKA-4766. -- Resolution: Fixed Reviewer: Ewen Cheslack-Postava Fix Version/s:

[jira] [Updated] (KAFKA-4832) kafka producer send Async message to the wrong IP cannot to stop producer.close()

2017-04-03 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4832?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ewen Cheslack-Postava updated KAFKA-4832: - Fix Version/s: (was: 0.8.2.2) Removing invalid fix version, we'll need to revi

[jira] [Updated] (KAFKA-4837) Config validation in Connector plugins need to compare against both canonical and simple class names

2017-04-03 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4837?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ewen Cheslack-Postava updated KAFKA-4837: - Reviewer: Ewen Cheslack-Postava Status: Patch Available (was: Open) > Confi

[jira] [Commented] (KAFKA-5004) poll() timeout not enforced when connecting to 0.10.0 broker

2017-04-03 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5004?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15954390#comment-15954390 ] Ismael Juma commented on KAFKA-5004: This is by design. Do you have an alternative pro

[jira] [Resolved] (KAFKA-4977) kafka-connect: fix findbugs issues in connect/runtime

2017-04-03 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4977?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ewen Cheslack-Postava resolved KAFKA-4977. -- Resolution: Fixed Fix Version/s: 0.11.0.0 Issue resolved by pull request

[GitHub] kafka pull request #2763: Kafka 4977

2017-04-03 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2763 --- 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 enab

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

2017-04-03 Thread Apache Jenkins Server
See Changes: [ismael] MINOR: Close the producer batch append stream when the batch gets full [ismael] MINOR: Suppress ProducerConfig warning in MirrorMaker -- [...truncat

[jira] [Updated] (KAFKA-5004) poll() timeout not enforced when connecting to 0.10.0 broker

2017-04-03 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5004?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax updated KAFKA-5004: --- Affects Version/s: 0.10.2.0 > poll() timeout not enforced when connecting to 0.10.0 broker > -

[jira] [Commented] (KAFKA-5004) poll() timeout not enforced when connecting to 0.10.0 broker

2017-04-03 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5004?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15954476#comment-15954476 ] Matthias J. Sax commented on KAFKA-5004: I did not know that this is by design --

Jenkins build is back to normal : kafka-trunk-jdk8 #1398

2017-04-03 Thread Apache Jenkins Server
See

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

2017-04-03 Thread Apache Jenkins Server
See

[GitHub] kafka pull request #2732: KAFKA-4855 Struct SchemaBuilder should not allow d...

2017-04-03 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2732 --- 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 enab

  1   2   >