Re: [DISCUSS] KIP-35 - Retrieve protocol version

2016-04-05 Thread Magnus Edenhill
Empty arrays are already used in MetadataRequest to retrieve all topics in the cluster, ApiVersion request will have the same standard semantics. 2016-04-05 19:01 GMT+02:00 Ewen Cheslack-Postava : > Also, just a thought but is empty list the sentinel we really want to >

Re: [DISCUSS] KIP-35 - Retrieve protocol version

2016-04-05 Thread Magnus Edenhill
Hey, people have had concerns about the complexity of the client of mapping API versions to features, so I implemented this in librdkafka and it is rather straight forward. See here: https://github.com/edenhill/librdkafka/blob/KIP-35/src/rdkafka_feature.c#L52 Consider it a proof of concept at

Re: [VOTE] KIP-50 - Enhance Authorizer interface to be aware of supported Principal Types

2016-04-05 Thread Ashish Singh
Jun, KIP-50 is now updated. Mind taking a look. On Mon, Apr 4, 2016 at 10:40 PM, Ashish Singh wrote: > Jun, > > Your suggested approach works, will update the KIP and re-initiate voting. > Thanks! > > On Sun, Apr 3, 2016 at 8:37 AM, Jun Rao wrote: > >>

Re: [DISCUSS] KIP-35 - Retrieve protocol version

2016-04-05 Thread Ismael Juma
Yeah, we should use nullable arrays (which have been introduced in KIP-4-Metadata) instead of empty list to indicate all versions. Ismael On 5 Apr 2016 18:01, "Ewen Cheslack-Postava" wrote: > Also, just a thought but is empty list the sentinel we really want to > indicate we

Re: [DISCUSS] KIP-35 - Retrieve protocol version

2016-04-05 Thread Ashish Singh
Magnus, it is proposed to be changed in version 1, https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations#KIP-4-Commandlineandcentralizedadministrativeoperations-MetadataSchema . On Tue, Apr 5, 2016 at 10:23 AM, Magnus Edenhill

Re: [DISCUSS] KIP-35 - Retrieve protocol version

2016-04-05 Thread Ashish Singh
Null array sounds good to me as well. On Tue, Apr 5, 2016 at 10:06 AM, Ismael Juma wrote: > Yeah, we should use nullable arrays (which have been introduced in > KIP-4-Metadata) instead of empty list to indicate all versions. > > Ismael > On 5 Apr 2016 18:01, "Ewen

Re: [DISCUSS] KIP-35 - Retrieve protocol version

2016-04-05 Thread Magnus Edenhill
Ashish, thanks, didnt know that. For ApiVersionRequest requesting no Apis to be returned doesnt make sense so the distinction isn't necessary, but I'm fine with adding Null to be more in line with future protocol requests, as long as it doesn't delay this KIP any longer! :) 2016-04-05 19:38

Re: [DISCUSS] KIP-35 - Retrieve protocol version

2016-04-05 Thread Ashish Singh
Sounds fair. I am OK with putting down, permanent support of ApiVersion api versions, as a limitation in KIP. On Tue, Apr 5, 2016 at 9:23 AM, Ewen Cheslack-Postava wrote: > On Mon, Apr 4, 2016 at 11:24 AM, Gwen Shapira wrote: > > > > > > > > > >In case

Re: How will KIP-35 and KIP-43 play together?

2016-04-05 Thread Gwen Shapira
I think we pretty much agreed on KIP-35 and are just finalizing details. Given that we are merging both KIP-35 and KIP-43, I would like some direction on what this will look like. Magnus suggested adding new Request type as part of KIP-43, which will allow us to advertise the new extension to

[jira] [Updated] (KAFKA-3510) OffsetIndex thread safety

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

[GitHub] kafka pull request: KAFKA-3510; OffsetIndex thread safety

2016-04-05 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/1188 --- 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-3512) Add a foreach() operator in Kafka Streams DSL

2016-04-05 Thread Guozhang Wang (JIRA)
Guozhang Wang created KAFKA-3512: Summary: Add a foreach() operator in Kafka Streams DSL Key: KAFKA-3512 URL: https://issues.apache.org/jira/browse/KAFKA-3512 Project: Kafka Issue Type: Bug

[jira] [Commented] (KAFKA-3510) OffsetIndex thread safety

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

Search on queue?

2016-04-05 Thread Manish Degan
Hi, I have been thinking of some high volume use cases where I want to utilize Kafka. The message objects in my queue might have a date property. I want to pull all messages till a date to perform certain actions. Currently, I can continue listening but cannot confirm if there are any more

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

2016-04-05 Thread Apache Jenkins Server
See Changes: [wangguoz] KAFKA-3510; OffsetIndex thread safety -- [...truncated 5403 lines...] kafka.log.BrokerCompressionTest > testBrokerSideCompression[3] PASSED

[jira] [Created] (KAFKA-3511) Provide built-in aggregators sum() and avg() in Kafka Streams DSL

2016-04-05 Thread Guozhang Wang (JIRA)
Guozhang Wang created KAFKA-3511: Summary: Provide built-in aggregators sum() and avg() in Kafka Streams DSL Key: KAFKA-3511 URL: https://issues.apache.org/jira/browse/KAFKA-3511 Project: Kafka

Re: [DISCUSS] KIP-35 - Retrieve protocol version

2016-04-05 Thread Ewen Cheslack-Postava
On Mon, Apr 4, 2016 at 11:24 AM, Gwen Shapira wrote: > > > > > >In case of connection closures, the KIP recommends that clients should > >use some other method of determining the apiRequest version to use, > > like, > >probing. For instance, client will send V0

[jira] [Commented] (KAFKA-3503) Throw exception on missing/non-existent partition

2016-04-05 Thread Navin Markandeya (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3503?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15226573#comment-15226573 ] Navin Markandeya commented on KAFKA-3503: - So I would expect an exception instead of

Re: [DISCUSS] KIP-35 - Retrieve protocol version

2016-04-05 Thread Ewen Cheslack-Postava
Also, just a thought but is empty list the sentinel we really want to indicate we want all API versions? We've got nullable string and nullable bytes in the protocol. Should we have nullable array support as well and use that to indicate we want everything? I can't think of a use case for sending

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

2016-04-05 Thread Apache Jenkins Server
See Changes: [wangguoz] KAFKA-3510; OffsetIndex thread safety -- [...truncated lines...] org.apache.kafka.streams.kstream.internals.KTableKTableOuterJoinTest > testNotSendingOldValue

[jira] [Created] (KAFKA-3513) Transient failure of OffsetValidationTest

2016-04-05 Thread Ewen Cheslack-Postava (JIRA)
Ewen Cheslack-Postava created KAFKA-3513: Summary: Transient failure of OffsetValidationTest Key: KAFKA-3513 URL: https://issues.apache.org/jira/browse/KAFKA-3513 Project: Kafka

[jira] [Commented] (KAFKA-3511) Provide built-in aggregators sum() and avg() in Kafka Streams DSL

2016-04-05 Thread Jay Kreps (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3511?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15227096#comment-15227096 ] Jay Kreps commented on KAFKA-3511: -- The issue is that there are dozens of common aggregation functions,

Re: [DISCUSS] KIP-35 - Retrieve protocol version

2016-04-05 Thread Ashish Singh
Ismael, thanks for the review. On Fri, Apr 1, 2016 at 1:22 AM, Ismael Juma wrote: > A couple of questions: > > 1. The KIP says "Specific version may be deprecated through protocol > documentation but must still be supported (although it is fair to return an > error code if

Re: [DISCUSS] KIP-35 - Retrieve protocol version

2016-04-05 Thread Ashish Singh
On Fri, Apr 1, 2016 at 1:32 AM, Ismael Juma wrote: > Two more things: > > 3. We talk about backporting of new request versions to stable branches in > the KIP. In practice, we can't do that until the Java client is changed so > that it doesn't blindly use the latest protocol

Re: [VOTE] KIP-4 Metadata Schema

2016-04-05 Thread Grant Henke
After the discussion today about the clarity and flexibility of the flag/lists for internal topics and deleted topics, I think I will switch back to using booleans inside the topic metadata. This is a clearer representation of the intent, should not have too much overhead (especially because users

Re: [VOTE] KIP-4 Metadata Schema

2016-04-05 Thread Grant Henke
Hi Jun, See my responses below: 2. The issues that I was thinking are the following. (a) Say the controller > has topic deletion disabled and a topic deletion request is submitted to > ZK. In this case, the controller will ignore this request. However, the > broker may pick up the topic deletion

[GitHub] kafka pull request: KAFKA-3506: Kafka Connect restart APIs

2016-04-05 Thread hachikuji
GitHub user hachikuji opened a pull request: https://github.com/apache/kafka/pull/1189 KAFKA-3506: Kafka Connect restart APIs You can merge this pull request into a Git repository by running: $ git pull https://github.com/hachikuji/kafka KAFKA-3506 Alternatively you can

[jira] [Updated] (KAFKA-3504) Changelog partition configured to enable log compaction

2016-04-05 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3504?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang updated KAFKA-3504: - Fix Version/s: (was: 0.10.1.0) 0.10.0.0 > Changelog partition configured

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

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

[jira] [Created] (KAFKA-3514) Stream timestamp computation needs some further thoughts

2016-04-05 Thread Guozhang Wang (JIRA)
Guozhang Wang created KAFKA-3514: Summary: Stream timestamp computation needs some further thoughts Key: KAFKA-3514 URL: https://issues.apache.org/jira/browse/KAFKA-3514 Project: Kafka Issue

[GitHub] kafka pull request: KAFKA-3508: Fix transient SimpleACLAuthorizerT...

2016-04-05 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/1156 --- 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-3184) Add Checkpoint for In-memory State Store

2016-04-05 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3184?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang updated KAFKA-3184: - Fix Version/s: 0.10.1.0 > Add Checkpoint for In-memory State Store >

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

2016-04-05 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3440?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang updated KAFKA-3440: - Fix Version/s: (was: 0.10.0.1) 0.10.0.0 > Add Javadoc for KTable

[jira] [Updated] (KAFKA-3443) Support regex topics in addSource() and stream()

2016-04-05 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3443?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang updated KAFKA-3443: - Fix Version/s: (was: 0.10.0.1) 0.10.1.0 > Support regex topics in

[jira] [Updated] (KAFKA-3497) Streams ProcessorContext should support forward() based on child name

2016-04-05 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3497?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang updated KAFKA-3497: - Fix Version/s: 0.10.0.0 > Streams ProcessorContext should support forward() based on child name >

[jira] [Updated] (KAFKA-3430) Allow users to set key in KTable.toStream() and KStream

2016-04-05 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3430?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang updated KAFKA-3430: - Fix Version/s: (was: 0.10.0.1) 0.10.0.0 > Allow users to set key in

[GitHub] kafka pull request: KAFKA-3489; Update request metrics if a client...

2016-04-05 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/1172 --- 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-3489) Update request metrics if client closes connection while broker response is in flight

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

[jira] [Updated] (KAFKA-3489) Update request metrics if client closes connection while broker response is in flight

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

[GitHub] kafka pull request: KAFKA-3505: Fix punctuate generated record met...

2016-04-05 Thread guozhangwang
GitHub user guozhangwang opened a pull request: https://github.com/apache/kafka/pull/1190 KAFKA-3505: Fix punctuate generated record metadata You can merge this pull request into a Git repository by running: $ git pull https://github.com/guozhangwang/kafka K3505

[jira] [Commented] (KAFKA-3505) Set curRecord in punctuate() functions

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

[jira] [Commented] (KAFKA-3477) Add customizable StreamPartition into #to functions of Streams DSL

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

[jira] [Updated] (KAFKA-3185) Allow users to cleanup internal data

2016-04-05 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3185?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang updated KAFKA-3185: - Fix Version/s: 0.10.1.0 > Allow users to cleanup internal data >

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

2016-04-05 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3439?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang updated KAFKA-3439: - Fix Version/s: (was: 0.10.0.1) 0.10.0.0 > Document possible exception

[jira] [Updated] (KAFKA-3429) Remove Serdes needed for repartitioning in KTable stateful operations

2016-04-05 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3429?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang updated KAFKA-3429: - Fix Version/s: (was: 0.10.0.1) 0.10.1.0 > Remove Serdes needed for

[jira] [Updated] (KAFKA-3183) Add metrics for persistent store caching layer

2016-04-05 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3183?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang updated KAFKA-3183: - Fix Version/s: 0.10.1.0 > Add metrics for persistent store caching layer >

[jira] [Updated] (KAFKA-3262) Make KafkaStreams debugging friendly

2016-04-05 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3262?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang updated KAFKA-3262: - Fix Version/s: (was: 0.10.0.1) 0.10.1.0 > Make KafkaStreams debugging

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

2016-04-05 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3506?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson updated KAFKA-3506: --- Status: Patch Available (was: In Progress) > Kafka Connect Task Restart API >

[jira] [Updated] (KAFKA-3508) Transient failure in kafka.security.auth.SimpleAclAuthorizerTest.testHighConcurrencyModificationOfResourceAcls

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

[jira] [Resolved] (KAFKA-3477) Add customizable StreamPartition into #to functions of Streams DSL

2016-04-05 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3477?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang resolved KAFKA-3477. -- Resolution: Fixed Issue resolved by pull request 1180

[GitHub] kafka pull request: [KAFKA-3477] [Kafka Streams] extended KStream/...

2016-04-05 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/1180 --- 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] [Assigned] (KAFKA-3515) Migrate org.apache.kafka.connect.json.JsonSerializer / Deser to common package

2016-04-05 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3515?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang reassigned KAFKA-3515: Assignee: Guozhang Wang > Migrate org.apache.kafka.connect.json.JsonSerializer / Deser to

[VOTE] KIP-52: Kafka Connect Control APIs

2016-04-05 Thread Jason Gustafson
I'd like to open the vote on KIP-52, which adds several control APIs to Kafka Connect: https://cwiki.apache.org/confluence/display/KAFKA/KIP-52%3A+Connector+Control+APIs. Compared to some of the other active KIPs, this is a relatively small feature, but it makes administration of Connect clusters

[GitHub] kafka pull request: KAFKA-3515: migrate json serde from connect to...

2016-04-05 Thread guozhangwang
GitHub user guozhangwang opened a pull request: https://github.com/apache/kafka/pull/1191 KAFKA-3515: migrate json serde from connect to common You can merge this pull request into a Git repository by running: $ git pull https://github.com/guozhangwang/kafka K3515

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

2016-04-05 Thread Apache Jenkins Server
See Changes: [junrao] KAFKA-3489; Update request metrics if a client closes a connection while [me] KAFKA-3508: Fix transient SimpleACLAuthorizerTest failures [wangguoz] KAFKA-3477: extended KStream/KTable API to specify custom

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

2016-04-05 Thread Apache Jenkins Server
See

Rebased 0.10.0 on trunk

2016-04-05 Thread Gwen Shapira
Hi Team, In order to make sure that the eventual 0.10.0 release will include everything we are working on, we agreed to merge trunk into 0.10.0 branch on a weekly basis. I just rebased 0.10.0 on trunk and pushed: Chens-MacBook-Pro:kafka gwen$ git push apache 0.10.0 Counting objects: 266, done.

Re: [VOTE] KIP-52: Kafka Connect Control APIs

2016-04-05 Thread Gwen Shapira
+1 Super useful, thanks Jason. On Tue, Apr 5, 2016 at 4:59 PM, Jason Gustafson wrote: > I'd like to open the vote on KIP-52, which adds several control APIs to > Kafka Connect: > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-52%3A+Connector+Control+APIs > . >

Re: [VOTE] KIP-52: Kafka Connect Control APIs

2016-04-05 Thread Ewen Cheslack-Postava
+1 On Tue, Apr 5, 2016 at 5:12 PM, Gwen Shapira wrote: > +1 > > Super useful, thanks Jason. > > On Tue, Apr 5, 2016 at 4:59 PM, Jason Gustafson > wrote: > > > I'd like to open the vote on KIP-52, which adds several control APIs to > > Kafka Connect: > > >

Re: [VOTE] KIP-4 Metadata Schema

2016-04-05 Thread Jun Rao
5. You will return no error and 4,5,6 as replicas. The response also includes a list of live brokers. So the client can figure out 5 is not live directly w/o relying on the error code. Thanks, Jun On Tue, Apr 5, 2016 at 5:05 PM, Grant Henke wrote: > Hi Jun, > > See my

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

2016-04-05 Thread Apache Jenkins Server
See Changes: [wangguoz] KAFKA-3477: extended KStream/KTable API to specify custom partitioner -- [...truncated 1602 lines...] kafka.log.LogTest > testTruncateTo PASSED kafka.log.LogTest >

[GitHub] kafka pull request: MINOR: ensure original use of prop_file in ver...

2016-04-05 Thread apovzner
GitHub user apovzner opened a pull request: https://github.com/apache/kafka/pull/1192 MINOR: ensure original use of prop_file in verifiable producer This PR: https://github.com/apache/kafka/pull/958 fixed the use of prop_file in the situation when we have multiple producers

Build failed in Jenkins: kafka-0.10.0-jdk7 #33

2016-04-05 Thread Apache Jenkins Server
See Changes: [cshapi] Changing version to 0.10.1.0-SNAPSHOT [cshapi] MINOR: update new version in additional places [cshapi] KAFKA-3445: Validate TASKS_MAX_CONFIG's lower bound [cshapi] KAFKA-3407 - ErrorLoggingCallback trims

Re: [VOTE] KIP-52: Kafka Connect Control APIs

2016-04-05 Thread Jay Kreps
+1 On Tue, Apr 5, 2016 at 4:59 PM, Jason Gustafson wrote: > I'd like to open the vote on KIP-52, which adds several control APIs to > Kafka Connect: > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-52%3A+Connector+Control+APIs > . > Compared to some of the other

[jira] [Commented] (KAFKA-3493) Replica fetcher load is not balanced over fetcher threads

2016-04-05 Thread Maysam Yabandeh (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3493?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15227557#comment-15227557 ] Maysam Yabandeh commented on KAFKA-3493: [~nehanarkhede], git suggests that you have the most

Re: [DISCUSS] KIP-50 - Enhance Authorizer interface to be aware of supported Principal Types

2016-04-05 Thread Jay Kreps
Given that we're breaking compatibility anyway should we: 1. Remove the get prefix on this method and the existing one which violate our own code style guidelines (Oops! Kind of sad we went through the whole KIP process and no one even flagged this) 2. Move the interface out of scala to be a

Re: [VOTE] KIP-52: Kafka Connect Control APIs

2016-04-05 Thread Neha Narkhede
+1 On Tue, Apr 5, 2016 at 7:53 PM, Jay Kreps wrote: > +1 > > On Tue, Apr 5, 2016 at 4:59 PM, Jason Gustafson > wrote: > > > I'd like to open the vote on KIP-52, which adds several control APIs to > > Kafka Connect: > > > > >

Re: [VOTE] KIP-52: Kafka Connect Control APIs

2016-04-05 Thread Guozhang Wang
+1 On Tue, Apr 5, 2016 at 8:06 PM, Neha Narkhede wrote: > +1 > > On Tue, Apr 5, 2016 at 7:53 PM, Jay Kreps wrote: > > > +1 > > > > On Tue, Apr 5, 2016 at 4:59 PM, Jason Gustafson > > wrote: > > > > > I'd like to open the vote on

assigning jiras

2016-04-05 Thread sunil kalva
Hi I was trying to assign a jira to myself to start working, but looks like i don't have permission. Can someone give me access. my id: sunilkalva -- SunilKalva

[jira] [Commented] (KAFKA-3332) Consumer can't consume messages from zookeeper chroot

2016-04-05 Thread Sergey Vergun (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3332?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15225868#comment-15225868 ] Sergey Vergun commented on KAFKA-3332: -- Can I get any answer? > Consumer can't consume messages from

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

2016-04-05 Thread Apache Jenkins Server
See Changes: [wangguoz] KAFKA-2998: log warnings when client is disconnected from bootstrap [me] KAFKA-3384: Conform to POSIX kill usage -- [...truncated 2640 lines...]

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

2016-04-05 Thread Apache Jenkins Server
See Changes: [me] KAFKA-3384: Conform to POSIX kill usage -- [...truncated 1588 lines...] kafka.log.LogTest > testTruncateTo PASSED kafka.log.LogTest > testCleanShutdownFile PASSED

[jira] [Updated] (KAFKA-3508) Transient failure in kafka.security.auth.SimpleAclAuthorizerTest.testHighConcurrencyModificationOfResourceAcls

2016-04-05 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3508?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma updated KAFKA-3508: --- Assignee: Grant Henke > Transient failure in >

[jira] [Updated] (KAFKA-3508) Transient failure in kafka.security.auth.SimpleAclAuthorizerTest.testHighConcurrencyModificationOfResourceAcls

2016-04-05 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3508?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma updated KAFKA-3508: --- Status: Patch Available (was: Open) There's a PR for this here:

[GitHub] kafka pull request: KAFKA-3510; OffsetIndex thread safety

2016-04-05 Thread ijuma
GitHub user ijuma opened a pull request: https://github.com/apache/kafka/pull/1188 KAFKA-3510; OffsetIndex thread safety * Make all fields accessed outside of a lock `volatile` * Only allow mutation within the class * Remove unnecessary `AtomicInteger` since mutation always

[jira] [Commented] (KAFKA-3510) OffsetIndex thread safety

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

[jira] [Created] (KAFKA-3510) OffsetIndex thread safety

2016-04-05 Thread Ismael Juma (JIRA)
Ismael Juma created KAFKA-3510: -- Summary: OffsetIndex thread safety Key: KAFKA-3510 URL: https://issues.apache.org/jira/browse/KAFKA-3510 Project: Kafka Issue Type: Bug Reporter:

[jira] [Updated] (KAFKA-3510) OffsetIndex thread safety

2016-04-05 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3510?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma updated KAFKA-3510: --- Status: Patch Available (was: Open) > OffsetIndex thread safety > - > >