Re: [KAFKA-557] Add emit on change support for Kafka Streams

2020-02-05 Thread Ted Yu
Thanks for the comments, Matthias. w.r.t. requirement of an `equals()` implementation, each template type would have an equals() method. We can use the following code to know whether it is provided by JVM or provided by user. boolean customEquals = false; try { Class cls =

Re: [KAFKA-557] Add emit on change support for Kafka Streams

2020-02-02 Thread Ted Yu
w.r.t. new metric, there is already droppedRecordsSensor which logs: "Skipping record due to null key. topic=[{}] partition=[{}] offset=[{}]", It seems we should introduce another metric which records the skipped (duplicate) values. This way, it is easier to observe the effect when this feature

Re: [VOTE] KIP-526: Reduce Producer Metadata Lookups for Large Number of Topics

2020-01-28 Thread Ted Yu
+1 On Tue, Jan 28, 2020 at 10:52 AM Rajini Sivaram wrote: > +1 (binding) > > Thanks for the KIP, Brian! > > Regards, > > Rajini > > On Thu, Jan 23, 2020 at 7:34 PM Jason Gustafson > wrote: > > > Sounds good. +1 from me. > > > > On Thu, Jan 23, 2020 at 9:00 AM Brian Byrne wrote: > > > > >

[jira] [Resolved] (KAFKA-9471) Throw exception for DEAD StreamThread.State

2020-01-24 Thread Ted Yu (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-9471?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ted Yu resolved KAFKA-9471. --- Resolution: Duplicate > Throw exception for DEAD StreamThread.St

[jira] [Created] (KAFKA-9471) Return empty collection for PENDING_SHUTDOWN

2020-01-23 Thread Ted Yu (Jira)
Ted Yu created KAFKA-9471: - Summary: Return empty collection for PENDING_SHUTDOWN Key: KAFKA-9471 URL: https://issues.apache.org/jira/browse/KAFKA-9471 Project: Kafka Issue Type: Improvement

[jira] [Resolved] (KAFKA-9464) Close the producer in completeShutdown

2020-01-23 Thread Ted Yu (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-9464?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ted Yu resolved KAFKA-9464. --- Resolution: Not A Problem > Close the producer in completeShutd

[jira] [Created] (KAFKA-9465) Enclose consumer call with catching InvalidOffsetException

2020-01-22 Thread Ted Yu (Jira)
Ted Yu created KAFKA-9465: - Summary: Enclose consumer call with catching InvalidOffsetException Key: KAFKA-9465 URL: https://issues.apache.org/jira/browse/KAFKA-9465 Project: Kafka Issue Type

[jira] [Created] (KAFKA-9464) Close the producer in completeShutdown

2020-01-22 Thread Ted Yu (Jira)
Ted Yu created KAFKA-9464: - Summary: Close the producer in completeShutdown Key: KAFKA-9464 URL: https://issues.apache.org/jira/browse/KAFKA-9464 Project: Kafka Issue Type: Bug Reporter

[jira] [Created] (KAFKA-9463) Transient failure in KafkaAdminClientTest.testListOffsets

2020-01-21 Thread Ted Yu (Jira)
Ted Yu created KAFKA-9463: - Summary: Transient failure in KafkaAdminClientTest.testListOffsets Key: KAFKA-9463 URL: https://issues.apache.org/jira/browse/KAFKA-9463 Project: Kafka Issue Type: Test

[jira] [Created] (KAFKA-9462) Correct exception message in DistributedHerder

2020-01-21 Thread Ted Yu (Jira)
Ted Yu created KAFKA-9462: - Summary: Correct exception message in DistributedHerder Key: KAFKA-9462 URL: https://issues.apache.org/jira/browse/KAFKA-9462 Project: Kafka Issue Type: Bug

Re: [VOTE] KIP-553: Disable all SSL protocols except TLSV1.2 by default.

2020-01-21 Thread Ted Yu
+1 On Tue, Jan 21, 2020 at 8:24 AM Rajini Sivaram wrote: > +1 (binding) > > Thanks for the KIP! > > Regards, > > Rajini > > > On Tue, Jan 21, 2020 at 3:43 PM Николай Ижиков > wrote: > > > Hello. > > > > I would like to start vote for KIP-553: Disable all SSL protocols except > > TLSV1.2 by

Re: [DISCUSS] : KIP-562: Allow fetching a key from a single partition rather than iterating over all the stores on an instance

2020-01-18 Thread Ted Yu
I guess this also means getPartition() should either throw an exception or > return null if the partition is unspecified. > > Thanks, > John > > On Sat, Jan 18, 2020, at 08:43, Ted Yu wrote: > > I wonder if the following two methods can be combined: > > > > Integ

Re: [DISCUSS] : KIP-562: Allow fetching a key from a single partition rather than iterating over all the stores on an instance

2020-01-18 Thread Ted Yu
I wonder if the following two methods can be combined: Integer getPartition() // would be null if unset or if "all partitions" boolean getAllLocalPartitions() // true/false if "all partitions" requested into: Integer getPartition() // would be null if unset or -1 if "all partitions" Cheers On

Re: [VOTE] 2.1.0 RC0

2018-10-24 Thread Ted Yu
+1 InternalTopicIntegrationTest failed during test suite run but passed with rerun. On Wed, Oct 24, 2018 at 3:48 AM Andras Beni wrote: > +1 (non-binding) > > Verified signatures and checksums of release artifacts > Performed quickstart steps on rc artifacts (both scala 2.11 and 2.12) and > one

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 ( >

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

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-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: > >

[jira] [Created] (KAFKA-7345) Potentially unclosed FileChannel in StateDirectory#unlock

2018-08-26 Thread Ted Yu (JIRA)
Ted Yu created KAFKA-7345: - Summary: Potentially unclosed FileChannel in StateDirectory#unlock Key: KAFKA-7345 URL: https://issues.apache.org/jira/browse/KAFKA-7345 Project: Kafka Issue Type: Bug

[jira] [Created] (KAFKA-7344) Return early when all tasks are assigned in StickyTaskAssignor#assignActive

2018-08-25 Thread Ted Yu (JIRA)
Ted Yu created KAFKA-7344: - Summary: Return early when all tasks are assigned in StickyTaskAssignor#assignActive Key: KAFKA-7344 URL: https://issues.apache.org/jira/browse/KAFKA-7344 Project: Kafka

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

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 Thu, Aug 23, 2018 at

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: > > >

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) >

[jira] [Created] (KAFKA-7334) Suggest changing config for state.dir in case of FileNotFoundException

2018-08-23 Thread Ted Yu (JIRA)
Ted Yu created KAFKA-7334: - Summary: Suggest changing config for state.dir in case of FileNotFoundException Key: KAFKA-7334 URL: https://issues.apache.org/jira/browse/KAFKA-7334 Project: Kafka

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

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

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

[jira] [Created] (KAFKA-7316) Use of filter method in KTable.scala may result in StackOverflowError

2018-08-20 Thread Ted Yu (JIRA)
Ted Yu created KAFKA-7316: - Summary: Use of filter method in KTable.scala may result in StackOverflowError Key: KAFKA-7316 URL: https://issues.apache.org/jira/browse/KAFKA-7316 Project: Kafka Issue

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

Re: subscribe

2018-08-18 Thread Ted Yu
Please see instructions here: http://kafka.apache.org/contact On Sat, Aug 18, 2018 at 8:18 AM Aegeaner wrote: > > >

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

2018-08-16 Thread Ted Yu
+1 On Thu, Aug 16, 2018 at 5:15 PM Yishun Guan wrote: > Hi all, > > I want to start a vote on this KIP: > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-347%3A++Enable+batching+in+FindCoordinatorRequest > > Here is the discussion thread: > >

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

2018-08-16 Thread Ted Yu
+1 On Thu, Aug 16, 2018 at 12:05 PM Vahid S Hashemian < vahidhashem...@us.ibm.com> wrote: > I would like to start a vote on KIP-325 which aims at adding a beginning > offset column to consumer group command describe output. > > The KIP: > >

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

2018-08-14 Thread Ted Yu
+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, Bill On Thu, Aug 9, 2018 at 4:20 PM John Roesler wrote: >

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

2018-08-14 Thread Ted Yu
+1 On Tue, Aug 14, 2018 at 10:42 AM Guozhang Wang wrote: > Hello folks, > > I'd like to start a voting thread on the following KIP: > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-356%3A+Add+withCachingDisabled%28%29+to+StoreBuilder > > It is a pretty straightforward one, adding a

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

2018-08-10 Thread Ted Yu
bq. this is the foundation of some later possible optimizations(enable batching in *describeConsumerGroups ...* *Can you say more why this change lays the foundation for the future optimizations ?* *You mentioned **FIND_COORDINATOR_REQUEST_V3 in the wiki but I don't see it in PR.* *I assume you

[jira] [Created] (KAFKA-7276) Consider using re2j to speed up regex operations

2018-08-10 Thread Ted Yu (JIRA)
Ted Yu created KAFKA-7276: - Summary: Consider using re2j to speed up regex operations Key: KAFKA-7276 URL: https://issues.apache.org/jira/browse/KAFKA-7276 Project: Kafka Issue Type: Task

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

2018-08-08 Thread Ted Yu
+1 On Wed, Aug 8, 2018 at 4:09 AM Mickael Maison wrote: > +1 (non-binding) > Thanks Vahid > On Wed, Aug 8, 2018 at 11:26 AM Kamal Chandraprakash > wrote: > > > > +1 (non-binding) > > > > Thanks for the KIP. > > > > On Wed, Aug 8, 2018 at 3:11 PM Stanislav Kozlovski < > stanis...@confluent.io>

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

2018-08-07 Thread Ted Yu
+1 On Tue, Aug 7, 2018 at 5:25 AM Thomas Becker wrote: > +1 (non-binding) > > We've hit issues with the log cleaner in the past, and this would be a > great improvement. > On Tue, 2018-08-07 at 12:19 +0100, Stanislav Kozlovski wrote: > > Hey everybody, > > I'm starting a vote on KIP-346 > > < >

Re: [DISCUSS] KIP-353: Allow Users to Configure Kafka Streams Timestamp Synchronization

2018-08-03 Thread Ted Yu
e > instance. > > Guozhang > > > On Fri, Aug 3, 2018 at 3:33 PM, Ted Yu wrote: > > > Guozhang: > > > > Do you plan to support dynamic config for the new config entry ? > > > > Cheers > > > > On Fri, Aug 3, 2018 at 2:00 PM Guozhang Wang wr

Re: [DISCUSS] KIP-353: Allow Users to Configure Kafka Streams Timestamp Synchronization

2018-08-03 Thread Ted Yu
Guozhang: Do you plan to support dynamic config for the new config entry ? Cheers On Fri, Aug 3, 2018 at 2:00 PM Guozhang Wang wrote: > Hello all, > > I would like to kick off a discussion on the following KIP, to allow users > control when a task can be processed based on its buffered

Moving build job Was: Build failed in Jenkins: kafka-trunk-jdk10 #364

2018-08-03 Thread Ted Yu
Should the build job run on nodes with the {{Hadoop}} label ? Based on recent builds@a.o discussion about how small the disk capacity is on the general {{ubuntu}} label, I think we should try {{Hadoop}} labeled machines. Cheers On Fri, Aug 3, 2018 at 3:08 PM Apache Jenkins Server <

Re: [VOTE] KIP-341: Update Sticky Assignor's User Data Protocol

2018-08-03 Thread Ted Yu
+1 On Fri, Aug 3, 2018 at 10:40 AM Jason Gustafson wrote: > +1 Thanks Vahid. > > On Thu, Aug 2, 2018 at 1:27 PM, Vahid S Hashemian < > vahidhashem...@us.ibm.com > > wrote: > > > Hi everyone, > > > > I believe the feedback on this KIP has been addressed so far. So I'd like > > to start a vote. >

Re: [VOTE] KIP-332: Update AclCommand to use AdminClient API

2018-08-02 Thread Ted Yu
+1 On Thu, Aug 2, 2018 at 7:33 PM Manikumar wrote: > Hi All, > > I would like to start voting on KIP-332 which allows AclCommand to use > AdminClient API for acl management. > > KIP: > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-332%3A+Update+AclCommand+to+use+AdminClient+API > >

Re: [VOTE] KIP-334 Include partitions in exceptions raised during consumer record deserialization/validation

2018-08-02 Thread Ted Yu
+1 Original message From: Stanislav Kozlovski Date: 8/2/18 2:41 AM (GMT-08:00) To: dev@kafka.apache.org Subject: [VOTE] KIP-334 Include partitions in exceptions raised during consumer record deserialization/validation Hey everybody, I'd like to start a vote thread for

Re: [DISCUSS] KIP-332: Update AclCommand to use AdminClient API

2018-07-30 Thread Ted Yu
Look good to me. On Mon, Jul 23, 2018 at 7:30 AM Manikumar wrote: > Hi all, > > I have created a KIP to use AdminClient API in AclCommand (kafka-acls.sh) > > * > https://cwiki.apache.org/confluence/display/KAFKA/KIP-332%3A+Update+AclCommand+to+use+AdminClient+API* > < >

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

2018-07-30 Thread Ted Yu
+1 On Mon, Jul 30, 2018 at 11:46 AM John Roesler wrote: > Hello devs, > > The discussion of KIP-328 has gone some time with no new comments, so I am > calling for a vote! > > Here's the KIP: https://cwiki.apache.org/confluence/x/sQU0BQ > > The basic idea is to provide: > * more usable control

Re: [Vote] KIP-321: Update TopologyDescription to better represent Source and Sink Nodes

2018-07-25 Thread Ted Yu
+1 On Wed, Jul 25, 2018 at 7:24 PM Nishanth Pradeep wrote: > Hello, > > I'm calling a vote for KIP-321: > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-321%3A+Update+TopologyDescription+to+better+represent+Source+and+Sink+Nodes > > Best, > Nishanth Pradeep >

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

2018-07-25 Thread Ted Yu
+1 On Wed, Jul 25, 2018 at 9:49 AM Kevin Lu wrote: > Hi All, > > I am calling a vote for KIP-344 > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-344%3A+The+auto-generated+client+id+should+be+passed+to+MetricsReporter > . > > If any committer feels that this minor change does not need

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

2018-07-24 Thread Ted Yu
Stanislav Kozlovski wrote: > Hi Ted, > > Yes, absolutely. Thanks for pointing that out! > > On Mon, Jul 23, 2018 at 6:12 PM Ted Yu wrote: > > > For `uncleanable-partitions`, should the example include topic name(s) ? > > > > Cheers > > > > On

Re: Seeing old tombstones in compacted topic

2018-07-23 Thread Ted Yu
Looking at some recent JIRAs, such as KAFKA-6568, which came in after the release of 0.11.0 Would that possibly be related to what you observed ? Cheers On Mon, Jul 23, 2018 at 6:23 PM Mitch Seymour wrote: > Hi all, > > We're using version 0.11.0 of Kafka (broker and client), and our Kafka >

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

2018-07-23 Thread Ted Yu
For `uncleanable-partitions`, should the example include topic name(s) ? Cheers On Mon, Jul 23, 2018 at 5:46 PM Stanislav Kozlovski wrote: > I renamed the KIP and that changed the link. Sorry about that. Here is the > new link: > >

[jira] [Created] (KAFKA-7195) StreamStreamJoinIntegrationTest fails in 2.0 Jenkins

2018-07-23 Thread Ted Yu (JIRA)
Ted Yu created KAFKA-7195: - Summary: StreamStreamJoinIntegrationTest fails in 2.0 Jenkins Key: KAFKA-7195 URL: https://issues.apache.org/jira/browse/KAFKA-7195 Project: Kafka Issue Type: Test

Re: [DISCUSS] KIP-334 Include partitions in exceptions raised during consumer record deserialization/validation

2018-07-20 Thread Ted Yu
Hi, Stanislav: Do you plan to start VOTE thread ? Cheers On Fri, Jul 20, 2018 at 6:11 PM Stanislav Kozlovski wrote: > Hey group, > > I added a Pull Request for this KIP - here it is > https://github.com/apache/kafka/pull/5410 > Please take a look. > > Best, > Stanislav > > On Thu, Jul 5, 2018

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

2018-07-20 Thread Ted Yu
Looks good to me. On Fri, Jul 20, 2018 at 12:17 AM Kevin Lu wrote: > Hi All, > > This is the discussion thread for KIP-344 ( > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-344%3A+The+auto-generated+client+id+should+be+passed+to+MetricsReporter > ). > > Thanks! > > Regards, > Kevin >

[jira] [Created] (KAFKA-7175) Make version checking logic more flexible in streams_upgrade_test.py

2018-07-17 Thread Ted Yu (JIRA)
Ted Yu created KAFKA-7175: - Summary: Make version checking logic more flexible in streams_upgrade_test.py Key: KAFKA-7175 URL: https://issues.apache.org/jira/browse/KAFKA-7175 Project: Kafka Issue

[jira] [Created] (KAFKA-7174) Improve version probing of subscription info

2018-07-17 Thread Ted Yu (JIRA)
Ted Yu created KAFKA-7174: - Summary: Improve version probing of subscription info Key: KAFKA-7174 URL: https://issues.apache.org/jira/browse/KAFKA-7174 Project: Kafka Issue Type: Improvement

Re: [VOTE] KIP-340: Allow kafka-reassign-partitions.sh and kafka-log-dirs.sh to take admin client property file

2018-07-17 Thread Ted Yu
+1 On Tue, Jul 17, 2018 at 1:40 PM Jason Gustafson wrote: > +1. This is useful (though the naming inconsistencies in the tools are > vexing, as always). > > -Jason > > On Tue, Jul 17, 2018 at 12:24 PM, Dong Lin wrote: > > > Hi all, > > > > It seems that there is no further concern with the

Re: KIP-327: Add describe all topics API to AdminClient

2018-07-14 Thread Ted Yu
McCabe, wrote: > > > Good point. We should probably have a maximum number of results like > > 1000 or something. That can go in the request RPC as well... > > Cheers, > > Colin > > > > On Fri, Jul 13, 2018, at 18:15, Ted Yu wrote: > > > bq. describe topic

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

2018-07-13 Thread Ted Yu
, the notion of null value support would be conveyed to user in a friendly manner. Cheers On Fri, Jul 13, 2018 at 5:51 PM Colin McCabe wrote: > On Fri, Jul 13, 2018, at 17:45, Ted Yu wrote: > > Looking at modifyConfigs API, it doesn't seem that ConcurrentHashMap > should be used as th

Re: KIP-327: Add describe all topics API to AdminClient

2018-07-13 Thread Ted Yu
bq. describe topics by a regular expression on the server side Should caution be taken if the regex doesn't filter ("*") ? Cheers On Fri, Jul 13, 2018 at 6:02 PM Colin McCabe wrote: > As Jason wrote, this won't scale as the number of partitions increases. > We already have users who have tens

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

2018-07-13 Thread Ted Yu
in > > On Wed, Jul 11, 2018, at 21:28, Ted Yu wrote: > > bq. Map changes, Set > removals, > > > > Is it possible to combine the two parameters into one Map where null > Config > > value signifies removal of config ? > > This way, the following

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

2018-07-11 Thread Ted Yu
bq. Map changes, Set removals, Is it possible to combine the two parameters into one Map where null Config value signifies removal of config ? This way, the following wouldn't occur (reducing un-intended config removal): bq. If a configuration key is specified in both *changes* and

Re: [DISCUSS] KIP-340: Allow kafka-reassign-partitions.sh and kafka-log-dirs.sh to take admin client property file

2018-07-11 Thread Ted Yu
t of this change is worth the effort to deprecate the > old config. Maybe we should do it separately if more people thing it is > necessary? > > On Wed, Jul 11, 2018 at 8:01 PM, Ted Yu wrote: > > > bq. "--config-file" in ConfigCommand. > > > > Please update t

Re: [DISCUSS] KIP-340: Allow kafka-reassign-partitions.sh and kafka-log-dirs.sh to take admin client property file

2018-07-11 Thread Ted Yu
e updated the KIP as suggested. > > Thanks, > Dong > > On Wed, Jul 11, 2018 at 5:57 PM, Ted Yu wrote: > > > Looking at StreamsResetter.java : > > > >commandConfigOption = optionParser.accepts("config-file", > "Property > > f

Re: [DISCUSS] KIP-340: Allow kafka-reassign-partitions.sh and kafka-log-dirs.sh to take admin client property file

2018-07-11 Thread Ted Yu
Looking at StreamsResetter.java : commandConfigOption = optionParser.accepts("config-file", "Property file containing configs to be passed to admin cl Not sure you have considered naming the option in the above fashion. Probably add the above to Alternative section. Cheers On Wed, Jul

Re: [VOTE] 2.0.0 RC2

2018-07-10 Thread Ted Yu
+1 Ran thru test suite. Checked signatures. On Tue, Jul 10, 2018 at 10:17 AM Rajini Sivaram wrote: > Hello Kafka users, developers and client-developers, > > > This is the third candidate for release of Apache Kafka 2.0.0. > > > This is a major version release of Apache Kafka. It includes 40

Re: [VOTE] 1.1.1 RC3

2018-07-09 Thread Ted Yu
+1 Ran test suite. Checked signatures. On Sun, Jul 8, 2018 at 3:36 PM Dong Lin wrote: > Hello Kafka users, developers and client-developers, > > This is the fourth candidate for release of Apache Kafka 1.1.1. > > Apache Kafka 1.1.1 is a bug-fix release for the 1.1 branch that was first >

[jira] [Resolved] (KAFKA-6335) SimpleAclAuthorizerTest#testHighConcurrencyModificationOfResourceAcls fails intermittently

2018-07-08 Thread Ted Yu (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-6335?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ted Yu resolved KAFKA-6335. --- Resolution: Cannot Reproduce > SimpleAclAuthorizerTest#testHighConcurrencyModificationOfResourceAcls fa

Re: [VOTE] KIP-321: Add method to get TopicNameExtractor in TopologyDescription

2018-07-03 Thread Ted Yu
Hi, I don't seem to find response to John's comment : http://search-hadoop.com/m/Kafka/uyzND11alrn1G5N3Y1?subj=Re+Discuss+KIP+321+Add+method+to+get+TopicNameExtractor+in+TopologyDescription On Tue, Jul 3, 2018 at 7:38 PM, Nishanth Pradeep wrote: > Hello, > > I would like to start the vote on

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

2018-07-03 Thread Ted Yu
f it turns out to be a problem. What do you think? > > #2, please try this link and see if it works now: > https://drive.google.com/file/d/1QbPDqfT59A2X4To2p3OfD5YeJR8aW > DK7/view?usp=sharing > > Regards, > Lucas > > > On Mon, Jul 2, 2018 at 5:52 PM, Ted Yu wrote:

Re: [VOTE] KIP-322: Return new error code for DeleteTopics API when topic deletion disabled.

2018-07-03 Thread Ted Yu
+1 On Tue, Jul 3, 2018 at 9:05 AM, Mickael Maison wrote: > +1 (non binding) > Thanks for the KIP > > On Tue, Jul 3, 2018 at 4:59 PM, Vahid S Hashemian > wrote: > > +1 (non-binding) > > > > --Vahid > > > > > > > > From: Gwen Shapira > > To: dev > > Date: 07/03/2018 08:49 AM > >

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

2018-07-02 Thread Ted Yu
an exposing it as a config? > > 2. Sorry for losing the format of the table, I've attached it below as a > picture > > > Regards, > Lucas > > On Fri, Jun 29, 2018 at 5:28 PM, Ted Yu wrote: > >> bq. which is hard coded to be 300 milliseconds >> >> Have

[jira] [Resolved] (KAFKA-7124) Number of AnyLogDir should match the length of the replicas list

2018-07-01 Thread Ted Yu (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-7124?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ted Yu resolved KAFKA-7124. --- Resolution: Not A Problem > Number of AnyLogDir should match the length of the replicas l

[jira] [Created] (KAFKA-7124) Number of AnyLogDir should match the length of the replicas list

2018-07-01 Thread Ted Yu (JIRA)
Ted Yu created KAFKA-7124: - Summary: Number of AnyLogDir should match the length of the replicas list Key: KAFKA-7124 URL: https://issues.apache.org/jira/browse/KAFKA-7124 Project: Kafka Issue Type

Re: [kafka-clients] [VOTE] 2.0.0 RC1

2018-06-30 Thread Ted Yu
+1 Checked signatures Ran test suite On Sat, Jun 30, 2018 at 2:37 AM, Rajini Sivaram wrote: > Hi Manikumar, > > Thank you for pointing that out, I had forgotten to drop the old artifacts. > New artifacts should be there now. > > Regards, > > Rajini > > On Sat, Jun 30, 2018 at 7:44 AM,

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

2018-06-29 Thread Ted Yu
bq. which is hard coded to be 300 milliseconds Have you considered making the duration configurable ? The comparison at the end of your email seems to be copied where tabular form is lost. Do you mind posting that part again ? Thanks On Fri, Jun 29, 2018 at 4:53 PM, Lucas Wang wrote: > Hi

Re: [VOTE] 0.10.2.2 RC1

2018-06-29 Thread Ted Yu
+1 Ran test suite. Checked signatures. On Fri, Jun 29, 2018 at 10:21 AM, Jason Gustafson wrote: > +1 (binding). I checked release notes, documentation, and went through the > quickstart. > > Thanks Matthias! > > On Fri, Jun 22, 2018 at 6:43 PM, Matthias J. Sax > wrote: > > > Hello Kafka

Re: [VOTE] KIP-308: Support dynamic update of max.connections.per.ip/max.connections.per.ip.overrides configs

2018-06-29 Thread Ted Yu
+1 On Fri, Jun 29, 2018 at 7:29 AM, Manikumar wrote: > Hi All, > > I would like to start voting on KIP-308 which would add support for dynamic > update of max.connections.per.ip/max.connections.per.ip.overrides configs > >

Re: [VOTE] 1.1.1 RC2

2018-06-28 Thread Ted Yu
+1 Ran test suite which passed. On Thu, Jun 28, 2018 at 6:12 PM, Dong Lin wrote: > Hello Kafka users, developers and client-developers, > > This is the second candidate for release of Apache Kafka 1.1.1. > > Apache Kafka 1.1.1 is a bug-fix release for the 1.1 branch that was first > released

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

2018-06-28 Thread Ted Yu
+1 On Thu, Jun 28, 2018 at 4:56 AM, Luís Cabral wrote: > Hi Ted, > Can I also get your input on this? > > bq. +1 from my side for using `compaction.strategy` with values > "offset","timestamp" and "header" and `compaction.strategy.header` > -Matthias > > bq. +1 from me as well. > -Guozhang > >

Re: [VOTE] KIP-330: Add retentionPeriod in SessionBytesStoreSupplier

2018-06-27 Thread Ted Yu
+1 On Wed, Jun 27, 2018 at 4:40 PM, Bill Bejeck wrote: > +1 > > -Bill > > On Wed, Jun 27, 2018 at 7:39 PM Guozhang Wang wrote: > > > Hello folks, > > > > I'd like to start a voting thread on KIP-330. I've intentionally skipped > > the discuss phase since it is a pretty straight-forward public

Re: [VOTE] 2.0.0 RC0

2018-06-27 Thread Ted Yu
+1 Checked signatures Ran test suite On Mon, Jun 25, 2018 at 11:12 AM, Thomas Crayford wrote: > +1 (non-binding) Heroku has run our usual set of upgrade and performance > tests, and we haven't found any notable issues through that. > > On Sat, Jun 23, 2018 at 12:30 AM, Vahid S Hashemian < >

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

2018-06-27 Thread Ted Yu
hods I listed. > > For example: > > windowCounts > .suppress( > Suppression > .suppressLateEvents(Duration.ofMinutes(10)) > .suppressIntermediateEvents( > IntermediateSuppression.emitAfter(Duration.ofMinutes(10)) > )

Re: [DISCUSS] KIP-322: Return new error code for DeleteTopics API when topic deletion disabled.

2018-06-27 Thread Ted Yu
bq. set the serialVersionUID = 1L in Kafka exception classes Indeed. Should have checked earlier. On Wed, Jun 27, 2018 at 1:38 AM, Manikumar wrote: > Hi, > > Thanks for the review. > > > > nit: you are going to fill in an actual value below in your PR, right ? > > private static final

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

2018-06-26 Thread Ted Yu
I started to read this KIP which contains a lot of materials. One suggestion: .suppress( new Suppression() Do you think it would be more consistent with the rest of Streams data structures by supporting `of` ? Suppression.of(Duration.ofMinutes(10)) Cheers On Tue, Jun 26, 2018

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

2018-06-26 Thread Ted Yu
nit: bq. leaving this empty for compacted topics Some user(s) may be confused by empty partition size. How about emitting 'compacted' for compacted topics ? Cheers On Tue, Jun 26, 2018 at 4:42 PM, Gwen Shapira wrote: > It will be. In my experience most topics aren't compacted, so it will

Re: [DISCUSS] KIP-322: Return new error code for DeleteTopics API when topic deletion disabled.

2018-06-26 Thread Ted Yu
Looks good overall. nit: you are going to fill in an actual value below in your PR, right ? private static final long serialVersionUID = 1L; In Motivation, please mention the IllegalStateException scenario. On Tue, Jun 26, 2018 at 9:34 AM, Manikumar wrote: > Hi all, > > I have created a

Re: [DISCUSS] KIP-326: Schedulable KTable as Graph source

2018-06-26 Thread Ted Yu
What's the relationship between this KIP and KIP-323 ? Thanks On Tue, Jun 26, 2018 at 11:22 AM, Flávio Stutz wrote: > Hey, guys, I've just created a new KIP about creating a new DSL graph > source for realtime partitioned consolidations. > > We have faced the following scenario/problem in a

Re: [Discuss] KIP-321: Add method to get TopicNameExtractor in TopologyDescription

2018-06-25 Thread Ted Yu
gt; > -Matthias > > On 6/25/18 5:19 PM, Ted Yu wrote: > > If there are different TopicNameExtractor classes from multiple sink > nodes, > > the new method should throw exception alerting user of such scenario. > > > > > > On Mon, Jun 25, 2018 at

Re: SASL Unit test failing

2018-06-25 Thread Ted Yu
", arch: "amd64", family: "unix" On Mon, Jun 25, 2018 at 5:42 PM, Ted Yu wrote: > Here was the command I used: > > ./gradlew -Dtest.single=SaslAuthenticatorTest clients:test > > On Mon, Jun 25, 2018 at 5:39 PM, Ahmed A wrote: > >> I ran test

Re: SASL Unit test failing

2018-06-25 Thread Ted Yu
ahmed 3.10.0-514.36.5.el7.x86_64 #1 SMP Thu Dec 28 21:42:18 EST > 2017 x86_64 x86_64 x86_64 GNU/Linux > > > Can you please let me know how I can run an individual unit test, what > options do I provide? > > > Thank you, > Ahmed. > > > > On Mon, Jun 25, 2

Re: [Discuss] KIP-321: Add method to get TopicNameExtractor in TopologyDescription

2018-06-25 Thread Ted Yu
If there are different TopicNameExtractor classes from multiple sink nodes, the new method should throw exception alerting user of such scenario. On Mon, Jun 25, 2018 at 2:23 PM, Bill Bejeck wrote: > Thanks for the KIP! > > Overall I'm +1 on the KIP. I have one question. > > The KIP states

Re: [DISCUSS] KIP-323: Schedulable KTable as Graph source

2018-06-25 Thread Ted Yu
This would be useful feature. In the Public Interfaces section, the new method lacks a closing parenthesis. In the Proposed Changes section, if the order of the 3 bullets can match the order of the parameters of the new method, it would be easier to read. For Rejected Alternatives #2, can you

Re: [VOTE] KIP-319: Replace numSegments to segmentInterval in Streams window configurations

2018-06-25 Thread Ted Yu
+1 On Mon, Jun 25, 2018 at 2:56 PM, John Roesler wrote: > Hello All, > > Thanks for the discussion on KIP-319. I'd now like to start the voting. > > As a reminder, KIP-319 proposes a fix to an issue I identified in > KAFKA-7080. Specifically, the issue is that we're creating >

Re: SASL Unit test failing

2018-06-25 Thread Ted Yu
I ran the test alone which passed. Can you include -i on the command line to see if there is some clue from the output ? Here is my environment: Java version: 1.8.0_151, vendor: Oracle Corporation Java home: /Library/Java/JavaVirtualMachines/jdk1.8.0_151.jdk/Contents/Home/jre Default locale:

Re: [VOTE] KIP-312: Add Overloaded StreamsBuilder Build Method to Accept java.util.Properties

2018-06-25 Thread Ted Yu
+1 On Mon, Jun 25, 2018 at 9:45 AM, Guozhang Wang wrote: > +1. > > On Mon, Jun 25, 2018 at 8:12 AM, Matthias J. Sax > wrote: > > > +1 (binding) > > > > On 6/25/18 6:11 AM, Bill Bejeck wrote: > > > All, > > > I'd like to start a vote for this KIP now. > > > > > > Thanks, > > > Bill > > > > > >

Re: [VOTE] 1.0.2 RC0

2018-06-22 Thread Ted Yu
+1 Ran test suite. Checked signatures. On Fri, Jun 22, 2018 at 11:42 AM, Vahid S Hashemian < vahidhashem...@us.ibm.com> wrote: > +1 (non-binding) > > Built from source and ran quickstart successfully on Ubuntu (with Java 8). > > Thanks for running the release Matthias! > --Vahid > > > > >

Re: [VOTE] KIP-313: Add KStream.flatTransform and KStream.flatTransformValues

2018-06-22 Thread Ted Yu
+1 On Fri, Jun 22, 2018 at 2:50 AM, Bruno Cadonna wrote: > Hi list, > > I would like to voting on this KIP. > > I created a first PR[1] that adds flatTransform. Once I get some > feedback, I will start work on flatTransformValues. > > Best regards, > Bruno > > [1]

Re: [DISCUSS] KIP-319: Replace segments with segmentSize in WindowBytesStoreSupplier

2018-06-21 Thread Ted Yu
> > opposed > > > to "segments". Now it makes sense that you meant as opposed to > > > "segmentSize". > > > > > > I named it that way to match the peer method "windowSize", which is > also > > a > > > quan

  1   2   3   4   5   >