Re: [DISCUSS] KIP-87 - Add Compaction Tombstone Flag

2016-11-14 Thread Renu Tewari
Is upping the magic byte to 2 needed? In your example say For broker api version at or above 0.10.2 the tombstone bit will be used for log compaction deletion. If the producerequest version is less than 0.10.2 and the message is null, the broker will up convert to set the tombstone bit on If the

[GitHub] kafka pull request #2132: add a space to separate two words

2016-11-14 Thread ZhengQian1
GitHub user ZhengQian1 opened a pull request: https://github.com/apache/kafka/pull/2132 add a space to separate two words I think we should add a space here, otherwise the two words will join together. You can merge this pull request into a Git repository by running: $ git

Re: [DISCUSS] KIP-87 - Add Compaction Tombstone Flag

2016-11-14 Thread Becket Qin
If we follow the current way of doing this format change, it would work the following way: 0. Bump up the magic byte to 2 to indicate the tombstone bit is used. 1. On receiving a ProduceRequest, broker always convert the messages to the configured message.format.version. 1.1 If the message

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

2016-11-14 Thread Apache Jenkins Server
See

[jira] [Comment Edited] (KAFKA-4404) Add knowledge of sign to numeric schema types

2016-11-14 Thread Andy Bryant (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4404?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15665817#comment-15665817 ] Andy Bryant edited comment on KAFKA-4404 at 11/15/16 2:40 AM: -- Hi Ewen I'd

[jira] [Commented] (KAFKA-4404) Add knowledge of sign to numeric schema types

2016-11-14 Thread Andy Bryant (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4404?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15665817#comment-15665817 ] Andy Bryant commented on KAFKA-4404: Hi Ewen I'd actually be fine with Kafka Connect only supporting

Re: Kafka 0.10 Monitoring tool

2016-11-14 Thread Otis Gospodnetić
Hi, Why are these tools not working perfectly for you? Does it *have to* be open-source? If not, Sematext SPM collects a lot of Kafka metrics, with consumer lag being one of them -- https://sematext.com/blog/2016/06/07/kafka-consumer-lag-offsets-monitoring/ Otis -- Monitoring - Log Management -

[GitHub] kafka pull request #2131: Remove failing ConnectDistributedTest.test_bad_con...

2016-11-14 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2131 --- 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 #2131: Remove failing ConnectDistributedTest.test_bad_con...

2016-11-14 Thread shikhar
GitHub user shikhar opened a pull request: https://github.com/apache/kafka/pull/2131 Remove failing ConnectDistributedTest.test_bad_connector_class Since #1911 was merged it is hard to externally test a connector transitioning to FAILED state due to an initialization failure, which

[jira] [Commented] (KAFKA-4409) ZK consumer shutdown/topic event deadlock

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

[jira] [Commented] (KAFKA-2066) Replace FetchRequest / FetchResponse with their org.apache.kafka.common.requests equivalents

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

[jira] [Resolved] (KAFKA-2066) Replace FetchRequest / FetchResponse with their org.apache.kafka.common.requests equivalents

2016-11-14 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2066?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson resolved KAFKA-2066. Resolution: Fixed Fix Version/s: 0.10.2.0 Issue resolved by pull request 2069

[GitHub] kafka pull request #2069: KAFKA-2066: Use client-side FetchRequest/FetchResp...

2016-11-14 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2069 --- 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 #2129: KAFKA-4409; Fix deadlock between topic event handl...

2016-11-14 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2129 --- 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: [VOTE] KIP-84: Support SASL SCRAM mechanisms

2016-11-14 Thread Jun Rao
Hi, Rajini, Thanks for the proposal. +1. A few minor comments. 30. Could you add that the broker config sasl.enabled.mechanisms can now take more values? 31. Could you document the meaning of s,t,k,i used in /config/users/alice in ZK? 32. In the rejected section, could you document why we

[jira] [Updated] (KAFKA-3462) Allow SinkTasks to disable consumer offset commit

2016-11-14 Thread Shikhar Bhushan (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3462?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Shikhar Bhushan updated KAFKA-3462: --- Resolution: Fixed Status: Resolved (was: Patch Available) This will be handled with

[jira] [Commented] (KAFKA-4401) Change the KafkaServerTestHarness and IntegrationTestHarness from trait to abstract class.

2016-11-14 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4401?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15665238#comment-15665238 ] Ismael Juma commented on KAFKA-4401: Kafka Streams also has helpers for tests that are nicer to use

Kafka 0.10 Monitoring tool

2016-11-14 Thread Ghosh, Achintya (Contractor)
Hi there, What is the best open source tool for Kafka monitoring mainly to check the offset lag. We tried the following tools: 1. Burrow 2. KafkaOffsetMonitor 3. Prometheus and Grafana 4. Kafka Manager But nothing is working perfectly. Please help us on this. Thanks

[GitHub] kafka pull request #2130: MINOR: Extract SCALA_BINARY_VERSION from SCALA_VER...

2016-11-14 Thread kkonstantine
GitHub user kkonstantine opened a pull request: https://github.com/apache/kafka/pull/2130 MINOR: Extract SCALA_BINARY_VERSION from SCALA_VERSION Will allow users to set one fewer environment variable if they need to change scala version. Still, SCALA_BINARY_VERSION can be

[jira] [Commented] (KAFKA-4322) StateRestoreCallback begin and end indication

2016-11-14 Thread Mark Shelton (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4322?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15665194#comment-15665194 ] Mark Shelton commented on KAFKA-4322: - >> Is there any particular reasons that you want to commit

[jira] [Resolved] (KAFKA-4011) allow sizing RequestQueue in bytes

2016-11-14 Thread radai rosenblatt (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4011?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] radai rosenblatt resolved KAFKA-4011. - Resolution: Won't Fix Fix Version/s: (was: 0.10.2.0) this was the original

[jira] [Commented] (KAFKA-4011) allow sizing RequestQueue in bytes

2016-11-14 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4011?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15665144#comment-15665144 ] ASF GitHub Bot commented on KAFKA-4011: --- Github user radai-rosenblatt closed the pull request at:

[jira] [Updated] (KAFKA-4322) StateRestoreCallback begin and end indication

2016-11-14 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4322?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax updated KAFKA-4322: --- Affects Version/s: (was: 0.10.0.1) 0.10.1.0 > StateRestoreCallback

[jira] [Updated] (KAFKA-4322) StateRestoreCallback begin and end indication

2016-11-14 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4322?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax updated KAFKA-4322: --- Status: Patch Available (was: Open) > StateRestoreCallback begin and end indication >

[GitHub] kafka pull request #1714: KAFKA-4011 - fix issues and beef up tests around B...

2016-11-14 Thread radai-rosenblatt
Github user radai-rosenblatt closed the pull request at: https://github.com/apache/kafka/pull/1714 --- 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

Re: [DISCUSS] KIP-87 - Add Compaction Tombstone Flag

2016-11-14 Thread Mayuresh Gharat
Hi Michael, Just another thing that came up during my discussion with Renu and I wanted to share it. Other thing we can do to handle a mixture of old and new clients is when once the new broker with this KIP is deployed, the new code should check the request version from older producer we can up

[jira] [Commented] (KAFKA-4401) Change the KafkaServerTestHarness and IntegrationTestHarness from trait to abstract class.

2016-11-14 Thread Jiangjie Qin (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4401?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15665054#comment-15665054 ] Jiangjie Qin commented on KAFKA-4401: - [~ewencp] Interestingly, we have done the same thing as well

Re: [DISCUSS] KIP-82 - Add Record Headers

2016-11-14 Thread Ignacio Solis
1) Yes - Headers are worthwhile 2) Yes - Headers should be a top level option On Mon, Nov 14, 2016 at 9:16 AM, Michael Pearce wrote: > Hi Roger, > > The kip details/examples the original proposal for key spacing , not the > new mentioned as per discussion namespace idea.

Re: [VOTE] KIP-89: Allow sink connectors to decouple flush and offset commit

2016-11-14 Thread Shikhar Bhushan
The vote passed with +3 binding votes. Thanks all! On Sun, Nov 13, 2016 at 1:42 PM Gwen Shapira wrote: +1 (binding) On Nov 9, 2016 2:17 PM, "Shikhar Bhushan" wrote: > Hi, > > I would like to initiate a vote on KIP-89 > >

[jira] [Assigned] (KAFKA-4306) Connect workers won't shut down if brokers are not available

2016-11-14 Thread Konstantine Karantasis (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4306?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Konstantine Karantasis reassigned KAFKA-4306: - Assignee: Konstantine Karantasis (was: Ewen Cheslack-Postava) > Connect

[jira] [Assigned] (KAFKA-4154) Kafka Connect fails to shutdown if it has not completed startup

2016-11-14 Thread Konstantine Karantasis (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4154?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Konstantine Karantasis reassigned KAFKA-4154: - Assignee: Konstantine Karantasis (was: Shikhar Bhushan) > Kafka Connect

[jira] [Assigned] (KAFKA-3008) Connect should parallelize task start/stop

2016-11-14 Thread Konstantine Karantasis (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3008?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Konstantine Karantasis reassigned KAFKA-3008: - Assignee: Konstantine Karantasis (was: Liquan Pei) > Connect should

Re: [DISCUSS] KIP-87 - Add Compaction Tombstone Flag

2016-11-14 Thread Becket Qin
Michael, Yes, I am OK with stage 1. We can discuss about stage 2 later but this sounds really an organization specific decision to deprecate an API. It does not seem a general need in open source Kafka to only support tombstone bit , which is a bad thing for people who are still running old

Re: [VOTE] KIP-72 - Allow putting a bound on memory consumed by Incoming requests

2016-11-14 Thread Rajini Sivaram
+1 Thank you for the KIP, Radai. On Mon, Nov 14, 2016 at 6:07 PM, Mickael Maison wrote: > +1. We've also been hit by OOMs on the broker because we were not able > to properly bound its memory usage. > > On Mon, Nov 14, 2016 at 5:56 PM, radai

[jira] [Work started] (KAFKA-4393) Improve invalid/negative TS handling

2016-11-14 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4393?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Work on KAFKA-4393 started by Matthias J. Sax. -- > Improve invalid/negative TS handling >

Re: [DISCUSS] KIP-87 - Add Compaction Tombstone Flag

2016-11-14 Thread Mayuresh Gharat
I am not sure about "If I understand correctly, you want to let the broker to reject requests from old clients to ensure everyone in an organization has upgraded, right?" I don't think we will be rejecting requests. What phase2 (stage 2) meant was we will only do log compaction based on tombstone

Re: [DISCUSS] KIP-87 - Add Compaction Tombstone Flag

2016-11-14 Thread Michael Pearce
I'm ok with this, but I'd like to at least get phase 1 in, for the next release, this is what I'm very keen for, As such shall we say we have Kip-87a that delivers phase 1 And then Kip-87b in release after that delivers phase 2 but has a dependency on support of deprecating old api versions

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

2016-11-14 Thread Apache Jenkins Server
See Changes: [becket.qin] KAFKA-4409; Fix deadlock between topic event handling and shutdown in -- [...truncated 3875 lines...] kafka.utils.CommandLineUtilsTest > testParseSingleArg STARTED

Re: [DISCUSS] KIP-81: Max in-flight fetches

2016-11-14 Thread radai
+1 - there's is a need for an effective way to control kafka memory consumption - both on the broker and on clients. i think we could even reuse the exact same param name - *queued.max.bytes *- as it would serve the exact same purpose. also (and again its the same across the broker and clients)

Re: [DISCUSS] KIP-87 - Add Compaction Tombstone Flag

2016-11-14 Thread Becket Qin
Hey Michael and Mayuresh, If I understand correctly, you want to let the broker to reject requests from old clients to ensure everyone in an organization has upgraded, right? This is essentially deprecating an old protocol. I agree it is useful and that is why we have that baked in KIP-35.

[jira] [Updated] (KAFKA-4409) ZK consumer shutdown/topic event deadlock

2016-11-14 Thread Jiangjie Qin (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4409?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jiangjie Qin updated KAFKA-4409: Affects Version/s: 0.10.1.0 Fix Version/s: 0.10.1.1 Component/s: core

[jira] [Resolved] (KAFKA-4409) ZK consumer shutdown/topic event deadlock

2016-11-14 Thread Jiangjie Qin (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4409?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jiangjie Qin resolved KAFKA-4409. - Resolution: Fixed Assignee: Joel Koshy Reviewer: Jiangjie Qin > ZK consumer

Re: [VOTE] KIP-72 - Allow putting a bound on memory consumed by Incoming requests

2016-11-14 Thread Mickael Maison
+1. We've also been hit by OOMs on the broker because we were not able to properly bound its memory usage. On Mon, Nov 14, 2016 at 5:56 PM, radai wrote: > @rajini - fixed the hasBytesBuffered() method. also updated poll() so that > no latency is added for picking up

Re: [DISCUSS] KIP-81: Max in-flight fetches

2016-11-14 Thread Mickael Maison
Thanks for all the replies. I've updated the KIP: https://cwiki.apache.org/confluence/display/KAFKA/KIP-81%3A+Bound+Fetch+memory+usage+in+the+consumer The main point is to selectively read from sockets instead of throttling FetchRequests sends. I also mentioned it will be reusing the MemoryPool

Re: [VOTE] KIP-72 - Allow putting a bound on memory consumed by Incoming requests

2016-11-14 Thread radai
@rajini - fixed the hasBytesBuffered() method. also updated poll() so that no latency is added for picking up data stuck in ssl buffers (timeout is set to 0, just like with immediately connected keys and staged receives). thank you for pointing these out. added ssl (re) testing to the KIP testing

[jira] [Commented] (KAFKA-4409) ZK consumer shutdown/topic event deadlock

2016-11-14 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4409?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15664535#comment-15664535 ] ASF GitHub Bot commented on KAFKA-4409: --- GitHub user jjkoshy opened a pull request:

[GitHub] kafka pull request #2129: KAFKA-4409; Fix deadlock between topic event handl...

2016-11-14 Thread jjkoshy
GitHub user jjkoshy opened a pull request: https://github.com/apache/kafka/pull/2129 KAFKA-4409; Fix deadlock between topic event handling and shutdown in… The consumer can deadlock on shutdown if a topic event fires during shutdown. The shutdown acquires the rebalance lock and

[jira] [Created] (KAFKA-4409) ZK consumer shutdown/topic event deadlock

2016-11-14 Thread Joel Koshy (JIRA)
Joel Koshy created KAFKA-4409: - Summary: ZK consumer shutdown/topic event deadlock Key: KAFKA-4409 URL: https://issues.apache.org/jira/browse/KAFKA-4409 Project: Kafka Issue Type: Bug

Re: [DISCUSS] KIP-92 - Add per partition lag metrics to KafkaConsumer

2016-11-14 Thread Becket Qin
Hey Michael, Thanks for the comments. Exposing the lag on the client side may serve some cases a little different from monitoring. For example, one of the use case we have was that an application has some high priority and low priority topics to consume. They want to switch between consuming from

Re: [DISCUSS] KIP-92 - Add per partition lag metrics to KafkaConsumer

2016-11-14 Thread Michael Pearce
Should state I have no objections adding this client side, just more a question to why we don't look and propose to add this broker side also. Sent using OWA for iPhone From: Michael Pearce Sent: Monday, November 14, 2016 4:58:45

Re: [DISCUSS] KIP-82 - Add Record Headers

2016-11-14 Thread Michael Pearce
Hi Roger, The kip details/examples the original proposal for key spacing , not the new mentioned as per discussion namespace idea. We will need to update the kip, when we get agreement this is a better approach (which seems to be the case if I have understood the general feeling in the

[jira] [Commented] (KAFKA-4404) Add knowledge of sign to numeric schema types

2016-11-14 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4404?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15664439#comment-15664439 ] Ewen Cheslack-Postava commented on KAFKA-4404: -- What is the motivation for this? Support for

Re: [DISCUSS] KIP-92 - Add per partition lag metrics to KafkaConsumer

2016-11-14 Thread Michael Pearce
Why do we not look to expose the lag broker side centrally? Eg like burrow. >From an operations point it's a lot easier to monitor lag centrally than per >application. Also then you'd be able to see lag of consumers not alive or >stalled. The information if the consumer uses Kafka based or

Re: [DISCUSS] KIP-87 - Add Compaction Tombstone Flag

2016-11-14 Thread Michael Pearce
I agree with Mayuresh. I don't see how having a magic byte helps here. What we are saying is that on day 1 after an upgrade both tombstone flag or a null value will be treated as a marker to delete on compacted topic. During this time we expect organisations to migrate themselves over onto the

[jira] [Created] (KAFKA-4408) KTable doesn't work with ProcessorTopologyTestDriver in Kafka 0.10.1.0

2016-11-14 Thread Byron Nikolaidis (JIRA)
Byron Nikolaidis created KAFKA-4408: --- Summary: KTable doesn't work with ProcessorTopologyTestDriver in Kafka 0.10.1.0 Key: KAFKA-4408 URL: https://issues.apache.org/jira/browse/KAFKA-4408 Project:

[jira] [Created] (KAFKA-4407) Java consumer does not always send LEAVE_GROUP request during shut down

2016-11-14 Thread Igor (JIRA)
Igor created KAFKA-4407: --- Summary: Java consumer does not always send LEAVE_GROUP request during shut down Key: KAFKA-4407 URL: https://issues.apache.org/jira/browse/KAFKA-4407 Project: Kafka Issue

Re: [VOTE] KIP-72 - Allow putting a bound on memory consumed by Incoming requests

2016-11-14 Thread Rajini Sivaram
Open point 1. I would just retain the current long value that specifies queued.max.bytes as long and not as %heap since it is simple and easy to use. And keeps it consistent with other ".bytes" configs. Point 3. ssl buffers - I am not quite sure the implementation looks correct.

[VOTE] KIP-84: Support SASL SCRAM mechanisms

2016-11-14 Thread Rajini Sivaram
Hi all, I would like to initiate the voting process for *KIP-84: Support SASL/SCRAM mechanisms*: https://cwiki.apache.org/confluence/display/KAFKA/KIP-84%3A+Support+SASL+SCRAM+mechanisms This KIP adds support for four SCRAM mechanisms (SHA-224, SHA-256, SHA-384 and SHA-512) for SASL

[jira] [Created] (KAFKA-4406) Add support for custom Java Security Providers in configuration

2016-11-14 Thread Magnus Reftel (JIRA)
Magnus Reftel created KAFKA-4406: Summary: Add support for custom Java Security Providers in configuration Key: KAFKA-4406 URL: https://issues.apache.org/jira/browse/KAFKA-4406 Project: Kafka

[jira] [Resolved] (KAFKA-4268) huge server.logs during the error frequently happen "Message format version for partition 200 not found"

2016-11-14 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4268?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma resolved KAFKA-4268. Resolution: Duplicate Marking as a duplicate of KAFKA-4362 even though this was filed first as

[jira] [Created] (KAFKA-4405) Kafka consumer improperly send prefetch request

2016-11-14 Thread ysysberserk (JIRA)
ysysberserk created KAFKA-4405: -- Summary: Kafka consumer improperly send prefetch request Key: KAFKA-4405 URL: https://issues.apache.org/jira/browse/KAFKA-4405 Project: Kafka Issue Type: Bug