[jira] [Resolved] (KAFKA-6457) Error: NOT_LEADER_FOR_PARTITION leads to NPE

2018-09-06 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-6457?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax resolved KAFKA-6457. Resolution: Duplicate > Error: NOT_LEADER_FOR_PARTITION leads to NPE >

[jira] [Reopened] (KAFKA-6457) Error: NOT_LEADER_FOR_PARTITION leads to NPE

2018-09-06 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-6457?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax reopened KAFKA-6457: > Error: NOT_LEADER_FOR_PARTITION leads to NPE >

Re: [DISCUSS] KIP-361: Add Consumer Configuration to Disable Auto Topic Creation

2018-09-06 Thread Matthias J. Sax
What is the status of this KIP? I think you can start a VOTE Dhruvil. -Matthias On 8/23/18 9:52 AM, Ismael Juma wrote: > Yeah, the reason why we want to deprecate the auto create functionality is > that it happens when a metadata request is done instead of when a write > operation happens. So,

Re: [DISCUSS] KIP-362: Dynamic Session Window Support

2018-09-06 Thread Matthias J. Sax
I cannot follow the example: >> (10, 10), (15, 3), (19, 5) ... First, [10,10] is created, second the window is extended to [10,15], and third [19,19] is created. Why would there be a [15,15]? And why would (19,5) be merged into [15,15] -- the gap was set to 3 via (15,3) and thus [19,19] should

Re: [DISCUSS] KIP-367 Introduce close(Duration) to Producer instead of close(long, TimeUnit)

2018-09-06 Thread Matthias J. Sax
Thanks for the KIP. It's a little hard to read -- it's easier if you just list the methods (without JavaDocs) and indicate if the get deprecated or added. Please don't show a diff as in a patch :) Is there already a JIRA for this? If not, please create on and link it in the KIP. Besides this, I

Re: [DISCUSS] KIP-360: Improve handling of unknown producer

2018-09-06 Thread Matthias J. Sax
I am +1 on this :) -Matthias On 9/4/18 9:55 AM, Jason Gustafson wrote: > Bump. Thanks to Magnus for noticing that I forgot to link to the KIP: > https://cwiki.apache.org/confluence/display/KAFKA/KIP-360%3A+Improve+handling+of+unknown+producer > . > > -Jason > > On Tue, Aug 21, 2018 at 4:37

Re: [DISCUSS] KIP-358: Migrate Streams API to Duration instead of long ms times

2018-09-06 Thread Matthias J. Sax
Thanks for updating the KIP! Couple of minor follow ups: (1) Some methods declare `throws IllegalArgumentException`, others don't. It's runtime exception and thus it's not required to declare it -- it just looks inconsistent in the KIP and maybe it's inconsistent in the code, too. I am not sure

Re: [DISCUSS] KIP-349 Priorities for Source Topics

2018-09-06 Thread Matthias J. Sax
I am still not sure how Samza's MessageChooser actually works and how this would align with KafkaConsumer fetch requests. Maybe I can give some background (conceptually); @Colin, please correct me if I say anything wrong: When a fetch request is send, all assigned topic partitions of the

Re: [VOTE] KIP-366 - Make FunctionConversations private

2018-09-06 Thread Matthias J. Sax
Can you please update the KIP accordingly? It still says "make private" instead of "deprecating" -Matthias On 9/6/18 10:07 AM, Attila Sasvári wrote: > +1 (non-binding) > > On Thu, Sep 6, 2018 at 6:38 PM Guozhang Wang wrote: > >> +1 for deprecating and copying the class over to internals. >>

Re: [DISCUSS] KIP-347: Enable batching in FindCoordinatorRequest

2018-09-06 Thread Yishun Guan
Hi Collin and Guozhang, I see. But even if the fall-back logic goes into AdminClient and ConsumerClient, it still have to be somehow type specific right? So either way, there will be api-specific process code somewhere? Thanks, Yishun On Tue, Sep 4, 2018 at 5:46 PM Colin McCabe wrote: > > Hi

Re: [VOTE] KIP-291: Have separate queues for control requests and data requests

2018-09-06 Thread Lucas Wang
@Jun Rao One clarification, currently on the selector level, we have the io-wait-ratio metric. For the new controller *network* thread, we can use it directly for IdlePct, instead of using 1- io-ratio, so that the logic is similar to the current average IdlePct for network threads. Is that

Re: [VOTE] KIP-320: Allow fetchers to detect and handle log truncation

2018-09-06 Thread Jason Gustafson
Hey All, One update while implementing this. To go along with the addition of the leader epoch to the OffsetCommit API, we have also updated TxnOffsetCommit. See the KIP for the changes: https://cwiki.apache. org/confluence/display/KAFKA/KIP-320%3A+Allow+fetchers+to+

[ANNOUNCE] Kafka Committers Guidelines

2018-09-06 Thread Guozhang Wang
Hello folks, Many of the Kafka PMC members have received emails asking about "how can I become a committer in Apache Kafka", and so we figured it is a good time to publish concrete guidelines on this regard. The PMC has discussed and voted on a short list of areas that we are looking for

Re: [VOTE] KIP-336: Consolidate ExtendedSerializer/Serializer and ExtendedDeserializer/Deserializer

2018-09-06 Thread Ismael Juma
Thanks, +1 (binding). On Mon, Sep 3, 2018 at 6:28 AM Viktor Somogyi-Vass wrote: > Apologies, miscounted the binding votes but the good news is that we need > only one. > > Cheers, > Viktor > > On Mon, Sep 3, 2018 at 11:09 AM Viktor Somogyi-Vass < > viktorsomo...@gmail.com> > wrote: > > > Hi

Re: [DISCUSS] KIP 368: Allow SASL Connections to Periodically Re-Authenticate

2018-09-06 Thread Rajini Sivaram
Hi Ron, The commit https://github.com/rajinisivaram/kafka/commit/b9d711907ad843c11d17e80d6743bfb1d4e3f3fd shows the kind of flow I was thinking of. It is just a prototype with a fixed re-authentication period to explore the possibility of implementing re-authentication similar to authentication.

[jira] [Created] (KAFKA-7383) Verify leader epoch in produce requests (KIP-359)

2018-09-06 Thread Jason Gustafson (JIRA)
Jason Gustafson created KAFKA-7383: -- Summary: Verify leader epoch in produce requests (KIP-359) Key: KAFKA-7383 URL: https://issues.apache.org/jira/browse/KAFKA-7383 Project: Kafka Issue

Re: [VOTE] KIP-359: Verify leader epoch in produce requests

2018-09-06 Thread Jason Gustafson
Thanks everyone for the reviews! Here is the final tally: Binding: +4 (Me, Dong Lin, Matthias Sax, Jun Rao) Non-binding: +4 (Dongjin Lee, Satish Duggana, Manikumar Reddy, Attila Sasvári) -Jason On Thu, Sep 6, 2018 at 11:37 AM, Jun Rao wrote: > Hi, Jason, > > Thanks for the KIP. Well

Re: [VOTE] KIP-359: Verify leader epoch in produce requests

2018-09-06 Thread Jun Rao
Hi, Jason, Thanks for the KIP. Well documented. +1 Jun On Thu, Aug 30, 2018 at 3:56 PM, Jason Gustafson wrote: > Hi All, > > I'd like to start the vote on KIP-359: https://cwiki.apache.org/ > confluence/display/KAFKA/KIP-359%3A+Verify+leader+epoch+in+ > produce+requests. > Thanks in advance

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

2018-09-06 Thread Jason Gustafson
Hi Dongjin, The KIP looks good to me. I'd suggest starting a vote. A couple minor points that might be worth calling out explicitly in the compatibility section: 1. Zstd will only be allowed for the bumped produce API. For older versions, we return UNSUPPORTED_COMPRESSION_TYPE regardless of the

Re: [VOTE] KIP-366 - Make FunctionConversations private

2018-09-06 Thread Attila Sasvári
+1 (non-binding) On Thu, Sep 6, 2018 at 6:38 PM Guozhang Wang wrote: > +1 for deprecating and copying the class over to internals. > > On Thu, Sep 6, 2018 at 6:56 AM, Bill Bejeck wrote: > > > +1 > > > > -Bill > > > > On Thu, Sep 6, 2018 at 4:29 AM Joan Goyeau wrote: > > > > > Sournds good,

Re: [VOTE] KIP-366 - Make FunctionConversations private

2018-09-06 Thread Guozhang Wang
+1 for deprecating and copying the class over to internals. On Thu, Sep 6, 2018 at 6:56 AM, Bill Bejeck wrote: > +1 > > -Bill > > On Thu, Sep 6, 2018 at 4:29 AM Joan Goyeau wrote: > > > Sournds good, I'll make the deprecation and copy the class over. > > > > Thanks > > > > On Wed, 5 Sep 2018

[jira] [Resolved] (KAFKA-5882) NullPointerException in StreamTask

2018-09-06 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5882?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang resolved KAFKA-5882. -- Resolution: Fixed Fix Version/s: 1.1.1 > NullPointerException in StreamTask >

Re: [VOTE] KIP-366 - Make FunctionConversations private

2018-09-06 Thread Bill Bejeck
+1 -Bill On Thu, Sep 6, 2018 at 4:29 AM Joan Goyeau wrote: > Sournds good, I'll make the deprecation and copy the class over. > > Thanks > > On Wed, 5 Sep 2018 at 22:48 John Roesler wrote: > > > I'm a +1 (non-binding) because we doubt the class is in use. > > > > If you decide to copy it to a

Re: [DISCUSS] KIP 368: Allow SASL Connections to Periodically Re-Authenticate

2018-09-06 Thread Ron Dagostino
Yeah, regarding ControllerChannelManager, it is one of the synchronous I/O use cases (along with 2 others: KafkaProducer, via Sender; and ReplicaFetcherBlockingSend, via ReplicaFetcherThread) where the assumption is complete ownership of the connection. The PR's approach to dealing with that

Re: [DISCUSS] KIP-349 Priorities for Source Topics

2018-09-06 Thread Jan Filipiak
On 05.09.2018 17:18, Colin McCabe wrote: Hi all, I agree that DISCUSS is more appropriate than VOTE at this point, since I don't remember the last discussion coming to a definite conclusion. I guess my concern is that this will add complexity and memory consumption on the server side. In

[jira] [Created] (KAFKA-7382) We shoud guarantee at lest one replica of partition should be alive when create or update topic

2018-09-06 Thread zhaoshijie (JIRA)
zhaoshijie created KAFKA-7382: - Summary: We shoud guarantee at lest one replica of partition should be alive when create or update topic Key: KAFKA-7382 URL: https://issues.apache.org/jira/browse/KAFKA-7382

Re: [DISCUSS] KIP 368: Allow SASL Connections to Periodically Re-Authenticate

2018-09-06 Thread Rajini Sivaram
Hi Ron, Disconnections on the broker-side: I think we should do disconnections as a separate KIP and PR as you originally intended. But that one could be done separately without requiring KIP-368 as a pre-req. As a simpler implementation and one that can be used without KIP-368 in some cases, we

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

2018-09-06 Thread Magnus Edenhill
> Ismael wrote: > Jason, that's an interesting point regarding the Java client. Do we know > what clients in other languages do in these cases? librdkafka (and its bindings) passes unknown/future errors through to the application, the error code remains intact while the error string will be set

Re: [VOTE] KIP-366 - Make FunctionConversations private

2018-09-06 Thread Joan Goyeau
Sournds good, I'll make the deprecation and copy the class over. Thanks On Wed, 5 Sep 2018 at 22:48 John Roesler wrote: > I'm a +1 (non-binding) because we doubt the class is in use. > > If you decide to copy it to a private version and deprecate the original > instead, as Matthias suggested,

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

2018-09-06 Thread Dongjin Lee
I updated the KIP page following the discussion here. Please take a look when you are free. If you have any opinion, don't hesitate to give me a message. Best, Dongjin On Fri, Aug 31, 2018 at 11:35