Re: KIP-28 does not allow Processor to specify partition of output message

2015-10-14 Thread Randall Hauch
Created https://issues.apache.org/jira/browse/KAFKA-2649 and attached a PR with the proposed change. Thanks! On October 14, 2015 at 3:12:34 AM, Guozhang Wang (wangg...@gmail.com) wrote: Thanks! On Tue, Oct 13, 2015 at 9:34 PM, Randall Hauch wrote: Ok, cool. I agree we want

[GitHub] kafka pull request: KAFKA-2650: Change ConfigCommand --deleted-con...

2015-10-14 Thread granthenke
GitHub user granthenke opened a pull request: https://github.com/apache/kafka/pull/308 KAFKA-2650: Change ConfigCommand --deleted-config option to align wit… …h TopicCommand You can merge this pull request into a Git repository by running: $ git pull

[jira] [Updated] (KAFKA-2649) Add support for custom partitioner in sink nodes

2015-10-14 Thread Randall Hauch (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2649?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Randall Hauch updated KAFKA-2649: - Status: Patch Available (was: Open) > Add support for custom partitioner in sink nodes >

[jira] [Created] (KAFKA-2651) Remove deprecated config alteration from TopicCommand in 0.9.1.0

2015-10-14 Thread Grant Henke (JIRA)
Grant Henke created KAFKA-2651: -- Summary: Remove deprecated config alteration from TopicCommand in 0.9.1.0 Key: KAFKA-2651 URL: https://issues.apache.org/jira/browse/KAFKA-2651 Project: Kafka

[jira] [Commented] (KAFKA-2650) Change ConfigCommand --deleted-config option to align with TopicCommand

2015-10-14 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2650?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14957081#comment-14957081 ] ASF GitHub Bot commented on KAFKA-2650: --- GitHub user granthenke opened a pull request:

[jira] [Created] (KAFKA-2650) Change ConfigCommand --deleted-config option to align with TopicCommand

2015-10-14 Thread Grant Henke (JIRA)
Grant Henke created KAFKA-2650: -- Summary: Change ConfigCommand --deleted-config option to align with TopicCommand Key: KAFKA-2650 URL: https://issues.apache.org/jira/browse/KAFKA-2650 Project: Kafka

[jira] [Updated] (KAFKA-2650) Change ConfigCommand --deleted-config option to align with TopicCommand

2015-10-14 Thread Grant Henke (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2650?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Grant Henke updated KAFKA-2650: --- Status: Patch Available (was: Open) > Change ConfigCommand --deleted-config option to align with

[jira] [Commented] (KAFKA-2649) Add support for custom partitioner in sink nodes

2015-10-14 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2649?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14957113#comment-14957113 ] ASF GitHub Bot commented on KAFKA-2649: --- GitHub user rhauch opened a pull request:

[GitHub] kafka pull request: KAFKA-2649 Add support for custom partitioning...

2015-10-14 Thread rhauch
GitHub user rhauch opened a pull request: https://github.com/apache/kafka/pull/309 KAFKA-2649 Add support for custom partitioning in topology sinks Added option to use custom partitioning logic within each topology sink. You can merge this pull request into a Git repository by

[jira] [Commented] (KAFKA-2017) Persist Coordinator State for Coordinator Failover

2015-10-14 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2017?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14957268#comment-14957268 ] Jason Gustafson commented on KAFKA-2017: I think persistence in Kafka is a promising idea. Reusing

[jira] [Updated] (KAFKA-2516) Rename o.a.k.client.tools to o.a.k.tools

2015-10-14 Thread Grant Henke (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2516?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Grant Henke updated KAFKA-2516: --- Status: Patch Available (was: Open) > Rename o.a.k.client.tools to o.a.k.tools >

[jira] [Assigned] (KAFKA-2516) Rename o.a.k.client.tools to o.a.k.tools

2015-10-14 Thread Grant Henke (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2516?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Grant Henke reassigned KAFKA-2516: -- Assignee: Grant Henke > Rename o.a.k.client.tools to o.a.k.tools >

[jira] [Commented] (KAFKA-2017) Persist Coordinator State for Coordinator Failover

2015-10-14 Thread Joel Koshy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2017?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14957348#comment-14957348 ] Joel Koshy commented on KAFKA-2017: --- If we reused this topic, then I think we could end up doing

Re: KIP-28 does not allow Processor to specify partition of output message

2015-10-14 Thread Yasuhiro Matsuda
A partitioning scheme should be a cluster wide thing. Letting each sink have a different partitioning scheme does not make sense to me. A partitioning scheme is not specific to a stream job, each task or a sink. I think specifying it at sink level is more error prone. If a user wants to customize

[jira] [Commented] (KAFKA-1436) Idempotent Producer / Duplicate Detection

2015-10-14 Thread Manish (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1436?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14957497#comment-14957497 ] Manish commented on KAFKA-1436: --- Hi Neha, Is there an ETA for getting this issue fixed? Thanks, Manish >

[jira] [Commented] (KAFKA-2017) Persist Coordinator State for Coordinator Failover

2015-10-14 Thread Joel Koshy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2017?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14957204#comment-14957204 ] Joel Koshy commented on KAFKA-2017: --- I agree with those benefits. If we go that route then I would

[jira] [Commented] (KAFKA-2516) Rename o.a.k.client.tools to o.a.k.tools

2015-10-14 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2516?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14957253#comment-14957253 ] ASF GitHub Bot commented on KAFKA-2516: --- GitHub user granthenke opened a pull request:

Re: [DISCUSS] KIP-37 - Add namespaces in Kafka

2015-10-14 Thread Jason Gustafson
Hey Ashish, thanks for the write-up. I think having a namespace capability is a useful feature for Kafka, in particular with the addition of the authorization layer. I probably prefer Jay's hierarchical approach if we're going to embed the namespace in the topic name since it seems more general.

[jira] [Commented] (KAFKA-2017) Persist Coordinator State for Coordinator Failover

2015-10-14 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2017?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14957330#comment-14957330 ] Guozhang Wang commented on KAFKA-2017: -- I am not sure if we want to piggy-back the membership

[GitHub] kafka pull request: KAFKA-2516: Rename o.a.k.client.tools to o.a.k...

2015-10-14 Thread granthenke
GitHub user granthenke opened a pull request: https://github.com/apache/kafka/pull/310 KAFKA-2516: Rename o.a.k.client.tools to o.a.k.tools You can merge this pull request into a Git repository by running: $ git pull https://github.com/granthenke/kafka tools-packaging

Re: [DISCUSS] KIP-37 - Add namespaces in Kafka

2015-10-14 Thread Ashish Singh
On Mon, Oct 12, 2015 at 7:37 PM, Gwen Shapira wrote: > This works really nicely from the consumer side, but what about the > producer? If there are no more topics,do we allow producing to a directory > and have the Partitioner hash-partition messages between all partitions in

Re: [DISCUSS] KIP-37 - Add namespaces in Kafka

2015-10-14 Thread Ashish Singh
On Mon, Oct 12, 2015 at 7:16 PM, Jay Kreps wrote: > Okay this is similar to what I think we have talked about before. Let me > elaborate on the idea that I think has been floating around--it's pretty > similar with a few differences. > > I think what you are calling the

Re: Kafka Monitoring Framework

2015-10-14 Thread Dong Lin
Hey Lin, Thanks for your interest. As Kartik mentioned in the blog post, LinkedIn Kafka team is developing a Kafka monitoring framework to help detect problem that may only be found, for example, when you run server/client with different versions for a long time using production traffic. We hope

Re: KIP-28 does not allow Processor to specify partition of output message

2015-10-14 Thread Guozhang Wang
Thanks! On Tue, Oct 13, 2015 at 9:34 PM, Randall Hauch wrote: > Ok, cool. I agree we want something simple. I'll create an issue and > create a pull request with a proposal. Look for it tomorrow. > > On Oct 13, 2015, at 10:25 PM, Guozhang Wang wrote: > >

[jira] [Updated] (KAFKA-2641) Upgrade path for ZK authentication

2015-10-14 Thread Flavio Junqueira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2641?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Flavio Junqueira updated KAFKA-2641: Description: If a cluster isn't secure and wants to migrate to secure, then it will need to

[jira] [Updated] (KAFKA-2644) Run relevant ducktape tests with SASL_PLAINTEXT and SASL_SSL

2015-10-14 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2644?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma updated KAFKA-2644: --- Summary: Run relevant ducktape tests with SASL_PLAINTEXT and SASL_SSL (was: Run relevant ducktape

[jira] [Updated] (KAFKA-2643) Run mirror maker tests in ducktape with SSL

2015-10-14 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2643?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma updated KAFKA-2643: --- Issue Type: Sub-task (was: Test) Parent: KAFKA-1682 > Run mirror maker tests in ducktape

[jira] [Updated] (KAFKA-2642) Run replication tests in ducktape with SSL for clients

2015-10-14 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2642?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma updated KAFKA-2642: --- Issue Type: Sub-task (was: Test) Parent: KAFKA-1682 > Run replication tests in ducktape with

[jira] [Commented] (KAFKA-1554) Corrupt index found on clean startup

2015-10-14 Thread Bo Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1554?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14956579#comment-14956579 ] Bo Wang commented on KAFKA-1554: @ Jun Rao, @Mayuresh Gharat , I don't think this patch is available. The

[jira] [Commented] (KAFKA-2629) Enable getting SSL password from an executable rather than passing plaintext password

2015-10-14 Thread Ashish K Singh (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2629?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14957626#comment-14957626 ] Ashish K Singh commented on KAFKA-2629: --- [~junrao] [~gwenshap] what do you guys think. As [~yoderme]

[jira] [Commented] (KAFKA-2017) Persist Coordinator State for Coordinator Failover

2015-10-14 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2017?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14957570#comment-14957570 ] Jason Gustafson commented on KAFKA-2017: One additional point worth considering is how

Re: [DISCUSS] KIP-37 - Add namespaces in Kafka

2015-10-14 Thread Jay Kreps
Hey Jason, I actually think this is one of the advantages. The problem we have today is that you can't really do bidirectional replication between clusters because it would actually be a feedback loop. So the intended use would be that you would have a structure where the top-level directory was

[jira] [Commented] (KAFKA-2477) Replicas spuriously deleting all segments in partition

2015-10-14 Thread Chinmay Soman (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2477?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14957645#comment-14957645 ] Chinmay Soman commented on KAFKA-2477: -- [~ewencp] Totally agree. Its just that the current 0.8.2.0

[GitHub] kafka pull request: KAFKA-2603: Add timeout arg to ConsoleConsumer...

2015-10-14 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/274 --- 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: KIP-28 does not allow Processor to specify partition of output message

2015-10-14 Thread Randall Hauch
It absolutely is important that the partitioning logic for a single topic be the same across an entire cluster. IOW, if a topology has a single sink, then no matter where that topology is run in the cluster, it had better use the same partitioning logic. I would argue that when the partitioning

Re: KIP-28 does not allow Processor to specify partition of output message

2015-10-14 Thread Guozhang Wang
I think part of Yasu's motivation for cross-cluster partitioning is that, for example, there could be multiple stream jobs reading / writing to some shared topics but controlled by different teams or services inside an organization, and if one team mistakenly specifying the partitioning in a wrong

Re: KIP-28 does not allow Processor to specify partition of output message

2015-10-14 Thread Yasuhiro Matsuda
>The bottom line is that *different* topics will likely need to be partitioned differently. You can do it with the existing Partitioner interface. Centralizing the logic doesn't mean all topics must use the same partitioning scheme. On Wed, Oct 14, 2015 at 1:03 PM, Randall Hauch

[jira] [Commented] (KAFKA-2397) leave group request

2015-10-14 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2397?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14957718#comment-14957718 ] Guozhang Wang commented on KAFKA-2397: -- For consumer shutdown, in the ZK-based consumer we will

RE: Kafka Monitoring Framework

2015-10-14 Thread Lin Ma
Thanks Dong. Then I bet it will take a while before its release. Lin -Original Message- From: Dong Lin [mailto:lindon...@gmail.com] Sent: Tuesday, October 13, 2015 11:21 PM To: dev@kafka.apache.org Subject: Re: Kafka Monitoring Framework Hey Lin, Thanks for your interest. As Kartik

[jira] [Commented] (KAFKA-2397) leave group request

2015-10-14 Thread Jay Kreps (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2397?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14957550#comment-14957550 ] Jay Kreps commented on KAFKA-2397: -- Dunno if we closed the loop on the approach. [~onurkaraman] Yeah the

Re: KIP-28 does not allow Processor to specify partition of output message

2015-10-14 Thread Guozhang Wang
I agree that cluster-wide partitioning would be preferable in cases where multiple producers from different services sharing the same topics, and this may well be resolved by the same manner like the schema registry service. On the other hand, I think this is not a problem that would be solved at

[jira] [Commented] (KAFKA-2017) Persist Coordinator State for Coordinator Failover

2015-10-14 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2017?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14957665#comment-14957665 ] Guozhang Wang commented on KAFKA-2017: -- Here is my two cents comparing those two approaches: 1.

[jira] [Commented] (KAFKA-2603) Add timeout to ConsoleConsumer running with new consumer

2015-10-14 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2603?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14957693#comment-14957693 ] ASF GitHub Bot commented on KAFKA-2603: --- Github user asfgit closed the pull request at:

[jira] [Updated] (KAFKA-2603) Add timeout to ConsoleConsumer running with new consumer

2015-10-14 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2603?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang updated KAFKA-2603: - Resolution: Fixed Status: Resolved (was: Patch Available) Issue resolved by pull request

[jira] [Commented] (KAFKA-2629) Enable getting SSL password from an executable rather than passing plaintext password

2015-10-14 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2629?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14957726#comment-14957726 ] Gwen Shapira commented on KAFKA-2629: - [~singhashish], I see the need for this type of integration - I

Re: Kafka Monitoring Framework

2015-10-14 Thread Dong Lin
Lin: Yes, I think so. On Wed, Oct 14, 2015 at 1:48 PM, Lin Ma wrote: > Thanks Dong. Then I bet it will take a while before its release. > > Lin > > > -Original Message- > From: Dong Lin [mailto:lindon...@gmail.com] > Sent: Tuesday, October 13, 2015 11:21 PM > To:

[jira] [Updated] (KAFKA-2593) KeyValueStores should not require use of the context's default serializers and deserializers

2015-10-14 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2593?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang updated KAFKA-2593: - Resolution: Fixed Status: Resolved (was: Patch Available) Issue resolved by pull request

[jira] [Created] (KAFKA-2653) Stateful operations in the KStream DSL layer

2015-10-14 Thread Guozhang Wang (JIRA)
Guozhang Wang created KAFKA-2653: Summary: Stateful operations in the KStream DSL layer Key: KAFKA-2653 URL: https://issues.apache.org/jira/browse/KAFKA-2653 Project: Kafka Issue Type:

[jira] [Commented] (KAFKA-2397) leave group request

2015-10-14 Thread Onur Karaman (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2397?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14957784#comment-14957784 ] Onur Karaman commented on KAFKA-2397: - Cool. It sounds like we all generally agree on the explicit

[jira] [Created] (KAFKA-2655) Consumer.poll(0)'s overhead too large

2015-10-14 Thread Guozhang Wang (JIRA)
Guozhang Wang created KAFKA-2655: Summary: Consumer.poll(0)'s overhead too large Key: KAFKA-2655 URL: https://issues.apache.org/jira/browse/KAFKA-2655 Project: Kafka Issue Type: Sub-task

[GitHub] kafka pull request: KAFKA-2536: topics tool should allow users to ...

2015-10-14 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/305 --- 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: TRIVIAL: add @throws ConsumerWakeupException i...

2015-10-14 Thread guozhangwang
GitHub user guozhangwang opened a pull request: https://github.com/apache/kafka/pull/311 TRIVIAL: add @throws ConsumerWakeupException in KafkaConsumer You can merge this pull request into a Git repository by running: $ git pull https://github.com/guozhangwang/kafka

[jira] [Updated] (KAFKA-2656) Default SSL keystore and truststore config are unusable

2015-10-14 Thread Rajini Sivaram (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2656?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Rajini Sivaram updated KAFKA-2656: -- Status: Patch Available (was: Open) Removed default keystore and truststore for Kafka server

[jira] [Updated] (KAFKA-2590) Kafka Streams Checklist

2015-10-14 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2590?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang updated KAFKA-2590: - Issue Type: New Feature (was: Task) > Kafka Streams Checklist > --- > >

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

2015-10-14 Thread Apache Jenkins Server
See Changes: [wangguoz] KAFKA-2603: Add timeout arg to ConsoleConsumer for new consumer -- [...truncated 1825 lines...] kafka.coordinator.ConsumerGroupMetadataTest >

[jira] [Updated] (KAFKA-2650) Change ConfigCommand --deleted-config option to align with TopicCommand

2015-10-14 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2650?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gwen Shapira updated KAFKA-2650: Resolution: Fixed Status: Resolved (was: Patch Available) Issue resolved by pull request

[jira] [Commented] (KAFKA-2650) Change ConfigCommand --deleted-config option to align with TopicCommand

2015-10-14 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2650?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14957758#comment-14957758 ] ASF GitHub Bot commented on KAFKA-2650: --- Github user asfgit closed the pull request at:

[GitHub] kafka pull request: KAFKA-2650: Change ConfigCommand --deleted-con...

2015-10-14 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/308 --- 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] [Created] (KAFKA-2654) Avoid calling Consumer.poll(0) in each iteration

2015-10-14 Thread Guozhang Wang (JIRA)
Guozhang Wang created KAFKA-2654: Summary: Avoid calling Consumer.poll(0) in each iteration Key: KAFKA-2654 URL: https://issues.apache.org/jira/browse/KAFKA-2654 Project: Kafka Issue Type:

[jira] [Commented] (KAFKA-2629) Enable getting SSL password from an executable rather than passing plaintext password

2015-10-14 Thread Sriharsha Chintalapani (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2629?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14957801#comment-14957801 ] Sriharsha Chintalapani commented on KAFKA-2629: --- [~singhashish] I still disagree with this

[jira] [Created] (KAFKA-2656) Default SSL keystore and truststore config are unusable

2015-10-14 Thread Rajini Sivaram (JIRA)
Rajini Sivaram created KAFKA-2656: - Summary: Default SSL keystore and truststore config are unusable Key: KAFKA-2656 URL: https://issues.apache.org/jira/browse/KAFKA-2656 Project: Kafka

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

2015-10-14 Thread Apache Jenkins Server
See Changes: [cshapi] KAFKA-2536: topics tool should allow users to alter topic configuration [cshapi] TRIVIAL: add @throws ConsumerWakeupException in KafkaConsumer -- [...truncated 362

[jira] [Created] (KAFKA-2652) Incorporate the new consumer protocol with partition-group interface

2015-10-14 Thread Guozhang Wang (JIRA)
Guozhang Wang created KAFKA-2652: Summary: Incorporate the new consumer protocol with partition-group interface Key: KAFKA-2652 URL: https://issues.apache.org/jira/browse/KAFKA-2652 Project: Kafka

[GitHub] kafka pull request: KAFKA-2516: Rename o.a.k.client.tools to o.a.k...

2015-10-14 Thread granthenke
Github user granthenke closed the pull request at: https://github.com/apache/kafka/pull/310 --- 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] [Commented] (KAFKA-2516) Rename o.a.k.client.tools to o.a.k.tools

2015-10-14 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2516?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14957774#comment-14957774 ] ASF GitHub Bot commented on KAFKA-2516: --- GitHub user granthenke reopened a pull request:

[jira] [Commented] (KAFKA-2516) Rename o.a.k.client.tools to o.a.k.tools

2015-10-14 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2516?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14957773#comment-14957773 ] ASF GitHub Bot commented on KAFKA-2516: --- Github user granthenke closed the pull request at:

[GitHub] kafka pull request: KAFKA-2516: Rename o.a.k.client.tools to o.a.k...

2015-10-14 Thread granthenke
GitHub user granthenke reopened a pull request: https://github.com/apache/kafka/pull/310 KAFKA-2516: Rename o.a.k.client.tools to o.a.k.tools You can merge this pull request into a Git repository by running: $ git pull https://github.com/granthenke/kafka tools-packaging

[jira] [Commented] (KAFKA-2536) topics tool should allow users to alter topic configuration

2015-10-14 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2536?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14957819#comment-14957819 ] ASF GitHub Bot commented on KAFKA-2536: --- Github user asfgit closed the pull request at:

[jira] [Updated] (KAFKA-2536) topics tool should allow users to alter topic configuration

2015-10-14 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2536?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gwen Shapira updated KAFKA-2536: Resolution: Fixed Status: Resolved (was: Patch Available) Issue resolved by pull request

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

2015-10-14 Thread Apache Jenkins Server
See Changes: [wangguoz] KAFKA-2603: Add timeout arg to ConsoleConsumer for new consumer [wangguoz] KAFKA-2593: Key value stores can use specified serializers and [cshapi] KAFKA-2650: Change ConfigCommand --deleted-config option to

[GitHub] kafka pull request: KAFKA-2656: Remove hardcoded default key and t...

2015-10-14 Thread rajinisivaram
GitHub user rajinisivaram opened a pull request: https://github.com/apache/kafka/pull/312 KAFKA-2656: Remove hardcoded default key and truststores Removed default hardcoded keystore and truststore in /tmp so that default JVM keystore/truststore may be used when keystore/truststore

[jira] [Commented] (KAFKA-2656) Default SSL keystore and truststore config are unusable

2015-10-14 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2656?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14957933#comment-14957933 ] ASF GitHub Bot commented on KAFKA-2656: --- GitHub user rajinisivaram opened a pull request:

Slow request log in Kafka

2015-10-14 Thread Aditya Auradkar
Hey everyone, We were recently discussing a small logging improvement for Kafka. Basically, add a request log for queries that took longer than a certain configurable time to execute. This can be quite useful for debugging purposes, in fact it would have proven handy while investigating a recent

[GitHub] kafka pull request: KAFKA-2593 Key value stores can use specified ...

2015-10-14 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/255 --- 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] [Commented] (KAFKA-2593) KeyValueStores should not require use of the context's default serializers and deserializers

2015-10-14 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2593?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14957739#comment-14957739 ] ASF GitHub Bot commented on KAFKA-2593: --- Github user asfgit closed the pull request at:

[jira] [Commented] (KAFKA-2655) Consumer.poll(0)'s overhead too large

2015-10-14 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2655?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14957824#comment-14957824 ] Ismael Juma commented on KAFKA-2655: It would be good to get some actual numbers before we try to

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

2015-10-14 Thread Apache Jenkins Server
See Changes: [wangguoz] KAFKA-2593: Key value stores can use specified serializers and [cshapi] KAFKA-2650: Change ConfigCommand --deleted-config option to align wit… [cshapi] KAFKA-2536: topics tool should allow users to alter

Re: [DISCUSS] KIP-36 - Rack aware replica assignment

2015-10-14 Thread Gwen Shapira
Can you clarify the workflow for the following scenarios: 1. I currently have 6 brokers and want to add rack information for each 2. I'm adding a new broker and I want to specify which rack it belongs on while adding it. Thanks! On Tue, Oct 13, 2015 at 2:21 PM, Allen Wang

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

2015-10-14 Thread Apache Jenkins Server
See Changes: [cshapi] TRIVIAL: add @throws ConsumerWakeupException in KafkaConsumer -- [...truncated 322 lines...] :kafka-trunk-jdk7:clients:jar UP-TO-DATE :kafka-trunk-jdk7:clients:javadoc

[jira] [Updated] (KAFKA-2490) support new consumer in ConsumerGroupCommand

2015-10-14 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2490?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang updated KAFKA-2490: - Reviewer: Guozhang Wang > support new consumer in ConsumerGroupCommand >

[jira] [Commented] (KAFKA-2500) Make logEndOffset available in the 0.8.3 Consumer

2015-10-14 Thread James Cheng (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2500?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14958045#comment-14958045 ] James Cheng commented on KAFKA-2500: Is there any chance this will make it into 0.9.0? > Make

[jira] [Commented] (KAFKA-2017) Persist Coordinator State for Coordinator Failover

2015-10-14 Thread Ashish K Singh (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2017?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14958100#comment-14958100 ] Ashish K Singh commented on KAFKA-2017: --- I agree with [~guozhang] that if we are not expecting group

[jira] [Commented] (KAFKA-2515) handle oversized messages properly in new consumer

2015-10-14 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2515?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14958099#comment-14958099 ] Guozhang Wang commented on KAFKA-2515: -- [~junrao] just to clarify, for step 3) above you have

[jira] [Commented] (KAFKA-2295) Dynamically loaded classes (encoders, etc.) may not be found by Kafka Producer

2015-10-14 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2295?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14958017#comment-14958017 ] Guozhang Wang commented on KAFKA-2295: -- [~omkreddy] Could you rebase the patch again? > Dynamically

[jira] [Comment Edited] (KAFKA-2295) Dynamically loaded classes (encoders, etc.) may not be found by Kafka Producer

2015-10-14 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2295?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14958017#comment-14958017 ] Guozhang Wang edited comment on KAFKA-2295 at 10/14/15 11:40 PM: -

[jira] [Updated] (KAFKA-2449) Update mirror maker (MirrorMaker) docs

2015-10-14 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2449?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang updated KAFKA-2449: - Reviewer: Ismael Juma > Update mirror maker (MirrorMaker) docs >

[jira] [Updated] (KAFKA-2412) Documentation bug: Add information for key.serializer and value.serializer to New Producer Config sections

2015-10-14 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2412?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang updated KAFKA-2412: - Reviewer: Gwen Shapira > Documentation bug: Add information for key.serializer and

[jira] [Updated] (KAFKA-2464) Client-side assignment and group generalization

2015-10-14 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2464?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang updated KAFKA-2464: - Reviewer: Guozhang Wang > Client-side assignment and group generalization >

[jira] [Updated] (KAFKA-2417) Ducktape tests for SSL/TLS

2015-10-14 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2417?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang updated KAFKA-2417: - Reviewer: Jun Rao > Ducktape tests for SSL/TLS > -- > >

[jira] [Updated] (KAFKA-2391) Blocking call such as position(), partitionsFor(), committed() and listTopics() should have a timeout

2015-10-14 Thread Onur Karaman (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2391?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Onur Karaman updated KAFKA-2391: Priority: Blocker (was: Major) > Blocking call such as position(), partitionsFor(), committed()

[jira] [Commented] (KAFKA-2629) Enable getting SSL password from an executable rather than passing plaintext password

2015-10-14 Thread Ashish K Singh (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2629?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14958020#comment-14958020 ] Ashish K Singh commented on KAFKA-2629: --- [~gwenshap] sure. We can pick this up once 0.9.0.0 has been

[jira] [Commented] (KAFKA-2629) Enable getting SSL password from an executable rather than passing plaintext password

2015-10-14 Thread Ashish K Singh (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2629?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14958025#comment-14958025 ] Ashish K Singh commented on KAFKA-2629: --- [~sriharsha] I am proposing this as an optional config

[jira] [Updated] (KAFKA-2516) Rename o.a.k.client.tools to o.a.k.tools

2015-10-14 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2516?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang updated KAFKA-2516: - Reviewer: Gwen Shapira > Rename o.a.k.client.tools to o.a.k.tools >

[jira] [Updated] (KAFKA-2441) SSL/TLS in official docs

2015-10-14 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2441?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang updated KAFKA-2441: - Reviewer: Jun Rao > SSL/TLS in official docs > > > Key:

[jira] [Updated] (KAFKA-2562) check Kafka scripts for 0.9.0.0

2015-10-14 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2562?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang updated KAFKA-2562: - Reviewer: Ismael Juma > check Kafka scripts for 0.9.0.0 > --- > >

[jira] [Updated] (KAFKA-2106) Partition balance tool between borkers

2015-10-14 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2106?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang updated KAFKA-2106: - Assignee: chenshangan > Partition balance tool between borkers >

[jira] [Commented] (KAFKA-2146) adding partition did not find the correct startIndex

2015-10-14 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2146?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14958050#comment-14958050 ] Guozhang Wang commented on KAFKA-2146: -- [~chenshangan...@163.com] your updated patch seems not apply

[jira] [Created] (KAFKA-2649) Add support for custom partitioner in sink nodes

2015-10-14 Thread Randall Hauch (JIRA)
Randall Hauch created KAFKA-2649: Summary: Add support for custom partitioner in sink nodes Key: KAFKA-2649 URL: https://issues.apache.org/jira/browse/KAFKA-2649 Project: Kafka Issue Type:

[jira] [Updated] (KAFKA-2649) Add support for custom partitioner in sink nodes

2015-10-14 Thread Randall Hauch (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2649?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Randall Hauch updated KAFKA-2649: - Description: The only way for Processor implementations to control partitioning of forwarded

[jira] [Commented] (KAFKA-1554) Corrupt index found on clean startup

2015-10-14 Thread Mayuresh Gharat (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1554?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14958225#comment-14958225 ] Mayuresh Gharat commented on KAFKA-1554: [~wangbo23] thanks for the review. We did not iterate on

  1   2   >