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

2018-08-13 Thread Jan Filipiak
Sorry for missing the discussion -1 nonbinding see https://samza.apache.org/learn/documentation/0.7.0/api/javadocs/org/apache/samza/system/chooser/MessageChooser.html Best Jan On 14.08.2018 03:19, n...@afshartous.com wrote: Hi All, Calling for a vote on KIP-349

Build failed in Jenkins: kafka-2.0-jdk8 #116

2018-08-13 Thread Apache Jenkins Server
See Changes: [matthias] KAFKA-7284: streams should unwrap fenced exception (#5499) -- [...truncated 2.48 MB...]

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

2018-08-13 Thread Apache Jenkins Server
See Changes: [mjsax] KAFKA-7284: streams should unwrap fenced exception (#5499) [jason] MINOR: Use statically compiled regular expressions for efficiency --

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

2018-08-13 Thread Apache Jenkins Server
See Changes: [jason] MINOR: Remove AbstractFetcherThread.PartitionData (#5233) [rajinisivaram] KAFKA-7266: Fix MetricsTest.testMetrics flakiness using compression --

Re: [DISCUSS] KIP-354 Time-based log compaction policy

2018-08-13 Thread Dong Lin
Hey Xiongqi, Thanks for the KIP. I have two questions regarding the use-case for meeting GDPR requirement. 1) If I recall correctly, one of the GDPR requirement is that we can not keep messages longer than e.g. 30 days in storage (e.g. Kafka). Say there exists a partition p0 which contains

[VOTE] KIP-349 Priorities for Source Topics

2018-08-13 Thread nick
Hi All, Calling for a vote on KIP-349 https://cwiki.apache.org/confluence/display/KAFKA/KIP-349%3A+Priorities+for+Source+Topics -- Nick

[jira] [Resolved] (KAFKA-7284) Producer getting fenced may cause Streams to shut down

2018-08-13 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-7284?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax resolved KAFKA-7284. Resolution: Fixed > Producer getting fenced may cause Streams to shut down >

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

2018-08-13 Thread Jason Gustafson
I'm going to go ahead and call the vote. Here is the final tally: Binding: Dong Lin, Jun Rao, Jason Gustafson Non-binding: Anna Povzner Thanks everyone! -Jason On Mon, Aug 13, 2018 at 2:35 PM, Anna Povzner wrote: > +1 > > Thanks for the KIP! > > On Thu, Aug 9, 2018 at 5:16 PM Jun Rao wrote:

Re: [VOTE] KIP-280: Enhanced log compaction

2018-08-13 Thread Jason Gustafson
Hey Luis, Thanks for the explanation. I'd suggest adding the use case to the motivation section. I think my only hesitation about the header-based compaction is that it is the first time we are putting a schema requirement on header values. I wonder if it's better to leave Kafka agnostic. For

Re: [VOTE] KIP-289: Improve the default group id behavior in KafkaConsumer

2018-08-13 Thread Satish Duggana
+1 (non binding) Thanks Vahid, On Tue, Aug 14, 2018 at 2:22 AM, Gwen Shapira wrote: > +1 (binding) > > On Tue, Aug 7, 2018 at 11:14 AM, Vahid S Hashemian < > vahidhashem...@us.ibm.com> wrote: > > > Hi all, > > > > I'd like to start a vote on KIP-289 to modify the default group id of > >

[jira] [Created] (KAFKA-7286) Loading offsets and group metadata hangs with large group metadata records

2018-08-13 Thread Flavien Raynaud (JIRA)
Flavien Raynaud created KAFKA-7286: -- Summary: Loading offsets and group metadata hangs with large group metadata records Key: KAFKA-7286 URL: https://issues.apache.org/jira/browse/KAFKA-7286

Re: [DISCUSS] KIP-354 Time-based log compaction policy

2018-08-13 Thread xiongqi wu
Hi Eno, The GDPR request we are getting here at linkedin is if we get a request to delete a record through a null key on a log compacted topic, we want to delete the record via compaction in a given time period like 2 days (whatever is required by the policy). There might be other issues (such

Re: [VOTE] KIP-328: Ability to suppress updates for KTables

2018-08-13 Thread John Roesler
Hey all, I just wanted to let you know that a few small issues surfaced during implementation and review. I've updated the KIP. Here's the diff: https://cwiki.apache.org/confluence/pages/diffpagesbyversion.action?pageId=87295409=9=8 Basically: * the metrics named "*-event-*" are inconsistent

Re: Permission to create KIP

2018-08-13 Thread Jun Rao
Hi, Nikolay, Thanks for your interest. Just gave you the wiki permission. Jun On Mon, Aug 13, 2018 at 10:53 AM, Nikolay Izhikov wrote: > Hello, Guys. > > I want to create KIP for ticket [1]. > Please, give me sufficient permissions. > > My JIRA ID - nizhikov. > > [1]

Re: [DISCUSS] KIP-354 Time-based log compaction policy

2018-08-13 Thread xiongqi wu
Yes. we want to enforce a max time interval from a message arrival time to the time the corresponding log segment needs to be compacted. Today, if the message arriving rate is low for a log compacted topic, the dirty ratio increases very slowly. As a result, a log segment might be un-compacted

[jira] [Created] (KAFKA-7285) Streams should be more fencing-sensitive during task suspension under EOS

2018-08-13 Thread Guozhang Wang (JIRA)
Guozhang Wang created KAFKA-7285: Summary: Streams should be more fencing-sensitive during task suspension under EOS Key: KAFKA-7285 URL: https://issues.apache.org/jira/browse/KAFKA-7285 Project:

Re: [DISCUSS] KIP-354 Time-based log compaction policy

2018-08-13 Thread Eno Thereska
Hello, Thanks for the KIP. I'd like to see a more precise definition of what part of GDPR you are targeting as well as some sort of verification that this KIP actually addresses the problem. Right now I find this a bit vague: "Ability to delete a log message through compaction in a timely manner

Re: [DISCUSS] KIP-354 Time-based log compaction policy

2018-08-13 Thread Guozhang Wang
Guess I need to carefully read the wiki page before asking :) Thanks! Another qq after reading the proposal: is it a complimentary to KIP-58 ( https://cwiki.apache.org/confluence/display/KAFKA/KIP-58+-+Make+Log+Compaction+Point+Configurable), just that KIP-58 is a "upper-bound" on what messages

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

2018-08-13 Thread Apache Jenkins Server
See

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

2018-08-13 Thread Anna Povzner
+1 Thanks for the KIP! On Thu, Aug 9, 2018 at 5:16 PM Jun Rao wrote: > Hi, Jason, > > Thanks for the KIP. +1 from me. > > Jun > > On Wed, Aug 8, 2018 at 1:04 PM, Jason Gustafson > wrote: > > > Hi All, > > > > I'd like to start a vote for KIP-320: > >

Re: [DISCUSS] KIP-354 Time-based log compaction policy

2018-08-13 Thread xiongqi wu
HI Guozhang, As I mentioned in the motivation section, KIP-280 focuses on how to compact the log segment to resolve the out of order messages compaction issue. The issue we try to address in this KIP is different: we want to introduce a compaction policy so that a log segment can be pickup for

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

2018-08-13 Thread Lucas Wang
@Becket Makes sense. I've updated the KIP by adding the following paragraph to the motivation section > Today there is no separate between controller requests and regular data > plane requests. Specifically (1) a controller in a cluster uses the same > advertised endpoints to connect to brokers

Re: [VOTE] KIP-289: Improve the default group id behavior in KafkaConsumer

2018-08-13 Thread Gwen Shapira
+1 (binding) On Tue, Aug 7, 2018 at 11:14 AM, Vahid S Hashemian < vahidhashem...@us.ibm.com> wrote: > Hi all, > > I'd like to start a vote on KIP-289 to modify the default group id of > KafkaConsumer. > The KIP: > https://cwiki.apache.org/confluence/display/KAFKA/KIP- >

Re: [DISCUSS] KIP-354 Time-based log compaction policy

2018-08-13 Thread Guozhang Wang
Hello Xiongqi, I think this KIP is already been covered in KIP-280? Could you check out that one and see if it is the case. Guozhang On Mon, Aug 13, 2018 at 1:23 PM, xiongqi wu wrote: > Hi Kafka, > > Just updated the confluence page to include the link to this KIP. > > Any comment will be

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

2018-08-13 Thread Guozhang Wang
Regarding 3): Today we do not have this logic with the existing client, because defer the decision about the version to use (we always assume that an new versioned request need to be down-converted to a single old versioned request: i.e. an one-to-one mapping), but in principle, we should be able

Re: [DISCUSS] KIP-354 Time-based log compaction policy

2018-08-13 Thread xiongqi wu
Hi Kafka, Just updated the confluence page to include the link to this KIP. Any comment will be appreciated: https://cwiki.apache.org/confluence/display/KAFKA/KIP-354%3A+Time-based+log+compaction+policy Thank you. Xiongqi (Wesley) Wu On Thu, Aug 9, 2018 at 4:18 PM, xiongqi wu wrote: > Hi

[jira] [Created] (KAFKA-7284) Producer getting fenced may cause Streams to shut down

2018-08-13 Thread John Roesler (JIRA)
John Roesler created KAFKA-7284: --- Summary: Producer getting fenced may cause Streams to shut down Key: KAFKA-7284 URL: https://issues.apache.org/jira/browse/KAFKA-7284 Project: Kafka Issue

[jira] [Created] (KAFKA-7283) mmap indexes lazily and skip sanity check for segments below recovery point

2018-08-13 Thread Zhanxiang (Patrick) Huang (JIRA)
Zhanxiang (Patrick) Huang created KAFKA-7283: Summary: mmap indexes lazily and skip sanity check for segments below recovery point Key: KAFKA-7283 URL: https://issues.apache.org/jira/browse/KAFKA-7283

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

2018-08-13 Thread Yishun Guan
@Guozhang, thank you so much! 1. I agree, fixed. 2. Added. 3. I see, that is something that I haven't think about. How does Kafka handle other api's different version problem now? So we have a specific convertor that convect a new version request to a old version one for each API (is this what the

Re: [VOTE] KIP-289: Improve the default group id behavior in KafkaConsumer

2018-08-13 Thread Vahid S Hashemian
Thanks Colin, just updated the table. I had skipped them because they were listed in the earlier table in the KIP. --Vahid From: Colin McCabe To: dev@kafka.apache.org Date: 08/13/2018 09:45 AM Subject:Re: [VOTE] KIP-289: Improve the default group id behavior in

Permission to create KIP

2018-08-13 Thread Nikolay Izhikov
Hello, Guys. I want to create KIP for ticket [1]. Please, give me sufficient permissions. My JIRA ID - nizhikov. [1] https://issues.apache.org/jira/browse/KAFKA-7277 signature.asc Description: This is a digitally signed message part

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

2018-08-13 Thread Jason Gustafson
> > But in my opinion, since the client will fail with the API version, so we > don't need to down-convert the messages anyway. Isn't it? So, I think we > don't care about this case. (I'm sorry, I am not familiar with down-convert > logic.) Currently the broker down-converts automatically when

Re: [DISCUSS] Applying scalafmt to core code

2018-08-13 Thread Colin McCabe
On Wed, Aug 8, 2018, at 10:19, Ray Chiang wrote: > By doing piecemeal formatting, I don't think we can do a "hard" > enforcement on using scalafmt with every PR, but by allowing the tool to > run on already modified files in a patch, we can slowly migrate towards > getting the entire code base

Re: [VOTE] KIP-346 - Improve LogCleaner behavior on error

2018-08-13 Thread Dhruvil Shah
Thanks for the KIP, Stanislav! +1 (non-binding) - Dhruvil On Mon, Aug 13, 2018 at 9:39 AM Colin McCabe wrote: > +1 (non-binding) > > best, > Colin > > On Tue, Aug 7, 2018, at 04:19, Stanislav Kozlovski wrote: > > Hey everybody, > > I'm starting a vote on KIP-346 > > < >

Re: [VOTE] KIP-289: Improve the default group id behavior in KafkaConsumer

2018-08-13 Thread Colin McCabe
+1 (non-binding). Thanks, Vahid! One last question: can you fill in the behavior for the group.id="" case, in the table following "This is how these two group ids will work". It would be helpful to have a description of this behavior in the KIP for documentation purposes. cheers, Colin On

Re: [VOTE] KIP-346 - Improve LogCleaner behavior on error

2018-08-13 Thread Colin McCabe
+1 (non-binding) best, Colin On Tue, Aug 7, 2018, at 04:19, Stanislav Kozlovski wrote: > Hey everybody, > I'm starting a vote on KIP-346 > > > -- > Best, > Stanislav

Re: KIP-213 - Scalable/Usable Foreign-Key KTable joins - Rebooted.

2018-08-13 Thread Adam Bellemare
CC Jan On Mon, Aug 13, 2018 at 12:16 PM, Adam Bellemare wrote: > Hi Jan > > If you do not use headers or other metadata, how do you ensure that > changes to the foreign-key value are not resolved out-of-order? > ie: If an event has FK = A, but you change it to FK = B, you need to > propagate

Re: KIP-213 - Scalable/Usable Foreign-Key KTable joins - Rebooted.

2018-08-13 Thread Adam Bellemare
Hi Jan If you do not use headers or other metadata, how do you ensure that changes to the foreign-key value are not resolved out-of-order? ie: If an event has FK = A, but you change it to FK = B, you need to propagate both a delete (FK=A -> null) and an addition (FK=B). In my solution, without

[jira] [Resolved] (KAFKA-7257) --property (silently) fails when configuring SSL parameters

2018-08-13 Thread Manikumar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-7257?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar resolved KAFKA-7257. -- Resolution: Not A Problem > --property (silently) fails when configuring SSL parameters >

Re: I have issue in Kafka 2.0

2018-08-13 Thread Steve Tian
Have you checked the javadoc of KafkaConsumer? On Mon, Aug 13, 2018, 11:10 PM Kailas Biradar wrote: > I have issue more time this ConcurrentModificationException because > KafkaConsumer is not safe for multi-threaded access > > -- > kailas >

I have issue in Kafka 2.0

2018-08-13 Thread Kailas Biradar
I have issue more time this ConcurrentModificationException because KafkaConsumer is not safe for multi-threaded access -- kailas

Re: [DISCUSS] KIP-258: Allow to Store Record Timestamps in RocksDB

2018-08-13 Thread Eno Thereska
Hi Matthias, Good stuff. Could you comment a bit on how future-proof is this change? For example, if we want to store both event timestamp "and" processing time in RocksDB will we then need another interface (e.g. called KeyValueWithTwoTimestampsStore)? Thanks Eno On Thu, Aug 9, 2018 at 2:30

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

2018-08-13 Thread Becket Qin
Hi Lucas, Thanks for the explanation. It might be a nitpick, but it seems better to mention in the motivation part that today the client requests and controller requests are not only sharing the same queue, but also a bunch of things else, so that we can avoid asking people to read the rejected

Re: [DISCUSS] KIP-342 Add Customizable SASL extensions to OAuthBearer authentication

2018-08-13 Thread Stanislav Kozlovski
Hi, I've written up an initial implementation of what has been discussed. Take a look at it here: https://github.com/apache/kafka/pull/5497/ I will make sure to update the KIP once a review of the PR passes On Mon, Aug 13, 2018 at 10:19 AM Rajini Sivaram wrote: > Hi Stanislav, > > I think

[jira] [Created] (KAFKA-7282) Failed to read `log header` from file channel

2018-08-13 Thread Alastair Munro (JIRA)
Alastair Munro created KAFKA-7282: - Summary: Failed to read `log header` from file channel Key: KAFKA-7282 URL: https://issues.apache.org/jira/browse/KAFKA-7282 Project: Kafka Issue Type:

Build failed in Jenkins: kafka-2.0-jdk8 #115

2018-08-13 Thread Apache Jenkins Server
See Changes: [jason] KAFKA-7164; Follower should truncate after every missed leader epoch -- [...truncated 882.89 KB...] kafka.controller.PartitionStateMachineTest >

Re: [DISCUSS] KIP-342 Add Customizable SASL extensions to OAuthBearer authentication

2018-08-13 Thread Rajini Sivaram
Hi Stanislav, I think `token` and `extensions` on `OAuthBearerExtensionsValidatorCallback` should be immutable. The getters should return whatever was provided in the constructor and these should be stored as `final` objects. The whole point of separating out

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

2018-08-13 Thread Apache Jenkins Server
See Changes: [jason] KAFKA-7164; Follower should truncate after every missed leader epoch [jason] KAFKA-5638; Improve the Required ACL of ListGroups API (KIP-231) (#5352)

[jira] [Created] (KAFKA-7281) Fix documentation and error message regarding cleanup.policy=[compact,delete]

2018-08-13 Thread Patrik Kleindl (JIRA)
Patrik Kleindl created KAFKA-7281: - Summary: Fix documentation and error message regarding cleanup.policy=[compact,delete] Key: KAFKA-7281 URL: https://issues.apache.org/jira/browse/KAFKA-7281

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

2018-08-13 Thread Viktor Somogyi
Bumping. If there are no more thoughts on this for a few more days, I'll start a vote. (Linking the KIP to avoid people having to scroll back in the conversation: https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=87298242 ) Viktor On Thu, Aug 2, 2018 at 11:30 AM Viktor Somogyi

[jira] [Created] (KAFKA-7280) ConcurrentModificationException in FetchSessionHandler in heartbeat thread

2018-08-13 Thread Rajini Sivaram (JIRA)
Rajini Sivaram created KAFKA-7280: - Summary: ConcurrentModificationException in FetchSessionHandler in heartbeat thread Key: KAFKA-7280 URL: https://issues.apache.org/jira/browse/KAFKA-7280 Project:

Re: KIP-213 - Scalable/Usable Foreign-Key KTable joins - Rebooted.

2018-08-13 Thread Jan Filipiak
Hi, Happy to see that you want to make an effort here. Regarding the ProcessSuppliers I couldn't find a way to not rewrite the joiners + the merger. The re-partitioners can be reused in theory. I don't know if repartition is optimized in 2.0 now. I made this

[jira] [Resolved] (KAFKA-7164) Follower should truncate after every leader epoch change

2018-08-13 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-7164?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson resolved KAFKA-7164. Resolution: Fixed Fix Version/s: 2.1.0 2.0.1