Re: [ANNOUNCE] Apache Kafka 1.1.1 Released

2018-07-19 Thread Guozhang Wang
Dong, thanks for driving the release! On Thu, Jul 19, 2018 at 6:51 PM, Ismael Juma wrote: > Thank you for managing the release Dong! > > Ismael > > On Thu, 19 Jul 2018, 16:54 Dong Lin, wrote: > > > The Apache Kafka community is pleased to announce the release for Apache > > Kafka 1.1.1. > > >

Re: [ANNOUNCE] Apache Kafka 1.1.1 Released

2018-07-19 Thread Ismael Juma
Thank you for managing the release Dong! Ismael On Thu, 19 Jul 2018, 16:54 Dong Lin, wrote: > The Apache Kafka community is pleased to announce the release for Apache > Kafka 1.1.1. > > This is a bug fix release and it includes fixes and improvements from 43 > JIRAs, including a few critical

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

2018-07-19 Thread Apache Jenkins Server
See Changes: [wangguoz] MINOR: Remove deprecated ZkUtils usage from EmbeddedKafkaCluster (#5324) [wangguoz] KAFKA-5037: Fix infinite loop if all input topics are unknown at startup

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

2018-07-19 Thread Nishanth Pradeep
Understood, Guozhang. Thanks for the help, everyone! I have updated the KIP. Let me know if you any other thoughts or suggestions. Best, Nishanth Pradeep On Thu, Jul 19, 2018 at 7:33 PM Guozhang Wang wrote: > I see. > > Well, I think if we add a new function like topicSet() it is less needed

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

2018-07-19 Thread Guozhang Wang
I see. Well, I think if we add a new function like topicSet() it is less needed to deprecate topics() as it returns "{topic1, topic2, ..}" which is sort of non-overlapping in usage with the new API. Guozhang On Thu, Jul 19, 2018 at 5:31 PM, Nishanth Pradeep wrote: > That is what I meant. I

Jenkins build is back to normal : kafka-trunk-jdk10 #295

2018-07-19 Thread Apache Jenkins Server
See

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

2018-07-19 Thread Nishanth Pradeep
That is what I meant. I will add topicSet() instead of changing the signature of topics() for compatibility reasons. But should we not add a @deprecated flag for topics() or do you want to keep it around for the long run? On Thu, Jul 19, 2018 at 7:27 PM Guozhang Wang wrote: > We cannot change

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

2018-07-19 Thread Guozhang Wang
We cannot change the signature of the function named "topics" from "String" to "Set", as Matthias mentioned it is a compatibility breaking change. That's why I was proposing add a new function like "Set topicSet()", while keeping "String topics()" as is. Guozhang On Thu, Jul 19, 2018 at 5:22

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

2018-07-19 Thread Nishanth Pradeep
Right, adding topicNames() instead of changing the return type of topics() in order preserve backwards compatibility is a good idea. But is it not better to depreciate topics() because it would be redundant? In our case, it would only be calling topicNames/topicSet#toString(). I still agree that

Re: [DISCUSS] KIP-307: Allow to define custom processor names with KStreams DSL

2018-07-19 Thread Guozhang Wang
Hello Florian, Sorry for being late... Found myself keep apologizing for late replies these days. But I do want to push this KIP's progress forward as I see it very important and helpful feature for extensibility. About the exceptions, I've gone through them and hopefully it is an exhaustive

Re: [DISCUSS] KIP-342 Add Customizable SASL extensions to OAuthBearer authentication

2018-07-19 Thread Stanislav Kozlovski
I have updated the PR and KIP to address the comments made so far. Please take another look at them and share your thoughts. KIP: https://cwiki.apache.org/confluence/display/KAFKA/KIP-342%3A+Add+support+for+Custom+SASL+extensions+in+OAuthBearer+authentication PR: Pull request

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

2018-07-19 Thread Apache Jenkins Server
See

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

2018-07-19 Thread Jun Rao
I agree that there is no strong ordering when there are more than one socket connections. Currently, we rely on controllerEpoch and leaderEpoch to ensure that the receiving broker picks up the latest state for each partition. One potential issue with the dequeue approach is that if the queue is

Re: [ANNOUNCE] Apache Kafka 1.1.1 Released

2018-07-19 Thread Dong Lin
On Thu, Jul 19, 2018 at 4:53 PM, Dong Lin wrote: > The Apache Kafka community is pleased to announce the release for Apache > Kafka 1.1.1. > > This is a bug fix release and it includes fixes and improvements from 43 > JIRAs, including a few critical bugs. > > All of the changes in this release

[ANNOUNCE] Apache Kafka 1.1.1 Released

2018-07-19 Thread Dong Lin
The Apache Kafka community is pleased to announce the release for Apache Kafka 1.1.1. This is a bug fix release and it includes fixes and improvements from 43 JIRAs, including a few critical bugs. All of the changes in this release can be found in the release notes:

Plan for new Kafka Connect Transform

2018-07-19 Thread karri saisatish kumar reddy
Respected Sir, I have recently worked on kafka as a part of my project.I have used debezium connector and SMT for changing schema in my project. I need to rename the following deeper field but found no suitable SMT . My schema has following structure ( -> implies subfield) Header - > General

[jira] [Created] (KAFKA-7186) Controller uses too much memory when sending out UpdateMetadataRequest that can cause OutOfMemoryError

2018-07-19 Thread Zhanxiang (Patrick) Huang (JIRA)
Zhanxiang (Patrick) Huang created KAFKA-7186: Summary: Controller uses too much memory when sending out UpdateMetadataRequest that can cause OutOfMemoryError Key: KAFKA-7186 URL:

[jira] [Resolved] (KAFKA-6587) Kafka Streams hangs when not able to access internal topics

2018-07-19 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-6587?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang resolved KAFKA-6587. -- Resolution: Fixed Assignee: Ted Yu Fix Version/s: 2.1.0 > Kafka Streams hangs

[jira] [Resolved] (KAFKA-6437) Streams does not warn about missing input topics, but hangs

2018-07-19 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-6437?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang resolved KAFKA-6437. -- Resolution: Fixed Assignee: Ted Yu (was: Mariam John) Fix Version/s: 2.1.0 >

[jira] [Resolved] (KAFKA-5037) Infinite loop if all input topics are unknown at startup

2018-07-19 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5037?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang resolved KAFKA-5037. -- Resolution: Fixed Fix Version/s: 2.1.0 Issue resolved by pull request 5322

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

2018-07-19 Thread Mayuresh Gharat
Yea, the correlationId is only set to 0 in the NetworkClient constructor. Since we reuse the same NetworkClient between Controller and the broker, a disconnection should not cause it to reset to 0, in which case it can be used to reject obsolete requests. Thanks, Mayuresh On Thu, Jul 19, 2018

Re: [DISCUSS] KIP-342 Add Customizable SASL extensions to OAuthBearer authentication

2018-07-19 Thread Stanislav Kozlovski
Hi Ron, Agreed. `SaslExtensionsCallback` will be the only public API addition and new documentation for the extension strings. A question that came up - should the LoginCallbackHandler throw an exception or simply ignore key/value extension pairs who do not match the validation regex pattern? I

[jira] [Created] (KAFKA-7185) getMatchingAcls throws StringIndexOutOfBoundsException for empty resource name

2018-07-19 Thread Dhruvil Shah (JIRA)
Dhruvil Shah created KAFKA-7185: --- Summary: getMatchingAcls throws StringIndexOutOfBoundsException for empty resource name Key: KAFKA-7185 URL: https://issues.apache.org/jira/browse/KAFKA-7185 Project:

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

2018-07-19 Thread Lucas Wang
@Dong, Great example and explanation, thanks! @All Regarding the example given by Dong, it seems even if we use a queue, and a dedicated controller request handling thread, the same result can still happen because R1_a will be sent on one connection, and R1_b & R2 will be sent on a different

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

2018-07-19 Thread Apache Jenkins Server
See Changes: [rajinisivaram] KAFKA-7182: SASL/OAUTHBEARER client response missing %x01 seps (#5391) -- [...truncated 2.48 MB...] org.apache.kafka.streams.TopologyTest >

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

2018-07-19 Thread Apache Jenkins Server
See Changes: [rajinisivaram] KAFKA-7182: SASL/OAUTHBEARER client response missing %x01 seps (#5391) -- [...truncated 1.98 MB...]

Re: [DISCUSS] KIP-342 Add Customizable SASL extensions to OAuthBearer authentication

2018-07-19 Thread Rajini Sivaram
Hi Ron, Thanks for the summary - that matches my understanding. It is a good idea to support unsecuredLoginExtension_=value for the default implementation and that would make it easy to test the KIP. Also agree with extension name restrictions, we should keep the patterns in the initial client

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

2018-07-19 Thread Apache Jenkins Server
See Changes: [github] KAFKA-3514: Remove min timestamp tracker (#5382) -- [...truncated 887.72 KB...] kafka.server.epoch.EpochDrivenReplicationProtocolAcceptanceTest >

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

2018-07-19 Thread Mayuresh Gharat
Actually nvm, correlationId is reset in case of connection loss, I think. Thanks, Mayuresh On Thu, Jul 19, 2018 at 11:11 AM Mayuresh Gharat wrote: > I agree with Dong that out-of-order processing can happen with having 2 > separate queues as well and it can even happen today. > Can we use the

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

2018-07-19 Thread Guozhang Wang
Personally I'd prefer to keep the deprecation-related changes as small as possible unless they are really necessary, and hence I'd prefer to just add List topicList() /* or Set topicSet() */ in addition to topicPattern to Source, in addition to `topicNameExtractor` to Sink, and leaving the

Re: [VOTE] KIP-338 Support to exclude the internal topics in kafka-topics.sh command

2018-07-19 Thread Manikumar
+1 (non-binding) On Thu, Jul 19, 2018 at 11:47 PM Colin McCabe wrote: > +1 (non-binding) > > cheers, > Colin > > On Mon, Jul 16, 2018, at 01:10, Chia-Ping Tsai wrote: > > hi folks, > > > > The discussion[1] of KIP-338[2] did not get any objection for last 6 > > days so it is time to start the

Re: [VOTE] KIP-338 Support to exclude the internal topics in kafka-topics.sh command

2018-07-19 Thread Colin McCabe
+1 (non-binding) cheers, Colin On Mon, Jul 16, 2018, at 01:10, Chia-Ping Tsai wrote: > hi folks, > > The discussion[1] of KIP-338[2] did not get any objection for last 6 > days so it is time to start the voting thread. > > Thanks for your time! > > [1] >

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

2018-07-19 Thread Mayuresh Gharat
I agree with Dong that out-of-order processing can happen with having 2 separate queues as well and it can even happen today. Can we use the correlationId in the request from the controller to the broker to handle ordering ? Thanks, Mayuresh On Thu, Jul 19, 2018 at 6:41 AM Becket Qin wrote:

Re: [DISCUSS] KIP-335 Consider configurations for Kafka Streams

2018-07-19 Thread Matthias J. Sax
Thanks for the KIP. I just double checked the code. It seems that `retries` and `retries.backoff.ms` is only used in `GlobalStateManagerImpl` any longer to catch a timeout exception and retry. Thus, it seems reasonable to me to deprecate those configs and rewrite `GlobalStateManager`

[jira] [Resolved] (KAFKA-1420) Replace AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK with TestUtils.createTopic in unit tests

2018-07-19 Thread Manikumar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1420?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar resolved KAFKA-1420. -- Resolution: Fixed Fixed as part of https://github.com/apache/kafka/pull/5303 > Replace

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

2018-07-19 Thread Matthias J. Sax
Thanks for updating the KIP. The current `Source` interface has a method `String topics()` atm. Thus, we cannot just add `Set Source#topics()` because this would replace the existing method and would be an incompatible change. I think, we should deprecate `String topics()` and add a method with

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

2018-07-19 Thread Apache Jenkins Server
See Changes: [github] KAFKA-3514: Remove min timestamp tracker (#5382) -- [...truncated 1.97 MB...]

[jira] [Resolved] (KAFKA-7182) SASL/OAUTHBEARER client response is missing %x01 separators

2018-07-19 Thread Rajini Sivaram (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-7182?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Rajini Sivaram resolved KAFKA-7182. --- Resolution: Fixed Reviewer: Rajini Sivaram Fix Version/s: 2.0.0 >

Re: [DISCUSS] KIP-342 Add Customizable SASL extensions to OAuthBearer authentication

2018-07-19 Thread Ron Dagostino
Hi Rajini and Stanislav. Rajini, yes, I think you are right about the login callback handler being more appropriate for retrieving the SASL extensions than the login module itself (how many times am I going to have to be encouraged to leverage the callback handlers?!? lol). OAuthBearerLoginModule

Re: [DISCUSS] KIP-342 Add Customizable SASL extensions to OAuthBearer authentication

2018-07-19 Thread Stanislav Kozlovski
Hey Ron, Come to think of it, I think what Rajini said makes more sense than my initial proposal. Having the OAuthBearerClientCallbackHandler populate SaslExtensionsCallback by taking a Map from the Subject would ease users' implementation - they'd only have to provide a login callback handler

Re: [DISCUSS] KIP-342 Add Customizable SASL extensions to OAuthBearer authentication

2018-07-19 Thread Rajini Sivaram
Hi Ron, Sorry, I meant that the login callback handler would populate Subject with extensions (along with OAuthBearerToken) using a custom login callback handler and the default OAuthBearerSaslClientCallbackHandler would obtain extensions along with OAuthBearerToken from Subject. Thanks to

Re: Contribution request

2018-07-19 Thread Matthias J. Sax
Done On 7/19/18 8:49 AM, András Katona wrote: > Hello Team, > > Could you add me to the contributors on Apache Kafka project in JIRA, > please? > Username: akatona > > Thank you, > Andras Katona > signature.asc Description: OpenPGP digital signature

Re: [DISCUSS] KIP-342 Add Customizable SASL extensions to OAuthBearer authentication

2018-07-19 Thread Rajini Sivaram
Hi Stanislav, Thanks for the KIP. Since SaslExtensions will be an internal class, can we remove it from the KIP to avoid confusion? Also, can we add the package name for SaslExtensionsCallback? The PR has it in org.apache.kafka.common.security which is an internal package. As a public class, it

Re: [DISCUSS] KIP-342 Add Customizable SASL extensions to OAuthBearer authentication

2018-07-19 Thread Rajini Sivaram
Hi Ron, Is there a reason why wouldn't want to provide extensions using a login callback handler in the same way as we inject tokens? The easiest way to inject custom extensions would be using the JAAS config. So we could have both OAuthBearerTokenCallback and SaslExtensionsCallback processed by

Contribution request

2018-07-19 Thread András Katona
Hello Team, Could you add me to the contributors on Apache Kafka project in JIRA, please? Username: akatona Thank you, Andras Katona

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

2018-07-19 Thread Viktor Somogyi
Hi Ismael, Yea, I think that's the most painless given the current circumstances, although a bit weird that we have an interface that doesn't enforce anything. I guess we can mitigate this on some level with documentation. However yes, new implementations can freely choose whatever they want to

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

2018-07-19 Thread Apache Jenkins Server
See Changes: [github] MINOR: Update build.gradle and release.py to upload streams-scala_2.12 -- [...truncated 2.47 MB...]

Re: [DISCUSS] KIP-342 Add Customizable SASL extensions to OAuthBearer authentication

2018-07-19 Thread Ron Dagostino
Hi Stanislav. Implementers of a custom sasl.client.callback.handler.class must be sure to provide the existing logic in org.apache.kafka.common.security.oauthbearer.internals.OAuthBearerSaslClientCallbackHandler that handles instances of OAuthBearerTokenCallback (by retrieving the private

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

2018-07-19 Thread Apache Jenkins Server
See Changes: [rajinisivaram] MINOR: Update build.gradle and release.py to upload streams-scala_2.12 -- [...truncated 2.48 MB...]

[jira] [Resolved] (KAFKA-7184) Kafka is going down with issue ERROR Failed to clean up log for __consumer_offsets-0 in dir /tmp/kafkadev2-logs due to IOException (kafka.server.LogDirFailureChannel)

2018-07-19 Thread Dhruvil Shah (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-7184?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Dhruvil Shah resolved KAFKA-7184. - Resolution: Not A Problem > Kafka is going down with issue ERROR Failed to clean up log for >

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

2018-07-19 Thread Becket Qin
Good point, Joel. I agree that a dedicated controller request handling thread would be a better isolation. It also solves the reordering issue. On Thu, Jul 19, 2018 at 2:23 PM, Joel Koshy wrote: > Good example. I think this scenario can occur in the current code as well > but with even lower

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

2018-07-19 Thread Apache Jenkins Server
See Changes: [github] MINOR: Update build.gradle and release.py to upload streams-scala_2.12 -- [...truncated 1.98 MB...]

Re: [VOTE] 2.0.0 RC2

2018-07-19 Thread Rajini Sivaram
Hi all, We found a blocker in 2.0.0 which is a bug in the newly added OAuth protocol implementation (https://issues.apache.org/jira/browse/KAFKA-7182). Since the current implementation doesn't conform to the SASL/OAUTHBEARER spec in RFC-7628, we need to fix this before the release to conform to

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

2018-07-19 Thread Viktor Somogyi
Hi Chia-Ping, Thanks for the idea, although I have some concers with it. As discussed on the jira, I think it would be a good practice to keep the current structure as I think the "headers" parameter would really be optional in our case. There are multiple ways to express this in Java, such as

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

2018-07-19 Thread Apache Jenkins Server
See Changes: [jason] KAFKA-4994; Remove findbugs suppression for OffsetStorageWriter (#5385) -- [...truncated 426.56 KB...] kafka.zookeeper.ZooKeeperClientTest >

Re: Ask For permissions

2018-07-19 Thread Rajini Sivaram
Hi, Done, welcome to the team! Regards, Rajini On Thu, Jul 19, 2018 at 10:18 AM, ryanyyang(杨原) wrote: > Hi Team, > > I am trying to claim a bug in Jira, Could you please assign me permissions. > my JIRA usernane is Ryanyyang. > > thanks >

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

2018-07-19 Thread Apache Jenkins Server
See Changes: [jason] KAFKA-4994; Remove findbugs suppression for OffsetStorageWriter (#5385) -- [...truncated 1.98 MB...]

Ask For permissions

2018-07-19 Thread 杨原
Hi Team, I am trying to claim a bug in Jira, Could you please assign me permissions. my JIRA usernane is Ryanyyang. thanks

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

2018-07-19 Thread Apache Jenkins Server
See

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

2018-07-19 Thread Apache Jenkins Server
See Changes: [lindong28] MINOR: Consolidate Topic create calls in Test classes -- [...truncated 1.98 MB...]

[jira] [Created] (KAFKA-7184) Kafka is going down with issue ERROR Failed to clean up log for __consumer_offsets-0 in dir /tmp/kafkadev2-logs due to IOException (kafka.server.LogDirFailureChannel)

2018-07-19 Thread Sandeep Muddamsetty (JIRA)
Sandeep Muddamsetty created KAFKA-7184: -- Summary: Kafka is going down with issue ERROR Failed to clean up log for __consumer_offsets-0 in dir /tmp/kafkadev2-logs due to IOException (kafka.server.LogDirFailureChannel) Key:

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

2018-07-19 Thread Joel Koshy
Good example. I think this scenario can occur in the current code as well but with even lower probability given that there are other non-controller requests interleaved. It is still sketchy though and I think a safer approach would be separate queues and pinning controller request handling to one

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

2018-07-19 Thread Dong Lin
Hey Becket, I think you are right that there may be out-of-order processing. However, it seems that out-of-order processing may also happen even if we use a separate queue. Here is the example: - Controller sends R1 and got disconnected before receiving response. Then it reconnects and sends

[jira] [Resolved] (KAFKA-4994) Fix findbugs warning about OffsetStorageWriter#currentFlushId

2018-07-19 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4994?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson resolved KAFKA-4994. Resolution: Fixed Fix Version/s: 2.1.0 > Fix findbugs warning about

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

2018-07-19 Thread Dong Lin
Hey Lucas, Let me present a more specific sequence of events to show why out-of-order processing can happen. Can you see if it makes sense? Say broker has 12 request handler threads and there are many large ProduceRequest in the queue. Each ProduceRequest takes 200 ms to be processed. Let's also