Re: [VOTE] KIP-229: DeleteGroups API

2018-01-16 Thread Edoardo Comar
+1 (non binding) - thanks Vahid -- Edoardo Comar IBM Message Hub IBM UK Ltd, Hursley Park, SO21 2JN From: Ted Yu To: dev@kafka.apache.org Date: 15/01/2018 20:33 Subject:Re: [VOTE] KIP-229: DeleteGroups API +1 On Mon, Jan 15

[jira] [Created] (KAFKA-6449) KafkaConsumer happen 40s timeOut when poll data after pollThread sleep more than request.timeout.ms

2018-01-16 Thread zhaoshijie (JIRA)
zhaoshijie created KAFKA-6449: - Summary: KafkaConsumer happen 40s timeOut when poll data after pollThread sleep more than request.timeout.ms Key: KAFKA-6449 URL: https://issues.apache.org/jira/browse/KAFKA-6449

Re: [DISCUSS] KIP-248 Create New ConfigCommand That Uses The New AdminClient

2018-01-16 Thread Rajini Sivaram
Hi Viktor, Thank you for the KIP. It is looking good. A few comments: 1. --bootstrap-server option: "*Help Message*" uses --bootstrap-servers. I think other tools use the singular form even though it should probably have been plural to start with. Can we use* --bootstrap-server* for consistency?

Re: [VOTE] KIP-229: DeleteGroups API

2018-01-16 Thread Rajini Sivaram
Hi Vahid, +1 (binding) Thanks for the KIP. Regards, Rajini On Tue, Jan 16, 2018 at 10:24 AM, Edoardo Comar wrote: > +1 (non binding) - thanks Vahid > > -- > > Edoardo Comar > > IBM Message Hub > > IBM UK Ltd, Hursley Park, SO21 2JN > > > > From

1.1 KIPs

2018-01-16 Thread Damian Guy
Hi, This is a reminder that we have one week left until the KIP deadline of Jan 23. There are still some KIPs that are under discussion and/or being voted on. Please keep in mind that the voting needs to be complete before the deadline for the KIP to be added to the release. https://cwiki.apache.

Re: [VOTE] KIP-229: DeleteGroups API

2018-01-16 Thread Mickael Maison
+1 (non binding) Thanks for the KIP On Tue, Jan 16, 2018 at 11:54 AM, Rajini Sivaram wrote: > Hi Vahid, > > +1 (binding) > > Thanks for the KIP. > > Regards, > > Rajini > > On Tue, Jan 16, 2018 at 10:24 AM, Edoardo Comar wrote: > >> +1 (non binding) - thanks Vahid >> >> -

Re: [DISCUSS] KIP-248 Create New ConfigCommand That Uses The New AdminClient

2018-01-16 Thread Viktor Somogyi
Hi Rajini, 1 and 2: corrected it in my code. So there will be 3 properties in this group: --bootstrap-server, --config.properties and --adminclient-property (following the conventions established elsewhere, like the console-producer). 3: Let me explain the reason for ListQuotas. In the current ve

[jira] [Created] (KAFKA-6450) Kafka Transaction with auto-commit=false, showing transaction log as a consumer lag

2018-01-16 Thread Mayur Patil (JIRA)
Mayur Patil created KAFKA-6450: -- Summary: Kafka Transaction with auto-commit=false, showing transaction log as a consumer lag Key: KAFKA-6450 URL: https://issues.apache.org/jira/browse/KAFKA-6450 Projec

Re: [VOTE] KIP-229: DeleteGroups API

2018-01-16 Thread Guozhang Wang
Thanks Vahid, +1 (binding) from me. A minor question as for tooling improvements you mentioned in the wiki, are we going to add a `--delete` option on the `kafka-consumer-groups.sh` script? I thought it is "yes" but the Proposed Changes section does not explicitly mention it. Guozhang On Tue,

[jira] [Created] (KAFKA-6451) Simplify KStreamReduce

2018-01-16 Thread Matthias J. Sax (JIRA)
Matthias J. Sax created KAFKA-6451: -- Summary: Simplify KStreamReduce Key: KAFKA-6451 URL: https://issues.apache.org/jira/browse/KAFKA-6451 Project: Kafka Issue Type: Bug Components

Re: Vote for KIP-245: Use Properties instead of StreamsConfig in KafkaStreams constructor

2018-01-16 Thread Damian Guy
Thanks for the KIP! +1 On Sat, 13 Jan 2018 at 12:30 Boyang Chen wrote: > Hey Matt and Guozhang, > > > I have already updated the pull request: > https://github.com/apache/kafka/pull/4354 > > and the KIP: > https://cwiki.apache.org/confluence/display/KAFKA/KIP-245%3A+Use+Properties+instead+of+St

Re: Vote for KIP-245: Use Properties instead of StreamsConfig in KafkaStreams constructor

2018-01-16 Thread Matthias J. Sax
Thanks for updating the KIP. I am recasting my vote +1 (binding). -Matthias On 1/13/18 4:30 AM, Boyang Chen wrote: > Hey Matt and Guozhang, > > > I have already updated the pull > request: https://github.com/apache/kafka/pull/4354 > > and the > KIP:  > https://cwiki.apache.org/confluence/dis

Re: Vote for KIP-245: Use Properties instead of StreamsConfig in KafkaStreams constructor

2018-01-16 Thread Colin McCabe
Why not just have a StreamsConfig constructor that takes a Properties object? This has a few advantages. Firstly, because it's purely additive, it doesn't create any deprecated functions or compatibility issues that we have to clean up later. Secondly, if we decide to do something interestin

Re: [DISCUSS] KIP-247: Add public test utils for Kafka Streams

2018-01-16 Thread Guozhang Wang
Thanks Matthias, I made a pass over the wiki and left some comments; I see you have addressed most of them. Here are a few more: 1. "TopologyTestDriver#process()": how about rename it to "pipeInput" or "sendInput"? 2. For "ConsumerRecordFactory" constructor where "startTimestampMs" is not specifi

[DISCUSS] KIP-249: Add Delegation Token Operations to Kafka Admin Client

2018-01-16 Thread Manikumar
Hi all, I have created a KIP to add delegation token operations to Java Admin Client. This KIP proposes new API additions to admin client. There are no new wire protocol changes. https://cwiki.apache.org/confluence/display/KAFKA/KIP-249%3A+Add+Delegation+Token+Operations+to+KafkaAdminClient Fee

Re: [VOTE] KIP-229: DeleteGroups API

2018-01-16 Thread Colin McCabe
Thanks for this KIP, Vahid. +1 (non-binding). How about creating a GROUP_ID_NOT_FOUND error, rather than re-using INVALID_GROUP_ID here? INVALID_GROUP_ID is used to indicate that the group id itself is bad (contains invalid characters, is an empty string, etc.). A group ID not being found o

Re: Vote for KIP-245: Use Properties instead of StreamsConfig in KafkaStreams constructor

2018-01-16 Thread Matthias J. Sax
ATM, StreamsConfig is immutable. Thus, it's just boiler plate code for the user. If we really add a builder pattern or change StreamsConfig, we can update the API accordingly. We will need a KIP for those changes anyway. But for now, it seems to be best to avoid boiler plate if possible. Just my

[jira] [Resolved] (KAFKA-4542) Add authentication based on delegation token.

2018-01-16 Thread Manikumar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4542?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar resolved KAFKA-4542. -- Resolution: Fixed Fixed via [https://github.com/apache/kafka/pull/3616] > Add authentication based on d

[jira] [Resolved] (KAFKA-4543) Add capability to renew/expire delegation tokens.

2018-01-16 Thread Manikumar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4543?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar resolved KAFKA-4543. -- Resolution: Fixed Fix Version/s: 1.1.0 Fixed via [https://github.com/apache/kafka/pull/3616] > A

[jira] [Created] (KAFKA-6452) Add documentation for delegation token authentication mechanism

2018-01-16 Thread Manikumar (JIRA)
Manikumar created KAFKA-6452: Summary: Add documentation for delegation token authentication mechanism Key: KAFKA-6452 URL: https://issues.apache.org/jira/browse/KAFKA-6452 Project: Kafka Issue

[jira] [Created] (KAFKA-6453) Reconsider timestamp propagation semantics

2018-01-16 Thread Matthias J. Sax (JIRA)
Matthias J. Sax created KAFKA-6453: -- Summary: Reconsider timestamp propagation semantics Key: KAFKA-6453 URL: https://issues.apache.org/jira/browse/KAFKA-6453 Project: Kafka Issue Type: Impr

[jira] [Created] (KAFKA-6454) Allow timestamp manipulation in Processor API

2018-01-16 Thread Matthias J. Sax (JIRA)
Matthias J. Sax created KAFKA-6454: -- Summary: Allow timestamp manipulation in Processor API Key: KAFKA-6454 URL: https://issues.apache.org/jira/browse/KAFKA-6454 Project: Kafka Issue Type: I

[jira] [Created] (KAFKA-6455) Improve timestamp propagation at DSL level

2018-01-16 Thread Matthias J. Sax (JIRA)
Matthias J. Sax created KAFKA-6455: -- Summary: Improve timestamp propagation at DSL level Key: KAFKA-6455 URL: https://issues.apache.org/jira/browse/KAFKA-6455 Project: Kafka Issue Type: Impr

Re: [VOTE] KIP-229: DeleteGroups API

2018-01-16 Thread Vahid S Hashemian
Thanks Guozhang for the vote and the feedback. There is a paragraph in the KIP that, I think, covers what you mentioned: To mimic the tooling behavior of the old consumer group tool, this KIP proposes supporting the --delete switch for Kafka stored group offsets (new consumer) too. The behavior

Re: [VOTE] KIP-229: DeleteGroups API

2018-01-16 Thread Vahid S Hashemian
Thanks Colin. Your suggestion is fair. I added a GROUP_ID_NOT_FOUND error to cover the group deletion case where the group id does not exist. I hope this change is what you had in mind, and also hope it does not void the votes cast so far :) --Vahid From: Colin McCabe To: dev@kafka.a

Build failed in Jenkins: kafka-trunk-jdk9 #312

2018-01-16 Thread Apache Jenkins Server
See Changes: [junrao] KAFKA-4541; Support for delegation token mechanism [wangguoz] MINOR: Fix typo (#4426) [wangguoz] KAFKA-4218: Enable access to key in ValueTransformer and ValueMapper --

[DISCUSS] Release Plan for 1.0.1

2018-01-16 Thread Ewen Cheslack-Postava
Hi all, I'd like to start the process for doing a 1.0.1 bug fix release. 1.0.0 was released Nov 1, 2017, and about 2.5 mos have passed and 32 bug fixes have accumulated so far. A few of the more notable fixes that we've merged so far: https://issues.apache.org/jira/browse/KAFKA-6269 - KTable rest

Re: 1.1 KIPs

2018-01-16 Thread Xavier Léauté
Hi Damian, I believe the list should also include KAFKA-5886 (KIP-91) which was voted for 1.0 but wasn't ready to be merged in time. On Tue, Jan 16, 2018 at 5:13 AM Damian Guy wrote: > Hi, > > This is a reminder that we have one week left until the KIP deadline of Jan > 23. There are still some

[jira] [Resolved] (KAFKA-6384) TransactionsTest#testFencingOnSendOffsets sometimes fails with ProducerFencedException

2018-01-16 Thread Ted Yu (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-6384?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ted Yu resolved KAFKA-6384. --- Resolution: Cannot Reproduce > TransactionsTest#testFencingOnSendOffsets sometimes fails with > ProducerFence

Re: [DISCUSS] KIP-247: Add public test utils for Kafka Streams

2018-01-16 Thread Bill Bejeck
Thanks for the KIP! One meta question: Will users that are currently using the existing testing code with the "classifier:test" approach: 1) have access to the new testing utilities without updating the gradle.build file 2) can they continue to use the current testing code with the cl

Re: [DISCUSS] KIP-249: Add Delegation Token Operations to Kafka Admin Client

2018-01-16 Thread Ted Yu
For createDelegationToken, what's the default value for maxLifeTimeMs ? For expireDelegationToken, what exception would be thrown if the operation is not permitted ? Thanks On Tue, Jan 16, 2018 at 10:03 AM, Manikumar wrote: > Hi all, > > I have created a KIP to add delegation token operations

Re: [DISCUSS] KIP-247: Add public test utils for Kafka Streams

2018-01-16 Thread Jeff Klukas
>From what I can tell, global state stores are managed separately from other state stores and are accessed via different methods. Do the proposed methods on TopologyTestDriver (such as getStateStore) cover global stores? If not, can we add an interface for accessing and testing global stores in th

Re: [DISCUSS] KIP-228 Negative record timestamp support

2018-01-16 Thread Konstantin Chukhlomin
Hi all, I tried to summarize below all approaches we were discussing. In case there are some difficulties in email formatting, see GoogleDocs: https://docs.google.com/document/d/1RjlcebpigOj9DyLCedyRxki9nZcFdkBchy-k7BiThMc/edit?usp=sharing

Re: [DISCUSS] Release Plan for 1.0.1

2018-01-16 Thread Guozhang Wang
Hello Ewen, Could you include one more notable changes in 1.0.1: https://issues.apache.org/jira/browse/KAFKA-6398 ? My PR is ready for reviews and should be mergable at any time. Guozhang On Tue, Jan 16, 2018 at 10:54 AM, Ewen Cheslack-Postava wrote: > Hi all, > > I'd like to start the proce

Re: [DISCUSS] KIP-228 Negative record timestamp support

2018-01-16 Thread Colin McCabe
I think we should just keep -1 as a special value, and allow negative timestamps. It just means one missing millisecond in 1969, right? There is just a lot of code everywhere checking for -1, and changing it now would be really likely to be buggy. We probably also want to limit the range of ti

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

2018-01-16 Thread Apache Jenkins Server
See Changes: [junrao] KAFKA-4541; Support for delegation token mechanism [wangguoz] MINOR: Fix typo (#4426) [wangguoz] KAFKA-4218: Enable access to key in ValueTransformer and ValueMapper -

Re: [DISCUSS] KIP-249: Add Delegation Token Operations to Kafka Admin Client

2018-01-16 Thread Ted Yu
Looks like DelegationTokenOwnerMismatchException would be thrown if token expiration is issued by non-owner. On Tue, Jan 16, 2018 at 11:16 AM, Ted Yu wrote: > For createDelegationToken, what's the default value for maxLifeTimeMs ? > > For expireDelegationToken, what exception would be thrown if

Build failed in Jenkins: kafka-1.0-jdk7 #123

2018-01-16 Thread Apache Jenkins Server
See Changes: [wangguoz] MINOR: Fix typo (#4426) -- [...truncated 1.81 MB...] org.apache.kafka.streams.integration.QueryableStateIntegrationTest > shouldBeAbleToQueryFilterSt

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

2018-01-16 Thread Jakub Scholz
I opened the PR: https://issues.apache.org/jira/browse/KAFKA-4029 It is still work in progress, missing mainly tests, docu etc. I will continue to work on it tomorrow. But it shows the implementation. One of the things which came to my mind - the PR is currently using the HttpURLConnection class f

Fwd: Re: [DISCUSS] KIP-247: Add public test utils for Kafka Streams

2018-01-16 Thread Matthias J. Sax
Forgot dev-list... Forwarded Message Subject: Re: [DISCUSS] KIP-247: Add public test utils for Kafka Streams Date: Tue, 16 Jan 2018 13:56:38 -0800 From: Matthias J. Sax Organization: Confluent Inc To: us...@kafka.apache.org Thanks a lot for the comments. @Guozhang: I updated

Re: [VOTE] KIP-219 - Improve Quota Communication

2018-01-16 Thread Jun Rao
Hi, Jiangjie, You are right that the heartbeat is done in a channel different from the fetch request. I think it's still useful to return an empty fetch response when the quota is violated. This way, the throttle time for the heartbeat request won't be large. I agree that we can just mute the chan

Kafka Contributor Status

2018-01-16 Thread Chris Egerton
Hello Kafka dev community, I'd like to become an official Kafka contributor. I've only made one small patch to Kafka so far but would like to become more involved after becoming employed at Confluent, Inc. Please let me know if that'd be possible, and if not, what I can do in order to become eligi

Re: Kafka Contributor Status

2018-01-16 Thread Jun Rao
Hi, Chris, Thanks for your interest. Just added you to the contributor list. Jun On Tue, Jan 16, 2018 at 3:48 PM, Chris Egerton wrote: > Hello Kafka dev community, > > I'd like to become an official Kafka contributor. I've only made one small > patch to Kafka so far but would like to become mo

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

2018-01-16 Thread Apache Jenkins Server
See Changes: [wangguoz] MINOR: Fix typo (#4426) [wangguoz] KAFKA-4218: Enable access to key in ValueTransformer and ValueMapper -- [...truncated 1.84 MB...] org.apache.k

Re: [DISCUSS] KIP-232: Detect outdated metadata by adding ControllerMetadataEpoch field

2018-01-16 Thread Jun Rao
Hi, Dong, Thanks for the updated KIP. Looks good to me overall. Just a few minor comments. 60. OffsetAndMetadata positionAndOffsetEpoch(TopicPartition partition): It seems that there is no need to return metadata. We probably want to return sth like OffsetAndEpoch. 61. Should we store partition_

Re: [DISCUSS] KIP-232: Detect outdated metadata by adding ControllerMetadataEpoch field

2018-01-16 Thread Jun Rao
Hi, Dong, Thanks for the updated KIP. Looks good to me overall. Just a few minor comments. 60. OffsetAndMetadata positionAndOffsetEpoch(TopicPartition partition): It seems that there is no need to return metadata. We probably want to return sth like OffsetAndEpoch. 61. Should we store partition_

Jenkins build is back to normal : kafka-trunk-jdk9 #313

2018-01-16 Thread Apache Jenkins Server
See

Re: [DISCUSS] KIP-249: Add Delegation Token Operations to Kafka Admin Client

2018-01-16 Thread Jun Rao
Hi, Mani, Thanks for the KIP. Looks good to me overhead. Just a couple of minor comments below. 1. Should hmac be of type ByteBuffer? We return hmac as byte[] in DelegationToken. So, it seems it's more consistent to pass in hmac as byte[] too. 2. Does describeDelegationToken() return all tokens?

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

2018-01-16 Thread Apache Jenkins Server
See Changes: [wangguoz] MINOR: Improve Join integration test coverage, PART II -- [...truncated 1.88 MB...] org.apache.kafka.streams.tools.StreamsResetterTest > shouldSee

Re: [DISCUSS] KIP-247: Add public test utils for Kafka Streams

2018-01-16 Thread Matthias J. Sax
Colin, the TopologyTestDriver does not connect to any broker and simulates processing of single-partitioned input topics purely in-memory (the driver is basically a mock for a StreamThread). This is sufficient to test basic business logic. For more complex topologies that are actually divided into

Re: [DISCUSS] KIP-249: Add Delegation Token Operations to Kafka Admin Client

2018-01-16 Thread Manikumar
Hi Ted, Thanks for the review. 1. default value for maxLifeTimeMs is -1. with this, token MaxLifeTime will default to a server side config value (delegation.token.max.lifetime.ms). Added a comment to the KIP. 2. Yes, DelegationTokenOwnerMismatchException will be thrown, updated the KIP with

Re: [DISCUSS] KIP-249: Add Delegation Token Operations to Kafka Admin Client

2018-01-16 Thread Manikumar
Hi, Jun, Thanks for the review. 1. Yes, We can pass hmac as byte[]. Updated the KIP 2. Yes, describeDelegationToken() returns all the user owned tokens and tokens where user have Describe permission. Added a comment to KIP. 3. updated the KIP with possible exceptions. Thanks, On We

Re: [VOTE] KIP-219 - Improve Quota Communication

2018-01-16 Thread Becket Qin
Thanks for the reply, Jun. Currently the byte rate quota does not apply to HeartbeatRequest, JoinGroupRequest/SyncGroupRequest. So the only case those requests are throttled is because the request quota is violated. In that case, the throttle time does not really matter whether we return a full Fe

Build failed in Jenkins: kafka-trunk-jdk9 #314

2018-01-16 Thread Apache Jenkins Server
See Changes: [wangguoz] KAFKA-6328: Sort node groups considering global stores in -- [...truncated 1.44 MB...] kafka.zk.KafkaZkClientTest > testBrokerSequenceIdMethods PASS

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

2018-01-16 Thread Apache Jenkins Server
See

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

2018-01-16 Thread Apache Jenkins Server
See Changes: [wangguoz] MINOR: Improve Join integration test coverage, PART II -- [...truncated 3.90 MB...] org.apache.kafka.connect.runtime.TransformationConfigTest > t

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

2018-01-16 Thread Apache Jenkins Server
See Changes: [wangguoz] KAFKA-6328: Sort node groups considering global stores in -- [...truncated 7.68 MB...] Caused by: java.nio.file.FileSystemExceptio

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

2018-01-16 Thread Ewen Cheslack-Postava
On Sun, Jan 14, 2018 at 1:20 PM, Jakub Scholz wrote: > Hi Ewen, > > Thanks for your comments / questions. > > re 1) I was using the valuesWithPrefixOverride(...) method from > AbstractConfig class. That takes the overrides setting by setting. It > should not be hard to change the code to use only

Re: [DISCUSS] KIP-212: Enforce set of legal characters for connector names

2018-01-16 Thread Ewen Cheslack-Postava
Sonke, I'm fine filtering some control characters. The trimming also seems like it might be *somewhat* moot because the way connector names work in standalone mode is limited by ConfigDef, which already does trimming of settings. Not a great reason to be restrictive, but we'd partly just be codify

Re: [DISCUSS] KIP-242: Mask password fields in Kafka Connect REST response

2018-01-16 Thread Ewen Cheslack-Postava
Vincent, I think with the addition of a configuration to control this for compatibility, people would generally be ok with it. If you want to start a VOTE thread, the KIP deadline is coming up and the PR looks pretty small. I will take a pass at reviewing the PR so we'll be ready to merge if we ca

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

2018-01-16 Thread Jakub Scholz
I have been thinking about this a bit more yesterday while updating the code. I think you are right, we should use only the prefixed values if at least one of them exists. Even I got quite easily confused what setup is actually used when the fields are mixed :-). Randall was also in favour of this