[GitHub] kafka pull request #3764: view diff

2017-08-31 Thread liwuchuai
GitHub user liwuchuai opened a pull request: https://github.com/apache/kafka/pull/3764 view diff view diff You can merge this pull request into a Git repository by running: $ git pull https://github.com/apache/kafka 0.8.1 Alternatively you can review and apply these changes as

[GitHub] kafka pull request #3765: KAFKA-5642 [WIP]: Use async ZookeeperClient in Con...

2017-08-31 Thread onurkaraman
GitHub user onurkaraman opened a pull request: https://github.com/apache/kafka/pull/3765 KAFKA-5642 [WIP]: Use async ZookeeperClient in Controller Synchronous zookeeper writes means that we wait an entire round trip before doing the next write. These synchronous writes are happening

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

2017-08-31 Thread Attila Kreiner
Hi All, Thx for the comments, I pretty much see a consensus here. So I'd like to start the vote for: https://cwiki.apache.org/confluence/display/KAFKA/KIP-191%3A+KafkaConsumer.subscribe%28%29+overload+that+takes+just+Pattern Cheers, Attila

[jira] [Created] (KAFKA-5814) java.lang.OutOfMemoryError: Direct buffer memory

2017-08-31 Thread Alexander Petrovsky (JIRA)
Alexander Petrovsky created KAFKA-5814: -- Summary: java.lang.OutOfMemoryError: Direct buffer memory Key: KAFKA-5814 URL: https://issues.apache.org/jira/browse/KAFKA-5814 Project: Kafka Is

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

2017-08-31 Thread Ismael Juma
Thanks for the KIP, +1 (binding). Ismael On 31 Aug 2017 8:38 am, "Attila Kreiner" wrote: Hi All, Thx for the comments, I pretty much see a consensus here. So I'd like to start the vote for: https://cwiki.apache.org/confluence/display/KAFKA/KIP-191%3A+KafkaConsumer. subscribe%28%29+overload+tha

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

2017-08-31 Thread Michal Borowiecki
+0 ambivalent about the naming but do agree that should be kept consistent On 31/08/17 00:43, Matthias J. Sax wrote: +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 c

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

2017-08-31 Thread Manikumar
+1 (non-binding) On Thu, Aug 31, 2017 at 1:53 PM, Ismael Juma wrote: > Thanks for the KIP, +1 (binding). > > Ismael > > On 31 Aug 2017 8:38 am, "Attila Kreiner" wrote: > > Hi All, > > Thx for the comments, I pretty much see a consensus here. So I'd like to > start the vote for: > https://cwiki.

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

2017-08-31 Thread Molnár Bálint
+1 (non-binding) 2017-08-31 10:33 GMT+02:00 Manikumar : > +1 (non-binding) > > On Thu, Aug 31, 2017 at 1:53 PM, Ismael Juma wrote: > > > Thanks for the KIP, +1 (binding). > > > > Ismael > > > > On 31 Aug 2017 8:38 am, "Attila Kreiner" wrote: > > > > Hi All, > > > > Thx for the comments, I prett

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

2017-08-31 Thread dguy
Github user dguy closed the pull request at: https://github.com/apache/kafka/pull/3759 --- 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 enable

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

2017-08-31 Thread Manikumar
Hi, Currently, we have FailedProduceRequestsPerSec, FailedFetchRequestsPerSec metrics to indicate un-expected failures on a broker while handling producer/fetch requests. Will it be useful to add these metrics for other requests also? I don't want to include these metrics to this KIP. Just want to

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

2017-08-31 Thread Ismael Juma
Thanks for the KIP, +1 (binding) from me. Ismael On Tue, Aug 29, 2017 at 4:56 PM, Tom Bentley wrote: > Hi all, > > I would like to start the vote on KIP-183 which will provide an AdminClient > interface for electing the preferred replica, and refactor the > kafka-preferred-replica-election.sh t

[GitHub] kafka pull request #3766: MINOR: Test SASL authorization id

2017-08-31 Thread rajinisivaram
GitHub user rajinisivaram opened a pull request: https://github.com/apache/kafka/pull/3766 MINOR: Test SASL authorization id You can merge this pull request into a Git repository by running: $ git pull https://github.com/rajinisivaram/kafka MINOR-sasl Alternatively you can re

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

2017-08-31 Thread Rajini Sivaram
Hi Manikumar, We currently have topic-level metrics for FailedProduceRequestsPerSec, FailedFetchRequestsPerSec. For all requests including produce/fetch, this KIP adds a new error rate metric: MBean: kafka.network:type=RequestMetrics,name=ErrorsPerSec,request=api_key_name,error=error_code_name F

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

2017-08-31 Thread Rajini Sivaram
Hi Roger, Yes, the rate metrics `successful-authentication-rate` and ` failed-authentication-rate` will also have corresponding failed-authentication-count and successful-authentication-count to match KIP 187. Thank you, Rajini On Tue, Aug 29, 2017 at 1:07 PM, Roger Hoover wrote: > Great sugg

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

2017-08-31 Thread Rajini Sivaram
This vote has passed with 5 binding (Jun, Ismael, Jason, Gwen, me) and 5 non-binding (Edo, Vahid, Manikumar, Mickael and Roger) votes. Many thanks to every one for the feedback and votes. I will update the KIP page. Regards, Rajini On Wed, Aug 30, 2017 at 1:22 PM, Gwen Shapira wrote: > Enthus

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

2017-08-31 Thread Manikumar
Hi Rajini, Yes, It provides the failure rate for all errors. I got confused with old metric names. Thank you for the clarification. Thanks On Thu, Aug 31, 2017 at 6:34 PM, Rajini Sivaram wrote: > Hi Manikumar, > > We currently have topic-level metrics for FailedProduceRequestsPerSec, > Failed

[GitHub] kafka pull request #3767: KAFKA-5650: add StateStoreBuilder interface and im...

2017-08-31 Thread dguy
GitHub user dguy opened a pull request: https://github.com/apache/kafka/pull/3767 KAFKA-5650: add StateStoreBuilder interface and implementations Part of KIP-182 - Add `StateStoreBuilder` interface and `WindowStateStoreBuilder`, `KeyValueStateStoreBuilder`, and `SessionStat

Re: [VOTE] KIP-182 - Reduce Streams DSL overloads and allow easier use of custom storage engines

2017-08-31 Thread Damian Guy
Thanks everyone for voting! Unfortunately i've had to make a bit of an update based on some issues found during implementation. The main changes are: BytesStoreSupplier -> StoreSupplier Addition of: WindowBytesStoreSupplier, KeyValueBytesStoreSupplier, SessionBytesStoreSupplier that will restrict s

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

2017-08-31 Thread Vahid S Hashemian
+1 (non-binding) Thanks. --Vahid From: Molnár Bálint To: dev@kafka.apache.org Date: 08/31/2017 02:13 AM Subject:Re: [VOTE] KIP-191: KafkaConsumer.subscribe() overload that takes just Pattern +1 (non-binding) 2017-08-31 10:33 GMT+02:00 Manikumar : > +1 (non-binding) > > O

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

2017-08-31 Thread Bill Bejeck
+1 Thanks, Bill On Thu, Aug 31, 2017 at 10:31 AM, Vahid S Hashemian < vahidhashem...@us.ibm.com> wrote: > +1 (non-binding) > > Thanks. > --Vahid > > > > > From: Molnár Bálint > To: dev@kafka.apache.org > Date: 08/31/2017 02:13 AM > Subject:Re: [VOTE] KIP-191: KafkaConsumer.subsc

[GitHub] kafka pull request #3697: KAFKA-2105: add topic null check to KafkaProducer....

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

2017-08-31 Thread Guozhang Wang
+1 On Thu, Aug 31, 2017 at 7:57 AM, Bill Bejeck wrote: > +1 > > Thanks, > Bill > > On Thu, Aug 31, 2017 at 10:31 AM, Vahid S Hashemian < > vahidhashem...@us.ibm.com> wrote: > > > +1 (non-binding) > > > > Thanks. > > --Vahid > > > > > > > > > > From: Molnár Bálint > > To: dev@kafka.apache.

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

2017-08-31 Thread Mickael Maison
+1 non binding Thanks On Thu, Aug 31, 2017 at 8:35 AM, Guozhang Wang wrote: > +1 > > On Thu, Aug 31, 2017 at 7:57 AM, Bill Bejeck wrote: > >> +1 >> >> Thanks, >> Bill >> >> On Thu, Aug 31, 2017 at 10:31 AM, Vahid S Hashemian < >> vahidhashem...@us.ibm.com> wrote: >> >> > +1 (non-binding) >> > >>

[GitHub] kafka pull request #3757: KAFKA-5379 follow up: reduce redundant mock proces...

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

2017-08-31 Thread Jason Gustafson
+1 On Thu, Aug 31, 2017 at 8:36 AM, Mickael Maison wrote: > +1 non binding > Thanks > > On Thu, Aug 31, 2017 at 8:35 AM, Guozhang Wang wrote: > > +1 > > > > On Thu, Aug 31, 2017 at 7:57 AM, Bill Bejeck wrote: > > > >> +1 > >> > >> Thanks, > >> Bill > >> > >> On Thu, Aug 31, 2017 at 10:31 AM, V

[GitHub] kafka pull request #3768: KAFKA:5652 add Printed class and KStream#print(pri...

2017-08-31 Thread dguy
GitHub user dguy opened a pull request: https://github.com/apache/kafka/pull/3768 KAFKA:5652 add Printed class and KStream#print(printed) Part of KIP-182 - Add `Printed` class and `KStream#print(Printed)` - deprecate all other `print` and `writeAsText` methods You can merge

[jira] [Resolved] (KAFKA-818) Request failure metrics on the Kafka brokers is not instrumented for all requests

2017-08-31 Thread Manikumar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-818?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar resolved KAFKA-818. - Resolution: Duplicate Similar metrics are proposed in KAFKA-5746. > Request failure metrics on the Kafka b

Re: CredentialCache might leak through Log

2017-08-31 Thread Viktor Somogyi
Hi Rodrigo, What attacks are possible if this info is exposed? I think printing the first one on debug is ok since that mode shouldn't be enabled in production, while it might be very useful while debugging. For the second one I could agree that we should print a sanitized warn depending on the lo

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

2017-08-31 Thread Roger Hoover
Sorry, my math was sloppy. It's not twice as many requests taking longer. If the probability of replication latency longer than X is Px for both replicas then, acks=all will have probability of Px(2-Px) of replication lag longer than X while acks=minIsr will be Px On Wed, Aug 30, 2017 at 5:18 P

[jira] [Resolved] (KAFKA-4905) StreamPartitionAssignor doesn't respect subscriptions to assign partitions.

2017-08-31 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4905?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang resolved KAFKA-4905. -- Resolution: Not A Problem > StreamPartitionAssignor doesn't respect subscriptions to assign part

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

2017-08-31 Thread Apache Jenkins Server
See Changes: [ismael] KAFKA-2105; Check for null in KafkaProducer.partitionsFor -- [...truncated 914.53 KB...] kafka.server.epoch.LeaderEpochFileCacheTest > shouldNotRes

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

2017-08-31 Thread Jun Rao
Hi, Rajini, Thanks for the updated KIP. Should be also track the conversion time for the producer? If so, the name can just be MessageConversionTimeMs and only the produce and the fetch request may have such a component. Jun On Wed, Aug 30, 2017 at 1:37 PM, Rajini Sivaram wrote: > Jun, > > Tha

[GitHub] kafka pull request #3766: MINOR: Test SASL authorization id

2017-08-31 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/3766 --- 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-0.10.2-jdk7 #195

2017-08-31 Thread Apache Jenkins Server
See Changes: [rajinisivaram] MINOR: Test SASL authorization id -- Started by an SCM change [EnvInject] - Loading node environment variables. Building remotely on H26 (couc

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

2017-08-31 Thread Jun Rao
Hi, Tom, Thanks for the KIP. +1. Just one more minor comment. It seems that the ElectPreferredLeadersResponse should expect at least 3 other types of errors : (1) request timeout exception, (2) leader rebalance in-progress exception, (3) can't move to the preferred replica exception (i.e., preferr

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

2017-08-31 Thread Jun Rao
Hi, Jason, Thanks for the KIP. +1. Just one minor comment. It seems that the new KafkaPrincipalBuilder interface should support Configurable and close() as the existing PrincipalBuilder? Jun On Wed, Aug 30, 2017 at 8:51 AM, Jason Gustafson wrote: > I'd like to open the vote for KIP-189: > http

[jira] [Resolved] (KAFKA-1447) Controlled shutdown deadlock when trying to send state updates

2017-08-31 Thread Manikumar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1447?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar resolved KAFKA-1447. -- Resolution: Fixed Pl reopen if you think the issue still exists > Controlled shutdown deadlock when t

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

2017-08-31 Thread Jason Gustafson
Hey Jun, I will add a note to the KIP, but my plan was to dynamically check whether the KafkaPrincipalBuilder implemented Configurable or Closeable. That makes implementing close() and configure() optional. Does that seem reasonable? -Jason On Thu, Aug 31, 2017 at 9:53 AM, Jun Rao wrote: > Hi,

[GitHub] kafka pull request #3769: MINOR: Log encountered exception during rebalance

2017-08-31 Thread guozhangwang
GitHub user guozhangwang opened a pull request: https://github.com/apache/kafka/pull/3769 MINOR: Log encountered exception during rebalance Some other minor changes: 1. Do not throw the exception form callback as it would only be swallowed by consumer coordinator; rememberi

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

2017-08-31 Thread Jun Rao
That's also fine as long as it's documented in the interface somehow. The only thing with java Closeable is that it throws IOException. In most of our usage, we use close() with no exception since it's not clear what you will do when an exception is thrown in close(). Thanks, Jun On Thu, Aug 31,

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

2017-08-31 Thread Apache Jenkins Server
See

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

2017-08-31 Thread Matthias J. Sax
+1 On 8/31/17 8:49 AM, Jason Gustafson wrote: > +1 > > On Thu, Aug 31, 2017 at 8:36 AM, Mickael Maison > wrote: > >> +1 non binding >> Thanks >> >> On Thu, Aug 31, 2017 at 8:35 AM, Guozhang Wang wrote: >>> +1 >>> >>> On Thu, Aug 31, 2017 at 7:57 AM, Bill Bejeck wrote: >>> +1 Th

[jira] [Created] (KAFKA-5815) Add Printed class and KStream#print(Printed)

2017-08-31 Thread Damian Guy (JIRA)
Damian Guy created KAFKA-5815: - Summary: Add Printed class and KStream#print(Printed) Key: KAFKA-5815 URL: https://issues.apache.org/jira/browse/KAFKA-5815 Project: Kafka Issue Type: Sub-task

[jira] [Created] (KAFKA-5816) Add Produced class and new to and through overloads to KStream

2017-08-31 Thread Damian Guy (JIRA)
Damian Guy created KAFKA-5816: - Summary: Add Produced class and new to and through overloads to KStream Key: KAFKA-5816 URL: https://issues.apache.org/jira/browse/KAFKA-5816 Project: Kafka Issue

[GitHub] kafka pull request #3770: KAFKA-5816: add Produced class, KStream#to(topic, ...

2017-08-31 Thread dguy
GitHub user dguy opened a pull request: https://github.com/apache/kafka/pull/3770 KAFKA-5816: add Produced class, KStream#to(topic, Produced), and KStream#through(topic, Produced) Add the `Produced` class and `KStream` overloads that use it: `KStream#to(String, Produced)` `K

[jira] [Created] (KAFKA-5817) Add Serialized class and KStream groupBy and groupByKey overloads

2017-08-31 Thread Damian Guy (JIRA)
Damian Guy created KAFKA-5817: - Summary: Add Serialized class and KStream groupBy and groupByKey overloads Key: KAFKA-5817 URL: https://issues.apache.org/jira/browse/KAFKA-5817 Project: Kafka Is

[jira] [Resolved] (KAFKA-1402) Create unit test helper that stops and starts a cluster

2017-08-31 Thread Manikumar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1402?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar resolved KAFKA-1402. -- Resolution: Fixed We have KafkaServerTestHarness, EmbeddedKafkaCluster, EmbeddedZookeeper helper classe

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

2017-08-31 Thread Apache Jenkins Server
See

[jira] [Resolved] (KAFKA-1920) Add a metric to count client side errors in BrokerTopicMetrics

2017-08-31 Thread Manikumar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1920?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar resolved KAFKA-1920. -- Resolution: Duplicate Similar metrics are proposed in KAFKA-5746. > Add a metric to count client side

[GitHub] kafka pull request #3771: MINOR: logging improvements

2017-08-31 Thread guozhangwang
GitHub user guozhangwang opened a pull request: https://github.com/apache/kafka/pull/3771 MINOR: logging improvements This is a manual cherry-pick of https://github.com/apache/kafka/pull/3769 for 0.11.0 You can merge this pull request into a Git repository by running: $ git pu

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

2017-08-31 Thread Apache Jenkins Server
See Changes: [rajinisivaram] MINOR: Test SASL authorization id -- [...truncated 914.44 KB...] kafka.server.epoch.LeaderEpochFileCacheTest > shouldNotResetEpochHistoryTai

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

2017-08-31 Thread Apurva Mehta
Thanks for the message, Roger. I think having 'acks=all' imply 'acks=minIsr' will probably result in some improvement in the latency. However, I would note two things: 1. The numbers on the wiki are latency at max throughput, which should not be representative of actual latency degradation. We sh

[GitHub] kafka pull request #3772: KAFKA-5817: Add Serialized class and overloads to ...

2017-08-31 Thread dguy
GitHub user dguy opened a pull request: https://github.com/apache/kafka/pull/3772 KAFKA-5817: Add Serialized class and overloads to KStream#groupBy and KStream#groupByKey Part of KIP-182 - Add the `Serialized` class - implement overloads of `KStream#groupByKey` and KStream#g

[jira] [Resolved] (KAFKA-1138) Remote producer uses the hostname defined in broker

2017-08-31 Thread Manikumar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1138?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar resolved KAFKA-1138. -- Resolution: Fixed > Remote producer uses the hostname defined in broker > --

[jira] [Resolved] (KAFKA-908) Write duplicate messages during broker failure

2017-08-31 Thread Manikumar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-908?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar resolved KAFKA-908. - Resolution: Fixed Closing inactive issue. Pl reopen if you think the issue still exists > Write duplicate

[jira] [Resolved] (KAFKA-2028) Unable to start the ZK instance after myid file was missing and had to recreate it.

2017-08-31 Thread Manikumar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2028?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar resolved KAFKA-2028. -- Resolution: Fixed related to config issue. Pl reopen if you think the issue still exists > Unable to

Build failed in Jenkins: kafka-0.11.0-jdk7 #291

2017-08-31 Thread Apache Jenkins Server
See Changes: [ismael] KAFKA-2105; Check for null in KafkaProducer.partitionsFor -- [...truncated 2.44 MB...] org.apache.kafka.streams.integration.KTableKTableJoinIntegrati

[jira] [Resolved] (KAFKA-2343) Clarify KafkaConsumer.poll rebalance behavior

2017-08-31 Thread Manikumar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2343?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar resolved KAFKA-2343. -- Resolution: Fixed Javadocs updated in newer versions. > Clarify KafkaConsumer.poll rebalance behavior >

Re: [DISCUSS] KIP-190: Handle client-ids consistently between clients and brokers

2017-08-31 Thread Mickael Maison
Even though it's pretty non controversial, I was expecting a few comments. I'll wait until next week for comments then I'll start the vote. Thanks On Mon, Aug 21, 2017 at 6:51 AM, Mickael Maison wrote: > Hi all, > > I have created a KIP to cleanup the way client-ids are handled by > brokers and

[GitHub] kafka pull request #3773: MINOR: Fix unit test failures from last commit, ch...

2017-08-31 Thread rajinisivaram
GitHub user rajinisivaram opened a pull request: https://github.com/apache/kafka/pull/3773 MINOR: Fix unit test failures from last commit, cherry-pick fix for SASL principal You can merge this pull request into a Git repository by running: $ git pull https://github.com/rajin

[jira] [Resolved] (KAFKA-2750) Sender.java: handleProduceResponse does not check protocol version

2017-08-31 Thread Manikumar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2750?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar resolved KAFKA-2750. -- Resolution: Fixed This was fixed in KAFKA-4462 /KIP-97 for newer clients > Sender.java: handleProduceRe

[jira] [Resolved] (KAFKA-2842) BrokerEndPoint regex does't support hostname with _

2017-08-31 Thread Manikumar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2842?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar resolved KAFKA-2842. -- Resolution: Fixed This was fixed in KAFKA-3719 > BrokerEndPoint regex does't support hostname with _ >

[jira] [Resolved] (KAFKA-2897) Class NIOServerCnxn$Factory not found due to mismatch in dependencies

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

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

2017-08-31 Thread Mickael Maison
+1 (non binding) Thanks for the KIP On Thu, Aug 31, 2017 at 10:10 AM, Jun Rao wrote: > That's also fine as long as it's documented in the interface somehow. The > only thing with java Closeable is that it throws IOException. In most of > our usage, we use close() with no exception since it's not

[jira] [Resolved] (KAFKA-2986) Consumer group doesn't lend itself well for slow consumers with varying message size

2017-08-31 Thread Manikumar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2986?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar resolved KAFKA-2986. -- Resolution: Fixed Fixed in newer versions. Pl reopen if you think the issue still exists > Consumer g

[jira] [Resolved] (KAFKA-3079) org.apache.kafka.common.KafkaException: java.lang.SecurityException: Configuration Error:

2017-08-31 Thread Manikumar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3079?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar resolved KAFKA-3079. -- Resolution: Cannot Reproduce Mostly related to config error. Pl reopen if you think the issue still exi

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

2017-08-31 Thread Roger Hoover
Makes sense in terms of priorities. Thanks, Apurva. On Thu, Aug 31, 2017 at 11:15 AM, Apurva Mehta wrote: > Thanks for the message, Roger. > > I think having 'acks=all' imply 'acks=minIsr' will probably result in some > improvement in the latency. However, I would note two things: > > 1. The nu

[jira] [Resolved] (KAFKA-3647) Unable to set a ssl provider

2017-08-31 Thread Manikumar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3647?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar resolved KAFKA-3647. -- Resolution: Fixed Closing as per above comments. > Unable to set a ssl provider > -

[jira] [Resolved] (KAFKA-5806) Fix transient unit test failure in trogdor coordinator shutdown

2017-08-31 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5806?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson resolved KAFKA-5806. Resolution: Fixed Fix Version/s: 1.0.0 Issue resolved by pull request 3755 [https://g

[GitHub] kafka pull request #3755: KAFKA-5806. Fix transient unit test failure in tro...

2017-08-31 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/3755 --- 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-188 - Add new metrics to support health checks

2017-08-31 Thread Rajini Sivaram
Hi Jun, Thank you, that makes sense. Updated the KIP. Regards, Rajini On Thu, Aug 31, 2017 at 12:35 PM, Jun Rao wrote: > Hi, Rajini, > > Thanks for the updated KIP. Should be also track the conversion time for > the producer? If so, the name can just be MessageConversionTimeMs and only > the

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

2017-08-31 Thread Apache Jenkins Server
See Changes: [rajinisivaram] MINOR: Fix unit test failures from last commit, cherry-pick fix for SASL -- [...truncated 159.37 KB...] kafka.server.KafkaConfigTest > testUn

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

2017-08-31 Thread Apache Jenkins Server
See

[GitHub] kafka pull request #3774: MINOR: Increase timeout of Zookeeper service in sy...

2017-08-31 Thread rhauch
GitHub user rhauch opened a pull request: https://github.com/apache/kafka/pull/3774 MINOR: Increase timeout of Zookeeper service in system tests The previous timeout was 10 seconds, but system test failures have occurred when Zookeeper has started after about 11 seconds. Increasing

[GitHub] kafka pull request #3774: MINOR: Increase timeout of Zookeeper service in sy...

2017-08-31 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/3774 --- 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 #3753: Allow timestamp parameter in `ProcessorTopologyTes...

2017-08-31 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/3753 --- 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] [Created] (KAFKA-5818) Kafka Stream state transitions not correct

2017-08-31 Thread Matthias J. Sax (JIRA)
Matthias J. Sax created KAFKA-5818: -- Summary: Kafka Stream state transitions not correct Key: KAFKA-5818 URL: https://issues.apache.org/jira/browse/KAFKA-5818 Project: Kafka Issue Type: Bug

[GitHub] kafka pull request #3775: KAFKA-5818: KafkaStreams state transitions not cor...

2017-08-31 Thread mjsax
GitHub user mjsax opened a pull request: https://github.com/apache/kafka/pull/3775 KAFKA-5818: KafkaStreams state transitions not correct - need to check that state is CRATED at startup - some minor test cleanup You can merge this pull request into a Git repository by running:

Jenkins build is back to normal : kafka-0.10.2-jdk7 #197

2017-08-31 Thread Apache Jenkins Server
See

Jenkins build is back to normal : kafka-0.11.0-jdk7 #292

2017-08-31 Thread Apache Jenkins Server
See

[GitHub] kafka pull request #3773: MINOR: Fix unit test failures from last commit, ch...

2017-08-31 Thread rajinisivaram
Github user rajinisivaram closed the pull request at: https://github.com/apache/kafka/pull/3773 --- 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-4188) compilation issues with org.apache.kafka.clients.consumer.internals.Fetcher.java

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

Re: [DISCUSS] KIP-190: Handle client-ids consistently between clients and brokers

2017-08-31 Thread Gwen Shapira
Thanks for bumping this. I do have a concern: This proposal changes the names of existing metrics - as such, it will require all owners of monitoring systems to update their dashboards. It will also complicate monitoring of multiple clusters with different versions and require some modifications t

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

2017-08-31 Thread Gwen Shapira
Thank you! +1 (binding). On Thu, Aug 31, 2017 at 9:48 AM Jun Rao wrote: > Hi, Tom, > > Thanks for the KIP. +1. Just one more minor comment. It seems that the > ElectPreferredLeadersResponse > should expect at least 3 other types of errors : (1) request timeout > exception, (2) leader rebalance i

[jira] [Resolved] (KAFKA-4411) broker don't have access to kafka zookeeper nodes

2017-08-31 Thread Manikumar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4411?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar resolved KAFKA-4411. -- Resolution: Not A Problem It is necessary to have the same principal name across all brokers for ZK Au