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

2018-08-19 Thread Ted Yu
bq. // or just Duration windowSize(); +1 to the above choice. The duration is obvious from the return type. For getter methods, we don't use get as prefix (as least for new code). Cheers On Sun, Aug 19, 2018 at 8:03 AM Nikolay Izhikov wrote: > Hello, John. > > Thank you very much for your feed

Re: [ANNOUNCE] New Kafka PMC member: Dong Lin

2018-08-21 Thread Ted Yu
Congratulation Dong! On Tue, Aug 21, 2018 at 1:59 AM Viktor Somogyi-Vass wrote: > Congrats Dong! :) > > On Tue, Aug 21, 2018 at 10:09 AM James Cheng wrote: > > > Congrats Dong! > > > > -James > > > > > On Aug 20, 2018, at 3:54 AM, Ismael Juma wrote: > > > > > > Hi everyone, > > > > > > Dong Li

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

2018-08-22 Thread Ted Yu
+1 Original message From: Kamal Chandraprakash Date: 8/22/18 3:19 AM (GMT-08:00) To: dev@kafka.apache.org Subject: Re: [VOTE] KIP-336: Consolidate ExtendedSerializer/Serializer and ExtendedDeserializer/Deserializer +1 Thanks for the KIP! On Wed, Aug 22, 2018 at 2:48 PM Vik

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

2018-08-22 Thread Ted Yu
We may also consider : "suppress.auto.topic.creation" or "allow.auto.topic.creation" w.r.t. suppress or allow, I don't have strong opinion either. It's just a matter of choosing the proper default value. Cheers On Wed, Aug 22, 2018 at 6:00 PM Dhruvil Shah wrote: > Hi Matthias, > > Do you me

Re: [DISCUSS] KIP-363: Make FunctionConversions private

2018-08-23 Thread Ted Yu
+1 In the Motivation section, you can quote the comment from pull request so that reader doesn't have to click through. Cheers On Thu, Aug 23, 2018 at 12:13 PM Joan Goyeau wrote: > Hi, > > As pointed out in this comment #5539 (comment) >

Re: Build failed in Jenkins: kafka-trunk-jdk10 #429

2018-08-23 Thread Ted Yu
I ran streams unit tests as of commit 4156ea0a9bcca67d209fd3b43d2268c9abd5a0b5 . All tests passed locally. FYI On Thu, Aug 23, 2018 at 12:23 PM Joan Goyeau wrote: > I'm looking into this one. > > On Thu, 23 Aug 2018 at 20:19 Apache Jenkins Server < > jenk...@builds.apache.org> wrote: > > > See

Re: [VOTE] KIP-363: Make FunctionConversions private

2018-08-24 Thread Ted Yu
gt; > > > > On Fri, 24 Aug 2018 at 19:14 John Roesler wrote: > > > > > > > I'm also in favor of this. I don't think it's controversial either. > > > Should > > > > we just move to a vote? > > > > > > > > On

Re: unable to build schema registry

2018-08-24 Thread Ted Yu
The first error was complaining about this class from Kafka clients: clients/src/main/java/org/apache/kafka/common/utils/Timer.java It seems you haven't installed kafka clients jar into local maven repo. You can run the following command under kafka working dir: ./gradlew installAll I just buil

Re: [VOTE] KIP-357: Add support to list ACLs per principal

2018-08-27 Thread Ted Yu
+1 On Mon, Aug 27, 2018 at 11:42 AM Priyank Shah wrote: > +1 (Non-binding) > > On 8/27/18, 9:24 AM, "Manikumar" wrote: > > Hi All, > > I would like to start voting on KIP-357 which allows to list ACLs per > principal using AclCommand (kafka-acls.sh) > > KIP: > > https://cwiki.a

Re: [VOTE] KIP-365: Materialized, Serialized, Joined, Consumed and Produced with implicit Serde

2018-08-27 Thread Ted Yu
+1 On Mon, Aug 27, 2018 at 12:18 PM John Roesler wrote: > +1 (non-binding) > > On Sat, Aug 25, 2018 at 1:16 PM Joan Goyeau wrote: > > > Hi, > > > > We want to make sure that we always have a serde for all Materialized, > > Serialized, Joined, Consumed and Produced. > > For that we can make use

Re: 答复: [VOTE] KIP-347: Enable batching in FindCoordinatorRequest

2018-08-27 Thread Ted Yu
Looking at the code for solution #1: } else if (builder.build(version) instanceof List){ wouldn't AbstractRequest be gone due to type erasure ? Which solution do you favor ? Cheers On Mon, Aug 27, 2018 at 4:20 PM Yishun

Re: [DISCUSS] Replacing EasyMock with Mockito in Kafka

2018-10-05 Thread Ted Yu
+1 to moving to Mockito On Fri, Oct 5, 2018 at 12:11 PM Ron Dagostino wrote: > I have used Mockito and am a big fan -- I had never used EasyMock until > recently. The concept of record vs. replay mode in EasyMock really annoyed > me -- I'm a fan of the "NO MODES" idea ( > https://en.wikipedia.o

Re: Kafka + ACL Topic based security ,Spring Boot

2017-09-22 Thread Ted Yu
Attachment didn't come thru. Have you read ? https://cwiki.apache.org/confluence/display/KAFKA/Security especially the https://cwiki.apache.org/confluence/display/KAFKA/Security#Security-ImplementingthePermissionManager section ? On Fri, Sep 22, 2017 at 2:45 AM, Pooppillikudiyil, Joby < joby.poo

Re: [DISCUSS]: KIP-159: Introducing Rich functions to Streams

2017-09-22 Thread Ted Yu
t; > > >> > > +1 > >> > > > >> > > On Thu, 21 Sep 2017 at 13:46 Guozhang Wang > >> wrote: > >> > > > >> > > > +1 for me as well for collapsing. > >> > > > > >> > > > Jeyhu

Re: [DISCUSS]: KIP-159: Introducing Rich functions to Streams

2017-09-22 Thread Ted Yu
s/interfaces that extend/implement *RecordContext.* > In general, 1) we make RecordContext publicly available within > ProcessorContext, 2) initialize its instance within all required > Processors and 3) pass it as an argument to the related Rich interfaces > inside Processors. > > &g

Re: Spurious INFO message in SessionExpireListener (of KafkaHealthcheck)?

2017-09-22 Thread Ted Yu
See KafkaHealthcheck#register() I think it means registration for the zookeeper path. On Fri, Sep 22, 2017 at 12:38 PM, Jacek Laskowski wrote: > Hi, > > Just noticed that SessionExpireListener#handleNewSession > (of KafkaHealthcheck) prints out the INFO message to the logs [1] and does > nothin

Re: [DISCUSS] KIP-201: Rationalising Policy interfaces

2017-09-25 Thread Ted Yu
bq. topic.action.policy.class.name Since the policy would cover more than one action, how about using actions for the second word ? For TopicState interface, the abstract modifier for its methods are not needed. bq. KIP-113 Mind adding more to the above bullet ? bq. If this KIP is accepted for

Re: [DISCUSS] KIP-201: Rationalising Policy interfaces

2017-09-25 Thread Ted Yu
bq. deprecations that are added in 1.x (x>0) have to remain in all 2.y Makes sense. It is fine to exclude KIP-113 from your KIP. Thanks On Mon, Sep 25, 2017 at 9:32 AM, Tom Bentley wrote: > Hi Ted, > > Thanks for the feedback! > > bq. topic.action.policy.class.name > > > > Since the policy wo

Re: [DISCUSS] KIP-206: Add support for UUID serialization and deserialization

2017-09-26 Thread Ted Yu
Please add link to Discussion thread field. Looks good overall. On Tue, Sep 26, 2017 at 10:18 AM, Jakub Scholz wrote: > Hi, > > I'd like to start a discussion for KIP-206. It is about adding serializers > and deserializers for UUIDs. The details can be found on the wiki: > https://cwiki.apache.

Re: KIP-167 Updates

2017-09-28 Thread Ted Yu
Looks good. Please update the discussion thread link. On Thu, Sep 28, 2017 at 12:01 PM, Bill Bejeck wrote: > All, > > I have updated KIP-167 to include the bootstrapping status of any > GlobalKTables defined in the application. > > The KIP has been updated: > https://cwiki.apache.org/confluence

Re: How is CorrelationId used for matching request and response

2017-09-29 Thread Ted Yu
Which release / version are you looking at ? In trunk branch, I only see one toSend(): protected Send toSend(String destination, ResponseHeader header, short apiVersion) { return new NetworkSend(destination, serialize(apiVersion, header)); On Fri, Sep 29, 2017 at 4:49 PM, Javed, Hase

Re: Can't build Kafka project

2017-10-01 Thread Ted Yu
In streams/.classpath , you should have the following lines: FYI On Sun, Oct 1, 2017 at 11:02 AM, Waleed Fateem wrote: > Hi everyone, > > I'm quite new to Kafka from a development perspective and I'm having some > issues building the project in Eclipse. > > I recall being able t

Re: [VOTE] KIP-206: Add support for UUID serialization and deserialization

2017-10-03 Thread Ted Yu
+1 Original message From: Jakub Scholz Date: 10/3/17 2:24 PM (GMT-08:00) To: dev@kafka.apache.org Subject: [VOTE] KIP-206: Add support for UUID serialization and deserialization Hi, Since there were no further discussion points, I would like to start the voting for KIP-206.

Re: integration between pull request and JIRA

2017-10-04 Thread Ted Yu
ens, > when you get tagged or comment on one, ie, you get auto subscribed to > the PR.) > > There is a "Subscribe" button on the right hand side. > > -Matthias > > > On 9/5/17 8:57 PM, Ted Yu wrote: > > bq. I did get tagged or I did comment on etc. > > >

Re: [DISCUSS] KIP-158: Kafka Connect should allow source connectors to set topic-specific settings for new topics

2017-10-04 Thread Ted Yu
Randall: bq. AdminClient currently allows changing the replication factory. By 'replication factory' did you mean 'replication factor' ? Cheers On Wed, Oct 4, 2017 at 9:58 AM, Randall Hauch wrote: > Currently the KIP's scope is only topics that don't yet exist, and we have > to cognizant of ra

Re: want to join this mail list

2017-10-04 Thread Ted Yu
See https://kafka.apache.org/contact for instructions. 2017-10-04 19:31 GMT-07:00 吴晓菊 : > -- > > Chrysan Wu > 联系方式:17717640807 >

Re: [DISCUSS] KIP-186: Increase offsets retention default to 7 days

2017-10-06 Thread Ted Yu
+1 on the KIP. bq. introducing offsets.retention.hours config property Probably the introduction would cause confusion among users due to the existing minutes config. On Fri, Oct 6, 2017 at 10:53 AM, Manikumar wrote: > looks like VOTE thread is started for this KIP. > > On Wed, Aug 16, 2017 at

Re: Can't build Kafka project

2017-10-08 Thread Ted Yu
following: > > /kafka/core/build/classes/java/test > > The core project itself doesn't have any java test source code under src/ > > > > > > On Sun, Oct 1, 2017 at 1:04 PM, Ted Yu wrote: > > > In streams/.classpath , you should have the followi

Re: [DISCUSS] KIP-208: Add SSL support to Kafka Connect REST interface

2017-10-09 Thread Ted Yu
For rest.ssl.clientAuth , I don't find counterpart in existing code. Can you add explanation on the KIP ? Thanks On Mon, Oct 9, 2017 at 8:25 AM, Jakub Scholz wrote: > Hi, > > I would like to start a discussion about KIP-208: Add SSL support to Kafka > Connect REST interface ( > https://cwiki.ap

Re: [VOTE] 1.0.0 RC0

2017-10-10 Thread Ted Yu
Guozhang: I took a brief look under the staging tree. e.g. https://repository.apache.org/content/groups/staging/org/apache/kafka/kafka-clients/ I don't see 1.0.0 jars. Would the jars be populated later ? Thanks On Tue, Oct 10, 2017 at 6:34 PM, Guozhang Wang wrote: > Hello Kafka users, develop

Re: [VOTE] KIP-204 : adding records deletion operation to the new Admin Client API

2017-10-11 Thread Ted Yu
+1 On Mon, Oct 2, 2017 at 10:51 PM, Paolo Patierno wrote: > Hi all, > > I didn't see any further discussion around this KIP, so I'd like to start > the vote for it. > > Just for reference : https://cwiki.apache.org/ > confluence/display/KAFKA/KIP-204+%3A+adding+records+ > deletion+operation+to+t

Re: Kafka Consumer - org.apache.kafka.common.errors.TimeoutException: Failed to get offsets by times in 305000 ms

2017-10-11 Thread Ted Yu
I don't see values for the Consumer Properties. Can you try out 0.11.0.1 ? See http://search-hadoop.com/m/Kafka/uyzND1qxYr5prjxv?subj=Incorrect+consumer+offsets+after+broker+restart+0+11+0+0 On Wed, Oct 11, 2017 at 11:37 AM, SenthilKumar K wrote: > Hi All , Recently we starting seeing Kafka Co

Re: [VOTE] 1.0.0 RC0

2017-10-11 Thread Ted Yu
Looks like the following change is needed for some downstream project to compile their code (which was using 0.11.0.1): -import org.apache.kafka.common.protocol.SecurityProtocol; +import org.apache.kafka.common.security.auth.SecurityProtocol; I took a look at docs/upgrade.html but didn't see any

Re: Limit Jenkins jobs on H0 to H13

2017-10-11 Thread Ted Yu
; Thanks for following up with INFRA on the issues we've been seeing. I asked > a clarifying comment in that ticket (for some reason it only allowed me to > add an internal comment). > > Ismael > > On Wed, Sep 20, 2017 at 2:31 AM, Ted Yu wrote: > > > Hi, > &

Re: [VOTE] 1.0.0 RC0

2017-10-12 Thread Ted Yu
ooking into this for possible doc changes now. > > > > Guozhang > > > > On Wed, Oct 11, 2017 at 3:23 PM, Ted Yu wrote: > > > >> Looks like the following change is needed for some downstream project to > >> compile their code (which was using 0.11.0.1): &g

Re: [VOTE] 1.0.0 RC0

2017-10-12 Thread Ted Yu
s OK to not mention it in the upgrade guides. > > For developers rather than users of Kafka, they are assumed to be familiar > with the codebase and not only rely on upgrade guide docs for such > information. > > > Guozhang > > On Thu, Oct 12, 2017 at 2:58 PM, Ted Yu wrote

Re: [DISCUSS] KIP-208: Add SSL support to Kafka Connect REST interface

2017-10-13 Thread Ted Yu
I agree with Randall. Actually I had the same thought during first round of review. On Fri, Oct 13, 2017 at 9:25 AM, Randall Hauch wrote: > Also, do we need these properties to be preceded with `rest`? I'd argue > that we're just configuring the worker's SSL information, and that the REST > API

Re: [VOTE] KIP-171 - Extend Consumer Group Reset Offset for Stream Application

2017-10-13 Thread Ted Yu
+1 On Fri, Oct 13, 2017 at 3:32 PM, Matthias J. Sax wrote: > +1 > > > > On 9/11/17 3:04 PM, Jorge Esteban Quilcate Otoya wrote: > > Hi All, > > > > It seems that there is no further concern with the KIP-171. > > At this point we would like to start the voting process. > > > > The KIP can be foun

Re: [DISCUSS] KIP-59 : Proposal for a kafka broker command

2017-10-15 Thread Ted Yu
Please fill 'Discussion thread:' with URL to this thread. For 'Proposed Changes' section, is it possible to indent the lines from 'Broker Id' to 'Trailing partition count' ? This way, it is easier to read. bq. The command kafka-brokers.sh requires zookeeper information Is the above still true bas

Re: [DISCUSS] KIP-208: Add SSL support to Kafka Connect REST interface

2017-10-16 Thread Ted Yu
+1 to this proposal. On Mon, Oct 16, 2017 at 7:49 AM, Jakub Scholz wrote: > I was having some more thoughts about it. We can simply take over what > Kafka broker implements for the listeners: > - We can take over the "listener" and "listener.security.protocol.map" > options to define multiple RE

Re: [DISCUSS] KIP-209 Connection String Support

2017-10-16 Thread Ted Yu
Please update link for Discussion thread and JIRA There're two TBD's for Invalid conversion and parameters. Can you fill them out ? Thanks On Mon, Oct 16, 2017 at 2:30 PM, Clebert Suconic wrote: > I would like to start a discussion about KIP-209 > (https://cwiki.apache.org/confluence/display/K

Re: [DISCUSS] KIP-209 Connection String Support

2017-10-16 Thread Ted Yu
bq. I was waiting my email to go through the servers http://search-hadoop.com/ indexes mailing lists actively. The delay is very short. FYI On Mon, Oct 16, 2017 at 2:44 PM, Clebert Suconic wrote: > On Mon, Oct 16, 2017 at 5:41 PM, Ted Yu wrote: > > Please update link for Discussi

Re: [DISCUSS] KIP-209 Connection String Support

2017-10-16 Thread Ted Yu
Can't you use IllegalArgumentException ? Some example in current code base: clients/src/main/java/org/apache/kafka/clients/Metadata.java: throw new IllegalArgumentException("Max time to wait for metadata updates should not be < 0 milliseconds"); On Mon, Oct 16, 2017 at 3:06 PM, Clebert Suconic

Re: [VOTE] 1.0.0 RC1

2017-10-16 Thread Ted Yu
After specifying the location of zookeeper jar: export CLASSPATH= The command can be executed successfully: bin/zookeeper-server-start.sh config/zookeeper.properties This doesn't seem to be Java 9 specific issue. On Mon, Oct 16, 2017 at 5:30 PM, Vahid S Hashemian < vahidhashem...@us.ibm.com> w

Re: [VOTE] KIP-207:The Offsets which ListOffsetsResponse returns should monotonically increase even during a partition leader change

2017-10-17 Thread Ted Yu
+1 On Tue, Oct 17, 2017 at 11:23 AM, Apurva Mehta wrote: > +1 (non-binding) > > On Tue, Oct 17, 2017 at 11:11 AM, Colin McCabe wrote: > > > Hi all, > > > > I'd like to start the voting process for KIP-207:The Offsets which > > ListOffsetsResponse returns should monotonically increase even duri

Re: working on KAFKA-4928

2017-10-18 Thread Ted Yu
There was a PR https://github.com/apache/kafka/pull/2889 which was closed. It would be good for committers to chime in on the previous attempt. On Wed, Oct 18, 2017 at 8:39 AM, Pavel Drankov wrote: > Hi, > > My name is Pavel and I'm a very new to Kafka. I actually would like to work > on KAFKA-

Re: [DISCUSS] KIP-210: Provide for custom error handling when Kafka Streams fails to produce

2017-10-18 Thread Ted Yu
Can you create JIRA that corresponds to the KIP ? For the new config, how about naming it production.exception.processor.class ? This way it is clear that class name should be specified. Cheers On Wed, Oct 18, 2017 at 2:40 PM, Matt Farmer wrote: > Hello everyone, > > This is the discussion thr

Re: [DISCUSS] KIP-211: Revise Expiration Semantics of Consumer Group Offsets

2017-10-18 Thread Ted Yu
Please fill out 'Rejected Alternatives' section. Thanks On Wed, Oct 18, 2017 at 4:45 PM, Vahid S Hashemian < vahidhashem...@us.ibm.com> wrote: > Hi all, > > I created a KIP to address the group offset expiration issue reported in > KAFKA-4682: > https://cwiki.apache.org/confluence/display/KAFKA/

Re: [VOTE] 1.0.0 RC3

2017-10-23 Thread Ted Yu
bq. Tag to be voted upon (off 1.0 branch) is the 1.0.0-rc2 tag: There seems to be a typo above: 1.0.0-rc3 tag FYI On Mon, Oct 23, 2017 at 6:00 PM, Guozhang Wang wrote: > Hello Kafka users, developers and client-developers, > > This is the third candidate for release of Apache Kafka 1.0.0. The

Re: [DISCUSS] KIP-213: Add zookeeper.max.in.flight.requests config to the broker

2017-10-25 Thread Ted Yu
This is for KAFKA-5894, right ? Please fill out the JIRA link. +1 on this proposal. On Wed, Oct 25, 2017 at 4:11 PM, Onur Karaman wrote: > Hey everyone. > > I made a config kip, KIP-213: Add zookeeper.max.in.flight.requests config > to the broker: > https://cwiki.apache.org/confluence/display/K

Re: [DISCUSS] KIP-214: Add zookeeper.max.in.flight.requests config to the broker

2017-10-26 Thread Ted Yu
Can you fill out JIRA number ? Thanks Original message From: Onur Karaman Date: 10/25/17 10:37 PM (GMT-08:00) To: dev@kafka.apache.org Subject: [DISCUSS] KIP-214: Add zookeeper.max.in.flight.requests config to the broker Hey everyone. Giving this another shot since it looks

Re: Failed to start Apache Kafka

2017-10-26 Thread Ted Yu
Hmm. You ran into KAFKA-5077. Can you try 1.0.0 RC3 ? http://search-hadoop.com/m/Kafka/uyzND18Q3Vh1Q5jUV1?subj=+VOTE+1+0+0+RC3 On Thu, Oct 26, 2017 at 10:08 AM, Naren gs wrote: > Hello, > I am starter in Apache Kafka, I downloaded the latest version and on > running the config of zookeeper

Re: [DISCUSS] KIP-215: Add topic regex support for Connect sinks

2017-10-26 Thread Ted Yu
bq. Users may specify only one of 'topics' or 'topics.pattern'. Can you fill in which exception would be thrown if both of them are specified ? Cheers On Thu, Oct 26, 2017 at 6:27 PM, Jeff Klukas wrote: > Looking for feedback on > > https://cwiki.apache.org/confluence/display/KAFKA/KIP- > 215%

Re: [DISCUSS] KIP-213 Support non-key joining in KTable

2017-10-26 Thread Ted Yu
Do you mind addressing my previous comments ? http://search-hadoop.com/m/Kafka/uyzND1hzF8SRzUqb?subj=Re+DISCUSS+KIP+213+Support+non+key+joining+in+KTable On Thu, Oct 26, 2017 at 9:38 PM, Jan Filipiak wrote: > Hello everyone, > > this is the new discussion thread after the ID-clash. > > Best > J

Re: Re: [DISCUSS] KIP-213 Support non-key joining in KTable

2017-10-27 Thread Ted Yu
se to introduce A and B or just stick with > "this ktable", "other ktable" > > Thank you > Jan > > > On 27.10.2017 06:58, Ted Yu wrote: > >> Do you mind addressing my previous comments ? >> >> http://search-hadoop.com/m/Kafka/uyzND1hzF8SR

Re: Errors when running integration/kafka/tools/MirrorMakerIntegrationTest

2017-10-27 Thread Ted Yu
The error log appears in other tests as well. I ran MirrorMakerIntegrationTest on trunk just now - it passed. I didn't see the test failing in any recent Jenkins builds. There're flaky tests, such as KAFKA-6137 FYI On Fri, Oct 27, 2017 at 8:35 AM, Pavel Drankov wrote: > Hi, > > I see some er

Re: [VOTE] KIP-214: Add zookeeper.max.in.flight.requests config to the broker

2017-10-27 Thread Ted Yu
+1 On Fri, Oct 27, 2017 at 10:15 AM, Onur Karaman wrote: > I'd like to start the vote for KIP-214: Add > zookeeper.max.in.flight.requests config to the broker > > https://cwiki.apache.org/confluence/display/KAFKA/KIP- > 214%3A+Add+zookeeper.max.in.flight.requests+config+to+the+broker > > - Onur

Re: [DISCUSS] KIP-190: Handle client-ids consistently between clients and brokers

2017-10-29 Thread Ted Yu
Can the Discussion Thread link be filled out ? Cheers On Wed, Sep 27, 2017 at 2:03 AM, Mickael Maison wrote: > I don't know the history either, I quickly scanned the KIP-55 threads > and couldn't see it being discussed. > > Anyway, your suggestion sounds good to me, are you planning to do that

Re: [VOTE] 1.0.0 RC4

2017-10-31 Thread Ted Yu
+1 (non-binding) Verified signatures. Ran test suite. On Tue, Oct 31, 2017 at 8:53 AM, Manikumar wrote: > +1 (non-binding). Verified quickstart, ran producer/consumer perf scripts, > streams quickstart > ran tests on src distribution. > > On Tue, Oct 31, 2017 at 8:42 PM, Ismael Juma wrote: > >

Re: [VOTE] KIP-215: Add topic regex support for Connect sinks

2017-11-01 Thread Ted Yu
+1 Original message From: Jeff Klukas Date: 11/1/17 7:40 AM (GMT-08:00) To: dev@kafka.apache.org Subject: [VOTE] KIP-215: Add topic regex support for Connect sinks I haven't heard any additional concerns over the proposal, so I'd like to get the voting process started for: h

Re: [DISCUSS] KIP-218: Make KafkaFuture.Function java 8 lambda compatible

2017-11-01 Thread Ted Yu
bq. PR#4033 makes from those abstract classes interfaces. Please adjust the syntax. bq. Wait for a kafka release which will not support java 7 anymore Do you want to raise a separate thread for the above ? On Wed, Nov 1, 2017 at 7:50 AM, Steven Aerts

Re: [DISCUSS] KIP-218: Make KafkaFuture.Function java 8 lambda compatible

2017-11-01 Thread Ted Yu
KAFKA-4423 is still open. When would Java 7 be dropped ? Thanks On Wed, Nov 1, 2017 at 8:56 AM, Ismael Juma wrote: > On Wed, Nov 1, 2017 at 3:51 PM, Ted Yu wrote: > > > bq. Wait for a kafka release which will not support java 7 anymore > > > > Do you want to raise a

Re: [VOTE] KIP-210: Provide for custom error handling when Kafka Streams fails to produce

2017-11-02 Thread Ted Yu
+1 On Wed, Nov 1, 2017 at 4:50 PM, Guozhang Wang wrote: > +1 (binding) from me. Thanks! > > On Wed, Nov 1, 2017 at 4:50 PM, Guozhang Wang wrote: > > > The vote should stay open for at least 72 hours. The bylaws can be found > > here https://cwiki.apache.org/confluence/display/KAFKA/Bylaws > > >

Re: [DISCUSS] KIP-217: Expose a timeout to allow an expired ZK session to be re-created

2017-11-02 Thread Ted Yu
ZOOKEEPER-2184 is scheduled for 3.4.12 whose release is unknown. I think adding the session recreation on Kafka side should benefit Kafka users, especially those who don't plan to move to 3.4.12+ in the near future. On Wed, Nov 1, 2017 at 6:34 PM, Jun Rao wrote: > Hi, Stephane, > > 3) The diffe

Re: [DISCUSS] KIP-217: Expose a timeout to allow an expired ZK session to be re-created

2017-11-02 Thread Ted Yu
nfig and just retry > the creation of Zookeeper forever. This is an improvement from the current > situation and if zookeeper-2184 is fixed in the future, we don't need to > deprecate the config. > > Thanks, > > Jun > > On Thu, Nov 2, 2017 at 9:02 AM, Ted Yu wrote: > >

Re: [DISCUSS] KIP-217: Expose a timeout to allow an expired ZK session to be re-created

2017-11-02 Thread Ted Yu
The following JIRA provides some background on why upgrading immediately following new release may not be prudent (though I expect this to be rare): ZOOKEEPER-2347 On Thu, Nov 2, 2017 at 3:00 PM, Ted Yu wrote: > Stephane: > bq. hasn't acted in over a year > > The above

kafka-pr-jdk9-scala2.12 keeps failing

2017-11-02 Thread Ted Yu
Hi, I took a look at recent runs under https://builds.apache. org/job/kafka-pr-jdk9-scala2.12 All the recent runs failed with: Could not update commit status of the Pull Request on GitHub. org.kohsuke.github.HttpException: Server returned HTTP response code: 201, message: 'Created' for URL: https

Re: kafka-pr-jdk9-scala2.12 keeps failing

2017-11-02 Thread Ted Yu
aused the issue? > > > Guozhang > > On Thu, Nov 2, 2017 at 6:25 PM, Ted Yu wrote: > > > Hi, > > I took a look at recent runs under https://builds.apache. > > org/job/kafka-pr-jdk9-scala2.12 > > > > All the recent runs fa

Re: [DISCUSS] KIP-220: Add AdminClient into Kafka Streams' ClientSupplier

2017-11-03 Thread Ted Yu
Looks good overall. bq. the creation within StreamsPartitionAssignor Typo above: should be StreamPartitionAssignor On Fri, Nov 3, 2017 at 4:49 PM, Guozhang Wang wrote: > Hello folks, > > I have filed a new KIP on adding AdminClient into Streams for internal > topic management. > > Looking for

Re: [DISCUSS] KIP-221: Repartition Topic Hints in Streams

2017-11-04 Thread Ted Yu
API is given without much javadoc on the role / meaning of method parameters. Can you enrich with descriptive javadoc ? On Sat, Nov 4, 2017 at 8:01 AM, Jeyhun Karimov wrote: > Dear community, > > I would like to initiate discussion on KIP-221 [1] based on issue [2]. > Please feel free to commen

Re: [DISCUSS] KIP-221: Repartition Topic Hints in Streams

2017-11-04 Thread Ted Yu
gh()` to create a topics in the back ground > and picky back all to `Produced`. > > > -Matthias > > On 11/4/17 4:16 PM, Ted Yu wrote: > > API is given without much javadoc on the role / meaning of method > > parameters. > > > > Can you enrich with

Re: kafka-pr-jdk9-scala2.12 keeps failing

2017-11-05 Thread Ted Yu
3 Nov 2017 1:53 am, "Ted Yu" wrote: > > > Looking at earlier runs, e.g. : > > https://builds.apache.org/job/kafka-pr-jdk9-scala2.12/2384/console > > > > FAILURE: Build failed with an exception. > > > > * What went wrong: > > Could not determine

Re: [DISCUSS] KIP-222 - Add "describe consumer group" to KafkaAdminClient

2017-11-06 Thread Ted Yu
Please fill out Discussion thread and JIRA fields. Thanks On Mon, Nov 6, 2017 at 2:02 AM, Tom Bentley wrote: > Hi Jorge, > > Thanks for the KIP. A few initial comments: > > 1. The AdminClient doesn't have any API like `listConsumerGroups()` > currently, so in general how does a client know the

Re: kafka producer : number of record pushed to topic limited

2017-11-06 Thread Ted Yu
Can you give us some more information ? Were you using a single node setup (127.0.0.1) ? Which release of Kafka are you using ? Anything interesting from broker log ? On Mon, Nov 6, 2017 at 8:52 AM, Dhia Beji wrote: > Hello, > > Would you please help me, I m trying to create producer that push

Re: [ANNOUNCE] New committer: Onur Karaman

2017-11-06 Thread Ted Yu
Congratulations, Onur! On Mon, Nov 6, 2017 at 9:24 AM, Jun Rao wrote: > Hi, everyone, > > The PMC of Apache Kafka is pleased to announce a new Kafka committer Onur > Karaman. > > Onur's most significant work is the improvement of Kafka controller, which > is the brain of a Kafka cluster. Over ti

Re: [DISCUSS] 0.11.0.2 bug fix release

2017-11-06 Thread Ted Yu
After clicking the outstanding issues link, I only saw KAFKA-6007. Here is the filter in case anyone encounters the same : status in (Open, Reopened, "In Progress", "Patch Available") AND Project = kafka AND fixVersion=0.11.0.2 On Mon, Nov 6, 2017 at 2:47 AM, Rajini Sivaram wrote: > Hi all, >

Re: [DISCUSS] KIP-221: Repartition Topic Hints in Streams

2017-11-06 Thread Ted Yu
bq. enlarge the score of through() I guess you meant scope. On Mon, Nov 6, 2017 at 1:15 PM, Jeyhun Karimov wrote: > Hi, > > Sorry for the late reply. I am convinced that we should enlarge the score > of through() (add more overloads) instead of introducing a separate set of > overloads to other

Re: [DISCUSS] KIP-213 Support non-key joining in KTable

2017-11-06 Thread Ted Yu
> >>>>> On Wed, Nov 1, 2017 at 12:18 PM, Jan Filipiak < >>>>> jan.filip...@trivago.com> >>>>> wrote: >>>>> >>>>> Hi thanks for the feedback >>>>>> >>>>>> On 01.11.2017 12:58, Damian

Re: kafka-pr-jdk9-scala2.12 keeps failing

2017-11-07 Thread Ted Yu
25 PM, Ismael Juma wrote: > > > Thanks! > > > > Ismael > > > > On Mon, Nov 6, 2017 at 3:48 AM, Ted Yu wrote: > > > >> Logged https://issues.apache.org/jira/browse/INFRA-15448 > >> > >> On Thu, Nov 2, 2017 at 11:39 PM, Ismael Juma

Re: kafka-pr-jdk9-scala2.12 keeps failing

2017-11-08 Thread Ted Yu
Does anything get reverted? > > > Guozhang > > > On Tue, Nov 7, 2017 at 8:33 AM, Ted Yu wrote: > > > https://builds.apache.org/job/kafka-pr-jdk9-scala2.12/2470/ is green. > > > > Thanks Ismael. > > > > On Tue, Nov 7, 2017 at 3:46 AM, Ismael J

Re: kafka-pr-jdk9-scala2.12 keeps failing

2017-11-08 Thread Ted Yu
> https://builds.apache.org/job/kafka-pr-jdk9-scala2.12/2498/console > > Does anything get reverted? > > > Guozhang > > > On Tue, Nov 7, 2017 at 8:33 AM, Ted Yu wrote: > > > https://builds.apache.org/job/kafka-pr-jdk9-scala2.12/2470/ is green. > > > &g

Re: [DISCUSS] KIP-224: Add configuration parameters `retries` and `retry.backoff.ms` to Streams API

2017-11-09 Thread Ted Yu
lgtm bq. pass both parameter parameter should be in plural. Same with 'two new configuration parameter' Cheers On Thu, Nov 9, 2017 at 4:20 AM, Damian Guy wrote: > Thanks Matthias, LGTM > > On Thu, 9 Nov 2017 at 11:13 Matthias J. Sax wrote: > > > Hi, > > > > I want to propose a new KIP to mak

Re: Regarding kafka produce Image(18mb)

2017-11-09 Thread Ted Yu
bq. Each image if of 18mb in size. Seems the image size is non-trivial. Can you tell us more about use case ? e.g. what systems (Apache projects) are involved. Just want to see if there is another route for sending the images. On Thu, Nov 9, 2017 at 2:15 AM, SADHNA wrote: > Hi Team, > > I am

Re: [DISCUSS]KIP-223 - Add per-topic min lead and per-partition lead metrics to KafkaConsumer

2017-11-09 Thread Ted Yu
Can you fill out JIRA number ? "TOPIC-PARTITION_ID.record-lead" In the code sample, "records-" is used. Please make them consistent. On Thu, Nov 9, 2017 at 1:03 AM, Hu Xi wrote: > Hi all, > > > As per Jun Rao's suggestion, I opened up the KIP-223(https://cwiki.apache. > org/confluence/display

Re: [DISCUSS] KIP-224: Add configuration parameters `retries` and `retry.backoff.ms` to Streams API

2017-11-10 Thread Ted Yu
try-in-next-loop as we did for non-global > >> stores. > >>> Since global stores are per task not per thread, we would not process > the > >>> task after the global store is bootstrapped fully. > >>> > >>> > >>> Guozhang > &g

Re: [VOTE] 0.11.0.2 RC0

2017-11-11 Thread Ted Yu
Running test suite I saw: kafka.admin.ReassignPartitionsClusterTest > shouldExecuteThrottledReassignment FAILED java.lang.AssertionError: Expected replication to be < 1 but was 10138 at org.junit.Assert.fail(Assert.java:88) at org.junit.Assert.assertTrue(Assert.java:41)

Re: [VOTE] 0.11.0.2 RC0

2017-11-13 Thread Ted Yu
cult to make > integration tests for quotas reliable since they verify the time taken to > complete after running for a short duration. We should make the tests > reliable, but this shouldn't be a blocker for the release. > > > On Sat, Nov 11, 2017 at 5:29 PM, Ted Yu wr

Re: [VOTE] KIP-224: Add configuration parameters `retries` to Streams API

2017-11-13 Thread Ted Yu
+1 On Mon, Nov 13, 2017 at 4:20 PM, Matthias J. Sax wrote: > Hi @all, > > I would like to start the vote for KIP-224: > > https://cwiki.apache.org/confluence/display/KAFKA/KIP- > 224%3A+Add+configuration+parameter+%60retries%60+to+Streams+API > > > Thanks a lot! > > > -Matthias > >

Re: [VOTE] KIP-222 - Add "describe consumer group" to KafkaAdminClient

2017-11-14 Thread Ted Yu
Can you fill in JIRA number (KAFKA-6058 ) ? If one JIRA is used for the two additions, consider updating the JIRA title. On Tue, Nov 14, 2017 at 9:04 AM, Jorge Esteban Quilcate Otoya < quilcate.jo...@gmail.com> wrote: > Hi all, > > As I didn't se

Re: [VOTE] KIP-222 - Add "describe consumer group" to KafkaAdminClient

2017-11-14 Thread Ted Yu
Please fill in JIRA number in Status section. On Tue, Nov 14, 2017 at 9:57 AM, Jorge Esteban Quilcate Otoya < quilcate.jo...@gmail.com> wrote: > JIRA issue title updated. > > El mar., 14 nov. 2017 a las 18:45, Ted Yu () > escribió: > > > Can you fill in JIRA nu

Re: [VOTE] KIP-220: Add AdminClient into Kafka Streams' ClientSupplier

2017-11-17 Thread Ted Yu
+1 On Fri, Nov 17, 2017 at 9:34 AM, Bill Bejeck wrote: > +1 > > Thanks, > Bill > > On Fri, Nov 17, 2017 at 12:13 PM, Colin McCabe wrote: > > > +1 (non-binding) > > > > Colin > > > > On Tue, Nov 14, 2017, at 10:02, Guozhang Wang wrote: > > > Hello folks, > > > > > > I have filed a new KIP on add

Re: Interested in being a contributor

2017-11-18 Thread Ted Yu
Please read this: https://kafka.apache.org/contributing You can use this Filter to find issues for new contributor: https://issues.apache.org/jira/issues/?jql=project%20%3D%20KAFKA%20AND%20labels%20%3D%20newbie%20AND%20status%20%3D%20Open Cheers On Sat, Nov 18, 2017 at 2:12 AM, Panuwat Anawatmo

SessionKeySchema#segmentsToSearch()

2017-11-18 Thread Ted Yu
Hi, I was reading code for SessionKeySchema#segmentsToSearch() where: public List segmentsToSearch(final Segments segments, final long from, final long to) { return segments.segments(from, Long.MAX_VALUE); I wonder why the parameter to is ignored. WindowKeySchema#segmentsToSearch() pa

Re: SessionKeySchema#segmentsToSearch()

2017-11-18 Thread Ted Yu
, from, to); The minSegment would be first in List and maxSegment would be last in List. On Sat, Nov 18, 2017 at 11:09 AM, Ted Yu wrote: > Hi, > I was reading code for SessionKeySchema#segmentsToSearch() where: > > public List segmentsToSearch(final Segments segments, final > l

Re: SessionKeySchema#segmentsToSearch()

2017-11-19 Thread Ted Yu
rThanEqualToTimestamp` and ` > getMaxSegmentLessThanEqualToTimestamp`: I think you meant to leave it as a > comment on https://github.com/apache/kafka/pull/4162? This is only added > in > that PR. > > > Guozhang > > > On Sat, Nov 18, 2017 at 11:16 AM, Ted Yu wrote: > > >

Re: [DISCUSS] KIP 226 - Dynamic Broker Configuration

2017-11-20 Thread Ted Yu
bq. (e.g. increase network/IO threads) Would decreasing network/IO threads be supported ? bq. void reconfigure(Map configs); What if some keys in configs are not in the Set returned by reconfigurableConfigs() ? Would exception be thrown ? If so, please specify which exception would be thrown

Re: [DISCUSS] KIP 226 - Dynamic Broker Configuration

2017-11-20 Thread Ted Yu
config option. > > *ssl.keystore.password appears a few lines above. Would there be any > issue with mixture of connections (with old and new password) ?* > No, passwords (and the actual keystore) are only used during > authentication. Any channel created using the old SslFactory will n

Re: [DISCUSS] KIP 226 - Dynamic Broker Configuration

2017-11-20 Thread Ted Yu
Thanks for the quick response. It seems the config following --delete-config should be quoted. Cheers On Mon, Nov 20, 2017 at 12:02 PM, Rajini Sivaram wrote: > Ted, > > Have added an example for --delete-config. > > On Mon, Nov 20, 2017 at 7:42 PM, Ted Yu wrote: > > >

Re: [DISCUSS] KIP-227: Introduce Incremental FetchRequests to Increase Partition Scalability

2017-11-21 Thread Ted Yu
Fill out the JIRA number. bq. If the leader receives an *IncrementalFetchRequest* with a UUID that does not match that of the latest *FetchResponse* *By *latest *FetchResponse, you mean latest response for the broker Id, right ?* *Cheers* On Tue, Nov 21, 2017 at 1:02 PM, Colin McCabe wrote: >

Re: [DISCUSS] KIP-227: Introduce Incremental FetchRequests to Increase Partition Scalability

2017-11-21 Thread Ted Yu
For compatibility, I assume that the follower with new code would detect when the leader doesn't support this feature and fall back to the existing full request. Cheers On Tue, Nov 21, 2017 at 3:44 PM, Colin McCabe wrote: > On Tue, Nov 21, 2017, at 14:35, Ted Yu wrote: > > Fil

<    1   2   3   4   5   >