Re: RE: [DISCUSS] KIP-280: Enhanced log compaction

2018-04-20 Thread Luís Cabral
Guozhang, is this reply ok with you? If you insist on the byte[] comparison directly, then I would need some suggestions on how to represent a "version" with it, and then the KIP could be changed to that. On Tuesday, April 17, 2018, 2:44:16 PM GMT+2, Luís Cabral wrote: Oops, missed

[jira] [Created] (KAFKA-6810) Enable dynamic reconfiguration of SSL truststores

2018-04-20 Thread Rajini Sivaram (JIRA)
Rajini Sivaram created KAFKA-6810: - Summary: Enable dynamic reconfiguration of SSL truststores Key: KAFKA-6810 URL: https://issues.apache.org/jira/browse/KAFKA-6810 Project: Kafka Issue Type:

[jira] [Resolved] (KAFKA-5253) TopologyTestDriver must handle streams created with patterns

2018-04-20 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5253?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax resolved KAFKA-5253. Resolution: Fixed Fix Version/s: 1.2.0 > TopologyTestDriver must handle streams creat

Build failed in Jenkins: kafka-trunk-jdk7 #3358

2018-04-20 Thread Apache Jenkins Server
See -- Started by an SCM change [EnvInject] - Loading node environment variables. Building remotely on H24 (ubuntu xenial) in workspace

Build failed in Jenkins: kafka-trunk-jdk7 #3359

2018-04-20 Thread Apache Jenkins Server
See -- Started by an SCM change [EnvInject] - Loading node environment variables. Building remotely on H24 (ubuntu xenial) in workspace

[jira] [Resolved] (KAFKA-6797) Connect oracle database to kafka and stream data

2018-04-20 Thread Randall Hauch (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-6797?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Randall Hauch resolved KAFKA-6797. -- Resolution: Invalid Please use the Apache Kafka user discussion forum for questions. See https:

Build failed in Jenkins: kafka-trunk-jdk7 #3360

2018-04-20 Thread Apache Jenkins Server
See -- Started by an SCM change [EnvInject] - Loading node environment variables. Building remotely on H24 (ubuntu xenial) in workspace

Re: [DISCUSS]KIP-216: IQ should throw different exceptions for different errors

2018-04-20 Thread John Roesler
Hi Vito, Thanks for the KIP! I think it's much nicer to give callers different exceptions to tell them whether the state store got migrated, whether it's still initializing, or whether there's some unrecoverable error. In the KIP, it's typically not necessary to discuss non-user-facing details s

Re: RE: [DISCUSS] KIP-280: Enhanced log compaction

2018-04-20 Thread Guozhang Wang
Hi Luís, What I'm thinking primarily is that we only need to compare the compaction values as LONG for the offset and timestmap "type" (I still think it is worth defining `timestamp` as a special compaction value, with the reasons below). Not sure if you've seen my other comment earlier regarding

Re: [DISCUSS] KIP-286: producer.send() should not block on metadata update

2018-04-20 Thread Dong Lin
I am going to drop this KIP. Thinking about this more, the benefit of not having to wait for metadata does not seem to worth the complexity added in producer due to this KIP. Assuming that the Kafka cluster is available, which should be the case, it should be fast to wait for the first metadata. Af

Re: [DISCUSS]KIP-216: IQ should throw different exceptions for different errors

2018-04-20 Thread Guozhang Wang
Thanks for the KIP Vito! I made a pass over the wiki and it looks great to me. I'm +1 on the KIP. About the base class InvalidStateStoreException itself, I'd actually suggest we do not deprecate it but still expose it as part of the public API, for people who do not want to handle these cases dif

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

2018-04-20 Thread Apache Jenkins Server
See Changes: [jason] MINOR: Use distinct consumer groups in dynamic listener tests (#4870) -- [...truncated 1.48 MB...] kafka.zk.KafkaZkClientTest > testIsrChangeNotificat

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

2018-04-20 Thread Apache Jenkins Server
See

Build failed in Jenkins: kafka-trunk-jdk7 #3362

2018-04-20 Thread Apache Jenkins Server
See Changes: [jason] MINOR: Use distinct consumer groups in dynamic listener tests (#4870) -- [...truncated 412.98 KB...] kafka.admin.ResetConsumerGroupOffsetTest > testR

[jira] [Created] (KAFKA-6811) Tasks should have access to connector and task metadata

2018-04-20 Thread Jeremy Custenborder (JIRA)
Jeremy Custenborder created KAFKA-6811: -- Summary: Tasks should have access to connector and task metadata Key: KAFKA-6811 URL: https://issues.apache.org/jira/browse/KAFKA-6811 Project: Kafka

Re: [DISCUSS]KIP-216: IQ should throw different exceptions for different errors

2018-04-20 Thread Bill Bejeck
Hi Vito, Thanks for the KIP, overall it's a +1 from me. At this point, the only thing I would change is possibly removing the listing of all methods called by the user and the listing of all store types and focus on what states result in which exceptions thrown to the user. Thanks, Bill On Fri,

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

2018-04-20 Thread Apache Jenkins Server
See Changes: [jason] MINOR: Clarify meaning of end offset in consumer javadocs (#4885) -- [...truncated 1.48 MB...] kafka.zk.KafkaZkClientTest > testTopicAssignmentMethods

Build failed in Jenkins: kafka-trunk-jdk7 #3363

2018-04-20 Thread Apache Jenkins Server
See Changes: [jason] MINOR: Clarify meaning of end offset in consumer javadocs (#4885) -- [...truncated 414.77 KB...] kafka.api.DelegationTokenEndToEndAuthorizationTest >

Re: [VOTE] KIP-276 Add StreamsConfig prefix for different consumers

2018-04-20 Thread Bill Bejeck
+1 Thanks, Bill On Thu, Apr 19, 2018 at 9:50 AM, Boyang Chen wrote: > Thanks guys! > > Sent from my iPhone > > > On Apr 19, 2018, at 8:28 PM, Matthias J. Sax > wrote: > > > > +1 (binding) > > > > -Matthias > > > >> On 4/19/18 9:08 AM, Ted Yu wrote: > >> +1 > >> Original message --

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

2018-04-20 Thread Apache Jenkins Server
See Changes: [jason] MINOR: Clarify meaning of end offset in consumer javadocs (#4885) -- [...truncated 419.68 KB...] kafka.zookeeper.ZooKeeperClientTest > testGetChildren

Re: [VOTE] KIP-281: ConsumerPerformance: Increase Polling Loop Timeout and Make It Reachable by the End User

2018-04-20 Thread Dhruvil Shah
+1 from me as well. I was going to file a JIRA for this exact same problem. :-) On Thu, Apr 19, 2018 at 10:45 PM, Alex Dunayevsky wrote: > +1 > > > 4 votes total: > > 1 binding vote (Jason Gustafson) > > 3 non-binding votes (Moshe Blumberg, Ted Yu, Alex Dunayevsky) > > > Can we consider the

Re: [VOTE] KIP-281: ConsumerPerformance: Increase Polling Loop Timeout and Make It Reachable by the End User

2018-04-20 Thread Ted Yu
You need two more binding votes. FYI On Thu, Apr 19, 2018 at 10:45 PM, Alex Dunayevsky wrote: > +1 > > > 4 votes total: > > 1 binding vote (Jason Gustafson) > > 3 non-binding votes (Moshe Blumberg, Ted Yu, Alex Dunayevsky) > > > Can we consider the voting closed? > > > Thank you everyone! >