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

Requesting Permission To Create KIP And Assign JIRAs

2018-06-26 Thread Kevin Lu
Hi All, I would like to start contributing to Kafka but I do not have access to create KIPs or assign JIRA to myself. Can someone set it up for me? Confluence id: lu.kevin Jira username: lu.kevin Email: lu.ke...@berkeley.edu Thanks! Regards, Kevin

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

2018-06-26 Thread Apache Jenkins Server
See

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

2018-06-26 Thread Gwen Shapira
Thank you! On Tue, Jun 26, 2018 at 8:47 PM, Vahid S Hashemian < vahidhashem...@us.ibm.com> wrote: > Thanks for the feedback. The KIP is updated to also include a "partition > size" column. > > --Vahid > > > > > From: Ted Yu > To: dev@kafka.apache.org > Date: 06/26/2018 06:21 PM >

Build failed in Jenkins: kafka-2.0-jdk8 #58

2018-06-26 Thread Apache Jenkins Server
See Changes: [wangguoz] HOTFIX: update Streams security docs -- [...truncated 430.29 KB...] kafka.controller.ReplicaStateMachineTest >

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

2018-06-26 Thread Vahid S Hashemian
Thanks for the feedback. The KIP is updated to also include a "partition size" column. --Vahid From: Ted Yu To: dev@kafka.apache.org Date: 06/26/2018 06:21 PM Subject:Re: [DISCUSS] KIP-325: Extend Consumer Group Command to Show Beginning Offsets nit: bq. leaving this

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

2018-06-26 Thread Apache Jenkins Server
See Changes: [wangguoz] HOTFIX: update Streams security docs -- [...truncated 1.53 MB...] kafka.coordinator.transaction.TransactionCoordinatorTest >

Re: [DISCUSS] KIP-320: Allow fetchers to detect and handle log truncation

2018-06-26 Thread Dong Lin
Hey Jason, Thanks for the explanation. Please correct me if this is wrong. The "unknown truncation offset" scenario happens when consumer does not have the full leaderEpoch -> offset mapping. In this case we can still use the KIP-101-based approach to truncate offset to "start offset of the

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-325: Extend Consumer Group Command to Show Beginning Offsets

2018-06-26 Thread Gwen Shapira
It will be. In my experience most topics aren't compacted, so it will still be valuable. If not difficult, leaving this empty for compacted topics to avoid confusion will also be nice. On Tue, Jun 26, 2018 at 4:29 PM, Vahid S Hashemian < vahidhashem...@us.ibm.com> wrote: > Hi Gwen, > > Thanks

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

2018-06-26 Thread Vahid S Hashemian
Hi Gwen, Thanks for the feedback. Regarding the partition size, couldn't "end offset - start offset" be misleading for compacted topics? --Vahid From: Gwen Shapira To: dev Date: 06/26/2018 02:36 PM Subject:Re: [DISCUSS] KIP-325: Extend Consumer Group Command to Show

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

2018-06-26 Thread Matthias J. Sax
NP. Thanks! On 6/26/18 2:54 PM, John Roesler wrote: > Sorry for the misunderstanding, Matthias. > > I have created https://issues.apache.org/jira/browse/KAFKA-7106 and > https://issues.apache.org/jira/browse/KAFKA-7107 to track these issues. > > Thanks, > -John > > On Mon, Jun 25, 2018 at

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

2018-06-26 Thread Apache Jenkins Server
See Changes: [mjsax] MINOR: Fix comment in quick union (#5244) -- [...truncated 434.42 KB...] kafka.utils.SchedulerTest > testRestart STARTED kafka.utils.SchedulerTest >

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

2018-06-26 Thread John Roesler
Sorry for the late comment, Looking at the other pieces of TopologyDescription, I noticed that pretty much all of the "payload" of these description nodes are strings. Should we consider returning a string from `topicNameExtractor()` instead? In fact, if we did that, we could consider calling

Re: [DISCUSS] KIP-263: Allow broker to skip sanity check of inactive segments on broker startup

2018-06-26 Thread Jason Gustafson
Hey Dong, Sorry for being slow to catch up to this. I think the benefit of the sanity check seems a little dubious in the first place. We detect garbage at the end of the index file, but that's about it. Is there any reason to think that corruption is more likely to occur there or any other

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

2018-06-26 Thread John Roesler
Sorry for the misunderstanding, Matthias. I have created https://issues.apache.org/jira/browse/KAFKA-7106 and https://issues.apache.org/jira/browse/KAFKA-7107 to track these issues. Thanks, -John On Mon, Jun 25, 2018 at 10:06 PM Matthias J. Sax wrote: > KAFKA-7080 is for this KIP. > > I meant

[jira] [Created] (KAFKA-7107) Ability to configure state store for JoinWindows in KStream-KStream join

2018-06-26 Thread John Roesler (JIRA)
John Roesler created KAFKA-7107: --- Summary: Ability to configure state store for JoinWindows in KStream-KStream join Key: KAFKA-7107 URL: https://issues.apache.org/jira/browse/KAFKA-7107 Project: Kafka

[jira] [Created] (KAFKA-7106) Remove segment/segmentInterval from Window definition

2018-06-26 Thread John Roesler (JIRA)
John Roesler created KAFKA-7106: --- Summary: Remove segment/segmentInterval from Window definition Key: KAFKA-7106 URL: https://issues.apache.org/jira/browse/KAFKA-7106 Project: Kafka Issue

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

2018-06-26 Thread Gwen Shapira
Small suggestion: you can also add a "partition size" column - difference between log-end and log-start. We've had users ask for this. On Tue, Jun 26, 2018 at 2:34 PM, Gwen Shapira wrote: > This will be useful! Thank you :) > > On Tue, Jun 26, 2018 at 11:23 AM, Vahid S Hashemian < >

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

2018-06-26 Thread Gwen Shapira
This will be useful! Thank you :) On Tue, Jun 26, 2018 at 11:23 AM, Vahid S Hashemian < vahidhashem...@us.ibm.com> wrote: > Hi everyone, > > I have created a trivial KIP to improve the offset reporting of the > consumer group command: > https://cwiki.apache.org/confluence/display/KAFKA/KIP- >

Re: [VOTE] KIP-324: Add method to get metrics() in AdminClient

2018-06-26 Thread Guozhang Wang
+1. Thanks. On Tue, Jun 26, 2018 at 2:31 PM, Yishun Guan wrote: > Hi All, > > I am starting a vote on this KIP: > > https://cwiki.apache.org/confluence/x/lQg0BQ > > Thanks, > Yishun > -- -- Guozhang

[VOTE] KIP-324: Add method to get metrics() in AdminClient

2018-06-26 Thread Yishun Guan
Hi All, I am starting a vote on this KIP: https://cwiki.apache.org/confluence/x/lQg0BQ Thanks, Yishun

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

2018-06-26 Thread Apache Jenkins Server
See Changes: [wangguoz] MINOR: Add note about num.standby.replicas (#5271) -- [...truncated 435.06 KB...] kafka.utils.SchedulerTest > testRestart STARTED

[jira] [Created] (KAFKA-7105) Refactor RocksDBSegmentsBatchingRestoreCallback and RocksDBBatchingRestoreCallback into a single class

2018-06-26 Thread Liquan Pei (JIRA)
Liquan Pei created KAFKA-7105: - Summary: Refactor RocksDBSegmentsBatchingRestoreCallback and RocksDBBatchingRestoreCallback into a single class Key: KAFKA-7105 URL: https://issues.apache.org/jira/browse/KAFKA-7105

[jira] [Created] (KAFKA-7104) ReplicaFetcher thread may die because of inconsistent log start offset in fetch response

2018-06-26 Thread Anna Povzner (JIRA)
Anna Povzner created KAFKA-7104: --- Summary: ReplicaFetcher thread may die because of inconsistent log start offset in fetch response Key: KAFKA-7104 URL: https://issues.apache.org/jira/browse/KAFKA-7104

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

[jira] [Created] (KAFKA-7103) Use bulkloading for RocksDBSegmentedBytesStore during init

2018-06-26 Thread Liquan Pei (JIRA)
Liquan Pei created KAFKA-7103: - Summary: Use bulkloading for RocksDBSegmentedBytesStore during init Key: KAFKA-7103 URL: https://issues.apache.org/jira/browse/KAFKA-7103 Project: Kafka Issue

Re: [VOTE] KIP-324: Add method to get metrics() in AdminClient

2018-06-26 Thread Yishun Guan
Hi Colin, I agree with what Guozhang's opinion that because all the other clients have it (producer, consumer..) and this will gain more visibility for those application that use admin client. (Now I added this sentence to the KIP) Since this returns an unmodifiableMap(like all the other

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

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

2018-06-26 Thread John Roesler
Hello devs and users, Please take some time to consider this proposal for Kafka Streams: KIP-328: Ability to suppress updates for KTables link: https://cwiki.apache.org/confluence/x/sQU0BQ The basic idea is to provide: * more usable control over update rate (vs the current state store caches)

Re: [DISCUSS] KIP-320: Allow fetchers to detect and handle log truncation

2018-06-26 Thread Jason Gustafson
The other thing I forgot to mention is that resetting the offset using the leader epoch is only available with the latest message format. By supporting reset by timestamp, users on the v1 format can still get some benefit from this KIP. -Jason On Tue, Jun 26, 2018 at 11:47 AM, Jason Gustafson

Build failed in Jenkins: kafka-2.0-jdk8 #57

2018-06-26 Thread Apache Jenkins Server
See Changes: [lindong28] KAFKA-6949; alterReplicaLogDirs() should grab partition lock when -- [...truncated 434.51 KB...] kafka.zookeeper.ZooKeeperClientTest >

Re: SASL Unit test failing

2018-06-26 Thread Rajini Sivaram
Colin/Ahmed, Can you run with debug logging turned on and attach the logs? You could just change log level in clients/src/test/resources/log4j.properties and run the tests. Thanks, Rajini On Tue, Jun 26, 2018 at 7:40 PM, Colin McCabe wrote: > On trunk, testMultipleServerMechanisms failed

[jira] [Created] (KAFKA-7102) Error in fetch kafka.server.ReplicaFetcherThread$FetchRequest@10c8df20

2018-06-26 Thread sankar (JIRA)
sankar created KAFKA-7102: - Summary: Error in fetch kafka.server.ReplicaFetcherThread$FetchRequest@10c8df20 Key: KAFKA-7102 URL: https://issues.apache.org/jira/browse/KAFKA-7102 Project: Kafka

Re: [VOTE] KIP-324: Add method to get metrics() in AdminClient

2018-06-26 Thread Colin McCabe
Can you add a little more explanation to the KIP for why you are adding this method? Is it something streams needs, for example? Will it help other applications that use admin client and want to expose metrics? What are the thread-safety guarantees for the map which is returned? best, Colin

Re: [DISCUSS] KIP-320: Allow fetchers to detect and handle log truncation

2018-06-26 Thread Jason Gustafson
Hey Dong, Thanks for the comments. - The KIP says that, with auto.offset.reset="closest", timestamp is used to > find offset if truncation offset is unknown. It seems that if consumer > knows the timestamp of the last message, then the consumer should also know > the (offset, leaderEpoch) of the

Re: SASL Unit test failing

2018-06-26 Thread Colin McCabe
On trunk, testMultipleServerMechanisms failed for me, as well as testAuthenticateCallbackHandlerMechanisms and testMechanismPluggability. org.apache.kafka.common.security.authenticator.SaslAuthenticatorTest > testMultipleServerMechanisms FAILED java.lang.AssertionError at

[VOTE] KIP-324: Add method to get metrics() in AdminClient

2018-06-26 Thread Yishun Guan
Hi All, I am starting a vote on this KIP: https://cwiki.apache.org/confluence/x/lQg0BQ Thanks, Yishun

[jira] [Created] (KAFKA-7101) Session Window store should set topic policy `compact,cleanup`

2018-06-26 Thread Matthias J. Sax (JIRA)
Matthias J. Sax created KAFKA-7101: -- Summary: Session Window store should set topic policy `compact,cleanup` Key: KAFKA-7101 URL: https://issues.apache.org/jira/browse/KAFKA-7101 Project: Kafka

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

2018-06-26 Thread Vahid S Hashemian
Hi everyone, I have created a trivial KIP to improve the offset reporting of the consumer group command: https://cwiki.apache.org/confluence/display/KAFKA/KIP-325%3A+Extend+Consumer+Group+Command+to+Show+Beginning+Offsets Looking forward to your feedback! Thanks. --Vahid

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

2018-06-26 Thread Flávio Stutz
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 lot of situations with KStreams: - Huge incoming data being processed by numerous application instances - Need to aggregate

Re: [DISCUSSION] KIP-324: Add method to get metrics() in AdminClient

2018-06-26 Thread Yishun Guan
Sure, that sounds good. - Yishun On Tue, Jun 26, 2018 at 11:19 AM, Guozhang Wang wrote: > Thanks Yishun. > > I think this should be a straight-forward one as "metrics()" is simply an > overlooked API that AdminClient should have since day one, as other > clients. We can go directly into the

Re: [DISCUSSION] KIP-324: Add method to get metrics() in AdminClient

2018-06-26 Thread Guozhang Wang
Thanks Yishun. I think this should be a straight-forward one as "metrics()" is simply an overlooked API that AdminClient should have since day one, as other clients. We can go directly into the voting process. Guozhang On Tue, Jun 26, 2018 at 11:15 AM, Yishun Guan wrote: > Hi All, > > I

[DISCUSSION] KIP-324: Add method to get metrics() in AdminClient

2018-06-26 Thread Yishun Guan
Hi All, I created a KIP to extract metrics() from AdminClient, here is the link: https://cwiki.apache.org/confluence/x/lQg0BQ Let me know what you think. Thanks @guozhangwang and @mjsax for guiding me. Best, Yishun

Re: [DISCUSS] KIP-320: Allow fetchers to detect and handle log truncation

2018-06-26 Thread Dong Lin
Hey Jason, Thanks for the KIP! It is pretty useful. At high level the new set of reset policies may be a bit complicated and confusing to users. I am wondering whether we can simplify it. A few questions below: - The KIP says that, with auto.offset.reset="closest", timestamp is used to find

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

2018-06-26 Thread Harsha
Thanks for the pointer. Will take a look might suit our requirements better. Thanks, Harsha On Mon, Jun 25th, 2018 at 2:52 PM, Lucas Wang wrote: > > > > Hi Harsha, > > If I understand correctly, the replication quota mechanism proposed in > KIP-73 can be helpful in that scenario. > Have

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

2018-06-26 Thread Harsha
This is very useful. LGTM. Thanks, Harsha On Mon, Jun 25th, 2018 at 10:20 AM, Dong Lin wrote: > > > > Hey Manikumar, > > Thanks much for the KIP. It looks pretty good. > > Thanks, > Dong > > On Thu, Jun 21, 2018 at 11:38 PM, Manikumar < manikumar.re...@gmail.com > > wrote: > > > Hi all,

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

2018-06-26 Thread Manikumar
Hi all, I have created a minor KIP to return new error code for DeleteTopics API when topic deletion disabled. https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=87295558 Please take a look. Thanks,

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

2018-06-26 Thread Bill Bejeck
+1 On Mon, Jun 25, 2018 at 11:07 PM Matthias J. Sax wrote: > +1 (binding) > > On 6/25/18 3:00 PM, Guozhang Wang wrote: > > +1 > > > > On Mon, Jun 25, 2018 at 2:58 PM, Ted Yu wrote: > > > >> +1 > >> > >> On Mon, Jun 25, 2018 at 2:56 PM, John Roesler > wrote: > >> > >>> Hello All, > >>> > >>>

[jira] [Resolved] (KAFKA-7100) kafka.tools.GetOffsetShell with enable Kerberos Security on Kafka1.0

2018-06-26 Thread Manikumar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-7100?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar resolved KAFKA-7100. -- Resolution: Duplicate This is being tracked in KAFKA-5235. > kafka.tools.GetOffsetShell with enable

[jira] [Created] (KAFKA-7100) kafka.tools.GetOffsetShell with enable Kerberos Security on Kafka1.0

2018-06-26 Thread abdullah toraman (JIRA)
abdullah toraman created KAFKA-7100: --- Summary: kafka.tools.GetOffsetShell with enable Kerberos Security on Kafka1.0 Key: KAFKA-7100 URL: https://issues.apache.org/jira/browse/KAFKA-7100

[jira] [Created] (KAFKA-7099) KafkaLog4jAppender - not sending any message with level DEBUG

2018-06-26 Thread Vincent Lebreil (JIRA)
Vincent Lebreil created KAFKA-7099: -- Summary: KafkaLog4jAppender - not sending any message with level DEBUG Key: KAFKA-7099 URL: https://issues.apache.org/jira/browse/KAFKA-7099 Project: Kafka

[jira] [Resolved] (KAFKA-5079) ProducerBounceTest fails occasionally with a SocketTimeoutException

2018-06-26 Thread Manikumar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5079?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar resolved KAFKA-5079. -- Resolution: Fixed ProducerBounceTest is removed as part old consumer changes. > ProducerBounceTest

[jira] [Resolved] (KAFKA-5080) Convert ProducerBounceTest to use the new KafkaConsumer

2018-06-26 Thread Manikumar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5080?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar resolved KAFKA-5080. -- Resolution: Fixed ProducerBounceTest is removed as part of old consumer changes. > Convert

[jira] [Resolved] (KAFKA-2488) System tests: updated console_consumer.py to support new consumer

2018-06-26 Thread Manikumar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2488?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar resolved KAFKA-2488. -- Resolution: Fixed Support was added in > 0.10.1 Kafka versions. > System tests: updated

[jira] [Resolved] (KAFKA-2215) Improve Randomness for ConsoleConsumer

2018-06-26 Thread Manikumar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2215?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar resolved KAFKA-2215. -- Resolution: Not A Problem Closing inactive issue. Also the default for console consumer's

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

2018-06-26 Thread Apache Jenkins Server
See

Re: [VOTE] 2.0.0 RC0

2018-06-26 Thread Jakub Scholz
+1 (non-binding) ... I ran my tests and verified the RC0 against my applications. On Mon, Jun 25, 2018 at 8:12 PM 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,

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

2018-06-26 Thread Jakub Scholz
+1 (non-binding) ... I ran my tests and verified the RC1 with my applications. On Mon, Jun 25, 2018 at 7:31 PM Manikumar wrote: > +1 (non-binding) Ran tests, Verified quick start, producer/consumer perf > tests > > > On Sat, Jun 23, 2018 at 8:11 AM Dong Lin wrote: > > > Thank you for