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

2016-04-12 Thread Rajini Sivaram
Jun's tweaked proposal sounds good to me. In terms of completing KIP-43, this changes the format of the request-response for exchanging mechanisms, but not the overall logic. Since the request format in KIP-43 is worth changing anyway, I will update the KIP and the PR. On Tue, Apr 12, 2016 at 2:13

[jira] [Created] (KAFKA-3547) Broker does not disconnect client on unknown request

2016-04-12 Thread Magnus Edenhill (JIRA)
Magnus Edenhill created KAFKA-3547: -- Summary: Broker does not disconnect client on unknown request Key: KAFKA-3547 URL: https://issues.apache.org/jira/browse/KAFKA-3547 Project: Kafka Issue

[jira] [Commented] (KAFKA-3547) Broker does not disconnect client on unknown request

2016-04-12 Thread Magnus Edenhill (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3547?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15236783#comment-15236783 ] Magnus Edenhill commented on KAFKA-3547: Fixed in https://github.com/apache/kafka/

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

2016-04-12 Thread Ismael Juma
Hi Jun, I understand the point about the SASL tokens being similar to the SSL handshake in a way. However, is there any SASL library that handles the network communication for these tokens? I couldn't find any and without that, there isn't much benefit in deviating from Kafka's protocol (we basica

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

2016-04-12 Thread Rajini Sivaram
Ismael, My only concern about wrapping SASL tokens in Kafka headers is backward compatibility. We would either have a different format for GSSAPI alone to match 0.9.0.x or we would need to support two different wire protocols for GSSAPI. Neither sounds ideal. On Tue, Apr 12, 2016 at 9:18 AM, Isma

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

2016-04-12 Thread Ismael Juma
Hi Rajini, Yes, I agree that it's not ideal. However, doing this once at the broker is more manageable than pushing the additional complexity to the clients. Between the two options you outlined, the second one seemed the least bad (we do something similar for controlled shutdown because it was mi

[jira] [Resolved] (KAFKA-3205) Error in I/O with host (java.io.EOFException) raised in producer

2016-04-12 Thread Flavio Junqueira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3205?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Flavio Junqueira resolved KAFKA-3205. - Resolution: Won't Fix Reviewer: Flavio Junqueira Fix Version/s: (was: 0

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

2016-04-12 Thread Tanju Cataltepe (JIRA)
Tanju Cataltepe created KAFKA-3548: -- Summary: Locale is not handled properly in kafka-consumer Key: KAFKA-3548 URL: https://issues.apache.org/jira/browse/KAFKA-3548 Project: Kafka Issue Type

[jira] [Assigned] (KAFKA-3523) Capture org.apache.kafka.clients.consumer.CommitFailedException in UncaughtExceptionHandler

2016-04-12 Thread Eno Thereska (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3523?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Eno Thereska reassigned KAFKA-3523: --- Assignee: Eno Thereska > Capture org.apache.kafka.clients.consumer.CommitFailedException in

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

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

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

2016-04-12 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3548?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15237186#comment-15237186 ] Ismael Juma commented on KAFKA-3548: A possible solution is to pass a `Locale` to `Str

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

2016-04-12 Thread Michael Noll (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3511?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15237291#comment-15237291 ] Michael Noll commented on KAFKA-3511: - [~guozhang]: Could we leverage the parent class

Re: [VOTE] KIP-4 Metadata Schema (Round 2)

2016-04-12 Thread Dana Powers
+1 On Apr 11, 2016 21:55, "Gwen Shapira" wrote: > +1 > > On Mon, Apr 11, 2016 at 10:42 AM, Grant Henke wrote: > > Based on the discussion in the previous vote thread > > < > http://search-hadoop.com/m/uyzND1xlaiU10QlYX&subj=+VOTE+KIP+4+Metadata+Schema > > > > I also would like to include a behav

Re: [DISCUSS] KIP-43: Kafka SASL enhancements

2016-04-12 Thread Ismael Juma
Hi Jun, Comments inline. On Mon, Apr 11, 2016 at 1:57 AM, Jun Rao wrote: > Yes, that should be fine right? Since the new api key will start with a 0 > byte, it actually guarantees that it's different from 0x60 (1st byte in the > old protocol) even if we change the request version id in the futu

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

2016-04-12 Thread Flavio Junqueira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3042?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15236055#comment-15236055 ] Flavio Junqueira edited comment on KAFKA-3042 at 4/12/16 2:58 PM: --

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

2016-04-12 Thread Jun Rao
Hi, Ismael, The SASL engine that we used is the SASL library, right? How did the C client generate those SASL tokens? Once a SASL mechanism is chosen, the subsequent tokens are determined, right? So, my feeling is that those tokens are part of SaslHandshakeRequest and are just extended across mult

[jira] [Resolved] (KAFKA-3547) Broker does not disconnect client on unknown request

2016-04-12 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3547?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma resolved KAFKA-3547. Resolution: Fixed Assignee: Grant Henke And this is the relevant commit for trunk: https://gi

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

2016-04-12 Thread Flavio Junqueira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3042?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15237383#comment-15237383 ] Flavio Junqueira commented on KAFKA-3042: - I had a look at the zookeeper logs, and

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

2016-04-12 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3511?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang updated KAFKA-3511: - Fix Version/s: (was: 0.10.0.0) 0.10.1.0 > Provide built-in aggregators sum(

Re: [VOTE] KIP-4 Metadata Schema (Round 2)

2016-04-12 Thread Jason Gustafson
+1 (non-binding) On Mon, Apr 11, 2016 at 6:21 PM, Ismael Juma wrote: > +1 (non-binding) > > Ismael > > On Tue, Apr 12, 2016 at 2:19 AM, Jun Rao wrote: > > > Grant, > > > > Thanks for the updated version. +1 from me. > > > > Jun > > > > On Mon, Apr 11, 2016 at 10:42 AM, Grant Henke > wrote: > >

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

2016-04-12 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3513?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15237505#comment-15237505 ] Ismael Juma commented on KAFKA-3513: We have had failures with various different param

[jira] [Commented] (KAFKA-3518) Transient failure in ReplicationTest.test_replication_with_broker_failure with ConsumerTimeoutException

2016-04-12 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3518?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15237508#comment-15237508 ] Ismael Juma commented on KAFKA-3518: We have had failures with various different param

[GitHub] kafka pull request: MINOR: Add missing `@Override` to `KStreamImpl...

2016-04-12 Thread ijuma
GitHub user ijuma opened a pull request: https://github.com/apache/kafka/pull/1216 MINOR: Add missing `@Override` to `KStreamImpl.through` You can merge this pull request into a Git repository by running: $ git pull https://github.com/ijuma/kafka add-missing-override-to-throug

[jira] [Updated] (KAFKA-3117) Fail test at: PlaintextConsumerTest. testAutoCommitOnRebalance

2016-04-12 Thread Grant Henke (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3117?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Grant Henke updated KAFKA-3117: --- Issue Type: Sub-task (was: Bug) Parent: KAFKA-2054 > Fail test at: PlaintextConsumerTest. tes

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

2016-04-12 Thread James Cheng (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3042?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15237604#comment-15237604 ] James Cheng commented on KAFKA-3042: Thanks [~fpj]. Do you need any additional info fr

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

2016-04-12 Thread James Cheng (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3042?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15237604#comment-15237604 ] James Cheng edited comment on KAFKA-3042 at 4/12/16 5:37 PM: -

[GitHub] kafka pull request: MINOR: Add missing `@Override` to `KStreamImpl...

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

[jira] [Commented] (KAFKA-1981) Make log compaction point configurable

2016-04-12 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1981?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15237782#comment-15237782 ] Ismael Juma commented on KAFKA-1981: Thanks for the PR. This introduces new configs so

[jira] [Created] (KAFKA-3549) Close consumers instantiated in consumer tests

2016-04-12 Thread Grant Henke (JIRA)
Grant Henke created KAFKA-3549: -- Summary: Close consumers instantiated in consumer tests Key: KAFKA-3549 URL: https://issues.apache.org/jira/browse/KAFKA-3549 Project: Kafka Issue Type: Improvem

[jira] [Created] (KAFKA-3550) Broker does not honor MetadataRequest api version; always returns v0 MetadataResponse

2016-04-12 Thread Dana Powers (JIRA)
Dana Powers created KAFKA-3550: -- Summary: Broker does not honor MetadataRequest api version; always returns v0 MetadataResponse Key: KAFKA-3550 URL: https://issues.apache.org/jira/browse/KAFKA-3550 Proje

[jira] [Commented] (KAFKA-3550) Broker does not honor MetadataRequest api version; always returns v0 MetadataResponse

2016-04-12 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3550?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15237791#comment-15237791 ] Ismael Juma commented on KAFKA-3550: cc [~granthenke] > Broker does not honor Metadat

[jira] [Commented] (KAFKA-3549) Close consumers instantiated in consumer tests

2016-04-12 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3549?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15237795#comment-15237795 ] Ismael Juma commented on KAFKA-3549: Interesting. It should have been obvious given th

[jira] [Assigned] (KAFKA-3550) Broker does not honor MetadataRequest api version; always returns v0 MetadataResponse

2016-04-12 Thread Grant Henke (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3550?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Grant Henke reassigned KAFKA-3550: -- Assignee: Grant Henke > Broker does not honor MetadataRequest api version; always returns v0 >

[jira] [Commented] (KAFKA-3550) Broker does not honor MetadataRequest api version; always returns v0 MetadataResponse

2016-04-12 Thread Grant Henke (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3550?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15237794#comment-15237794 ] Grant Henke commented on KAFKA-3550: I will look into this and provide a detailed summ

[jira] [Commented] (KAFKA-3550) Broker does not honor MetadataRequest api version; always returns v0 MetadataResponse

2016-04-12 Thread Grant Henke (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3550?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15237800#comment-15237800 ] Grant Henke commented on KAFKA-3550: Note that KAFKA-2512 has an older pull request op

[GitHub] kafka pull request: KAFKA-3549: Close consumers instantiated in co...

2016-04-12 Thread granthenke
GitHub user granthenke opened a pull request: https://github.com/apache/kafka/pull/1217 KAFKA-3549: Close consumers instantiated in consumer tests You can merge this pull request into a Git repository by running: $ git pull https://github.com/granthenke/kafka close-consumers

[jira] [Commented] (KAFKA-3549) Close consumers instantiated in consumer tests

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

[jira] [Updated] (KAFKA-3549) Close consumers instantiated in consumer tests

2016-04-12 Thread Grant Henke (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3549?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Grant Henke updated KAFKA-3549: --- Status: Patch Available (was: Open) > Close consumers instantiated in consumer tests > --

[jira] [Commented] (KAFKA-3549) Close consumers instantiated in consumer tests

2016-04-12 Thread Grant Henke (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3549?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15237807#comment-15237807 ] Grant Henke commented on KAFKA-3549: [~ijuma] This patch tries to clean up a lot of th

Jenkins build is back to normal : kafka-0.10.0-jdk7 #36

2016-04-12 Thread Apache Jenkins Server
See

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

2016-04-12 Thread Flavio Junqueira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3042?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15237824#comment-15237824 ] Flavio Junqueira commented on KAFKA-3042: - hey [~wushujames] bq. you said that br

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

2016-04-12 Thread Robert Christ (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3042?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15237839#comment-15237839 ] Robert Christ commented on KAFKA-3042: -- We turned on unclean leader election due to e

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

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

[GitHub] kafka pull request: KAFKA-3439: Added exceptions thrown

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

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

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

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

2016-04-12 Thread Apache Jenkins Server
See Changes: [wangguoz] MINOR: Add missing `@Override` to `KStreamImpl.through` -- [...truncated 3152 lines...] kafka.integration.AutoOffsetResetTest > testResetToLatestWhenOffsetTooLow PASSED k

[jira] [Created] (KAFKA-3551) Update rocksdb, snappy-java, slf4j

2016-04-12 Thread Ismael Juma (JIRA)
Ismael Juma created KAFKA-3551: -- Summary: Update rocksdb, snappy-java, slf4j Key: KAFKA-3551 URL: https://issues.apache.org/jira/browse/KAFKA-3551 Project: Kafka Issue Type: Improvement

[GitHub] kafka pull request: MINOR: Remove unused hadoop version

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

[GitHub] kafka pull request: KAFKA-3461: Fix typos in Kafka web documentati...

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

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

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

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

2016-04-12 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3461?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gwen Shapira resolved KAFKA-3461. - Resolution: Fixed Fix Version/s: 0.10.1.0 Issue resolved by pull request 1138 [https://gith

[jira] [Updated] (KAFKA-3551) Update rocksdb to 4.2.0

2016-04-12 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3551?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma updated KAFKA-3551: --- Summary: Update rocksdb to 4.2.0 (was: Update rocksdb, snappy-java, slf4j) > Update rocksdb to 4.2.0

[jira] [Resolved] (KAFKA-3551) Update rocksdb to 4.2.0

2016-04-12 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3551?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma resolved KAFKA-3551. Resolution: Won't Fix Rocksdb 4.2.0 is even worse, running the test suite segfaulted on two consecu

[GitHub] kafka pull request: MINOR: Patch version updates for snappy and sl...

2016-04-12 Thread ijuma
GitHub user ijuma opened a pull request: https://github.com/apache/kafka/pull/1218 MINOR: Patch version updates for snappy and slf4j * slf4j 1.7.21 includes thread-safety fixes: http://www.slf4j.org/news.html * snappy 1.1.2.4 includes performance improvements requested by Spark:

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

2016-04-12 Thread Apache Jenkins Server
See

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

2016-04-12 Thread Kanak Biscuitwala (JIRA)
Kanak Biscuitwala created KAFKA-3552: Summary: New Consumer: java.lang.OutOfMemoryError: Direct buffer memory Key: KAFKA-3552 URL: https://issues.apache.org/jira/browse/KAFKA-3552 Project: Kafka

[jira] [Assigned] (KAFKA-3552) New Consumer: java.lang.OutOfMemoryError: Direct buffer memory

2016-04-12 Thread Liquan Pei (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3552?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Liquan Pei reassigned KAFKA-3552: - Assignee: Liquan Pei (was: Neha Narkhede) > New Consumer: java.lang.OutOfMemoryError: Direct buf

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

2016-04-12 Thread Apache Jenkins Server
See Changes: [cshapi] MINOR: Remove unused hadoop version [cshapi] KAFKA-3461: Fix typos in Kafka web documentations. -- [...truncated 3813 lines...] org.apache.kafka.common.SerializeCompatibil

Reg: Issue with Kafka Kerberos (Kafka Version 0.9.0.1)

2016-04-12 Thread BigData dev
Hi All, I am facing issue with kafka kerberoized cluster. After following the steps how to enables SASL on kafka by using below link. http://docs.confluent.io/2.0.0/kafka/sasl.html After this,when i start the kafka-server I am getting below error. [2016-04-12 16:59:26,201] ERROR [KafkaApi-1001]

[jira] [Commented] (KAFKA-3544) Missing topics on startup

2016-04-12 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3544?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15238307#comment-15238307 ] Guozhang Wang commented on KAFKA-3544: -- The topics that Kafka Streams reads as source

[GitHub] kafka pull request: KAFKA-3504: Log compaction for changelog parti...

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

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

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

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

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

[jira] [Commented] (KAFKA-3552) New Consumer: java.lang.OutOfMemoryError: Direct buffer memory

2016-04-12 Thread Liquan Pei (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3552?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15238372#comment-15238372 ] Liquan Pei commented on KAFKA-3552: --- Hi Kanak, Can you share with us the consumer confi

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

2016-04-12 Thread Apache Jenkins Server
See Changes: [wangguoz] KAFKA-3504; Log compaction for changelog partition -- [...truncated 3842 lines...] kafka.log.BrokerCompressionTest > testBrokerSideCompression[3] PASSED kafka.log.Broker

[jira] [Commented] (KAFKA-3377) add REST interface to JMX

2016-04-12 Thread Chien Le (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3377?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15238536#comment-15238536 ] Chien Le commented on KAFKA-3377: - Might also want to check out https://github.com/arnobr

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

2016-04-12 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3042?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15238651#comment-15238651 ] Jun Rao commented on KAFKA-3042: The issue seems to be the following. In 0.9.0, we changed

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

2016-04-12 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3042?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15238721#comment-15238721 ] Ismael Juma commented on KAFKA-3042: [~junrao], some people have said that they have s

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

2016-04-12 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3042?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma updated KAFKA-3042: --- Fix Version/s: 0.10.0.0 > updateIsr should stop after failed several times due to zkVersion issue > --