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

2018-08-15 Thread Jan Filipiak
even before message headers, the option for me always existed to just wrap the messages into my own custom envelop. So I of course thought this through. One sentence in your last email triggered all the thought process I put in the back then again to design it in the, what i think is the

Build failed in Jenkins: kafka-1.1-jdk7 #181

2018-08-15 Thread Apache Jenkins Server
See Changes: [matthias] KAFKA-7285: Create new producer on each rebalance if EOS enabled (#5501) -- [...truncated 1.93 MB...]

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

2018-08-15 Thread Brett Rann
> (segment.largestTimestamp is lastModified time of the log segment or max timestamp we see for the log segment. Due to the lack of record timestamp, segment.largestTimestamp might be earlier than the actual timestamp of latest record of that segment.). I'm curious about the mention of last

Build failed in Jenkins: kafka-trunk-jdk10 #408

2018-08-15 Thread Apache Jenkins Server
See Changes: [github] KAFKA-7285: Create new producer on each rebalance if EOS enabled (#5501) -- [...truncated 1.54 MB...]

[jira] [Created] (KAFKA-7299) batch LeaderAndIsr requests during auto preferred leader election

2018-08-15 Thread Jun Rao (JIRA)
Jun Rao created KAFKA-7299: -- Summary: batch LeaderAndIsr requests during auto preferred leader election Key: KAFKA-7299 URL: https://issues.apache.org/jira/browse/KAFKA-7299 Project: Kafka Issue

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

2018-08-15 Thread Brett Rann
An API was suggested by Gwen and James when I discussed it with them. For me I can think of it as a use case for scheduling compaction rather than relying on an config based time trigger. We're looking at creating some potentially very large compacted topics for event sourcing and from an

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

2018-08-15 Thread xiongqi wu
Brett, Thank you for your comments. I was thinking since we already has immediate compaction setting by setting min dirty ratio to 0, so I decide to use "0" as disabled state. I am ok to go with -1(disable), 0 (immediate) options. For the implementation, there are a few differences between mine

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

2018-08-15 Thread Brett Rann
Eno, For us as well the requirement is around compacted topics because they are the topics that already facilitate selective deletes. Currently they allow specifying a minimum life time, but lacks the ability to specify a maximum life time. For non compacted topics there's no ability to delete

Build failed in Jenkins: kafka-trunk-jdk10 #407

2018-08-15 Thread Apache Jenkins Server
See Changes: [wangguoz] MINOR: (re)add equals/hashCode to *Windows (#5510) -- [...truncated 1.98 MB...] org.apache.kafka.streams.StreamsConfigTest >

Build failed in Jenkins: kafka-1.1-jdk7 #180

2018-08-15 Thread Apache Jenkins Server
See Changes: [wangguoz] KAFKA-7284: streams should unwrap fenced exception (#5513) -- [...truncated 423.07 KB...] kafka.zk.KafkaZkClientTest >

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

2018-08-15 Thread Brett Rann
We've been looking into this too. Mailing list: https://lists.apache.org/thread.html/ed7f6a6589f94e8c2a705553f364ef599cb6915e4c3ba9b561e610e4@%3Cdev.kafka.apache.org%3E jira wish: https://issues.apache.org/jira/browse/KAFKA-7137 confluent slack discussion:

[jira] [Created] (KAFKA-7298) Concurrent DeleteRecords can lead to fatal OutOfSequence error in producer

2018-08-15 Thread Jason Gustafson (JIRA)
Jason Gustafson created KAFKA-7298: -- Summary: Concurrent DeleteRecords can lead to fatal OutOfSequence error in producer Key: KAFKA-7298 URL: https://issues.apache.org/jira/browse/KAFKA-7298

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

2018-08-15 Thread Yishun Guan
Hi, I am looking into AdminClient.scala and AdminClient.java, and also looking into ApiVersionRequest.java and ApiVersionResponse.java, but I don't see anywhere contains to logic of the one-to-one mapping from version to version, am i looking at the right place? On Mon, Aug 13, 2018 at 1:30 PM

[jira] [Created] (KAFKA-7297) Both read/write access to Log.segments should be protected by lock

2018-08-15 Thread Dong Lin (JIRA)
Dong Lin created KAFKA-7297: --- Summary: Both read/write access to Log.segments should be protected by lock Key: KAFKA-7297 URL: https://issues.apache.org/jira/browse/KAFKA-7297 Project: Kafka

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

2018-08-15 Thread xiongqi wu
Eno, Dong, I have updated the KIP. We decide not to address the issue that we might have for both compaction and time retention enabled topics (see the rejected alternative item 2). This KIP will only ensure log can be compacted after a specified time-interval. As suggested by Dong, we will

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

2018-08-15 Thread Guozhang Wang
I think Headers are not meant for user-space only: in fact, in KIP-258 there is also a proposal for compatibility of changelog topics relying on Headers. But you have a good point how to avoid conflicting with user header key space. I think there is no absolute-safe ways to avoid conflicts, but

[jira] [Created] (KAFKA-7296) Producer should handler COORDINATOR_LOADING error in TxnOffsetCommit

2018-08-15 Thread Jason Gustafson (JIRA)
Jason Gustafson created KAFKA-7296: -- Summary: Producer should handler COORDINATOR_LOADING error in TxnOffsetCommit Key: KAFKA-7296 URL: https://issues.apache.org/jira/browse/KAFKA-7296 Project:

Re: [VOTE] KIP-353: Allow Users to Configure Multi-Streams Timestamp Synchronization Behavior

2018-08-15 Thread Guozhang Wang
+1 from myself (binding). I'm closing this thread with 3 binding votes (Damian, Matthias, myself) and 3 non-binding votes (Bill, John, Ted). Thanks! Guozhang On Wed, Aug 15, 2018 at 1:18 AM, Damian Guy wrote: > +1 > > On Tue, 14 Aug 2018 at 19:35 Ted Yu wrote: > > > +1 > >

[jira] [Resolved] (KAFKA-6974) Changes the interaction between request handler threads and fetcher threads into an ASYNC model

2018-08-15 Thread Lucas Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-6974?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Lucas Wang resolved KAFKA-6974. --- Resolution: Won't Fix > Changes the interaction between request handler threads and fetcher threads

Re: Permission to create KIP

2018-08-15 Thread Matthias J. Sax
Done. On 8/15/18 8:55 AM, Attila Sasvári wrote: > Hi there, > > Can you please grant me permission to create a KIP? > > Wiki ID: asasvari > > Thanks, > Attila > signature.asc Description: OpenPGP digital signature

Re: [DISCUSS] KIP-325: Extend Consumer Group Command to Show Beginning Offsets

2018-08-15 Thread Vahid S Hashemian
In the absence of additional feedback to upvote one option against the other, I decided to roll the KIP back to an earlier version without the "partition size" support. I'll start another KIP to add the partition size info to kafka-topic command output, where I think would be a better fit. If

Re: [DISCUSS] KIP-346 - Limit blast radius of log compaction failure

2018-08-15 Thread Stanislav Kozlovski
Hi Jason, I was thinking about your suggestion. I agree that it makes sense to cap it at a certain threshold and it doesn't sound *too* restrictive to me either, considering the common case. The issue with the __consumer_offsets topic is problematic, that is true. Nevertheless, I have some

Re: [VOTE] KIP-344: The auto-generated client id should be passed to MetricsReporter

2018-08-15 Thread Kevin Lu
Hi All, Forgot to post this earlier, but I cancelled this KIP as we determined this is a bug fix. Could a committer review/merge the PR ( https://github.com/apache/kafka/pull/5383)? Colin has already finished and approved a review. Thanks! Regards, Kevin On Wed, Jul 25, 2018 at 10:12 AM Ted

Permission to create KIP

2018-08-15 Thread Attila Sasvári
Hi there, Can you please grant me permission to create a KIP? Wiki ID: asasvari Thanks, Attila

Anyone interested in helping out a little brother Apache project with their Kafka integration?

2018-08-15 Thread Christofer Dutz
Hi all, I am one of the Apache PLC4X (incubating) committers and am looking for people willing to help out with a little thing. PLC4X is aiming at industrial programmable logic controllers. So what we are doing is similar what JDBC did in the late 90s. We’ve implemented a universal API with

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

2018-08-15 Thread Apache Jenkins Server
See

Re: [DISCUSS]: KIP-339: Create a new ModifyConfigs API

2018-08-15 Thread Viktor Somogyi
Hi, To weigh-in, I agree with Colin on the API naming, overloads shouldn't change behavior. I think all of the Java APIs I've used so far followed this principle and I think we shouldn't diverge. Also I think I have an entry about this incremental thing in KIP-248. It died off a bit at voting (I

Jenkins build is back to normal : kafka-trunk-jdk10 #406

2018-08-15 Thread Apache Jenkins Server
See

[jira] [Resolved] (KAFKA-7288) Transient failure in SslSelectorTest.testCloseConnectionInClosingState

2018-08-15 Thread Rajini Sivaram (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-7288?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Rajini Sivaram resolved KAFKA-7288. --- Resolution: Fixed Reviewer: Jun Rao Fix Version/s: 2.1.0 > Transient failure

[jira] [Reopened] (KAFKA-7119) Intermittent test failure with GSSAPI authentication failure

2018-08-15 Thread Rajini Sivaram (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-7119?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Rajini Sivaram reopened KAFKA-7119: --- There was a test failure with the 1.0 build. It looks like we need to handle retriable Kerberos

Re: [VOTE] KIP-353: Allow Users to Configure Multi-Streams Timestamp Synchronization Behavior

2018-08-15 Thread Damian Guy
+1 On Tue, 14 Aug 2018 at 19:35 Ted Yu wrote: > +1 > Original message From: Bill Bejeck > Date: 8/14/18 11:09 AM (GMT-08:00) To: dev@kafka.apache.org Subject: > Re: [VOTE] KIP-353: Allow Users to Configure Multi-Streams Timestamp > Synchronization Behavior > +1 > > Thanks, >

Re: [VOTE] KIP-356: Add withCachingDisabled() to StoreBuilder

2018-08-15 Thread Damian Guy
+1 On Tue, 14 Aug 2018 at 22:58 Matthias J. Sax wrote: > +1 (binding) > > On 8/14/18 11:16 AM, Eno Thereska wrote: > > +1 (non binding) > > > > Thanks > > Eno > > > > On Tue, Aug 14, 2018 at 10:53 AM, Bill Bejeck wrote: > > > >> Thanks for the KIP. > >> > >> +1 > >> > >> -Bill > >> > >> On

Checking Connection with Kafka Broker from Client-side

2018-08-15 Thread Jorge Esteban Quilcate Otoya
Hi everyone, I'm evaluating how to validate connection to Kafka Brokers in an application that uses Consumer API by making health check using AdminClient. Is there any consideration around Authorization that I should take into consideration/any best practice? (I'm considering calling

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

2018-08-15 Thread Apache Jenkins Server
See Changes: [wangguoz] KAFKA-6761: Reduce streams footprint part IV add optimization (#5451) -- [...truncated 421.62 KB...]