[VOTE] KIP-223 - Add per-topic min lead and per-partition lead metrics to KafkaConsumer

2017-11-29 Thread Hu Xi
Hi all, As I didn't see any further discussion around this KIP, I'd like to start voting. KIP documentation: https://cwiki.apache.org/confluence/display/KAFKA/KIP-223+-+Add+per-topic+min+lead+and+per-partition+lead+metrics+to+KafkaConsumer Cheers, huxihx

Re: [DISCUSS] KIP-227: Introduce Incremental FetchRequests to Increase Partition Scalability

2017-11-29 Thread Dong Lin
Hey Colin, Thanks much for the update. I have a few questions below: 1. I am not very sure that we need Fetch Session Epoch. It seems that Fetch Session Epoch is only needed to help leader distinguish between "a full fetch request" and "a full fetch request and request a new incremental fetch ses

Jenkins build is back to normal : kafka-trunk-jdk9 #228

2017-11-29 Thread Apache Jenkins Server
See

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

2017-11-29 Thread Apache Jenkins Server
See

[jira] [Created] (KAFKA-6286) Consider reusing materialized store for multi-same-stream join

2017-11-29 Thread Guozhang Wang (JIRA)
Guozhang Wang created KAFKA-6286: Summary: Consider reusing materialized store for multi-same-stream join Key: KAFKA-6286 URL: https://issues.apache.org/jira/browse/KAFKA-6286 Project: Kafka

[GitHub] kafka pull request #3912: KAFKA-5936: KafkaProducer.close should throw Inter...

2017-11-29 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/3912 ---

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

2017-11-29 Thread Apache Jenkins Server
See

[jira] [Reopened] (KAFKA-4669) KafkaProducer.flush hangs when NetworkClient.handleCompletedReceives throws exception

2017-11-29 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4669?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson reopened KAFKA-4669: > KafkaProducer.flush hangs when NetworkClient.handleCompletedReceives throws > exception > ---

Re: [VOTE] KIP-203: Add toLowerCase support to sasl.kerberos.principal.to.local rule

2017-11-29 Thread Jason Gustafson
+1 Thanks for the KIP! On Fri, Nov 3, 2017 at 10:08 AM, Manikumar wrote: > Bump up. waiting for few more binding votes. > > On Wed, Oct 18, 2017 at 6:57 PM, Rajini Sivaram > wrote: > > > +1 (binding) > > > > On Mon, Oct 9, 2017 at 5:32 PM, Manikumar > > wrote: > > > > > I'm bumping this up to

Re: [DISCUSS] KIP-231: Improve the Required ACL of ListGroups API

2017-11-29 Thread Vahid S Hashemian
Completing the subject line :) From: "Vahid S Hashemian" To: dev Date: 11/29/2017 03:17 PM Subject:[DISCUSS] KIP-231: Hi everyone, I started KIP-231 to propose a small change to the required ACL of ListGroups API (in response to KAFKA-5638): https://urldefense.proofpoint.

[DISCUSS] KIP-231:

2017-11-29 Thread Vahid S Hashemian
Hi everyone, I started KIP-231 to propose a small change to the required ACL of ListGroups API (in response to KAFKA-5638): https://cwiki.apache.org/confluence/display/KAFKA/KIP-231%3A+Improve+the+Required+ACL+of+ListGroups+API Your feedback and suggestions are welcome! Thanks. --Vahid

Re: java.lang.IllegalStateException: Correlation id for response () does not match request ()

2017-11-29 Thread Aarti Gupta
https://issues.apache.org/jira/browse/KAFKA-4669?focusedCommentId=16271727&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-16271727 Can we reopen this issue ? We saw this on the consumer in production today. We are on 0.11.01 ERROR c.v.v.h.m.k.KafkaEventConsumerDeleg

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

2017-11-29 Thread Apache Jenkins Server
See

Re: [DISCUSS] KIP-227: Introduce Incremental FetchRequests to Increase Partition Scalability

2017-11-29 Thread Colin McCabe
I updated the KIP with the ideas we've been discussing. best, Colin On Tue, Nov 28, 2017, at 08:38, Colin McCabe wrote: > On Mon, Nov 27, 2017, at 22:30, Jan Filipiak wrote: > > Hi Colin, thank you for this KIP, it can become a really useful thing. > > > > I just scanned through the discussion

Jenkins build is back to normal : kafka-0.10.2-jdk7 #199

2017-11-29 Thread Apache Jenkins Server
See

[jira] [Created] (KAFKA-6285) OffsetCommitRequest should have read-after-write logic

2017-11-29 Thread Dong Lin (JIRA)
Dong Lin created KAFKA-6285: --- Summary: OffsetCommitRequest should have read-after-write logic Key: KAFKA-6285 URL: https://issues.apache.org/jira/browse/KAFKA-6285 Project: Kafka Issue Type: Bug

[jira] [Created] (KAFKA-6284) System Test failed: ConnectRestApiTest

2017-11-29 Thread Mikkin Patel (JIRA)
Mikkin Patel created KAFKA-6284: --- Summary: System Test failed: ConnectRestApiTest Key: KAFKA-6284 URL: https://issues.apache.org/jira/browse/KAFKA-6284 Project: Kafka Issue Type: Bug

[GitHub] kafka pull request #4277: STRUCT schema equality check failing for complex o...

2017-11-29 Thread ntrp
GitHub user ntrp opened a pull request: https://github.com/apache/kafka/pull/4277 STRUCT schema equality check failing for complex objects *The schema equality need to be done with the equals method to avoid unwanted side effects. When applied to complex objects with the same schema

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

2017-11-29 Thread Apache Jenkins Server
See Changes: [me] KAFKA-4827: Correctly encode special chars while creating URI objects -- [...truncated 81.02 KB...] kafka.server.DelayedOperationTest > testRequestPurge

[GitHub] kafka pull request #4273: KAFKA-4827: Porting fix for KAFKA-4827 to v0.10 an...

2017-11-29 Thread wicknicks
Github user wicknicks closed the pull request at: https://github.com/apache/kafka/pull/4273 ---

[GitHub] kafka pull request #4255: KAFKA-6259: Make KafkaStreams.cleanup() clean glob...

2017-11-29 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/4255 ---

[GitHub] kafka pull request #4272: KAFKA-4827: Porting fix for KAFKA-4827 to v0.10 an...

2017-11-29 Thread wicknicks
Github user wicknicks closed the pull request at: https://github.com/apache/kafka/pull/4272 ---

[GitHub] kafka pull request #4276: KAFKA-6260: Ensure selection keys are removed from...

2017-11-29 Thread hachikuji
GitHub user hachikuji opened a pull request: https://github.com/apache/kafka/pull/4276 KAFKA-6260: Ensure selection keys are removed from all collections on socket close When a socket is closed, we must remove corresponding selection keys from internal collections. This fixes an NP

Re: [DISCUSS] KIP-229: DeleteGroups API

2017-11-29 Thread Vahid S Hashemian
Hi Dong, Thanks a lot for your feedback. I updated the KIP and included those fields and also made a note of the corresponding AdminClient API that will be created to support deleting consumer groups. I hope the updates address your suggestions. Cheers! --Vahid From: Dong Lin To: de

[jira] [Resolved] (KAFKA-6038) Repartition topics could be much more transient

2017-11-29 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-6038?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang resolved KAFKA-6038. -- Resolution: Duplicate > Repartition topics could be much more transient > --

[GitHub] kafka pull request #4258: [KAFKA-4499] Add all() and fetchAll() API for quer...

2017-11-29 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/4258 ---

[jira] [Resolved] (KAFKA-4499) Add "getAllKeys" API for querying windowed KTable stores

2017-11-29 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4499?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang resolved KAFKA-4499. -- Resolution: Fixed Fix Version/s: 1.1.0 Issue resolved by pull request 4258 [https://githu

[GitHub] kafka pull request #4275: Close socketChannel in finally block

2017-11-29 Thread cmccabe
Github user cmccabe closed the pull request at: https://github.com/apache/kafka/pull/4275 ---

[GitHub] kafka pull request #4275: Close socketChannel in finally block

2017-11-29 Thread cmccabe
GitHub user cmccabe opened a pull request: https://github.com/apache/kafka/pull/4275 Close socketChannel in finally block You can merge this pull request into a Git repository by running: $ git pull https://github.com/cmccabe/kafka KAFKA-6260 Alternatively you can review and

[GitHub] kafka pull request #4274: KAFKA-6283: Configuration of custom SCRAM SaslServ...

2017-11-29 Thread tombentley
GitHub user tombentley opened a pull request: https://github.com/apache/kafka/pull/4274 KAFKA-6283: Configuration of custom SCRAM SaslServer implementations Pass the jaasContext to the ScramServerCallbackHandler, so that custom implementations of a SCRAM SaslServer have access to th

Re: 答复: REPLY: [DISCUSS]KIP-223 - Add per-topic min lead and per-partition lead metrics to KafkaConsumer

2017-11-29 Thread Jun Rao
Hi, Hu, Yes. Could you start a separate voting thread? Thanks, Jun On Tue, Nov 28, 2017 at 5:02 PM, Hu Xi wrote: > Hi Rao Jun, > > > Already updated the patch per your suggestion and it seems there are no > further feedbacks on this KIP. Could we vote now? > > > _

Re: Plans to extend streams?

2017-11-29 Thread Wim Van Leuven
What you are actually asking is if Kafka Streams should be reimplemented as Apache Storm? -wim On Wed, 29 Nov 2017 at 15:10 Adrienne Kole wrote: > Hi, > > The purpose of this email is to get overall intuition for the future plans > of streams library. > > The main question is that, will it be a

[jira] [Created] (KAFKA-6283) Configuration of custom SCRAM SaslServer implementations

2017-11-29 Thread Tom Bentley (JIRA)
Tom Bentley created KAFKA-6283: -- Summary: Configuration of custom SCRAM SaslServer implementations Key: KAFKA-6283 URL: https://issues.apache.org/jira/browse/KAFKA-6283 Project: Kafka Issue Type

Re: Flaky healthcheck when trying to check Kafka Stream processing app status

2017-11-29 Thread Bill Bejeck
Patrice, Thanks for reporting this. I'll have a look at what you've posted on Github. Thanks, Bill On Wed, Nov 29, 2017 at 7:04 AM, Patrice Chalcol wrote: > Hello, > > I have implemented a basic application which uses kafka streams stores and > interactive queries, available there : > https:/

Re: Plans to extend streams?

2017-11-29 Thread Adrienne Kole
Not necessarily. I would avoid the term "reimplemented". Btw, Apache Storm is not also the best (streaming) system that can utilize the network and it does not support runtime scale in/out (at least by design). So, can streams preserve its current selling points (ex:dynamicity) while introducing a

[jira] [Created] (KAFKA-6282) exactly_once semantics breaks demo application

2017-11-29 Thread Romans Markuns (JIRA)
Romans Markuns created KAFKA-6282: - Summary: exactly_once semantics breaks demo application Key: KAFKA-6282 URL: https://issues.apache.org/jira/browse/KAFKA-6282 Project: Kafka Issue Type: Bu

Plans to extend streams?

2017-11-29 Thread Adrienne Kole
Hi, The purpose of this email is to get overall intuition for the future plans of streams library. The main question is that, will it be a single threaded application in the long run and serve microservices use-cases, or are there any plans to extend it to multi-node execution framework with les

Re: [DISCUSS]: KIP-230: Name Windowing Joins

2017-11-29 Thread Matt Farmer
Hi Matthias, I certainly have found the auto-generated names unwieldy while doing cluster administration. I will point out that your KIP doesn't outline what would happen if you picked a name that resulted in a non unique topic name? What would be the error handling behavior there? On Wed, Nov 2

[DISCUSS]: KIP-230: Name Windowing Joins

2017-11-29 Thread Matthias Margush
Hi everyone, I created this KIP to allow windowing joins to be named. If named, then the associated internal topic names would be derived from that, instead of being randomly generated. https://cwiki.apache.org/confluence/display/KAFKA/KIP+230%3A+Name+Windowing+Joins Thanks, Matthias

Re: [DISCUSS]: KIP-159: Introducing Rich functions to Streams

2017-11-29 Thread Jan Filipiak
Hi, thank you for the summary and thanks for acknowledging that I do have a point here. I don't like the second Idea at all. Hence I started of this discussion. I am just disappointed, back then when we had the discussion about how to refactor store overload and IQ handling, I knew the path

Flaky healthcheck when trying to check Kafka Stream processing app status

2017-11-29 Thread Patrice Chalcol
Hello, I have implemented a basic application which uses kafka streams stores and interactive queries, available there : https://github.com/pchalcol/kstreams-healthcheck The healthcheck implementation is based on kafka streams metadata and the stream state, as illustrated below : ``` String healt

Re: [DISCUSS] KIP 226 - Dynamic Broker Configuration

2017-11-29 Thread Rajini Sivaram
Hi Jason, Thanks for reviewing the KIP. I hadn't included *inter.broker.protocol.version*, but you have provided a good reason to do that in order to avoid an additional rolling restart during upgrade. I had included *log.message.format.version* along with other default topic configs, but it prob

Re: [VOTE] KIP-225 - Use tags for consumer “records.lag” metrics

2017-11-29 Thread charly molter
Hi, Just a reminder that this vote is open. Let me know if you think this is not a valuable change. Thanks! On Wed, Nov 22, 2017 at 10:48 AM, charly molter wrote: > > Hi, > > I would like to start the voting thread for KIP-225. > This KIP proposes to correct some lag metrics emitted by the con

[jira] [Created] (KAFKA-6281) Kafka JavaAPI Producer failed with NotLeaderForPartitionException

2017-11-29 Thread Anil (JIRA)
Anil created KAFKA-6281: --- Summary: Kafka JavaAPI Producer failed with NotLeaderForPartitionException Key: KAFKA-6281 URL: https://issues.apache.org/jira/browse/KAFKA-6281 Project: Kafka Issue Type: Bu