[jira] [Commented] (KAFKA-2376) Add Copycat metrics

2017-02-02 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2376?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15851186#comment-15851186 ] Ewen Cheslack-Postava commented on KAFKA-2376: -- Not quite a direct comparison, but some

Re: Trying to understand design decision about producer ack and min.insync.replicas

2017-02-02 Thread Ewen Cheslack-Postava
On Thu, Feb 2, 2017 at 11:21 PM, James Cheng wrote: > Ewen, > > Ah right, that's a good point. > > My initial reaction to your examples was that "well, those should be in > separate topics", but then I realized that people choose their topics for a > variety of reasons.

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

2017-02-02 Thread Guozhang Wang
+1 (binding). Cheers. On Thu, Feb 2, 2017 at 10:35 PM, Ewen Cheslack-Postava wrote: > +1 > > I don't think this solves all the stickiness/incremental rebalancing > problems we'll eventually want to address, but it's a nice improvement, > would be a benefit for a fair number

Re: Trying to understand design decision about producer ack and min.insync.replicas

2017-02-02 Thread James Cheng
Ewen, Ah right, that's a good point. My initial reaction to your examples was that "well, those should be in separate topics", but then I realized that people choose their topics for a variety of reasons. Sometimes they organize it based on their producers, sometimes they organize it based on

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

2017-02-02 Thread Apache Jenkins Server
See Changes: [wangguoz] MINOR: added upgrade and API changes to docs -- [...truncated 1447 lines...] kafka.api.ConsumerBounceTest > testSeekAndCommitWithBrokerFailures STARTED

Re: [DISCUSS] KIP-110: Add Codec for ZStandard Compression

2017-02-02 Thread Ewen Cheslack-Postava
I think attachments get filtered by the Apache mailing lists. You might need to post it elsewhere. I think you could upload it to the corresponding JIRA, for example. -Ewen On Tue, Jan 31, 2017 at 3:06 AM, Dongjin Lee wrote: > No problem. I will elaborate my KIP with the

Re: KIP-54 voting status?

2017-02-02 Thread Ewen Cheslack-Postava
I bumped the thread, including another binding vote for the KIP. Nag messages are sometimes useful here and definitely acceptable -- much of the time the reviews of KIPs (or JIRAs, mailing list questions, etc) simply get lost in everyone's inboxes. -Ewen On Mon, Jan 30, 2017 at 5:57 PM, Jeff

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

2017-02-02 Thread Ewen Cheslack-Postava
+1 I don't think this solves all the stickiness/incremental rebalancing problems we'll eventually want to address, but it's a nice improvement, would be a benefit for a fair number of applications, and as it's a clean extension to the existing options it doesn't come with any significant

[jira] [Commented] (KAFKA-4385) producer is sending too many unnecessary meta data request if the meta data for a topic is not available and "auto.create.topics.enable" =false

2017-02-02 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4385?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15851100#comment-15851100 ] Ewen Cheslack-Postava commented on KAFKA-4385: -- [~sslavic] Your analysis seems fine, but the

Re: [DISCUSS] KIP-117: Add a public AdministrativeClient API for Kafka admin operations

2017-02-02 Thread Becket Qin
Hi Colin, Thanks for the KIP. An admin client is probably a must after we block direct access to ZK. Some comments and thoughts below: 1. Do we have a clear scope for the admin client? It might be worth thinking about the entire user experience of the admin client. Ideally we may want to have a

[jira] [Commented] (KAFKA-4614) Long GC pause harming broker performance which is caused by mmap objects created for OffsetIndex

2017-02-02 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4614?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15851097#comment-15851097 ] Guozhang Wang commented on KAFKA-4614: -- [~kawamuray] Thanks for sharing!! > Long GC pause harming

[jira] [Comment Edited] (KAFKA-4614) Long GC pause harming broker performance which is caused by mmap objects created for OffsetIndex

2017-02-02 Thread Yuto Kawamura (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4614?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15851093#comment-15851093 ] Yuto Kawamura edited comment on KAFKA-4614 at 2/3/17 5:42 AM: -- Just as a

Re: Trying to understand design decision about producer ack and min.insync.replicas

2017-02-02 Thread Ewen Cheslack-Postava
James, Great question, I probably should have been clearer. log data is an example where the app (or even instance of the app) might know best what the right tradeoff is. Depending on your strategy for managing logs, you may or may not be mixing multiple logs (and logs from different deployments)

[jira] [Comment Edited] (KAFKA-4614) Long GC pause harming broker performance which is caused by mmap objects created for OffsetIndex

2017-02-02 Thread Yuto Kawamura (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4614?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15851093#comment-15851093 ] Yuto Kawamura edited comment on KAFKA-4614 at 2/3/17 5:41 AM: -- Just as a

[jira] [Comment Edited] (KAFKA-4614) Long GC pause harming broker performance which is caused by mmap objects created for OffsetIndex

2017-02-02 Thread Yuto Kawamura (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4614?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15851093#comment-15851093 ] Yuto Kawamura edited comment on KAFKA-4614 at 2/3/17 5:37 AM: -- Just as a

[GitHub] kafka pull request #2461: MINOR: added upgrade and API changes to docs

2017-02-02 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2461 --- 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-4614) Long GC pause harming broker performance which is caused by mmap objects created for OffsetIndex

2017-02-02 Thread Yuto Kawamura (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4614?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Yuto Kawamura updated KAFKA-4614: - Attachment: kafka-produce-99th.png Just as a sequel, this is what happened to the 99th percentile

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

2017-02-02 Thread Apache Jenkins Server
See

Re: [DISCUSS] KIP-117: Add a public AdministrativeClient API for Kafka admin operations

2017-02-02 Thread Dong Lin
Hey Colin, Thanks for the KIP. I have a few comments below: - I share similar view with Ismael that a Future-based API is better. PurgeDataFrom() is an example API that user may want to do it asynchronously even though there is only one request in flight at a time. In the future we may also have

Re: [DISCUSS] KIP-116 - Add State Store Checkpoint Interval Configuration

2017-02-02 Thread Matthias J. Sax
Thanks Damian. One more question: "Checkpointing is disabled if the checkpoint interval is set to a value <=0." Does it make sense to disable check pointing? What's the tradeoff here? -Matthias On 2/2/17 1:51 AM, Damian Guy wrote: > Hi Matthias, > > Thanks for the comments. > > 1. TBD - i

[jira] [Updated] (KAFKA-4317) RocksDB checkpoint files lost on kill -9

2017-02-02 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4317?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax updated KAFKA-4317: --- Description: Right now, the checkpoint files for logged RocksDB stores are written during a

[jira] [Updated] (KAFKA-4423) Drop support for Java 7

2017-02-02 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4423?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma updated KAFKA-4423: --- Description: Java 7 was released in July 2011, it hasn't received public updates since April 2015,

Re: [VOTE] KIP-98: Exactly Once Delivery and Transactional Messaging

2017-02-02 Thread Apurva Mehta
The wiki has been updated with a section on authorization, as well a summary of the message format changes. On Thu, Feb 2, 2017 at 9:38 AM, Jason Gustafson wrote: > Thanks Tom, we'll update the wiki to reflect all the movement on the design > document. Did you have a

[jira] [Commented] (KAFKA-4708) Fix Transient failure in BrokerApiVersionsCommandTest.checkBrokerApiVersionCommandOutput

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

[jira] [Updated] (KAFKA-4708) Fix Transient failure in BrokerApiVersionsCommandTest.checkBrokerApiVersionCommandOutput

2017-02-02 Thread Colin P. McCabe (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4708?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Colin P. McCabe updated KAFKA-4708: --- Status: Patch Available (was: Open) > Fix Transient failure in >

[GitHub] kafka pull request #2490: kafka-4727: A Production server configuration need...

2017-02-02 Thread amethystic
GitHub user amethystic opened a pull request: https://github.com/apache/kafka/pull/2490 kafka-4727: A Production server configuration needs to be updated 1. Update value for queued.max.requests to 500 2. Removed invalid config 'controller.message.queue.size' 3. Removed flush

[GitHub] kafka pull request #2489: KAFKA-4708: Fix Transient Failure in BrokerApiVers...

2017-02-02 Thread cmccabe
GitHub user cmccabe opened a pull request: https://github.com/apache/kafka/pull/2489 KAFKA-4708: Fix Transient Failure in BrokerApiVersionsCommandTest.che… …ckBrokerApiVersionCommandOutput You can merge this pull request into a Git repository by running: $ git pull

[jira] [Assigned] (KAFKA-4727) A Production server configuration needs to be updated

2017-02-02 Thread huxi (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4727?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] huxi reassigned KAFKA-4727: --- Assignee: huxi > A Production server configuration needs to be updated >

[jira] [Updated] (KAFKA-4708) Fix Transient failure in BrokerApiVersionsCommandTest.checkBrokerApiVersionCommandOutput

2017-02-02 Thread Colin P. McCabe (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4708?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Colin P. McCabe updated KAFKA-4708: --- Summary: Fix Transient failure in

[GitHub] kafka pull request #2488: MINOR: add architecture section and configure / ex...

2017-02-02 Thread guozhangwang
GitHub user guozhangwang opened a pull request: https://github.com/apache/kafka/pull/2488 MINOR: add architecture section and configure / execution for streams You can merge this pull request into a Git repository by running: $ git pull https://github.com/guozhangwang/kafka

[jira] [Updated] (KAFKA-4423) Drop support for Java 7

2017-02-02 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4423?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma updated KAFKA-4423: --- Labels: kip (was: ) > Drop support for Java 7 > --- > > Key:

[jira] [Updated] (KAFKA-4423) Drop support for Java 7

2017-02-02 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4423?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma updated KAFKA-4423: --- Fix Version/s: 0.11.0.0 > Drop support for Java 7 > --- > > Key:

[GitHub] kafka pull request #2487: Kafka-4722 : Add application.id to StreamThread na...

2017-02-02 Thread sharad-develop
GitHub user sharad-develop opened a pull request: https://github.com/apache/kafka/pull/2487 Kafka-4722 : Add application.id to StreamThread name Kafka-4722 : Add application.id to StreamThread name You can merge this pull request into a Git repository by running: $ git pull

[jira] [Assigned] (KAFKA-3924) Data loss due to halting when LEO is larger than leader's LEO

2017-02-02 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3924?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma reassigned KAFKA-3924: -- Assignee: Maysam Yabandeh > Data loss due to halting when LEO is larger than leader's LEO >

[jira] [Updated] (KAFKA-4727) A Production server configuration needs to be updated

2017-02-02 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4727?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma updated KAFKA-4727: --- Labels: newbie (was: newbies) > A Production server configuration needs to be updated >

[jira] [Updated] (KAFKA-4727) A Production server configuration needs to be updated

2017-02-02 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4727?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma updated KAFKA-4727: --- Labels: newbies (was: ) > A Production server configuration needs to be updated >

[jira] [Created] (KAFKA-4727) A Production server configuration needs to be updated

2017-02-02 Thread Jun Rao (JIRA)
Jun Rao created KAFKA-4727: -- Summary: A Production server configuration needs to be updated Key: KAFKA-4727 URL: https://issues.apache.org/jira/browse/KAFKA-4727 Project: Kafka Issue Type:

[GitHub] kafka pull request #2483: MINOR: Ensure timestamp type is provided when up-c...

2017-02-02 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2483 --- 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] KIP-117: Add a public AdministrativeClient API for Kafka admin operations

2017-02-02 Thread Ismael Juma
Hi Colin, Thanks for the KIP, great to make progress on this. I have some initial comments, will post more later: 1. We have KafkaProducer that implements the Producer interface and KafkaConsumer that implements the Consumer interface. Maybe we could have KafkaAdminClient that implements the

[GitHub] kafka pull request #2461: MINOR: added upgrade and API changes to docs

2017-02-02 Thread mjsax
GitHub user mjsax reopened a pull request: https://github.com/apache/kafka/pull/2461 MINOR: added upgrade and API changes to docs You can merge this pull request into a Git repository by running: $ git pull https://github.com/mjsax/kafka addStreamsUpdateSecton Alternatively

[jira] [Assigned] (KAFKA-4039) Exit Strategy: using exceptions instead of inline invocation of exit/halt

2017-02-02 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4039?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma reassigned KAFKA-4039: -- Assignee: Maysam Yabandeh > Exit Strategy: using exceptions instead of inline invocation of

[jira] [Commented] (KAFKA-4039) Exit Strategy: using exceptions instead of inline invocation of exit/halt

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

[GitHub] kafka pull request #2474: KAFKA-4039: Fix deadlock during shutdown due to lo...

2017-02-02 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2474 --- 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-4039) Exit Strategy: using exceptions instead of inline invocation of exit/halt

2017-02-02 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4039?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma updated KAFKA-4039: --- Resolution: Fixed Fix Version/s: 0.10.3.0 Status: Resolved (was: Patch Available)

[GitHub] kafka pull request #2461: MINOR: added upgrade and API changes to docs

2017-02-02 Thread mjsax
Github user mjsax closed the pull request at: https://github.com/apache/kafka/pull/2461 --- 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-4726) ValueMapper should have (read) access to key

2017-02-02 Thread Steven Schlansker (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4726?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Steven Schlansker updated KAFKA-4726: - Description: {{ValueMapper}} should have read-only access to the key for the value it is

[jira] [Created] (KAFKA-4726) ValueMapper should have (read) access to key

2017-02-02 Thread Steven Schlansker (JIRA)
Steven Schlansker created KAFKA-4726: Summary: ValueMapper should have (read) access to key Key: KAFKA-4726 URL: https://issues.apache.org/jira/browse/KAFKA-4726 Project: Kafka Issue

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

2017-02-02 Thread Ismael Juma
Yes, I'd also prefer the option where we only have a checksum at the message set level. I didn't suggest it due to the mentioned auditing use cases, but if they can be satisfied in some other way, then that would be great. Ismael On 2 Feb 2017 7:03 pm, "Jason Gustafson"

Re: [VOTE] 0.10.2.0 RC0

2017-02-02 Thread Mathieu Fenniak
+1 (non-binding) Upgraded a KS app, custom KC connectors, and brokers, ran an end-to-end test suite. Looks like a great release to me. :-) Mathieu On Wed, Feb 1, 2017 at 4:44 PM, Ewen Cheslack-Postava wrote: > Hello Kafka users, developers and client-developers, > > This

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

2017-02-02 Thread Apache Jenkins Server
See Changes: [wangguoz] KAFKA-3896: Fix KStream-KStream leftJoin in RepartitionIntegrationTest [wangguoz] MINOR: Update Streams docs: quickstart and concepts [me] KAFKA-4719: Consumption timeout should take into account producer

[jira] [Commented] (KAFKA-4689) OffsetValidationTest fails validation with "Current position greater than the total number of consumed records"

2017-02-02 Thread Apurva Mehta (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4689?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15850342#comment-15850342 ] Apurva Mehta commented on KAFKA-4689: - This happened again in the system test run of 02/02/2017:

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

2017-02-02 Thread Jason Gustafson
One more: 1. I did some benchmarking and CRC32C seems to be a massive win when using > the hardware instruction (particularly for messages larger than 65k), so > I'm keen on taking advantage of the message format version bump to add > support for it. I can write a separate KIP for this as it's

[jira] [Commented] (KAFKA-4722) Add application.id to StreamThread name

2017-02-02 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4722?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15850331#comment-15850331 ] Matthias J. Sax commented on KAFKA-4722: Thanks for the hint. Removing the duplicate app-Id from

[jira] [Created] (KAFKA-4725) Kafka broker fails due to OOM when producer exceeds throttling quota for extended periods of time

2017-02-02 Thread Jeff Chao (JIRA)
Jeff Chao created KAFKA-4725: Summary: Kafka broker fails due to OOM when producer exceeds throttling quota for extended periods of time Key: KAFKA-4725 URL: https://issues.apache.org/jira/browse/KAFKA-4725

[DISCUSS] KIP-117: Add a public AdministrativeClient API for Kafka admin operations

2017-02-02 Thread Colin McCabe
Hi all, I wrote up a Kafka improvement proposal for adding an AdministrativeClient interface.  This is a continuation of the work on KIP-4 towards centralized administrative operations.  Please check out

[jira] [Commented] (KAFKA-4719) Timed out waiting for consumption in OffsetValidationTest.test_broker_failure

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

[jira] [Resolved] (KAFKA-4719) Timed out waiting for consumption in OffsetValidationTest.test_broker_failure

2017-02-02 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4719?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ewen Cheslack-Postava resolved KAFKA-4719. -- Resolution: Fixed Fix Version/s: 0.10.2.0 Issue resolved by pull request

[GitHub] kafka pull request #2479: KAFKA-4719: Consumption timeout should take into a...

2017-02-02 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2479 --- 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-3896) Unstable test KStreamRepartitionJoinTest.shouldCorrectlyRepartitionOnJoinOperations

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

[GitHub] kafka pull request #2482: MINOR: Update Streams docs: quickstart and concept...

2017-02-02 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2482 --- 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

[GitHub] kafka pull request #2485: KAFKA-3896: Fix KStream-KStream leftJoin in Repart...

2017-02-02 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2485 --- 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-4722) Add application.id to StreamThread name

2017-02-02 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4722?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15850282#comment-15850282 ] Guozhang Wang commented on KAFKA-4722: -- One side note is that we have seen some scenarios where

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

2017-02-02 Thread Jason Gustafson
Hey Ismael, 2. The message timestamp field is 8 bytes. Did we consider storing the > first timestamp in the message set and then storing deltas using varints in > the messages like we do for offsets (the difference would be the usage of > signed varints)? It seems like the deltas would be quite a

Kafka KIP meeting Feb 7 at 11:00am PST

2017-02-02 Thread Jun Rao
Hi, Everyone, We plan to have a Kafka KIP meeting this coming Tuesday at 11:00am PST. If you plan to attend but haven't received an invite, please let me know. The following is the tentative agenda. Agenda: KIP-112: Handle disk failure for JBOD KIP-113: Support replicas movement between log

[jira] [Assigned] (KAFKA-4722) Add application.id to StreamThread name

2017-02-02 Thread Sharad (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4722?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Sharad reassigned KAFKA-4722: - Assignee: Sharad > Add application.id to StreamThread name > --- > >

[jira] [Updated] (KAFKA-4722) Add application.id to StreamThread name

2017-02-02 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4722?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax updated KAFKA-4722: --- Issue Type: Improvement (was: New Feature) > Add application.id to StreamThread name >

[jira] [Updated] (KAFKA-4722) Add application.id to StreamThread name

2017-02-02 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4722?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax updated KAFKA-4722: --- Priority: Minor (was: Major) > Add application.id to StreamThread name >

[jira] [Updated] (KAFKA-4722) Add application.id to StreamThread name

2017-02-02 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4722?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax updated KAFKA-4722: --- Description: StreamThread currently sets its name thusly: {code} super("StreamThread-" +

[jira] [Updated] (KAFKA-4722) Add application.id to StreamThread name

2017-02-02 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4722?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax updated KAFKA-4722: --- Description: StreamThread currently sets its name thusly: {code} super("StreamThread-" +

Re: [VOTE] KIP-48 Support for delegation tokens as an authentication mechanism

2017-02-02 Thread Jun Rao
Hi, Mani, Sorry for the late response. A couple of more comments below. > 107.4 How is token deletion handled? Does every broker delete the same > > token independently or only one broker does the deletion? > > > > Only one broker does the deletion. Broker updates the expiration in its > local

[jira] [Updated] (KAFKA-4722) Add application.id to StreamThread name

2017-02-02 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4722?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax updated KAFKA-4722: --- Summary: Add application.id to StreamThread name (was: StreamThread should allow

[jira] [Updated] (KAFKA-4722) Add application.id to StreamThread name

2017-02-02 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4722?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax updated KAFKA-4722: --- Description: StreamThread currently sets its name thusly: {code} super("StreamThread-" +

[jira] [Updated] (KAFKA-4722) Add application.id to StreamThread name

2017-02-02 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4722?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax updated KAFKA-4722: --- Labels: beginner easyfix newbie (was: ) > Add application.id to StreamThread name >

[jira] [Commented] (KAFKA-4722) StreamThread should allow customization of thread prefix

2017-02-02 Thread Steven Schlansker (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4722?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15850243#comment-15850243 ] Steven Schlansker commented on KAFKA-4722: -- That would be perfect, thanks :) > StreamThread

Re: [VOTE] KIP-98: Exactly Once Delivery and Transactional Messaging

2017-02-02 Thread Jason Gustafson
Thanks Tom, we'll update the wiki to reflect all the movement on the design document. Did you have a specific concern with the new ACLs? -Jason On Thu, Feb 2, 2017 at 6:49 AM, Ismael Juma wrote: > Hi Tom, > > That is a good point. During the discussion, it was agreed that

[jira] [Resolved] (KAFKA-4182) Move the change logger out of RocksDB stores

2017-02-02 Thread Damian Guy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4182?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Damian Guy resolved KAFKA-4182. --- Resolution: Fixed Fix Version/s: 0.10.2.0 Fixed by: https://github.com/apache/kafka/pull/2360

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

2017-02-02 Thread Apurva Mehta
Good point Tom. We will update the KIP with the ACLs section and also the message format changes. > On Feb 2, 2017, at 06:45, Tom Crayford wrote: > > I said this in the voting thread, but can the authors include a section > about new ACLs if there are going to be ACLs

[jira] [Commented] (KAFKA-4723) offsets.storage=kafka - groups stuck in rebalancing with committed offsets

2017-02-02 Thread Jason Bew (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4723?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15850173#comment-15850173 ] Jason Bew commented on KAFKA-4723: -- On further examination it does in deed look as though the brokers are

Re: [DISCUS] consuming messages is polling - is there a reason? new KIP for poll?

2017-02-02 Thread Ismael Juma
On Thu, Feb 2, 2017 at 4:41 PM, radai wrote: > also - i dont think you need to shorten fetch.max.wait.ms to get lower > delays - you could still configure a relatively long fetch.max.wait.ms and > have the broker answer your poll the minute _any_ messags are

Re: [DISCUS] consuming messages is polling - is there a reason? new KIP for poll?

2017-02-02 Thread radai
kafka relies on the underlying OS' page cache for serving "popular" data. so "pre-assembling" push batches would move from page cache to heap storage, which is not as appealing. also, for trivial cases a lot of consumers read the same thing, which would make the heap caching even worse. also - i

Live-lock between consumer thread and heartbeat thread

2017-02-02 Thread Lukavský Jan
Hi all, I have a question about a very suspicious behavior I see during consuming messages using manual synchronous commit with Kafka 0.10.1.0. The code looks something like this: try (KafkaConsumer<...> consumer = ...) { Map commitMap =

Re: [VOTE] KIP-98: Exactly Once Delivery and Transactional Messaging

2017-02-02 Thread Ismael Juma
Hi Tom, That is a good point. During the discussion, it was agreed that changes to public interfaces (message format, protocol, ACLs, etc.) would be copied to the wiki once the things had settled down, but it looks like that hasn't been done yet. I agree that it makes sense to do it before people

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

2017-02-02 Thread Tom Crayford
I said this in the voting thread, but can the authors include a section about new ACLs if there are going to be ACLs for TransactionalId. It's mentioned in the google doc, but I think new ACLs should be in a KIP directly. On Thu, Feb 2, 2017 at 2:42 PM, Ismael Juma wrote: >

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

2017-02-02 Thread Ismael Juma
Thanks for the responses and updates to the document, Guozhang and Jason. They look good. One follow-up and one additional comment: 1. I did some benchmarking and CRC32C seems to be a massive win when using the hardware instruction (particularly for messages larger than 65k), so I'm keen on

Re: [VOTE] KIP-98: Exactly Once Delivery and Transactional Messaging

2017-02-02 Thread Tom Crayford
-1 (non-binding) I've been slow at keeping up with the KIP and the discussion thread. This is an exciting and quite complex new feature, which provides great new functionality. There's a thing I noticed missing from the KIP that's present in the google doc - the doc talks about ACLs for

Re: Live-lock between consumer thread and heartbeat thread

2017-02-02 Thread Jan Lukavský
Hi Ismael, short answer at the end of this email. :) On 02/02/2017 02:52 PM, Ismael Juma wrote: I hadn't quite understood this. So the asynchronous code continues to update the map, I see. Regarding your suggested fix, the required change is not to hold a lock to the map passed to

Re: Live-lock between consumer thread and heartbeat thread

2017-02-02 Thread Ismael Juma
Sorry, I replied a little too fast. It is true that my original suggestion would not work and I agree that it would be better to copy the map in `commitSync` (that's why I suggested it in my email). Please file a JIRA. However, that doesn't solve your immediate issue. More inline. On Thu, Feb 2,

Re: Live-lock between consumer thread and heartbeat thread

2017-02-02 Thread Rajini Sivaram
I agree with Jan. KafkaConsumer should take a copy of the offsetMap, now that there are multiple threads accessing the map. commitAsync already does take a copy and when there was only one thread in the consumer, it was reasonable to avoid cloning for commitSync. But now it makes sense to clone

[GitHub] kafka pull request #2486: KAFKA-4724: Clean up of state directories can poss...

2017-02-02 Thread dguy
GitHub user dguy opened a pull request: https://github.com/apache/kafka/pull/2486 KAFKA-4724: Clean up of state directories can possibly remove stores that are about to be used by another thread Delay the cleanup of state directories that are not locked and not owned by the

[jira] [Commented] (KAFKA-4724) Clean up of state directories can possibly remove stores that are about to be used by another thread

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

[jira] [Updated] (KAFKA-4724) Clean up of state directories can possibly remove stores that are about to be used by another thread

2017-02-02 Thread Damian Guy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4724?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Damian Guy updated KAFKA-4724: -- Summary: Clean up of state directories can possibly remove stores that are about to be used by another

Re: Live-lock between consumer thread and heartbeat thread

2017-02-02 Thread Jan Lukavský
I'd disagree that I can fix the issue as you suggest, because: - if I remove the `Collections.synchronizedMap` from the `commitMap` I get unsynchronized map and therefore the asynchronous writes to this map would result in undefined state - if I remove the manual synchronization then there

Re: Live-lock between consumer thread and heartbeat thread

2017-02-02 Thread Ismael Juma
OK, you can fix this by removing `Collections.synchronizedMap` from the following line or by removing the synchronized blocks. Map commitMap = Collections.synchronizedMap(...); There is no reason to do manual and automatic synchronization at the same time in

Re: Live-lock between consumer thread and heartbeat thread

2017-02-02 Thread Jan Lukavský
Hi Ismael, yes, no problem: The following thread is the main thread interacting with the KafkaConsumer (polling topic and committing offsets): "pool-3-thread-1" #14 prio=5 os_prio=0 tid=0x7f00f4434800 nid=0x32a9 runnable [0x7f00b6662000] java.lang.Thread.State: RUNNABLE

[jira] [Assigned] (KAFKA-4724) Clean up of state directories has a race condition and can possibly remove stores that are still used

2017-02-02 Thread Damian Guy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4724?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Damian Guy reassigned KAFKA-4724: - Assignee: Damian Guy > Clean up of state directories has a race condition and can possibly

[jira] [Updated] (KAFKA-4724) Clean up of state directories has a race condition and can possibly remove stores that are still used

2017-02-02 Thread Damian Guy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4724?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Damian Guy updated KAFKA-4724: -- Summary: Clean up of state directories has a race condition and can possibly remove stores that are

Re: Live-lock between consumer thread and heartbeat thread

2017-02-02 Thread Ismael Juma
Hi Jan, Do you have stacktraces showing the issue? That would help. Also, if you can test 0.10.1.1, which is the latest stable release, that would be even better. From looking at the code briefly, I don't see where the consumer is locking on the received offsets map, so not sure what would cause

[jira] [Created] (KAFKA-4724) Clean up of state directories has a race condition and can possibly remove stores that are sill used

2017-02-02 Thread Damian Guy (JIRA)
Damian Guy created KAFKA-4724: - Summary: Clean up of state directories has a race condition and can possibly remove stores that are sill used Key: KAFKA-4724 URL: https://issues.apache.org/jira/browse/KAFKA-4724

[jira] [Updated] (KAFKA-4723) offsets.storage=kafka - groups stuck in rebalancing with committed offsets

2017-02-02 Thread Jason Bew (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4723?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Bew updated KAFKA-4723: - Description: Hi, I have moved offset store to kafka only, when I now run; bin/kafka-consumer-groups.sh

Live-lock between consumer thread and heartbeat thread

2017-02-02 Thread je.ik
Hi all, I have a question about a very suspicious behavior I see during consuming messages using manual synchronous commit with Kafka 0.10.1.0. The code looks something like this: try (KafkaConsumer<...> consumer = ...) { Map commitMap =

  1   2   >