[GitHub] kafka pull request: MINOR: MetadataCache brokerId is not set on fi...

2016-04-29 Thread granthenke
GitHub user granthenke opened a pull request: https://github.com/apache/kafka/pull/1299 MINOR: MetadataCache brokerId is not set on first run with generated … …broker id This is because the id passed into the MetadataCache is the value from the config before the real

Re: [DISCUSS] KIP-55: Secure quotas for authenticated users

2016-04-29 Thread Ewen Cheslack-Postava
Rajini, I'm admittedly not very familiar with a lot of this code or implementation, so correct me if I'm making any incorrect assumptions. I've only scanned the KIP, but my main concern is the rejection of the alternative -- unifying client-id and principal quotas. In particular, doesn't this

Re: [DISCUSS] KIP-54 Sticky Partition Assignment Strategy

2016-04-29 Thread Ewen Cheslack-Postava
I think I'm unclear how we leverage the onPartitionsRevoked/onPartitionsAssigned here in any way that's different from our normal usage -- certainly you can use them to generate a diff, but you still need to commit when partitions are revoked and that has a non-trivial cost. Are we just saying

Re: [DISCUSS] KIP-53 Add custom policies for reconnect attempts to NetworkdClient

2016-04-29 Thread Ewen Cheslack-Postava
I'll agree w/ Jay and point out that the implementations of ReconnectionPolicy provided built-in with that driver are Constant, Exponential, and Counting. Constant and exponential can be combined with the right set of policy config parameters. I'm curious if there's a real need for something else,

Re: [VOTE] KIP-45: Standardize KafkaConsumer API to use Collection

2016-04-29 Thread Jason Gustafson
Hey Harsha, Just to clarify, are you ok with removing the methods in a later release (say 0.11)? As I mentioned above, the only weird ones are subscribe() and assign(), which will have a deprecated version which accepts List. Users will have to change their code to use another collection type or

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

2016-04-29 Thread Apache Jenkins Server
See Changes: [cshapi] KAFKA-3644; Use Boolean protocol type for StopReplicaRequest delete_p… -- [...truncated 1684 lines...] kafka.log.OffsetIndexTest > appendTooMany PASSED

Re: KIP-57: Interoperable LZ4 Framing

2016-04-29 Thread Ewen Cheslack-Postava
Two questions: 1. My understanding based on KIP-35 is that this won't be a problem for clients that want to support older broker versions since they will use v0 produce requests with broken checksum to send to those, and any broker advertising support for v1 produce requests will also support

[GitHub] kafka pull request: MINOR: Add version check on enable-systest-eve...

2016-04-29 Thread granders
GitHub user granders opened a pull request: https://github.com/apache/kafka/pull/1298 MINOR: Add version check on enable-systest-events flag Recent patch adding enable-systest-events flag without any version check breaks all uses of versioned console consumer. E.g. upgrade tests,

[jira] [Updated] (KAFKA-3644) Use Boolean protocol type for StopReplicaRequest delete_partitions

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

[GitHub] kafka pull request: KAFKA-3644: Use Boolean protocol type for Stop...

2016-04-29 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/1296 --- 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-3644) Use Boolean protocol type for StopReplicaRequest delete_partitions

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

[VOTE] 0.10.0.0 RC2

2016-04-29 Thread Gwen Shapira
Hello Kafka users, developers and client-developers, This is the first candidate for release of Apache Kafka 0.10.0.0. This is a major release that includes: (1) New message format including timestamps (2) client interceptor API (3) Kafka Streams. (4) Configurable SASL authentication mechanisms

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

2016-04-29 Thread Apache Jenkins Server
See

Re: [VOTE] KIP-45: Standardize KafkaConsumer API to use Collection

2016-04-29 Thread Harsha
Grant, I am sure this is discussed and voted. I've seen the discussion. Given that there is an opportunity to make it less painful for the users who shipped consumers using the 0.9.x we should consider that. ". However, for now the documentation of > > the

[jira] [Commented] (KAFKA-3581) Use timeout when joining threads in system test services

2016-04-29 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3581?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15264925#comment-15264925 ] ASF GitHub Bot commented on KAFKA-3581: --- GitHub user granders opened a pull request:

[GitHub] kafka pull request: KAFKA-3581: add timeouts to joins in backgroun...

2016-04-29 Thread granders
GitHub user granders opened a pull request: https://github.com/apache/kafka/pull/1297 KAFKA-3581: add timeouts to joins in background thread services This actually removes joins altogether, as well as references to self.worker_threads, which is best left as an implementation detail

Re: [DISCUSS] mbeans overwritten with identical clients on a single jvm

2016-04-29 Thread Joel Koshy
Currently, the only available mechanism to disambiguate between producer/consumer clients in the same JVM is the client-id. However, that does not play very well with the current definition of client-id and its use as the entity for quota enforcement. i.e., an application can thwart quota

[GitHub] kafka pull request: MINOR: change initial value of Min stat to Dou...

2016-04-29 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/1143 --- 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] [Updated] (KAFKA-3549) Close consumers instantiated in consumer tests

2016-04-29 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3549?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gwen Shapira updated KAFKA-3549: Fix Version/s: 0.10.0.0 > Close consumers instantiated in consumer tests >

[jira] [Updated] (KAFKA-3439) Document possible exception thrown in public APIs

2016-04-29 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3439?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gwen Shapira updated KAFKA-3439: Fix Version/s: 0.10.0.0 > Document possible exception thrown in public APIs >

[jira] [Updated] (KAFKA-3618) Handle ApiVersionRequest before SASL handshake

2016-04-29 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3618?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gwen Shapira updated KAFKA-3618: Fix Version/s: 0.10.0.0 > Handle ApiVersionRequest before SASL handshake >

[jira] [Updated] (KAFKA-3490) Multiple version support for ducktape performance tests

2016-04-29 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3490?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gwen Shapira updated KAFKA-3490: Fix Version/s: 0.10.0.0 > Multiple version support for ducktape performance tests >

[jira] [Updated] (KAFKA-2693) Run relevant ducktape tests with SASL/PLAIN and multiple mechanisms

2016-04-29 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2693?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gwen Shapira updated KAFKA-2693: Fix Version/s: 0.10.0.0 > Run relevant ducktape tests with SASL/PLAIN and multiple mechanisms >

[jira] [Updated] (KAFKA-3526) REST APIs return object representation instead of string for config values, default values and recommended values

2016-04-29 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3526?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gwen Shapira updated KAFKA-3526: Fix Version/s: 0.10.0.0 > REST APIs return object representation instead of string for config

[jira] [Updated] (KAFKA-3382) Add system test for ReplicationVerificationTool

2016-04-29 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3382?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gwen Shapira updated KAFKA-3382: Fix Version/s: 0.10.0.0 > Add system test for ReplicationVerificationTool >

[jira] [Updated] (KAFKA-2479) Add CopycatExceptions to indicate transient and permanent errors in a connector/task

2016-04-29 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2479?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gwen Shapira updated KAFKA-2479: Fix Version/s: 0.10.0.0 > Add CopycatExceptions to indicate transient and permanent errors in a >

[jira] [Updated] (KAFKA-3597) Enable query ConsoleConsumer and VerifiableProducer if they shutdown cleanly

2016-04-29 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3597?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gwen Shapira updated KAFKA-3597: Fix Version/s: 0.10.0.0 > Enable query ConsoleConsumer and VerifiableProducer if they shutdown

[jira] [Updated] (KAFKA-2658) Implement SASL/PLAIN

2016-04-29 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2658?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gwen Shapira updated KAFKA-2658: Fix Version/s: 0.10.0.0 > Implement SASL/PLAIN > > > Key:

[jira] [Updated] (KAFKA-3529) Transient kafka.api.PlaintextConsumerTest.testAsyncCommit failures

2016-04-29 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3529?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gwen Shapira updated KAFKA-3529: Fix Version/s: 0.10.0.0 > Transient kafka.api.PlaintextConsumerTest.testAsyncCommit failures >

[jira] [Updated] (KAFKA-3459) Returning zero task configurations from a connector does not properly clean up existing tasks

2016-04-29 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3459?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gwen Shapira updated KAFKA-3459: Fix Version/s: 0.10.0.0 > Returning zero task configurations from a connector does not properly

[jira] [Updated] (KAFKA-3615) Exclude test jars in CLASSPATH of kafka-run-class.sh

2016-04-29 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3615?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gwen Shapira updated KAFKA-3615: Fix Version/s: 0.10.0.0 > Exclude test jars in CLASSPATH of kafka-run-class.sh >

[jira] [Updated] (KAFKA-2547) Make DynamicConfigManager to use the ZkNodeChangeNotificationListener introduced as part of KAFKA-2211

2016-04-29 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2547?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gwen Shapira updated KAFKA-2547: Fix Version/s: 0.10.0.0 > Make DynamicConfigManager to use the ZkNodeChangeNotificationListener >

[jira] [Updated] (KAFKA-3307) Add ApiVersion request/response and server side handling.

2016-04-29 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3307?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gwen Shapira updated KAFKA-3307: Fix Version/s: 0.10.0.0 > Add ApiVersion request/response and server side handling. >

[jira] [Updated] (KAFKA-2800) Update outdated dependencies

2016-04-29 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2800?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gwen Shapira updated KAFKA-2800: Fix Version/s: 0.10.0.0 > Update outdated dependencies > > >

[jira] [Updated] (KAFKA-3569) commitAsync() sometimes fails with errors

2016-04-29 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3569?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gwen Shapira updated KAFKA-3569: Fix Version/s: 0.10.0.0 > commitAsync() sometimes fails with errors >

[jira] [Updated] (KAFKA-3548) Locale is not handled properly in kafka-consumer

2016-04-29 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3548?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gwen Shapira updated KAFKA-3548: Fix Version/s: 0.10.0.0 > Locale is not handled properly in kafka-consumer >

[jira] [Updated] (KAFKA-3418) Add section on detecting consumer failures in new consumer javadoc

2016-04-29 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3418?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gwen Shapira updated KAFKA-3418: Fix Version/s: 0.10.0.0 > Add section on detecting consumer failures in new consumer javadoc >

[jira] [Updated] (KAFKA-3461) Fix typos in Kafka web documentations

2016-04-29 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3461?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gwen Shapira updated KAFKA-3461: Fix Version/s: 0.10.0.0 > Fix typos in Kafka web documentations >

[jira] [Updated] (KAFKA-3611) Remove WARNs when using reflections

2016-04-29 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3611?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gwen Shapira updated KAFKA-3611: Fix Version/s: 0.10.0.0 > Remove WARNs when using reflections >

[jira] [Updated] (KAFKA-3558) Add compression_type parameter to benchmarks in benchmark_test.py

2016-04-29 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3558?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gwen Shapira updated KAFKA-3558: Fix Version/s: 0.10.0.0 > Add compression_type parameter to benchmarks in benchmark_test.py >

[jira] [Updated] (KAFKA-3563) Maintain MessageAndMetadata constructor compatibility

2016-04-29 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3563?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gwen Shapira updated KAFKA-3563: Fix Version/s: 0.10.0.0 > Maintain MessageAndMetadata constructor compatibility >

[jira] [Updated] (KAFKA-3358) Only request metadata updates once we have topics or a pattern subscription

2016-04-29 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3358?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gwen Shapira updated KAFKA-3358: Fix Version/s: 0.10.0.0 > Only request metadata updates once we have topics or a pattern

[jira] [Updated] (KAFKA-3306) Change metadata response to include required additional fields

2016-04-29 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3306?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gwen Shapira updated KAFKA-3306: Fix Version/s: 0.10.0.0 > Change metadata response to include required additional fields >

[jira] [Updated] (KAFKA-3506) Kafka Connect Task Restart API

2016-04-29 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3506?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gwen Shapira updated KAFKA-3506: Fix Version/s: 0.10.0.0 > Kafka Connect Task Restart API > -- > >

[jira] [Updated] (KAFKA-2370) Add pause/unpause connector support

2016-04-29 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2370?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gwen Shapira updated KAFKA-2370: Fix Version/s: 0.10.0.0 > Add pause/unpause connector support > ---

[jira] [Updated] (KAFKA-1236) Change producer performance tool to optionally use the new producer

2016-04-29 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1236?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gwen Shapira updated KAFKA-1236: Fix Version/s: 0.10.0.0 > Change producer performance tool to optionally use the new producer >

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

2016-04-29 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1554?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gwen Shapira updated KAFKA-1554: Fix Version/s: 0.10.0.0 > Corrupt index found on clean startup >

[jira] [Updated] (KAFKA-665) Outgoing responses delayed on a busy Kafka broker

2016-04-29 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-665?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gwen Shapira updated KAFKA-665: --- Fix Version/s: 0.10.0.0 > Outgoing responses delayed on a busy Kafka broker >

[jira] [Updated] (KAFKA-3605) Connector REST endpoint allows incorrectly overriding the connector name

2016-04-29 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3605?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gwen Shapira updated KAFKA-3605: Fix Version/s: 0.10.0.0 > Connector REST endpoint allows incorrectly overriding the connector name

[jira] [Updated] (KAFKA-3421) Update docs with new connector features

2016-04-29 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3421?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gwen Shapira updated KAFKA-3421: Fix Version/s: 0.10.0.0 > Update docs with new connector features >

[jira] [Updated] (KAFKA-3641) Fix RecordMetadata constructor backward compatibility

2016-04-29 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3641?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gwen Shapira updated KAFKA-3641: Fix Version/s: 0.10.0.0 > Fix RecordMetadata constructor backward compatibility >

[jira] [Updated] (KAFKA-3578) Allow cross origin HTTP requests on all HTTP methods

2016-04-29 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3578?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gwen Shapira updated KAFKA-3578: Fix Version/s: 0.10.0.0 > Allow cross origin HTTP requests on all HTTP methods >

[jira] [Updated] (KAFKA-3644) Use Boolean protocol type for StopReplicaRequest delete_partitions

2016-04-29 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3644?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gwen Shapira updated KAFKA-3644: Fix Version/s: (was: 0.10.0.0) 0.10.1.0 > Use Boolean protocol type for

[jira] [Updated] (KAFKA-3575) Use console consumer access topic that does not exist, can not use "Control + C" to exit process

2016-04-29 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3575?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gwen Shapira updated KAFKA-3575: Fix Version/s: (was: 0.10.0.0) 0.10.1.0 > Use console consumer access topic

[jira] [Updated] (KAFKA-3634) Add ducktape tests for upgrade with SASL

2016-04-29 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3634?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gwen Shapira updated KAFKA-3634: Fix Version/s: (was: 0.10.0.0) 0.10.1.0 > Add ducktape tests for upgrade

[jira] [Updated] (KAFKA-3600) Enhance java clients to use ApiVersion Req/Resp to check if the broker they are talking to supports required api versions

2016-04-29 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3600?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gwen Shapira updated KAFKA-3600: Fix Version/s: (was: 0.10.0.0) 0.10.1.0 > Enhance java clients to use

[jira] [Updated] (KAFKA-3128) Add metrics for ZooKeeper events

2016-04-29 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3128?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gwen Shapira updated KAFKA-3128: Fix Version/s: (was: 0.10.0.0) 0.10.1.0 > Add metrics for ZooKeeper events >

[jira] [Updated] (KAFKA-3042) updateIsr should stop after failed several times due to zkVersion issue

2016-04-29 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3042?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gwen Shapira updated KAFKA-3042: Fix Version/s: (was: 0.10.0.0) 0.10.1.0 > updateIsr should stop after failed

[jira] [Updated] (KAFKA-3186) KIP-50: Move Authorizer and related classes to separate package.

2016-04-29 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3186?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gwen Shapira updated KAFKA-3186: Fix Version/s: (was: 0.10.0.0) 0.10.1.0 > KIP-50: Move Authorizer and

[jira] [Updated] (KAFKA-3592) System tests - don't hardcode paths to scripts

2016-04-29 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3592?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gwen Shapira updated KAFKA-3592: Fix Version/s: (was: 0.10.0.0) 0.10.1.0 > System tests - don't hardcode

[jira] [Updated] (KAFKA-3517) Document configuration of SASL/PLAIN and multiple mechanisms

2016-04-29 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3517?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gwen Shapira updated KAFKA-3517: Fix Version/s: (was: 0.10.0.0) 0.10.1.0 > Document configuration of

[jira] [Updated] (KAFKA-3520) System tests of config validate and list connectors REST APIs

2016-04-29 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3520?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gwen Shapira updated KAFKA-3520: Fix Version/s: (was: 0.10.0.0) 0.10.1.0 > System tests of config validate

[jira] [Updated] (KAFKA-2955) Add Prompt to kafka-console-producer

2016-04-29 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2955?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gwen Shapira updated KAFKA-2955: Fix Version/s: (was: 0.10.0.0) 0.10.1.0 > Add Prompt to

[jira] [Updated] (KAFKA-3500) KafkaOffsetBackingStore set method needs to handle null

2016-04-29 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3500?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gwen Shapira updated KAFKA-3500: Fix Version/s: (was: 0.10.0.0) 0.10.1.0 > KafkaOffsetBackingStore set method

[jira] [Updated] (KAFKA-3163) KIP-33 - Add a time based log index to Kafka

2016-04-29 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3163?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gwen Shapira updated KAFKA-3163: Fix Version/s: (was: 0.10.0.0) 0.10.1.0 > KIP-33 - Add a time based log

[jira] [Commented] (KAFKA-3173) Error while moving some partitions to OnlinePartition state

2016-04-29 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3173?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15264904#comment-15264904 ] Gwen Shapira commented on KAFKA-3173: - This is marked as critical. [~fpj] and [~ijuma] - are you

[jira] [Commented] (KAFKA-3565) Producer's throughput lower with compressed data after KIP-31/32

2016-04-29 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3565?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15264905#comment-15264905 ] Gwen Shapira commented on KAFKA-3565: - This is marked as critical. [~becket_qin] and [~ijuma] - are

[jira] [Commented] (KAFKA-3644) Use Boolean protocol type for StopReplicaRequest delete_partitions

2016-04-29 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3644?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15264892#comment-15264892 ] ASF GitHub Bot commented on KAFKA-3644: --- GitHub user granthenke opened a pull request:

[GitHub] kafka pull request: KAFKA-3644: Use Boolean protocol type for Stop...

2016-04-29 Thread granthenke
GitHub user granthenke opened a pull request: https://github.com/apache/kafka/pull/1296 KAFKA-3644: Use Boolean protocol type for StopReplicaRequest delete_p… …artitions You can merge this pull request into a Git repository by running: $ git pull

[jira] [Updated] (KAFKA-3644) Use Boolean protocol type for StopReplicaRequest delete_partitions

2016-04-29 Thread Grant Henke (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3644?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Grant Henke updated KAFKA-3644: --- Status: Patch Available (was: Open) > Use Boolean protocol type for StopReplicaRequest

[jira] [Created] (KAFKA-3644) Use Boolean protocol type for StopReplicaRequest delete_partitions

2016-04-29 Thread Grant Henke (JIRA)
Grant Henke created KAFKA-3644: -- Summary: Use Boolean protocol type for StopReplicaRequest delete_partitions Key: KAFKA-3644 URL: https://issues.apache.org/jira/browse/KAFKA-3644 Project: Kafka

[jira] [Commented] (KAFKA-3627) New consumer doesn't run delayed tasks while under load

2016-04-29 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3627?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15264870#comment-15264870 ] ASF GitHub Bot commented on KAFKA-3627: --- GitHub user hachikuji opened a pull request:

[GitHub] kafka pull request: KAFKA-3627: consumer fails to execute delayed ...

2016-04-29 Thread hachikuji
GitHub user hachikuji opened a pull request: https://github.com/apache/kafka/pull/1295 KAFKA-3627: consumer fails to execute delayed tasks in poll when records are available You can merge this pull request into a Git repository by running: $ git pull

[GitHub] kafka pull request: HOTFIX: Fix equality semantics of KeyValue

2016-04-29 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/1294 --- 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

[RELEASE] Final merge of trunk into 0.10.0

2016-04-29 Thread Gwen Shapira
Hi, I just merged trunk into 0.10.0 branch and pushed. 0.10.0 is updated as of commit d0dedc6 (KAFKA-3459: Returning zero task configurations from a connector does not properly clean up existing tasks). Committers: Please cherry-pick only critical bug fixes and/or low-risk changes (preferably

[jira] [Commented] (KAFKA-3459) Returning zero task configurations from a connector does not properly clean up existing tasks

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

[jira] [Updated] (KAFKA-3459) Returning zero task configurations from a connector does not properly clean up existing tasks

2016-04-29 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3459?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ewen Cheslack-Postava updated KAFKA-3459: - Resolution: Fixed Fix Version/s: 0.10.1.0 Status: Resolved

[GitHub] kafka pull request: KAFKA-3459: Returning zero task configurations...

2016-04-29 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/1248 --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is

Re: [DISCUSS] mbeans overwritten with identical clients on a single jvm

2016-04-29 Thread Jay Kreps
The definition for client id has always been "a logical name for an application which (potentially) spans more than one process". >From my point of view the rationalization that is most needed is client id with "user" for the authenticated cases. There not quite the same but they're similar. I

[GitHub] kafka pull request: HOTFIX: Fix equality semantics of KeyValue

2016-04-29 Thread miguno
GitHub user miguno opened a pull request: https://github.com/apache/kafka/pull/1294 HOTFIX: Fix equality semantics of KeyValue Fixes wrong KeyValue equals logic when keys not equal but values equal. Original hotfix PR at https://github.com/apache/kafka/pull/1293 (/cc

[jira] [Commented] (KAFKA-3641) Fix RecordMetadata constructor backward compatibility

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

[GitHub] kafka pull request: KAFKA-3641: Fix RecordMetadata constructor bac...

2016-04-29 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/1292 --- 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] [Updated] (KAFKA-3641) Fix RecordMetadata constructor backward compatibility

2016-04-29 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3641?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gwen Shapira updated KAFKA-3641: Resolution: Fixed Fix Version/s: (was: 0.10.0.0) 0.10.1.0

[jira] [Commented] (KAFKA-3643) Data Duplication on clean restart of Kafka Broker

2016-04-29 Thread Arun Mathew (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3643?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15264820#comment-15264820 ] Arun Mathew commented on KAFKA-3643: Further Details of the Issue. When an event is received by the

[jira] [Commented] (KAFKA-3128) Add metrics for ZooKeeper events

2016-04-29 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3128?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15264750#comment-15264750 ] Ismael Juma commented on KAFKA-3128: Thanks [~junrao], I'll update the PR to use `KafkaMetricsGroup`.

[jira] [Commented] (KAFKA-3128) Add metrics for ZooKeeper events

2016-04-29 Thread James Cheng (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3128?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15264741#comment-15264741 ] James Cheng commented on KAFKA-3128: +1. These metrics would be super useful. I know that there's an

[jira] [Closed] (KAFKA-3611) Remove WARNs when using reflections

2016-04-29 Thread Liquan Pei (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3611?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Liquan Pei closed KAFKA-3611. - > Remove WARNs when using reflections > > > Key:

[jira] [Closed] (KAFKA-3578) Allow cross origin HTTP requests on all HTTP methods

2016-04-29 Thread Liquan Pei (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3578?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Liquan Pei closed KAFKA-3578. - > Allow cross origin HTTP requests on all HTTP methods >

[jira] [Closed] (KAFKA-3606) Traverse CLASSPATH during herder start to list connectors

2016-04-29 Thread Liquan Pei (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3606?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Liquan Pei closed KAFKA-3606. - > Traverse CLASSPATH during herder start to list connectors >

[jira] [Closed] (KAFKA-3582) remove references to Copcyat from connect property files

2016-04-29 Thread Liquan Pei (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3582?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Liquan Pei closed KAFKA-3582. - > remove references to Copcyat from connect property files >

[jira] [Closed] (KAFKA-3615) Exclude test jars in CLASSPATH of kafka-run-class.sh

2016-04-29 Thread Liquan Pei (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3615?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Liquan Pei closed KAFKA-3615. - > Exclude test jars in CLASSPATH of kafka-run-class.sh >

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

2016-04-29 Thread Apache Jenkins Server
See Changes: [me] KAFKA-3440: Update streams javadocs -- [...truncated 1641 lines...] kafka.log.LogTest > testLogRolls PASSED kafka.log.LogTest > testMessageSizeCheck PASSED kafka.log.LogTest

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

2016-04-29 Thread Apache Jenkins Server
See

[GitHub] kafka pull request: KAFKA-3641: Fix RecordMetadata constructor bac...

2016-04-29 Thread granthenke
GitHub user granthenke reopened a pull request: https://github.com/apache/kafka/pull/1292 KAFKA-3641: Fix RecordMetadata constructor backward compatibility You can merge this pull request into a Git repository by running: $ git pull https://github.com/granthenke/kafka

[jira] [Commented] (KAFKA-3641) Fix RecordMetadata constructor backward compatibility

2016-04-29 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3641?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15264649#comment-15264649 ] ASF GitHub Bot commented on KAFKA-3641: --- GitHub user granthenke reopened a pull request:

[jira] [Commented] (KAFKA-3209) Support single message transforms in Kafka Connect

2016-04-29 Thread Nisarg Shah (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3209?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15264644#comment-15264644 ] Nisarg Shah commented on KAFKA-3209: That does sound good. I realise it is not insanely complicated,

[jira] [Commented] (KAFKA-3641) Fix RecordMetadata constructor backward compatibility

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

[GitHub] kafka pull request: KAFKA-3641: Fix RecordMetadata constructor bac...

2016-04-29 Thread granthenke
Github user granthenke closed the pull request at: https://github.com/apache/kafka/pull/1292 --- 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-3209) Support single message transforms in Kafka Connect

2016-04-29 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3209?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15264633#comment-15264633 ] Ewen Cheslack-Postava commented on KAFKA-3209: -- [~snisarg] I haven't started work in earnest.

[jira] [Updated] (KAFKA-3440) Add Javadoc for KTable (changelog stream) and KStream (record stream)

2016-04-29 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3440?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ewen Cheslack-Postava updated KAFKA-3440: - Resolution: Fixed Reviewer: Ewen Cheslack-Postava Status: Resolved

  1   2   >