[jira] [Created] (KAFKA-5813) Unexpected unclean leader election due to leader/controller's unusual event handling order

2017-08-30 Thread Allen Wang (JIRA)
Allen Wang created KAFKA-5813: - Summary: Unexpected unclean leader election due to leader/controller's unusual event handling order Key: KAFKA-5813 URL: https://issues.apache.org/jira/browse/KAFKA-5813

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

2017-08-30 Thread Roger Hoover
Sorry if this is a bit out of left field but can't help wondering... One way to improve producer performance while still having good guarantees would be to allow a setting between acks=1 and acks=all. We could introduce "acks=minIsr". This is already the guarantee you get when the ISR set

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

2017-08-30 Thread Apurva Mehta
Hi Ted, int16 is sufficient. I forgot to specify initially. I have updated the KIP. Thanks for pointing it out! Apurva On Wed, Aug 30, 2017 at 4:43 PM, Ted Yu wrote: > For ProduceRequest v4, would int32 or int16 be enough for idempotenceLevel > ? > > Cheers > > On Wed, Aug

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

2017-08-30 Thread Matthias J. Sax
+1 On 8/30/17 12:00 PM, Bill Bejeck wrote: > +1 > > On Wed, Aug 30, 2017 at 1:06 PM, Damian Guy wrote: > >> +1 >> >> On Wed, 30 Aug 2017 at 17:49 Guozhang Wang wrote: >> >>> Hello Michal and community: >>> >>> While working on updating the web docs

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

2017-08-30 Thread Ted Yu
For ProduceRequest v4, would int32 or int16 be enough for idempotenceLevel ? Cheers On Wed, Aug 30, 2017 at 3:47 PM, Apurva Mehta wrote: > Thanks Ismael and Jason, I filed a separate KIP to solve the problems > identified through this discussion. I also incorporated

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

2017-08-30 Thread Apurva Mehta
Thanks Ismael and Jason, I filed a separate KIP to solve the problems identified through this discussion. I also incorporated Jason's comments in that document: https://cwiki.apache.org/confluence/display/KAFKA/KIP-192+%3A+Provide+cleaner+semantics+when+idempotence+is+enabled Please have a look,

[GitHub] kafka pull request #3763: KAFKA-5812; Represent logDir with case class where...

2017-08-30 Thread lindong28
GitHub user lindong28 opened a pull request: https://github.com/apache/kafka/pull/3763 KAFKA-5812; Represent logDir with case class where absolute path is required You can merge this pull request into a Git repository by running: $ git pull https://github.com/lindong28/kafka

[jira] [Created] (KAFKA-5812) Represent logDir with case class where absolute path is required

2017-08-30 Thread Dong Lin (JIRA)
Dong Lin created KAFKA-5812: --- Summary: Represent logDir with case class where absolute path is required Key: KAFKA-5812 URL: https://issues.apache.org/jira/browse/KAFKA-5812 Project: Kafka Issue

[jira] [Created] (KAFKA-5811) Trogdor should handle injecting disk faults

2017-08-30 Thread Colin P. McCabe (JIRA)
Colin P. McCabe created KAFKA-5811: -- Summary: Trogdor should handle injecting disk faults Key: KAFKA-5811 URL: https://issues.apache.org/jira/browse/KAFKA-5811 Project: Kafka Issue Type:

[GitHub] kafka pull request #3404: KAFKA-5476: Implement a system test that creates n...

2017-08-30 Thread cmccabe
Github user cmccabe closed the pull request at: https://github.com/apache/kafka/pull/3404 --- 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 #3762: KAFKA-5807 - Check Connector.config() and Transfor...

2017-08-30 Thread jcustenborder
GitHub user jcustenborder opened a pull request: https://github.com/apache/kafka/pull/3762 KAFKA-5807 - Check Connector.config() and Transformation.config() returns a valid ConfigDef You can merge this pull request into a Git repository by running: $ git pull

Re: [DISCUSS] KIP-188 - Add new metrics to support health checks

2017-08-30 Thread Rajini Sivaram
Jun, Thank you, your suggestions sound good. I have updated the KIP. Regards, Rajini On Tue, Aug 29, 2017 at 9:12 PM, Jun Rao wrote: > Hi, Rajini, > > Thanks for the updated KIP. I agree that those additional metrics can be > useful. I was thinking what would an admin do

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

2017-08-30 Thread Apache Jenkins Server
See Changes: [wangguoz] MINOR: KIP-160 docs -- [...truncated 2.02 MB...] org.apache.kafka.common.security.scram.ScramMessagesTest > invalidServerFirstMessage STARTED

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

2017-08-30 Thread Jun Rao
Hi, Jiangjie, I mis-understood Jason's approach earlier. It does seem to be a good one. We still need to calculate the selector timeout based on the remaining delivery.timeout.ms to call the callback on time, but we can always wait for an inflight request based on request.timeout.ms. Thanks,

[jira] [Resolved] (KAFKA-4520) Kafka broker fails with not so user-friendly error msg when log.dirs is not set

2017-08-30 Thread Manikumar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4520?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar resolved KAFKA-4520. -- Resolution: Fixed Fix Version/s: 0.11.0.0 > Kafka broker fails with not so user-friendly error

[GitHub] kafka pull request #3761: KAFKA-5763: Refactor NetworkClient to use LogConte...

2017-08-30 Thread adyach
GitHub user adyach opened a pull request: https://github.com/apache/kafka/pull/3761 KAFKA-5763: Refactor NetworkClient to use LogContext This PR lets logging client id in every log line in NetworkClient You can merge this pull request into a Git repository by running: $ git

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

2017-08-30 Thread Apache Jenkins Server
See

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

2017-08-30 Thread Apache Jenkins Server
See Changes: [wangguoz] MINOR: KIP-160 docs -- [...truncated 914.24 KB...] kafka.server.epoch.LeaderEpochFileCacheTest > shouldNotResetEpochHistoryTailIfUndefinedPassed

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

2017-08-30 Thread Bill Bejeck
+1 On Wed, Aug 30, 2017 at 1:06 PM, Damian Guy wrote: > +1 > > On Wed, 30 Aug 2017 at 17:49 Guozhang Wang wrote: > > > Hello Michal and community: > > > > While working on updating the web docs and java docs for this KIP, I felt > > that the term

[jira] [Resolved] (KAFKA-5786) Yet another exception is causing that streamming app is zombie

2017-08-30 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5786?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax resolved KAFKA-5786. Resolution: Duplicate > Yet another exception is causing that streamming app is zombie >

[jira] [Resolved] (KAFKA-270) sync producer / consumer test producing lot of kafka server exceptions & not getting the throughput mentioned here http://incubator.apache.org/kafka/performance.html

2017-08-30 Thread Manikumar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-270?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar resolved KAFKA-270. - Resolution: Won't Fix Closing due to inactivity. Pl reopen if you think the issue still exists > sync

[GitHub] kafka pull request #3733: MINOR: KIP-160 docs

2017-08-30 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/3733 --- 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-1492) Getting error when sending producer request at the broker end with a single broker

2017-08-30 Thread Manikumar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1492?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar resolved KAFKA-1492. -- Resolution: Cannot Reproduce Pl reopen if you think the issue still exists > Getting error when

[jira] [Resolved] (KAFKA-1632) No such method error on KafkaStream.head

2017-08-30 Thread Manikumar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1632?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar resolved KAFKA-1632. -- Resolution: Cannot Reproduce Mostly related to Kafka version mismatch. Pl reopen if you think the

[jira] [Resolved] (KAFKA-1463) producer fails with scala.tuple error

2017-08-30 Thread Manikumar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1463?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar resolved KAFKA-1463. -- Resolution: Won't Fix Pl reopen if you think the issue still exists > producer fails with

[jira] [Resolved] (KAFKA-888) problems when shutting down the java consumer .

2017-08-30 Thread Manikumar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-888?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar resolved KAFKA-888. - Resolution: Cannot Reproduce Pl reopen if you think the issue still exists > problems when shutting

[jira] [Resolved] (KAFKA-1980) Console consumer throws OutOfMemoryError with large max-messages

2017-08-30 Thread Manikumar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1980?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar resolved KAFKA-1980. -- Resolution: Won't Fix [~ndimiduk] Agree. Updated the JIRA. > Console consumer throws OutOfMemoryError

[jira] [Reopened] (KAFKA-1980) Console consumer throws OutOfMemoryError with large max-messages

2017-08-30 Thread Manikumar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1980?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar reopened KAFKA-1980: -- > Console consumer throws OutOfMemoryError with large max-messages >

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

2017-08-30 Thread Apache Jenkins Server
See Changes: [wangguoz] MINOR: doc changes for KIP-138 -- [...truncated 913.99 KB...] kafka.server.epoch.LeaderEpochFileCacheTest >

[jira] [Reopened] (KAFKA-5763) Refactor NetworkClient to use LogContext

2017-08-30 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5763?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma reopened KAFKA-5763: NetworkClient is a different class and it doesn't use LogContext yet. > Refactor NetworkClient to use

Re: [VOTE] KIP-152 - Improve diagnostics for SASL authentication failures

2017-08-30 Thread Gwen Shapira
Enthusiastic +1 from me :) It will be so helpful. On Wed, Aug 30, 2017 at 9:22 AM Roger Hoover wrote: > Great. Thank you, Rajini. > > On Wed, Aug 30, 2017 at 7:53 AM, Rajini Sivaram > wrote: > > > Hi Roger, > > > > Thank you for the

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

2017-08-30 Thread Damian Guy
+1 On Wed, 30 Aug 2017 at 17:49 Guozhang Wang wrote: > Hello Michal and community: > > While working on updating the web docs and java docs for this KIP, I felt > that the term SYSTEM_TIME a bit confusing sometimes from a reader's > perspective as we are actually talking

[jira] [Resolved] (KAFKA-5763) Refactor NetworkClient to use LogContext

2017-08-30 Thread Kamal Chandraprakash (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5763?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Kamal Chandraprakash resolved KAFKA-5763. - Resolution: Fixed Fix Version/s: 1.0.0 [~ijuma] `ConsumerNetworkClient`

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

2017-08-30 Thread Apache Jenkins Server
See Changes: [wangguoz] KAFKA-5797: Delay checking of partition existence in -- [...truncated 2.64 MB...]

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

2017-08-30 Thread Guozhang Wang
Hello Michal and community: While working on updating the web docs and java docs for this KIP, I felt that the term SYSTEM_TIME a bit confusing sometimes from a reader's perspective as we are actually talking about wall-clock time. I'd hence like to propose an minor addendum to this adopted KIP

[GitHub] kafka pull request #3732: MINOR: doc changes for KIP-138

2017-08-30 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/3732 --- 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-site pull request #72: Jason and Becket are PMC members

2017-08-30 Thread hachikuji
Github user hachikuji closed the pull request at: https://github.com/apache/kafka-site/pull/72 --- 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

[jira] [Resolved] (KAFKA-5769) Transient test failure org.apache.kafka.streams.integration.KStreamRepartitionJoinTest.shouldCorrectlyRepartitionOnJoinOperationsWithZeroSizedCache

2017-08-30 Thread Damian Guy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5769?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Damian Guy resolved KAFKA-5769. --- Resolution: Duplicate > Transient test failure >

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

2017-08-30 Thread Apache Jenkins Server
See Changes: [wangguoz] KAFKA-5797: Delay checking of partition existence in -- [...truncated 914.04 KB...] kafka.server.epoch.LeaderEpochFileCacheTest >

[jira] [Resolved] (KAFKA-4835) Allow users control over repartitioning

2017-08-30 Thread Damian Guy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4835?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Damian Guy resolved KAFKA-4835. --- Resolution: Duplicate > Allow users control over repartitioning >

[jira] [Reopened] (KAFKA-4835) Allow users control over repartitioning

2017-08-30 Thread Damian Guy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4835?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Damian Guy reopened KAFKA-4835: --- > Allow users control over repartitioning > --- > >

Re: [VOTE] KIP-152 - Improve diagnostics for SASL authentication failures

2017-08-30 Thread Roger Hoover
Great. Thank you, Rajini. On Wed, Aug 30, 2017 at 7:53 AM, Rajini Sivaram wrote: > Hi Roger, > > Thank you for the suggestions. > > I think we should have a separate JIRA to address logging improvements for > authentication. That shouldn't need a KIP. The way the code

[GitHub] kafka-site issue #72: Jason and Becket are PMC members

2017-08-30 Thread guozhangwang
Github user guozhangwang commented on the issue: https://github.com/apache/kafka-site/pull/72 LGTM. --- 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

Re: [VOTE] KIP-189 - Improve principal builder interface and add support for SASL

2017-08-30 Thread Manikumar
+1 (non-binding). On Wed, Aug 30, 2017 at 9:29 PM, Mayuresh Gharat wrote: > +1 (non-binding) > > Thanks, > > Mayuresh > > On Wed, Aug 30, 2017 at 8:51 AM, Jason Gustafson > wrote: > > > I'd like to open the vote for KIP-189: > >

Re: [VOTE] KIP-189 - Improve principal builder interface and add support for SASL

2017-08-30 Thread Mayuresh Gharat
+1 (non-binding) Thanks, Mayuresh On Wed, Aug 30, 2017 at 8:51 AM, Jason Gustafson wrote: > I'd like to open the vote for KIP-189: > https://cwiki.apache.org/confluence/display/KAFKA/KIP- > 189%3A+Improve+principal+builder+interface+and+add+support+for+SASL. > Thanks to

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

2017-08-30 Thread Tom Bentley
I've updated in the KIP. Thanks, Tom On 30 August 2017 at 16:42, Ismael Juma wrote: > If you agree with the change, yes, please rename. It's OK to make changes > after the VOTE thread starts. In cases where some people have already > voted, it's recommended to mention the

[VOTE] KIP-189 - Improve principal builder interface and add support for SASL

2017-08-30 Thread Jason Gustafson
I'd like to open the vote for KIP-189: https://cwiki.apache.org/confluence/display/KAFKA/KIP-189%3A+Improve+principal+builder+interface+and+add+support+for+SASL. Thanks to everyone who helped review. -Jason

Re: [VOTE] KIP-152 - Improve diagnostics for SASL authentication failures

2017-08-30 Thread Vahid S Hashemian
Hi Rajini, The error_message field sounds good to me too. Thanks. --Vahid From: Ismael Juma To: dev@kafka.apache.org Date: 08/30/2017 08:05 AM Subject:Re: [VOTE] KIP-152 - Improve diagnostics for SASL authentication failures Sent by:

[GitHub] kafka pull request #3748: KAFKA-5797: Delay checking of partition existence ...

2017-08-30 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/3748 --- 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-5797) StoreChangelogReader should be resilient to broker-side metadata not available

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

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

2017-08-30 Thread Ismael Juma
If you agree with the change, yes, please rename. It's OK to make changes after the VOTE thread starts. In cases where some people have already voted, it's recommended to mention the changes in the VOTE thread as a heads up. Generally, we don't restart the vote unless the changes are significant.

Re: [VOTE] KIP-187 - Add cumulative count metric for all Kafka rate metrics

2017-08-30 Thread Rajini Sivaram
The vote has passed with four binding (Ismael, Gwen, Jason, me) and three non-binding (Stevo, Manikumar, Mickael) votes. Thanks to everyone for the feedback and votes. I will update the KIP page. Regards, Rajini On Tue, Aug 29, 2017 at 12:13 PM, Jason Gustafson wrote: >

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

2017-08-30 Thread Tom Bentley
Hi Ismael, I agree that `electPreferredReplicaLeader` is a mouthful and am happy to change it to `electPreferredLeaders`. I'd rename the correspond request and response similarly. Should I rename it in the KIP now, even though I initiated a VOTE thread yesterday? Cheers, Tom On 30 August 2017

[GitHub] kafka pull request #3760: MINOR: add table of contents

2017-08-30 Thread enothereska
GitHub user enothereska opened a pull request: https://github.com/apache/kafka/pull/3760 MINOR: add table of contents You can merge this pull request into a Git repository by running: $ git pull https://github.com/enothereska/kafka minor-docs-toc Alternatively you can review

Re: [DISCUSS] KIP-179: Change ReassignPartitionsCommand to use AdminClient

2017-08-30 Thread Tom Bentley
Hi all, I've updated the KIP as follows: * remove the APIs supporting progress reporting in favour of the APIs being implemented in KIP-113. * added some APIs to cover the existing functionality around throttling inter-broker transfers, which was previously a TODO. To respond to Colin's

Re: [VOTE] KIP-152 - Improve diagnostics for SASL authentication failures

2017-08-30 Thread Ismael Juma
The error_message field sounds good to me Rajini. I agree that we should file a separate JIRA for an authentication log. Ismael On Wed, Aug 30, 2017 at 3:53 PM, Rajini Sivaram wrote: > Hi Roger, > > Thank you for the suggestions. > > I think we should have a separate

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

2017-08-30 Thread Ismael Juma
Hi Tom, Thanks for the KIP, it's a useful one. I find the proposed method name `electPreferredReplicaLeader` a little hard to read. It seems that a small change would make it clearer: `electPreferredReplicaAsLeader`. The next point is that this is a batch API, so it should ideally be plural like

[jira] [Created] (KAFKA-5810) Improve authentication logging on the broker-side

2017-08-30 Thread Rajini Sivaram (JIRA)
Rajini Sivaram created KAFKA-5810: - Summary: Improve authentication logging on the broker-side Key: KAFKA-5810 URL: https://issues.apache.org/jira/browse/KAFKA-5810 Project: Kafka Issue

Re: [VOTE] KIP-152 - Improve diagnostics for SASL authentication failures

2017-08-30 Thread Rajini Sivaram
Hi Roger, Thank you for the suggestions. I think we should have a separate JIRA to address logging improvements for authentication. That shouldn't need a KIP. The way the code is structured at the moment, SSL implementation is in the network package. And that makes it a bit messy to move

[GitHub] kafka pull request #3759: [WIP]: extract and make public KeySchema & HasNext...

2017-08-30 Thread dguy
GitHub user dguy opened a pull request: https://github.com/apache/kafka/pull/3759 [WIP]: extract and make public KeySchema & HasNextCondition You can merge this pull request into a Git repository by running: $ git pull https://github.com/dguy/kafka keyschema-extract

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

2017-08-30 Thread Apache Jenkins Server
See Changes: [damian.guy] KAFKA-5804; retain duplicates in ChangeLoggingWindowBytesStore -- [...truncated 919.57 KB...] kafka.server.epoch.LeaderEpochFileCacheTest >

[GitHub] kafka pull request #3754: KAFKA-5804: retain duplicates in ChangeLoggingWind...

2017-08-30 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/3754 --- 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-5804) ChangeLoggingWindowBytesStore needs to retain duplicates when writing to the log

2017-08-30 Thread Damian Guy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5804?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Damian Guy resolved KAFKA-5804. --- Resolution: Fixed Issue resolved by pull request 3754 [https://github.com/apache/kafka/pull/3754] >

[GitHub] kafka pull request #3758: KAFKA-3131: enable error level for SSLException lo...

2017-08-30 Thread omkreddy
GitHub user omkreddy opened a pull request: https://github.com/apache/kafka/pull/3758 KAFKA-3131: enable error level for SSLException logs You can merge this pull request into a Git repository by running: $ git pull https://github.com/omkreddy/kafka KAFKA-3131 Alternatively

[jira] [Resolved] (KAFKA-5762) Refactor AdminClient to use LogContext

2017-08-30 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5762?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma resolved KAFKA-5762. Resolution: Fixed > Refactor AdminClient to use LogContext > --

[GitHub] kafka pull request #3741: KAFKA-5762; LogContext used to capture the clientI...

2017-08-30 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/3741 --- 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-4389) kafka-server.stop.sh not work

2017-08-30 Thread Manikumar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4389?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar resolved KAFKA-4389. -- Resolution: Duplicate > kafka-server.stop.sh not work > - > >

[jira] [Resolved] (KAFKA-4297) Cannot Stop Kafka with Shell Script

2017-08-30 Thread Manikumar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4297?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar resolved KAFKA-4297. -- Resolution: Duplicate Closing this as there is a latest PR for KAFKA-4931. > Cannot Stop Kafka with

[GitHub] kafka pull request #3545: KAFKA-5499: Minor cleanup of commit code

2017-08-30 Thread enothereska
Github user enothereska closed the pull request at: https://github.com/apache/kafka/pull/3545 --- 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

[GitHub] kafka pull request #3730: MINOR: stateful docs for aggregates

2017-08-30 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/3730 --- 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-191: KafkaConsumer.subscribe() overload that takes just Pattern

2017-08-30 Thread Magnus Edenhill
For what it is worth: librdkafka-based clients will treat any subscribed topic string that is prefixed with the regexp start-of-line anchor ("^") as a regexp, thus providing regex and actual topic subscriptions with the same API, also allowing a mix of actuals and regexps: E.g.:

Re: [DISCUSS] KIP-191: KafkaConsumer.subscribe() overload that takes just Pattern

2017-08-30 Thread Guozhang Wang
Sounds good. On Tue, Aug 29, 2017 at 6:12 PM, Becket Qin wrote: > Sounds good to me as well. > > On Tue, Aug 29, 2017 at 2:43 AM, Ismael Juma wrote: > > > Sounds good to me too. Since this is a non controversial change, I > suggest > > starting the vote

[jira] [Created] (KAFKA-5809) when zookeeper set acl on path /. then kafka can't connect zookeeper

2017-08-30 Thread heping (JIRA)
heping created KAFKA-5809: - Summary: when zookeeper set acl on path /. then kafka can't connect zookeeper Key: KAFKA-5809 URL: https://issues.apache.org/jira/browse/KAFKA-5809 Project: Kafka Issue