Re: [DISCUSS] KIP-78: Cluster Id

2016-09-02 Thread Ismael Juma
Hi Dong, Thanks for your feedback. Comments inline. On Thu, Sep 1, 2016 at 7:51 PM, Dong Lin wrote: > > I share the view with Harsha and would like to understand how the current > approach of randomly generating cluster.id compares with the approach of > manually specifying

Re: [DISCUSS] KIP-79 - ListOffsetRequest v1 and offsetForTime() method in new consumer.

2016-09-02 Thread Becket Qin
Thanks for the feedback Jun, Jay and Ismael. To answer Jun's question > 1. In ListOffsetResponse, I am not sure if it's useful to return the > timestamp since the user can always find the timestamp by fetching the > message at the returned offset. Returning timestamp with offset may be useful if

Re: [DISCUSS] Remove beta label from the new Java consumer

2016-09-02 Thread Ismael Juma
Hi Jaikiran, Thanks for your feedback. Comments inline. On Wed, Aug 31, 2016 at 5:40 AM, Jaikiran Pai wrote: > Personally, I would be OK if the beta label is removed from it if the dev > team is sure the API isn't going to change. I don't know if that's true or > not

Re: [DISCUSS] KIP-79 - ListOffsetRequest v1 and offsetForTime() method in new consumer.

2016-09-02 Thread Ismael Juma
Thanks for the proposal Becket. Looks good overall, a few comments: ListOffsetResponse => [TopicName [PartitionOffsets]] > PartitionOffsets => Partition ErrorCode Timestamp [Offset] > Partition => int32 > ErrorCode => int16 > Timestamp => int64 > Offset => int It should be int64 for

[jira] [Commented] (KAFKA-4118) StreamsSmokeTest.test_streams started failing since 18 August build

2016-09-02 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4118?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15459966#comment-15459966 ] Jason Gustafson commented on KAFKA-4118: [~guozhang] I have a patch available for KAFKA-3807 which

[jira] [Created] (KAFKA-4118) StreamsSmokeTest.test_streams started failing since 18 August build

2016-09-02 Thread Ismael Juma (JIRA)
Ismael Juma created KAFKA-4118: -- Summary: StreamsSmokeTest.test_streams started failing since 18 August build Key: KAFKA-4118 URL: https://issues.apache.org/jira/browse/KAFKA-4118 Project: Kafka

[jira] [Commented] (KAFKA-3340) Add support for rebalance and adding concurrently records with MockConsumer

2016-09-02 Thread Florian Hussonnois (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3340?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15459658#comment-15459658 ] Florian Hussonnois commented on KAFKA-3340: --- I have updated a pull request for this JIRA with

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

2016-09-02 Thread Apache Jenkins Server
See Changes: [junrao] KAFKA-4099; Fix the potential frequent log rolling -- [...truncated 3801 lines...] kafka.tools.ConsoleProducerTest > testParseKeyProp PASSED

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

2016-09-02 Thread Apache Jenkins Server
See Changes: [junrao] KAFKA-4099; Fix the potential frequent log rolling -- [...truncated 3421 lines...] kafka.integration.SaslPlaintextTopicMetadataTest > testAutoCreateTopic STARTED

Re: [DISCUSS] KIP-79 - ListOffsetRequest v1 and offsetForTime() method in new consumer.

2016-09-02 Thread Jay Kreps
This looks great, big improvements for the list offset protocol which is currently quite odd. One minor thing. I think the old v0 list offsets request also gave you the highwater mark, it kind of shoves it in as the last thing in the array of offsets. This is used internally to implement

Re: [DISCUSS] KIP-72 Allow Sizing Incoming Request Queue in Bytes

2016-09-02 Thread Jun Rao
Hi, Radi, Thanks for the update. At the high level, this looks promising. A few comments below. 1. If we can bound the requests by bytes, it seems that we don't need queued.max.requests any more? Could we just deprecate the config and make the queue size unbounded? 2. How do we communicate back

Re: [DISCUSS] KIP-79 - ListOffsetRequest v1 and offsetForTime() method in new consumer.

2016-09-02 Thread Jun Rao
Hi, Jiangjie, Thanks for the wiki. Looks good overall. Just one comment below. 1. In ListOffsetResponse, I am not sure if it's useful to return the timestamp since the user can always find the timestamp by fetching the message at the returned offset. Jun On Tue, Aug 30, 2016 at 8:38 PM,

[jira] [Commented] (KAFKA-3807) OffsetValidationTest - transient failure on test_broker_rolling_bounce

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

[GitHub] kafka pull request #1821: KAFKA-3807: Fix transient test failure caused by r...

2016-09-02 Thread hachikuji
GitHub user hachikuji opened a pull request: https://github.com/apache/kafka/pull/1821 KAFKA-3807: Fix transient test failure caused by race on future completion You can merge this pull request into a Git repository by running: $ git pull https://github.com/hachikuji/kafka

[jira] [Updated] (KAFKA-4099) Change the time based log rolling to only based on the message timestamp.

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

[GitHub] kafka pull request #1809: KAFKA-4099: Fix the potential frequent log rolling

2016-09-02 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/1809 --- 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-4099) Change the time based log rolling to only based on the message timestamp.

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

[GitHub] kafka pull request #1820: MINOR: Add timeout to testRenegotiation.

2016-09-02 Thread SinghAsDev
GitHub user SinghAsDev opened a pull request: https://github.com/apache/kafka/pull/1820 MINOR: Add timeout to testRenegotiation. You can merge this pull request into a Git repository by running: $ git pull https://github.com/SinghAsDev/kafka MinorTestRenogtiation

[jira] [Updated] (KAFKA-4117) Cleanup StreamPartitionAssignor behavior

2016-09-02 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4117?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang updated KAFKA-4117: - Description: I went through the whole assignment logic once again and I feel the logic has now

[jira] [Created] (KAFKA-4117) Cleanup StreamPartitionAssignor behavior

2016-09-02 Thread Guozhang Wang (JIRA)
Guozhang Wang created KAFKA-4117: Summary: Cleanup StreamPartitionAssignor behavior Key: KAFKA-4117 URL: https://issues.apache.org/jira/browse/KAFKA-4117 Project: Kafka Issue Type: Bug

[jira] [Commented] (KAFKA-3478) Finer Stream Flow Control

2016-09-02 Thread Bill Bejeck (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3478?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15458985#comment-15458985 ] Bill Bejeck commented on KAFKA-3478: Makes sense, but I didn't realize that was the case, thanks. >

[jira] [Assigned] (KAFKA-4114) Allow for different "auto.offset.reset" strategies for different input streams

2016-09-02 Thread Bill Bejeck (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4114?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Bill Bejeck reassigned KAFKA-4114: -- Assignee: Bill Bejeck (was: Guozhang Wang) > Allow for different "auto.offset.reset"

[GitHub] kafka pull request #1819: WIP: add context to some exceptions

2016-09-02 Thread dguy
GitHub user dguy opened a pull request: https://github.com/apache/kafka/pull/1819 WIP: add context to some exceptions KafkaExceptions currently thrown from within StreamThread/StreamTask currently bubble up without any additional context. This makes it hard to figure out where

[jira] [Commented] (KAFKA-3478) Finer Stream Flow Control

2016-09-02 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3478?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15458831#comment-15458831 ] Matthias J. Sax commented on KAFKA-3478: All newly creates "Streams" Jiras are assigned to him by

[jira] [Commented] (KAFKA-4024) First metadata update always take retry.backoff.ms milliseconds to complete

2016-09-02 Thread Yuto Kawamura (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4024?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15458699#comment-15458699 ] Yuto Kawamura commented on KAFKA-4024: -- Updated PR to fix this issue not only about the first

[jira] [Commented] (KAFKA-4024) First metadata update always take retry.backoff.ms milliseconds to complete

2016-09-02 Thread Yuto Kawamura (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4024?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15458639#comment-15458639 ] Yuto Kawamura commented on KAFKA-4024: -- I reconsidered this issue and think I found that this is much

[jira] [Updated] (KAFKA-4024) First metadata update always take retry.backoff.ms milliseconds to complete

2016-09-02 Thread Yuto Kawamura (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4024?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Yuto Kawamura updated KAFKA-4024: - Description: Recently I updated our KafkaProducer configuration, specifically we adjusted

[jira] [Updated] (KAFKA-4024) First metadata update always take retry.backoff.ms milliseconds to complete

2016-09-02 Thread Yuto Kawamura (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4024?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Yuto Kawamura updated KAFKA-4024: - Description: gtgtRecently I updated our KafkaProducer configuration, specifically we adjusted

[jira] [Commented] (KAFKA-3587) LogCleaner fails due to incorrect offset map computation on a replica

2016-09-02 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3587?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15458428#comment-15458428 ] ASF GitHub Bot commented on KAFKA-3587: --- GitHub user id opened a pull request:

[GitHub] kafka pull request #1818: Backport KAFKA-3587

2016-09-02 Thread id
GitHub user id opened a pull request: https://github.com/apache/kafka/pull/1818 Backport KAFKA-3587 You can merge this pull request into a Git repository by running: $ git pull https://github.com/klarna/kafka backport-KAFKA-3587 Alternatively you can review and apply these

[jira] [Commented] (KAFKA-3478) Finer Stream Flow Control

2016-09-02 Thread Bill Bejeck (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3478?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15458306#comment-15458306 ] Bill Bejeck commented on KAFKA-3478: [~mjsax] thanks for the heads up, but it looks like they are

[jira] [Commented] (KAFKA-4113) Allow KTable bootstrap

2016-09-02 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4113?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15458144#comment-15458144 ] Matthias J. Sax commented on KAFKA-4113: Thanks for pointing out! I read it once. Good post! :) >

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

2016-09-02 Thread Apache Jenkins Server
See Changes: [wangguoz] KAFKA-4104: Queryable state metadata is sometimes invalid -- [...truncated 7235 lines...] kafka.log.LogTest > testReadOutOfRange STARTED kafka.log.LogTest >

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

2016-09-02 Thread Apache Jenkins Server
See Changes: [wangguoz] KAFKA-4104: Queryable state metadata is sometimes invalid -- [...truncated 4478 lines...] kafka.network.SocketServerTest > tooBigRequestIsRejected PASSED

[jira] [Updated] (KAFKA-3703) Selector.close() doesn't complete outgoing writes

2016-09-02 Thread Rajini Sivaram (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3703?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Rajini Sivaram updated KAFKA-3703: -- Status: Patch Available (was: Open) > Selector.close() doesn't complete outgoing writes >

[jira] [Updated] (KAFKA-3703) Selector.close() doesn't complete outgoing writes

2016-09-02 Thread Rajini Sivaram (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3703?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Rajini Sivaram updated KAFKA-3703: -- Description: Outgoing writes may be discarded when a connection is closed. For instance, when

[jira] [Updated] (KAFKA-3703) Selector.close() doesn't complete outgoing writes

2016-09-02 Thread Rajini Sivaram (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3703?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Rajini Sivaram updated KAFKA-3703: -- Affects Version/s: 0.10.0.1 Component/s: clients Summary:

[GitHub] kafka pull request #1817: KAFKA-3703: Flush outgoing writes before closing c...

2016-09-02 Thread rajinisivaram
GitHub user rajinisivaram opened a pull request: https://github.com/apache/kafka/pull/1817 KAFKA-3703: Flush outgoing writes before closing client selector Close client connections only after outgoing writes complete or timeout. You can merge this pull request into a Git repository

[jira] [Commented] (KAFKA-3703) PlaintextTransportLayer.close() doesn't complete outgoing writes

2016-09-02 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3703?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15457896#comment-15457896 ] ASF GitHub Bot commented on KAFKA-3703: --- GitHub user rajinisivaram opened a pull request:

[jira] [Commented] (KAFKA-4116) Specifying 0.0.0.0 in "listeners" doesn't work

2016-09-02 Thread Yuto Kawamura (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4116?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15457774#comment-15457774 ] Yuto Kawamura commented on KAFKA-4116: -- [~gwenshap] PTAL. > Specifying 0.0.0.0 in "listeners"

[jira] [Updated] (KAFKA-4116) Specifying 0.0.0.0 in "listeners" doesn't work

2016-09-02 Thread Yuto Kawamura (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4116?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Yuto Kawamura updated KAFKA-4116: - Status: Patch Available (was: Open) > Specifying 0.0.0.0 in "listeners" doesn't work >

[jira] [Commented] (KAFKA-4116) Specifying 0.0.0.0 in "listeners" doesn't work

2016-09-02 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4116?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15457768#comment-15457768 ] ASF GitHub Bot commented on KAFKA-4116: --- GitHub user kawamuray opened a pull request:

[GitHub] kafka pull request #1816: KAFKA-4116: Handle 0.0.0.0 as a special case when ...

2016-09-02 Thread kawamuray
GitHub user kawamuray opened a pull request: https://github.com/apache/kafka/pull/1816 KAFKA-4116: Handle 0.0.0.0 as a special case when using advertised.listeners Issue: https://issues.apache.org/jira/browse/KAFKA-4116 You can merge this pull request into a Git repository by

[jira] [Created] (KAFKA-4116) Specifying 0.0.0.0 in "listeners" doesn't work

2016-09-02 Thread Yuto Kawamura (JIRA)
Yuto Kawamura created KAFKA-4116: Summary: Specifying 0.0.0.0 in "listeners" doesn't work Key: KAFKA-4116 URL: https://issues.apache.org/jira/browse/KAFKA-4116 Project: Kafka Issue Type: Bug