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

2015-10-13 Thread Apache Jenkins Server
See Changes: [cshapi] KAFKA-2632: move fetchable check ahead in handleFetchResponse [cshapi] MINOR: ignore wakeups when committing offsets on consumer close -- [...truncated 4191 lines...]

[jira] [Commented] (KAFKA-2120) Add a request timeout to NetworkClient

2015-10-13 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2120?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14956184#comment-14956184 ] Guozhang Wang commented on KAFKA-2120: -- [~mgharat] Your commit seems contains some debugging code in

[jira] [Created] (KAFKA-2648) Coordinator should not allow empty groupIds

2015-10-13 Thread Jason Gustafson (JIRA)
Jason Gustafson created KAFKA-2648: -- Summary: Coordinator should not allow empty groupIds Key: KAFKA-2648 URL: https://issues.apache.org/jira/browse/KAFKA-2648 Project: Kafka Issue Type:

Kafka Monitoring Framework

2015-10-13 Thread Lin Ma
Hi, I read from website that effort has been put on developing Kafka monitoring framework recently (detailed below). Anyone knows when this framework can be released into kafka major version? Thanks. https://engineering.linkedin.com/apache-kafka/how-we_re-improving-and-advancing-kafka-linkedin

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

2015-10-13 Thread Apache Jenkins Server
See Changes: [cshapi] KAFKA-2632: move fetchable check ahead in handleFetchResponse [cshapi] MINOR: ignore wakeups when committing offsets on consumer close -- [...truncated 2769 lines...]

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

2015-10-13 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2017?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14956186#comment-14956186 ] Guozhang Wang commented on KAFKA-2017: -- I agree, and I think that was originally proposed by

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

2015-10-13 Thread Guozhang Wang
I see your point. Yeah I think it is a good way to add a Partitioner into addSink(...) but the Partitioner interface in producer is a bit overkill: "partition(String topic, Object key, byte[] keyBytes, Object value, byte[] valueBytes, Cluster cluster)" whereas for us we only want to partition on

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

2015-10-13 Thread Randall Hauch
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: > > I see your point. Yeah I think it is a good way to add a Partitioner into >

Re: [DISCUSS] KIP-32 - Add CreateTime and LogAppendTime to Kafka message

2015-10-13 Thread Jiangjie Qin
Hey Jay and Guozhang, Thanks a lot for the reply. So if I understand correctly, Jay's proposal is: 1. Let client stamp the message create time. 2. Broker build index based on client-stamped message create time. 3. Broker only takes message whose create time is withing current time plus/minus T

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

2015-10-13 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2477?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14954511#comment-14954511 ] Ewen Cheslack-Postava commented on KAFKA-2477: -- [~cpsoman] Beyond applying to 0.8.2.0 with

[jira] [Reopened] (KAFKA-2633) Default logging from tools to Stderr

2015-10-13 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2633?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ewen Cheslack-Postava reopened KAFKA-2633: -- Commented on PR about potential compatibility issues. Would be glad to see this

[jira] [Created] (KAFKA-2637) Cipher suite setting should be configurable for SSL

2015-10-13 Thread Ben Stopford (JIRA)
Ben Stopford created KAFKA-2637: --- Summary: Cipher suite setting should be configurable for SSL Key: KAFKA-2637 URL: https://issues.apache.org/jira/browse/KAFKA-2637 Project: Kafka Issue Type:

Re: [DISCUSS] KIP-32 - Add CreateTime and LogAppendTime to Kafka message

2015-10-13 Thread Jay Kreps
Here's my basic take: - I agree it would be nice to have a notion of time baked in if it were done right - All the proposals so far seem pretty complex--I think they might make things worse rather than better overall - I think adding 2x8 byte timestamps to the message is probably a non-starter

Re: [DISCUSS] KIP-32 - Add CreateTime and LogAppendTime to Kafka message

2015-10-13 Thread Jay Kreps
I think it should be possible to index out-of-order timestamps. The timestamp index would be similar to the offset index, a memory mapped file appended to as part of the log append, but would have the format timestamp offset The timestamp entries would be monotonic and as with the offset index

[GitHub] kafka pull request: KAFKA-2637: Cipher suite setting should be con...

2015-10-13 Thread benstopford
GitHub user benstopford opened a pull request: https://github.com/apache/kafka/pull/301 KAFKA-2637: Cipher suite setting should be configurable for SSL Enables Cipher suite setting. Code was previously reviewed by @ijuma, @harshach. Moving to an independent PR. You can merge this

[jira] [Assigned] (KAFKA-2637) Cipher suite setting should be configurable for SSL

2015-10-13 Thread Ben Stopford (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2637?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ben Stopford reassigned KAFKA-2637: --- Assignee: Ben Stopford > Cipher suite setting should be configurable for SSL >

[jira] [Commented] (KAFKA-2637) Cipher suite setting should be configurable for SSL

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

[jira] [Updated] (KAFKA-2637) Cipher suite setting should be configurable for SSL

2015-10-13 Thread Ben Stopford (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2637?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ben Stopford updated KAFKA-2637: Status: Patch Available (was: In Progress) > Cipher suite setting should be configurable for SSL >

[jira] [Work started] (KAFKA-2637) Cipher suite setting should be configurable for SSL

2015-10-13 Thread Ben Stopford (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2637?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Work on KAFKA-2637 started by Ben Stopford. --- > Cipher suite setting should be configurable for SSL >

[jira] [Updated] (KAFKA-2637) Cipher suite setting should be configurable for SSL

2015-10-13 Thread Ben Stopford (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2637?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ben Stopford updated KAFKA-2637: Component/s: security > Cipher suite setting should be configurable for SSL >

[jira] [Resolved] (KAFKA-2564) SSL: Received fatal alert: handshake_failure occurs sporadically

2015-10-13 Thread Ben Stopford (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2564?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ben Stopford resolved KAFKA-2564. - Resolution: Duplicate Assignee: Ben Stopford Duplicate of

[jira] [Updated] (KAFKA-2581) Run some existing ducktape tests with SSL-enabled clients and brokers

2015-10-13 Thread Rajini Sivaram (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2581?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Rajini Sivaram updated KAFKA-2581: -- Summary: Run some existing ducktape tests with SSL-enabled clients and brokers (was: Run all

[jira] [Commented] (KAFKA-2581) Run some existing ducktape tests with SSL-enabled clients and brokers

2015-10-13 Thread Rajini Sivaram (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2581?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14954959#comment-14954959 ] Rajini Sivaram commented on KAFKA-2581: --- Tests that have been SSL enabled: # Console consumer sanity

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

2015-10-13 Thread Rajini Sivaram (JIRA)
Rajini Sivaram created KAFKA-2643: - Summary: Run mirror maker tests in ducktape with SSL Key: KAFKA-2643 URL: https://issues.apache.org/jira/browse/KAFKA-2643 Project: Kafka Issue Type: Test

[jira] [Created] (KAFKA-2644) Run relevant ducktape tests with SASL_PLAIN and SASL_SSL

2015-10-13 Thread Ismael Juma (JIRA)
Ismael Juma created KAFKA-2644: -- Summary: Run relevant ducktape tests with SASL_PLAIN and SASL_SSL Key: KAFKA-2644 URL: https://issues.apache.org/jira/browse/KAFKA-2644 Project: Kafka Issue

[jira] [Commented] (KAFKA-2581) Run some existing ducktape tests with SSL-enabled clients and brokers

2015-10-13 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2581?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14954964#comment-14954964 ] Ismael Juma commented on KAFKA-2581: Thanks! > Run some existing ducktape tests with SSL-enabled

[jira] [Commented] (KAFKA-2637) Cipher suite setting should be configurable for SSL

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

[GitHub] kafka pull request: KAFKA-2637: Cipher suite setting should be con...

2015-10-13 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/301 --- 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-2642) Run replication tests in ducktape with SSL for clients

2015-10-13 Thread Rajini Sivaram (JIRA)
Rajini Sivaram created KAFKA-2642: - Summary: Run replication tests in ducktape with SSL for clients Key: KAFKA-2642 URL: https://issues.apache.org/jira/browse/KAFKA-2642 Project: Kafka Issue

[jira] [Created] (KAFKA-2638) ConsumerPerformance (kafka.tools) should support properties via file

2015-10-13 Thread Ben Stopford (JIRA)
Ben Stopford created KAFKA-2638: --- Summary: ConsumerPerformance (kafka.tools) should support properties via file Key: KAFKA-2638 URL: https://issues.apache.org/jira/browse/KAFKA-2638 Project: Kafka

[GitHub] kafka pull request: Kafka 2639: Refactoring of ZkUtils

2015-10-13 Thread fpj
GitHub user fpj opened a pull request: https://github.com/apache/kafka/pull/303 Kafka 2639: Refactoring of ZkUtils I've split the work of KAFKA-1695 because this refactoring touches a large number of files. Most of the changes are trivial, but I feel it will be easier to review

[jira] [Commented] (KAFKA-2638) ConsumerPerformance (kafka.tools) should support properties via file

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

[GitHub] kafka pull request: KAFKA-2638: Added default properties file to C...

2015-10-13 Thread benstopford
GitHub user benstopford opened a pull request: https://github.com/apache/kafka/pull/302 KAFKA-2638: Added default properties file to ConsumerPerformance Blocker for SSL integration You can merge this pull request into a Git repository by running: $ git pull

[jira] [Resolved] (KAFKA-2638) ConsumerPerformance (kafka.tools) should support properties via file

2015-10-13 Thread Ben Stopford (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2638?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ben Stopford resolved KAFKA-2638. - Resolution: Duplicate > ConsumerPerformance (kafka.tools) should support properties via file >

[jira] [Commented] (KAFKA-2638) ConsumerPerformance (kafka.tools) should support properties via file

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

[jira] [Created] (KAFKA-2640) Add tests for ZK authentication

2015-10-13 Thread Flavio Junqueira (JIRA)
Flavio Junqueira created KAFKA-2640: --- Summary: Add tests for ZK authentication Key: KAFKA-2640 URL: https://issues.apache.org/jira/browse/KAFKA-2640 Project: Kafka Issue Type: Sub-task

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

2015-10-13 Thread Flavio Junqueira (JIRA)
Flavio Junqueira created KAFKA-2641: --- Summary: Upgrade path for ZK authentication Key: KAFKA-2641 URL: https://issues.apache.org/jira/browse/KAFKA-2641 Project: Kafka Issue Type: Sub-task

[jira] [Commented] (KAFKA-1695) Authenticate connection to Zookeeper

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

[GitHub] kafka pull request: KAFKA-2638: Added default properties file to C...

2015-10-13 Thread benstopford
Github user benstopford closed the pull request at: https://github.com/apache/kafka/pull/302 --- 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-2639) Refactoring of ZkUtils

2015-10-13 Thread Flavio Junqueira (JIRA)
Flavio Junqueira created KAFKA-2639: --- Summary: Refactoring of ZkUtils Key: KAFKA-2639 URL: https://issues.apache.org/jira/browse/KAFKA-2639 Project: Kafka Issue Type: Sub-task

[jira] [Commented] (KAFKA-2644) Run relevant ducktape tests with SASL_PLAIN and SASL_SSL

2015-10-13 Thread Rajini Sivaram (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2644?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14955056#comment-14955056 ] Rajini Sivaram commented on KAFKA-2644: --- [~ijuma] Are you planning to work on this? If not, I am

[jira] [Commented] (KAFKA-2644) Run relevant ducktape tests with SASL_PLAIN and SASL_SSL

2015-10-13 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2644?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14955099#comment-14955099 ] Ismael Juma commented on KAFKA-2644: It would be awesome if you could do this. > Run relevant

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

2015-10-13 Thread Apache Jenkins Server
See

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

2015-10-13 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2643?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14954979#comment-14954979 ] Ismael Juma commented on KAFKA-2643: This probably depends on KAFKA-2452 too, right? > Run mirror

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

2015-10-13 Thread Rajini Sivaram (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2643?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14955046#comment-14955046 ] Rajini Sivaram commented on KAFKA-2643: --- [~ijuma] Thank you, yes, this task depends on KAFKA-2452 as

[jira] [Created] (KAFKA-2646) Re-enable altering topic config in the topics command to maintain backwards compatibility

2015-10-13 Thread Grant Henke (JIRA)
Grant Henke created KAFKA-2646: -- Summary: Re-enable altering topic config in the topics command to maintain backwards compatibility Key: KAFKA-2646 URL: https://issues.apache.org/jira/browse/KAFKA-2646

[jira] [Commented] (KAFKA-2372) Copycat distributed config storage

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

[jira] [Updated] (KAFKA-2372) Copycat distributed config storage

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

[jira] [Updated] (KAFKA-2620) Introduce Scalariform

2015-10-13 Thread Grant Henke (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2620?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Grant Henke updated KAFKA-2620: --- Status: Patch Available (was: Open) > Introduce Scalariform > - > >

[jira] [Resolved] (KAFKA-2633) Default logging from tools to Stderr

2015-10-13 Thread Grant Henke (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2633?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Grant Henke resolved KAFKA-2633. Resolution: Fixed > Default logging from tools to Stderr > > >

[jira] [Commented] (KAFKA-2633) Default logging from tools to Stderr

2015-10-13 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2633?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14955241#comment-14955241 ] Gwen Shapira commented on KAFKA-2633: - [~brocknoland] - technically it was fixed, the change was

[GitHub] kafka pull request: KAFKA-2372: Add Kafka-backed storage of Copyca...

2015-10-13 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/241 --- 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-2633) Default logging from tools to Stderr

2015-10-13 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2633?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14955194#comment-14955194 ] Gwen Shapira commented on KAFKA-2633: - [~ewencp], thanks for highlighting a possible issue. I'm not

[jira] [Created] (KAFKA-2645) Document potentially breaking changes in the release notes for 0.9.0

2015-10-13 Thread Grant Henke (JIRA)
Grant Henke created KAFKA-2645: -- Summary: Document potentially breaking changes in the release notes for 0.9.0 Key: KAFKA-2645 URL: https://issues.apache.org/jira/browse/KAFKA-2645 Project: Kafka

[jira] [Commented] (KAFKA-2633) Default logging from tools to Stderr

2015-10-13 Thread Brock Noland (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2633?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14955206#comment-14955206 ] Brock Noland commented on KAFKA-2633: - >From a user perspective, I'd like to see this in ASAP. I

[jira] [Commented] (KAFKA-2633) Default logging from tools to Stderr

2015-10-13 Thread Grant Henke (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2633?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14955220#comment-14955220 ] Grant Henke commented on KAFKA-2633: _Duplicating my comment here_ I am happy to send a docs patch

[jira] [Created] (KAFKA-2647) Migrate System Tools to work with SSL

2015-10-13 Thread Ben Stopford (JIRA)
Ben Stopford created KAFKA-2647: --- Summary: Migrate System Tools to work with SSL Key: KAFKA-2647 URL: https://issues.apache.org/jira/browse/KAFKA-2647 Project: Kafka Issue Type: Improvement

Re: Kafka KIP meeting Oct 13 at 11:00am PST

2015-10-13 Thread Jun Rao
The following are the notes from today's meeting. * 0.9.0 release: We discussed if KAFKA-2397 should be a blocker in 0.9.0. Jason and Guozhang will follow up on the jira. * KIP-32 and KIP-33: We discussed Jay's alternative proposal of just keeping CreateTime in the message and having a config to

[jira] [Commented] (KAFKA-2620) Introduce Scalariform

2015-10-13 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2620?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14955538#comment-14955538 ] Gwen Shapira commented on KAFKA-2620: - So the plan is to wait until after 0.9.0 is released and then

[jira] [Commented] (KAFKA-2636) Producer connectivity obscured connection failure logging

2015-10-13 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2636?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14955355#comment-14955355 ] Gwen Shapira commented on KAFKA-2636: - This specific error is in debug level because the KafkaProducer

[jira] [Commented] (KAFKA-2620) Introduce Scalariform

2015-10-13 Thread Grant Henke (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2620?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14955382#comment-14955382 ] Grant Henke commented on KAFKA-2620: They should probably go in at a similar time, or at least the

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

2015-10-13 Thread Apache Jenkins Server
See Changes: [cshapi] KAFKA-2372: Add Kafka-backed storage of Copycat configs. -- [...truncated 362 lines...] :kafka-trunk-jdk8:core:compileScala UP-TO-DATE

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

2015-10-13 Thread Grant Henke (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2536?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Grant Henke reassigned KAFKA-2536: -- Assignee: Grant Henke > topics tool should allow users to alter topic configuration >

[jira] [Resolved] (KAFKA-2646) Re-enable altering topic config in the topics command to maintain backwards compatibility

2015-10-13 Thread Grant Henke (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2646?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Grant Henke resolved KAFKA-2646. Resolution: Duplicate > Re-enable altering topic config in the topics command to maintain backwards

[jira] [Commented] (KAFKA-2620) Introduce Scalariform

2015-10-13 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2620?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14955351#comment-14955351 ] Gwen Shapira commented on KAFKA-2620: - Is there any benefit for getting this in before we agree on

[jira] [Assigned] (KAFKA-2647) Migrate System Tools to work with SSL

2015-10-13 Thread Ben Stopford (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2647?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ben Stopford reassigned KAFKA-2647: --- Assignee: Ben Stopford > Migrate System Tools to work with SSL >

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

2015-10-13 Thread Grant Henke (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2536?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14955381#comment-14955381 ] Grant Henke commented on KAFKA-2536: If we are deprecating it from the topic command, I can log that

[jira] [Updated] (KAFKA-1804) Kafka network thread lacks top exception handler

2015-10-13 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1804?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma updated KAFKA-1804: --- Fix Version/s: 0.9.0.0 > Kafka network thread lacks top exception handler >

[jira] [Resolved] (KAFKA-1804) Kafka network thread lacks top exception handler

2015-10-13 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1804?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma resolved KAFKA-1804. Resolution: Fixed Resolving this as it has been fixed. > Kafka network thread lacks top exception

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

2015-10-13 Thread Randall Hauch
The new streams API added with KIP-28 is great. I’ve been using it on a prototype for a few weeks, and I’m looking forward to it being included in 0.9.0. However, at the moment, a Processor implementation is not able to specify the partition number when it outputs messages. I’d be happy to log a

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

2015-10-13 Thread Aditya Auradkar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2536?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14955375#comment-14955375 ] Aditya Auradkar commented on KAFKA-2536: [~gwenshap] - Thanks for reporting. Do we plan to keep

[GitHub] kafka pull request: KAFKA-2527; System Test for Quotas in Ducktape

2015-10-13 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/275 --- 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-658) Implement "Exact Mirroring" functionality in mirror maker

2015-10-13 Thread Anil Sadineni (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-658?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14955656#comment-14955656 ] Anil Sadineni commented on KAFKA-658: - +1, I too have exact same use case > Implement "Exact

[jira] [Commented] (KAFKA-2527) System Test for Quotas in Ducktape

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

[jira] [Assigned] (KAFKA-2644) Run relevant ducktape tests with SASL_PLAIN and SASL_SSL

2015-10-13 Thread Rajini Sivaram (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2644?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Rajini Sivaram reassigned KAFKA-2644: - Assignee: Rajini Sivaram > Run relevant ducktape tests with SASL_PLAIN and SASL_SSL >

[jira] [Commented] (KAFKA-2527) System Test for Quotas in Ducktape

2015-10-13 Thread Aditya Auradkar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2527?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14955720#comment-14955720 ] Aditya Auradkar commented on KAFKA-2527: [~gwenshap] - Thanks! > System Test for Quotas in

[GitHub] kafka pull request: MINOR: flush record collector after local stat...

2015-10-13 Thread ymatsuda
GitHub user ymatsuda opened a pull request: https://github.com/apache/kafka/pull/304 MINOR: flush record collector after local state flush @guozhangwang Fix the order of flushing. Undoing the change I did sometime ago. You can merge this pull request into a Git repository by

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

2015-10-13 Thread Apache Jenkins Server
See Changes: [cshapi] KAFKA-2527; System Test for Quotas in Ducktape -- Started by an SCM change [EnvInject] - Loading node environment variables. Building remotely on ubuntu-5 (docker Ubuntu

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

2015-10-13 Thread Onur Karaman (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2397?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14955673#comment-14955673 ] Onur Karaman commented on KAFKA-2397: - My pull request had diverged again from trunk, so I force

[jira] [Commented] (KAFKA-658) Implement "Exact Mirroring" functionality in mirror maker

2015-10-13 Thread James Cheng (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-658?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14955679#comment-14955679 ] James Cheng commented on KAFKA-658: --- I would also like this, so that consumers can transition from one

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

2015-10-13 Thread Allen Wang
We discussed the KIP in the hangout today. The recommendation is to make rack as a broker property in ZooKeeper. For users with existing rack information stored somewhere, they would need to retrieve the information at broker start up and dynamically set the rack property, which can be implemented

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

2015-10-13 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2017?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14956001#comment-14956001 ] Guozhang Wang commented on KAFKA-2017: -- [~hachikuji] [~onurkaraman] [~junrao] With the new protocol,

Re: [DISCUSS] KIP-32 - Add CreateTime and LogAppendTime to Kafka message

2015-10-13 Thread Guozhang Wang
Just to complete Jay's option, here is my understanding: 1. For log retention: if we want to remove data before time t, we look into the index file of each segment and find the largest timestamp t' < t, find the corresponding timestamp and start scanning to the end of the segment, if there is no

[GitHub] kafka pull request: MINOR: flush record collector after local stat...

2015-10-13 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/304 --- 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-2536) topics tool should allow users to alter topic configuration

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

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

2015-10-13 Thread granthenke
GitHub user granthenke opened a pull request: https://github.com/apache/kafka/pull/305 KAFKA-2536: topics tool should allow users to alter topic configuration This is a minimal revert of some backward incompatible changes made in KAFKA-2205, with the addition of the deprecation

[GitHub] kafka pull request: MINOR: ignore wakeups when committing offsets ...

2015-10-13 Thread hachikuji
GitHub user hachikuji opened a pull request: https://github.com/apache/kafka/pull/306 MINOR: ignore wakeups when committing offsets on consumer close You can merge this pull request into a Git repository by running: $ git pull https://github.com/hachikuji/kafka

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

2015-10-13 Thread Grant Henke (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2536?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Grant Henke updated KAFKA-2536: --- Fix Version/s: 0.9.0.0 > topics tool should allow users to alter topic configuration >

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

2015-10-13 Thread Grant Henke (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2536?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Grant Henke updated KAFKA-2536: --- Status: Patch Available (was: Open) > topics tool should allow users to alter topic configuration >

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

2015-10-13 Thread Grant Henke (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2536?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Grant Henke updated KAFKA-2536: --- Component/s: tools > topics tool should allow users to alter topic configuration >

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

2015-10-13 Thread Apache Jenkins Server
See Changes: [wangguoz] MINOR: flush record collector after local state flush -- Started by an SCM change [EnvInject] - Loading node environment variables. Building remotely on ubuntu-6 (docker

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

2015-10-13 Thread Apache Jenkins Server
See Changes: [wangguoz] MINOR: flush record collector after local state flush -- [...truncated 6015 lines...] org.apache.kafka.copycat.file.FileStreamSourceConnectorTest > testSourceTasksStdin

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

2015-10-13 Thread Apache Jenkins Server
See Changes: [cshapi] KAFKA-2527; System Test for Quotas in Ducktape -- [...truncated 6337 lines...] org.apache.kafka.copycat.file.FileStreamSourceConnectorTest > testSourceTasksStdin PASSED

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

2015-10-13 Thread Guozhang Wang
Hi Randall, You can try to set the partitioner class as ProducerConfig.PARTITIONER_CLASS_CONFIG in the StreamsConfig, its interface can be found in org.apache.kafka.clients.producer.Partitioner Let me know if it works for you. Guozhang On Tue, Oct 13, 2015 at 10:59 AM, Randall Hauch

[jira] [Commented] (KAFKA-2484) Add schema projection utilities

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

[GitHub] kafka pull request: KAFKA-2484: Add schema projection utilities

2015-10-13 Thread Ishiihara
GitHub user Ishiihara opened a pull request: https://github.com/apache/kafka/pull/307 KAFKA-2484: Add schema projection utilities This PR adds schema projection utilities to copycat. You can merge this pull request into a Git repository by running: $ git pull

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

2015-10-13 Thread Randall Hauch
This overrides the partitioning logic for all topics, right? That means I have to explicitly call the default partitioning logic for all topics except those that my Producer forwards. I’m guess the best way to do by extending org.apache.kafka.clients.producer.DefaultProducer. Of course, with

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

2015-10-13 Thread Joel Koshy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2017?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14956108#comment-14956108 ] Joel Koshy commented on KAFKA-2017: --- In offline threads we were going to discuss the options of

[GitHub] kafka pull request: KAFKA-2632: move fetchable check ahead in hand...

2015-10-13 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/295 --- 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-2632) Move fetchable check from fetchedRecords to fetch response handler

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

  1   2   >