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

2017-11-16 Thread James Cheng
How fast does the in-memory cache grow? As a random datapoint... 10 months ago we set our offsets.retention.minutes to 1 year. So, for the past 10 months, we essentially have not expired any offsets. Via JMX, one of our brokers says kafka.coordinator.group:type=GroupMetadataManager,name=NumOff

Re: [DISCUSS] KIP-225 - Use tags for consumer “records.lag” metrics

2017-11-16 Thread James Cheng
This KIP will break backwards compatibility for anyone who is using the existing attribute names. Kafka devs, I believe that metrics are a supported interface, and so this would be a breaking change. In order to do this, we would need a deprecation timeframe for the old metric, and a transition

[jira] [Resolved] (KAFKA-4) Confusing Error mesage from producer when no kafka brokers are available

2017-11-16 Thread JIRA
[ https://issues.apache.org/jira/browse/KAFKA-4?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Sönke Liebau resolved KAFKA-4. -- Resolution: Fixed (was: Unresolved) > Confusing Error mesage from producer when no kafka brokers are avail

[jira] [Created] (KAFKA-6220) ReplicaFetcherThread throws UnknownTopicOrPartitionExeption on topic creation

2017-11-16 Thread Alex (JIRA)
Alex created KAFKA-6220: --- Summary: ReplicaFetcherThread throws UnknownTopicOrPartitionExeption on topic creation Key: KAFKA-6220 URL: https://issues.apache.org/jira/browse/KAFKA-6220 Project: Kafka I

[jira] [Created] (KAFKA-6221) ReplicaFetcherThread throws UnknownTopicOrPartitionExeption on topic creation

2017-11-16 Thread Alex (JIRA)
Alex created KAFKA-6221: --- Summary: ReplicaFetcherThread throws UnknownTopicOrPartitionExeption on topic creation Key: KAFKA-6221 URL: https://issues.apache.org/jira/browse/KAFKA-6221 Project: Kafka I

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

2017-11-16 Thread Jan Filipiak
Hi Trevor, thank you very much for your interested. Too keep discussion mailing list focused and not Jira or Confluence I decided to reply here. 1. its tricky activity is indeed very low. In the KIP-213 there are 2 proposals about the return type of the join. I would like to settle on one. Un

[jira] [Resolved] (KAFKA-6220) ReplicaFetcherThread throws UnknownTopicOrPartitionExeption on topic creation

2017-11-16 Thread Mickael Maison (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-6220?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Mickael Maison resolved KAFKA-6220. --- Resolution: Duplicate DUP of https://issues.apache.org/jira/browse/KAFKA-6221 > ReplicaFetche

Re: [VOTE] 0.11.0.2 RC0

2017-11-16 Thread Rajini Sivaram
+1 from me The vote has passed with 4 binding votes (Gwen, Guozhang, Ismael and Rajini) and 3 non-binding votes (Ted, Satish and Tim). I will close the voting thread and complete the release process. Many thanks to everyone for voting. Regards, Rajini On Thu, Nov 16, 2017 at 3:01 AM, Ismael Ju

Re: [DISCUSS] KIP-225 - Use tags for consumer “records.lag” metrics

2017-11-16 Thread charly molter
Yes James you are right. I wasn't sure what to do about it and followed what happened with BytesOut in KIP-153 which completely changed meaning without any deprecation window. I'm happy to adapt my KIP if the community thinks we should duplicate the metric for a while. Thanks! On Thu, Nov 16, 201

Re: [VOTE] 0.11.0.2 RC0

2017-11-16 Thread Rajini Sivaram
Correction from previous note: Vote closed with 3 binding PMC votes (Gwen, Guozhang, Ismael ) and 4 non-binding votes. On Thu, Nov 16, 2017 at 10:03 AM, Rajini Sivaram wrote: > +1 from me > > The vote has passed with 4 binding votes (Gwen, Guozhang, Ismael and > Rajini) and 3 non-binding votes

[RESULTS] [VOTE] Release Kafka version 0.11.0.2

2017-11-16 Thread Rajini Sivaram
This vote passes with 7 +1 votes (3 bindings) and no 0 or -1 votes. +1 votes PMC Members: * Gwen Shapira * Guozhang Wang * Ismael Juma Committers: * Rajini Sivaram Community: * Ted Yu * Satish Duggana * Tim Carey-Smith 0 votes * No votes -1 votes * No votes Vote thread: http://mail-archives

Build failed in Jenkins: kafka-0.11.0-jdk7 #336

2017-11-16 Thread Apache Jenkins Server
See Changes: [rajinisivaram] Bump version to 0.11.0.2 [rajinisivaram] MINOR: Update version numbers to 0.11.0.3-SNAPSHOT -- [...truncated 2.44 MB...] org.apache.kafka.st

Fwd: The KafkaConsumer reads randomly from the offset 0

2017-11-16 Thread dali dali
-- Forwarded message -- From: dali dali Date: 2017-11-16 10:54 GMT+01:00 Subject: The KafkaConsumer reads randomly from the offset 0 To: us...@kafka.apache.org Hi, I want to test a Kafka example. I am using Kafka 0.10.0.1. The producer: object ProducerApp extends App { val topi

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

2017-11-16 Thread Randall Hauch
No, we need to keep the KIP since we want to change/correct the existing behavior. But we do need to clarify in the KIP these edge cases that will change. Thanks for the continued work on this, Sönke. Regards, Randall > On Nov 16, 2017, at 1:56 AM, Sönke Liebau > wrote: > > Hi Randall, >

[GitHub] kafka pull request #4150: MINOR: Add valid values for message.timestamp.type

2017-11-16 Thread makearl
GitHub user makearl reopened a pull request: https://github.com/apache/kafka/pull/4150 MINOR: Add valid values for message.timestamp.type The documentation for `message.timestamp.type` is missing valid values (https://kafka.apache.org/documentation/#topicconfigs). This change adds v

[GitHub] kafka pull request #4150: MINOR: Add valid values for message.timestamp.type

2017-11-16 Thread makearl
Github user makearl closed the pull request at: https://github.com/apache/kafka/pull/4150 ---

[jira] [Resolved] (KAFKA-4675) Subsequent CreateTopic command could be lost after a DeleteTopic command

2017-11-16 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4675?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma resolved KAFKA-4675. Resolution: Duplicate KAFKA-6098 is the same issue and has more information. > Subsequent CreateTop

[jira] [Reopened] (KAFKA-1993) Enable topic deletion as default

2017-11-16 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1993?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma reopened KAFKA-1993: > Enable topic deletion as default > > > Key: KAFKA-199

[jira] [Resolved] (KAFKA-1993) Enable topic deletion as default

2017-11-16 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1993?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma resolved KAFKA-1993. Resolution: Duplicate > Enable topic deletion as default > > >

Re: [ANNOUNCE] Apache Kafka 1.0.0 Released

2017-11-16 Thread sebb
On 1 November 2017 at 15:38, Guozhang Wang wrote: > The Apache Kafka community is pleased to announce the release for Apache > Kafka 1.0.0. ... > > All of the changes in this release can be found in the release notes: > > https://dist.apache.org/repos/dist/release/kafka/1.0.0/RELEASE_NOTES.html

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

2017-11-16 Thread Sönke Liebau
Hi Randall, I had mentioned this edge case in the KIP, but will add some further detail to further clarify all changing scenarios post pull request. Kind regards, Sönke On Thu, Nov 16, 2017 at 2:06 PM, Randall Hauch wrote: > No, we need to keep the KIP since we want to change/correct the e

[jira] [Created] (KAFKA-6222) Download page must not link to dist.apache.org

2017-11-16 Thread Sebb (JIRA)
Sebb created KAFKA-6222: --- Summary: Download page must not link to dist.apache.org Key: KAFKA-6222 URL: https://issues.apache.org/jira/browse/KAFKA-6222 Project: Kafka Issue Type: Bug Report

[jira] [Created] (KAFKA-6223) Please delete old releases from mirroring system

2017-11-16 Thread Sebb (JIRA)
Sebb created KAFKA-6223: --- Summary: Please delete old releases from mirroring system Key: KAFKA-6223 URL: https://issues.apache.org/jira/browse/KAFKA-6223 Project: Kafka Issue Type: Bug Environ

Re: [ANNOUNCE] Apache Kafka 1.0.0 Released

2017-11-16 Thread Ismael Juma
Thanks Sebb. For reference, Sebb filed KAFKA-6222 and KAFKA-6223, so let's keep the discussion in the relevant JIRAs. Ismael On Thu, Nov 16, 2017 at 1:49 PM, sebb wrote: > On 1 November 2017 at 15:38, Guozhang Wang wrote: > > The Apache Kafka community is pleased to announce the release for Ap

[jira] [Resolved] (KAFKA-381) Changes made by a request do not affect following requests in the same packet.

2017-11-16 Thread JIRA
[ https://issues.apache.org/jira/browse/KAFKA-381?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Sönke Liebau resolved KAFKA-381. Resolution: Not A Bug I think we can safely close this issue, the behavior was sufficiently investig

[jira] [Resolved] (KAFKA-1408) Kafk broker can not stop itself normaly after problems with connection to ZK

2017-11-16 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1408?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma resolved KAFKA-1408. Resolution: Duplicate Closing as duplicate of KAFKA-1317 based on other comments and the fact that

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

2017-11-16 Thread Sönke Liebau
I've added some more detail to the KIP [1] around current scenarios that might break in the future. I actually came up with a second limitation that we'd impose on users and also documented this. Let me know what you think. Kind regards, Sönke [1] https://cwiki.apache.org/confluence/display/KAFK

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

2017-11-16 Thread Randall Hauch
Nice job updating the KIP. The PR ( https://github.com/apache/kafka/pull/2755/files) for the proposed implementation does prevent names from being empty, and it trims whitespace from the name only when creating a new connector. However, the KIP's "Proposed Change" section should probably be very cl

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

2017-11-16 Thread Sönke Liebau
Sounds good. I've added a few sentences to this effect to the KIP. On Thu, Nov 16, 2017 at 5:02 PM, Randall Hauch wrote: > Nice job updating the KIP. The PR ( > https://github.com/apache/kafka/pull/2755/files) for the proposed > implementation does prevent names from being empty, and it trims wh

[jira] [Resolved] (KAFKA-414) Evaluate mmap-based writes for Log implementation

2017-11-16 Thread Jay Kreps (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-414?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jay Kreps resolved KAFKA-414. - Resolution: Won't Fix > Evaluate mmap-based writes for Log implementation > ---

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

2017-11-16 Thread Jan Filipiak
We are running this perfectly fine. for us the smaller table changes rather infrequent say. only a few times per day. The performance of the flush is way lower than the computing power you need to bring to the table to account for all the records beeing emmited after the one single update. On

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

2017-11-16 Thread Trevor Huey
1. Going over KIP-213, I am leaning toward the "less intrusive" approach. In my use case, I am planning on performing a sequence of several oneToMany joins, From my understanding, the more intrusive approach would result in several nested levels of CombinedKey's. For example, consider Tables A, B,

log retention policy issure

2017-11-16 Thread 张明富
Hi, From kafka's document I found: "The Kafka cluster retains all published records—whether or not they have been consumed—using a configurable retention period. For example, if the retention policy is set to two days, then for the two days after a record is published, it is available for co

[GitHub] kafka pull request #4132: KAFKA-5925: Adding records deletion operation to t...

2017-11-16 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/4132 ---

[jira] [Resolved] (KAFKA-5811) Trogdor should handle injecting disk faults

2017-11-16 Thread Rajini Sivaram (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5811?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Rajini Sivaram resolved KAFKA-5811. --- Resolution: Fixed Fix Version/s: 1.1.0 Issue resolved by pull request 4195 [https://git

[GitHub] kafka pull request #4195: KAFKA-5811: Add Kibosh integration for Trogdor and...

2017-11-16 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/4195 ---

[jira] [Resolved] (KAFKA-6213) Stream processor receives messages after close() is invoked

2017-11-16 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-6213?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax resolved KAFKA-6213. Resolution: Not A Problem > Stream processor receives messages after close() is invoked > --

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

2017-11-16 Thread Apache Jenkins Server
See Changes: [wangguoz] KAFKA-5925: Adding records deletion operation to the new Admin Client -- [...truncated 385.37 KB...] kafka.security.auth.ResourceTypeTest > testFr

[GitHub] kafka pull request #3904: [MINOR] Added equals() method to Stamped

2017-11-16 Thread KoenDG
Github user KoenDG closed the pull request at: https://github.com/apache/kafka/pull/3904 ---

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

2017-11-16 Thread Vahid S Hashemian
James, thanks for the feedback, and sharing the datapoint. Just as a reference, here is how the key-value in an offsets topic record is formed: Key ** group id: string ** topic, partition: string, int Value ** offset, metadata: long, string ** commit timestamp: long ** expire timestamp: lo

[GitHub] kafka pull request #3891: [WIP input needed] MINOR: Further code cleanup inv...

2017-11-16 Thread KoenDG
Github user KoenDG closed the pull request at: https://github.com/apache/kafka/pull/3891 ---

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

2017-11-16 Thread Rajini Sivaram
Hi Becket, The current user quota doesn't solve the problem. But I was thinking that if we could ensure we don't read more from the network than the quota allows, we may be able to fix the issue in a different way (throttling all connections, each for a limited time prior to reading large requests

[GitHub] kafka pull request #4226: MINOR: Arrays.toList replaced with Collections.sin...

2017-11-16 Thread KoenDG
GitHub user KoenDG opened a pull request: https://github.com/apache/kafka/pull/4226 MINOR: Arrays.toList replaced with Collections.singletonList() where possible. This is something I did after my working hours, I would ask people reviewing this do the same, don't take time for this

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

2017-11-16 Thread Matthias J. Sax
Any thoughts about my latest proposal? -Matthias On 11/10/17 10:02 PM, Jan Filipiak wrote: > Hi, > > i think this is the better way. Naming is always tricky Source is kinda > taken > I had TopicBackedK[Source|Table] in mind > but for the user its way better already IMHO > > Thank you for recons

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

2017-11-16 Thread Matthias J. Sax
@Jan: The `Produced` class was introduced in 1.0 to specify key and valud Serdes (and partitioner) if data is written into a topic. Old API: KStream#to("topic", keySerde, valueSerde); New API: KStream#to("topic", Produced.with(keySerde, valueSerde)); This allows to reduce the number of overl

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

2017-11-16 Thread Apache Jenkins Server
See

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

2017-11-16 Thread Matthias J. Sax
Hi, I am just catching up on this discussion and did re-read the KIP and discussion thread. In contrast to you, I prefer the second approach with CombinedKey as return type for the following reasons: 1) the oneToManyJoin() method had less parameter 2) those parameters are easy to understand 3

[jira] [Resolved] (KAFKA-6218) Optimize condition in if statement to reduce the number of comparisons

2017-11-16 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-6218?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ewen Cheslack-Postava resolved KAFKA-6218. -- Resolution: Fixed Reviewer: Ewen Cheslack-Postava > Optimize condition in

[GitHub] kafka pull request #4225: KAFKA-6218 : Optimize condition in if statement to...

2017-11-16 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/4225 ---

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

2017-11-16 Thread Becket Qin
Thanks Rajini, I updated the KIP wiki to clarify the scope of the KIP. To summarize, the current quota has a few caveats: 1. The brokers are only throttling the NEXT request even if the current request is already violating quota. So the broker will always process at least one request from the clie

[GitHub] kafka pull request #4227: MINOR: Log unexpected exceptions in Connect REST c...

2017-11-16 Thread ewencp
GitHub user ewencp opened a pull request: https://github.com/apache/kafka/pull/4227 MINOR: Log unexpected exceptions in Connect REST calls that generate 500s at a higher log level The ConnectExceptionMapper was originally intended to handle ConnectException errors for some expected

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

2017-11-16 Thread Apache Jenkins Server
See Changes: [me] KAFKA-6218: Optimize condition in if statement to reduce the number of -- [...truncated 385.85 KB...] kafka.utils.CoreUtilsTest > testCircularIterator PA

[GitHub] kafka pull request #4228: MINOR: improve StateStore JavaDocs

2017-11-16 Thread mjsax
GitHub user mjsax opened a pull request: https://github.com/apache/kafka/pull/4228 MINOR: improve StateStore JavaDocs Clarify that state directory must use `storeName` ### Committer Checklist (excluded from commit message) - [ ] Verify design and implementation - [

[jira] [Created] (KAFKA-6224) Can not build Kafka 1.0.0 with gradle 3.2.1

2017-11-16 Thread Chao Ren (JIRA)
Chao Ren created KAFKA-6224: --- Summary: Can not build Kafka 1.0.0 with gradle 3.2.1 Key: KAFKA-6224 URL: https://issues.apache.org/jira/browse/KAFKA-6224 Project: Kafka Issue Type: Bug Com

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

2017-11-16 Thread Guozhang Wang
Thanks for the explanation Jan. On top of my head I'm leaning towards the "more intrusive" approach to resolve the race condition issue we discussed above. Matthias has some arguments for this approach already, so I would not re-iterate them here. To me I find the "ValueMapper joinPrefixFaker" is a

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

2017-11-16 Thread Guozhang Wang
Matthias, For this idea, are your proposing that for any many-to-one mapping operations (for now only Join operators), we will strip off the record context in the resulted records and claim "we cannot infer its traced context anymore"? Guozhang On Thu, Nov 16, 2017 at 1:03 PM, Matthias J. Sax

Re: [DISCUSS] KIP-225 - Use tags for consumer “records.lag” metrics

2017-11-16 Thread James Cheng
Ah, that's a great point. KIP-153 didn't *rename* the metric but changed its meaning, yet we didn't seem to discuss compatibility much when we made that change. If the Kafka devs can comment on the backwards-compatibility-ness of metrics and how we treat that, that would be helpful. -James >