Re: [DISCUSS] KIP-379: Multiple Consumer Group Management

2018-10-18 Thread Jason Gustafson
Hi Alex, Thanks for the KIP. I think it makes sense, especially since most of the group apis are intended for batching anyway. The only questions I have are about compatibility. For example, the csv format for resetting offsets is changed, so will we continue to support the old format? Also, if o

Re: [DISCUSS] KIP-382: MirrorMaker 2.0

2018-10-18 Thread Harsha Chintalapani
Ryanne,        Makes sense. Can you please add this under rejected alternatives so that everyone has context on why it  wasn’t picked. Thanks, Harsha On Oct 18, 2018, 8:02 AM -0700, Ryanne Dolan , wrote: > Harsha, concerning uReplicator specifically, the project is a major > inspiration for MM2,

Re: [VOTE] KIP-376: Implement AutoClosable on appropriate classes that want to be used in a try-with-resource statement

2018-10-18 Thread Harsha Chintalapani
LGTM. +1 (binding) -- Harsha On Oct 18, 2018, 5:53 PM -0700, Matthias J. Sax , wrote: > +1 (binding) > > Thanks for the KIP. And thanks for bumping the thread regularly. As > 2.1.0 and 2.0.1 releases are running atm, it takes some time to get > attention. > > -Matthias > > On 10/18/18 11:47 AM,

Re: Request for contributor permissions

2018-10-18 Thread Gwen Shapira
Done :) Welcome on board. On Thu, Oct 18, 2018 at 6:58 AM Kent wrote: > JIRA ID: cjfan2...@gmail.com > Cwiki ID: cjfan2...@gmail.com > > -- > Best Regards, > > Kent Fan > -- *Gwen Shapira* Product Manager | Confluent 650.450.2760 | @gwenshap Follow us: Twitter

Request for contributor permissions

2018-10-18 Thread Stone Huang
JIRA ID: godisren Cwiki ID: Ren Jr Huang Thanks in advance!

Re: [VOTE] KIP-376: Implement AutoClosable on appropriate classes that want to be used in a try-with-resource statement

2018-10-18 Thread Matthias J. Sax
+1 (binding) Thanks for the KIP. And thanks for bumping the thread regularly. As 2.1.0 and 2.0.1 releases are running atm, it takes some time to get attention. -Matthias On 10/18/18 11:47 AM, Yishun Guan wrote: > Bumping this thread up again, thanks! > On Tue, Oct 16, 2018 at 11:24 AM Yishun Gua

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

2018-10-18 Thread Apache Jenkins Server
See Changes: [lindong28] MINOR: Add a note about Zstandard compression in the upgrade docs [lindong28] KAFKA-7464; catch exceptions in "leaderEndpoint.close()" when shutting ---

[jira] [Resolved] (KAFKA-7464) Fail to shutdown ReplicaManager during broker cleaned shutdown

2018-10-18 Thread Dong Lin (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-7464?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Dong Lin resolved KAFKA-7464. - Resolution: Fixed > Fail to shutdown ReplicaManager during broker cleaned shutdown >

Build failed in Jenkins: kafka-trunk-jdk11 #43

2018-10-18 Thread Apache Jenkins Server
See Changes: [lindong28] MINOR: Add a note about Zstandard compression in the upgrade docs [lindong28] KAFKA-7464; catch exceptions in "leaderEndpoint.close()" when shutting

Jenkins build is back to normal : kafka-2.1-jdk8 #31

2018-10-18 Thread Apache Jenkins Server
See

[jira] [Created] (KAFKA-7519) Transactional Ids Left in Pending State by TransactionStateManager During Transactional Id Expiration Are Unusable

2018-10-18 Thread Bridger Howell (JIRA)
Bridger Howell created KAFKA-7519: - Summary: Transactional Ids Left in Pending State by TransactionStateManager During Transactional Id Expiration Are Unusable Key: KAFKA-7519 URL: https://issues.apache.org/jira/b

Re: [VOTE] KIP-376: Implement AutoClosable on appropriate classes that want to be used in a try-with-resource statement

2018-10-18 Thread Yishun Guan
Bumping this thread up again, thanks! On Tue, Oct 16, 2018 at 11:24 AM Yishun Guan wrote: > > Bumping this thread up again, thanks! > > On Fri, Oct 12, 2018, 4:53 PM Colin McCabe wrote: >> >> On Fri, Oct 12, 2018, at 15:45, Yishun Guan wrote: >> > Hi Colin, >> > >> > Thanks for your suggestions.

Re: Throwing away prefetched records optimisation.

2018-10-18 Thread Jan Filipiak
The idea for you would be that Messagechooser could hang on to the prefetched messages. ccing cmcc...@apache.org @Collin just for you to see that MessageChooser is a powerfull abstraction. :) Best jan On 18.10.2018 13:59, Zahari Dichev wrote: > Jan, > > Quite insightful indeed. I think your p

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

2018-10-18 Thread nick
> On Oct 12, 2018, at 5:06 PM, Colin McCabe wrote: > > Maybe there's some really cool use-case that I haven't thought of. But so > far I can't really think of any time I would need topic priorities if I was > muting topics and offloading blocking operations in a reasonable way. It > would

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

2018-10-18 Thread Apache Jenkins Server
See

RE: [DISCUSS] KIP-383 Pluggable interface for SSL Factory

2018-10-18 Thread Pellerin, Clement
I guess it could but unless I'm mistaken, you have the same problem as before since it is not pluggable. The idea is you should be able to replace the implementation through configuration, without rebuilding a custom Kafka distribution or resorting to classpath tricks to shadow Kafka classes. I

Re: [DISCUSS] KIP-382: MirrorMaker 2.0

2018-10-18 Thread Ryanne Dolan
Harsha, concerning uReplicator specifically, the project is a major inspiration for MM2, but I don't think it is a good foundation for anything included in Apache Kafka. uReplicator uses Helix to solve problems that Connect also solves, e.g. REST API, live configuration changes, cluster management,

Re: [DISCUSS] KIP-383 Pluggable interface for SSL Factory

2018-10-18 Thread Harsha
Hi, Thanks for the KIP. Curious to understand why the ChannelBuilder interface doesn't solve the stated reasons in Motiviation section. Thanks, Harsha On Wed, Oct 17, 2018, at 12:10 PM, Pellerin, Clement wrote: > I would like feedback on this proposal to make it possible to replace > Ssl

Re: [DISCUSS] KIP-381 Connect: Tell about records that had their offsets flushed in callback

2018-10-18 Thread Ryanne Dolan
Per Steffenson, getting sequence numbers correct is definitely difficult, but this is not Connect's fault. I'd like to see Connect implement exactly-once from end-to-end, but that requires coordination between sources and sinks along the lines that you allude to, using sequence numbers and transact

[jira] [Resolved] (KAFKA-7505) Flaky test: SslTransportLayerTest.testListenerConfigOverride

2018-10-18 Thread Rajini Sivaram (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-7505?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Rajini Sivaram resolved KAFKA-7505. --- Resolution: Fixed Reviewer: Ismael Juma Fix Version/s: 2.1.0 > Flaky test: Ss

Request for contributor permissions

2018-10-18 Thread Kent
JIRA ID: cjfan2...@gmail.com Cwiki ID: cjfan2...@gmail.com -- Best Regards, Kent Fan

Re: Throwing away prefetched records optimisation.

2018-10-18 Thread Zahari Dichev
Thanks Ryanne, I am glad it makes sense. Should I put a KIP together and call for discussion on it ? Its my first KIP, so have not quite locked in the process yet. Zahari On Thu, Oct 18, 2018 at 3:26 PM Ryanne Dolan wrote: > Zahari, that makes sense, thanks for reframing your question. I suspec

Re: [DISCUSS] KIP-382: MirrorMaker 2.0

2018-10-18 Thread Ryanne Dolan
Jan, thanks for the share. Also similar are Pulsar's concepts of namespaces and global topics. I don't think these need to be supported in Kafka itself, but there are many benefits to adopting naming conventions along these lines, esp for tooling, dashboards etc. > use it to copy my messages from

Re: Throwing away prefetched records optimisation.

2018-10-18 Thread Ryanne Dolan
Zahari, that makes sense, thanks for reframing your question. I suspect that pause/resume was not intended to be called at high frequency like that, but I agree with you that the current behavior is needlessly inefficient. I like your idea of making it configurable. Ryanne On Thu, Oct 18, 2018, 6

Re: [DISCUSS] KIP-381 Connect: Tell about records that had their offsets flushed in callback

2018-10-18 Thread Per Steffensen
On 17/10/2018 18.17, Ryanne Dolan wrote: > this does not guarantee that the > offsets of R have been written/flushed at the next commit() call True, but does it matter? So long as you can guarantee the records are delivered to the downstream Kafka cluster, it shouldn't matter if they have been

Re: Throwing away prefetched records optimisation.

2018-10-18 Thread Zahari Dichev
Jan, Quite insightful indeed. I think your propositions are valid. Ryanne, I understand that consumers are using a pull model... And yes, indeed if a consumer is not ready for more records it surely should not call poll. Except that it needs to do so periodically in order to indicate that its li

[jira] [Created] (KAFKA-7518) FutureRecordMetadata.get deadline calculation from timeout is not using timeunit

2018-10-18 Thread Andras Katona (JIRA)
Andras Katona created KAFKA-7518: Summary: FutureRecordMetadata.get deadline calculation from timeout is not using timeunit Key: KAFKA-7518 URL: https://issues.apache.org/jira/browse/KAFKA-7518 Projec

Re: [DISCUSS] KIP-382: MirrorMaker 2.0

2018-10-18 Thread Jan Filipiak
then I just hope that in the midsts of all this new features I can still at least use it to copy my messages from A to B later. Another hint you should be aware of: https://cwiki.apache.org/confluence/display/KAFKA/Hierarchical+Topics That was always a design I admired, with active / active re