Re: [DISCUSS] KIP-115: Enforce offsets.topic.replication.factor

2017-01-30 Thread Onur Karaman
I get where Ewen's coming from but personally, I have trouble understanding a use case where end users would specifically rely on existing behavior of group coordination and offset commits succeeding with nondeterministic RF (which is only later to be manually fixed) during the window of time when

[GitHub] kafka pull request #2469: MINOR: Logging improvements in consumer internals

2017-01-30 Thread hachikuji
GitHub user hachikuji opened a pull request: https://github.com/apache/kafka/pull/2469 MINOR: Logging improvements in consumer internals You can merge this pull request into a Git repository by running: $ git pull https://github.com/hachikuji/kafka improve-consumer-logging

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

2017-01-30 Thread Apache Jenkins Server
See Changes: [wangguoz] KAFKA-4613: Follow-up to fix JavaDocs -- [...truncated 5246 lines...] kafka.server.OffsetCommitTest > testOffsetsDeleteAfterTopicDeletion PASSED

Re: [DISCUSS] KIP-115: Enforce offsets.topic.replication.factor

2017-01-30 Thread Ewen Cheslack-Postava
So, we have one other blocker bug in system tests that we're trying to make sure can safely be removed, so we've had a bit of slack time with this. Obviously having this all happen very last minute isn't really ideal since it didn't allow enough time to address the feedback -- Stevo's questions

KIP-54 voting status?

2017-01-30 Thread Jeff Widman
I joined the dev list after KIP-54 voting started, so unfortunately don't have the old thread to bump. But wanted to check if there was any news on this? >From KAFKA-2273 sounds like there are no outstanding objections to the design, but there also aren't yet enough +1's, so is this just

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

2017-01-30 Thread Eugen Dueck
> The PID and sequence numbers are totally transparent to applications. Now that you say it, the cwiki makes that point already pretty clear by the way the Producer API is (not) being changed. Sorry for taking your time on this. In other words, at the point where messages enter the Kafka

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

2017-01-30 Thread Apache Jenkins Server
See Changes: [wangguoz] KAFKA-4613: Treat null-key records the same way for joins and -- [...truncated 3385 lines...] kafka.security.auth.SimpleAclAuthorizerTest > testAclManagementAPIs PASSED

[GitHub] kafka pull request #2468: KAFKA-4613: Follow-up to fix JavaDocs

2017-01-30 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2468 --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is

[GitHub] kafka pull request #2468: KAFKA-4613: Follow-up to fix JavaDocs

2017-01-30 Thread mjsax
GitHub user mjsax opened a pull request: https://github.com/apache/kafka/pull/2468 KAFKA-4613: Follow-up to fix JavaDocs You can merge this pull request into a Git repository by running: $ git pull https://github.com/mjsax/kafka kafka-4613-null-keys-follow-up Alternatively

[GitHub] kafka pull request #2420: KAFKA-4613: Treat null-key records the same way fo...

2017-01-30 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2420 --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

2017-01-30 Thread Guozhang Wang
Hello Folks, We have addressed all the comments collected so far, and would like to propose a voting thread this Wednesday. If you have any further comments on this KIP, please feel free to continue sending them on this thread before that. Guozhang On Mon, Jan 30, 2017 at 1:10 PM, Jason

Re: [VOTE] KIP-115: Enforce offsets.topic.replication.factor

2017-01-30 Thread Onur Karaman
I've updated the KIP title to further clarify the scope of this change: https://cwiki.apache.org/confluence/display/KAFKA/KIP-115%3A+Enforce+offsets.topic.replication.factor+upon+__consumer_offsets+auto+topic+creation On Sat, Jan 28, 2017 at 4:46 PM, Onur Karaman

Re: [DISCUSS] KIP-115: Enforce offsets.topic.replication.factor

2017-01-30 Thread Onur Karaman
I've updated the KIP title to reflect this distinction: https://cwiki.apache.org/confluence/display/KAFKA/KIP-115%3A+Enforce+offsets.topic.replication.factor+upon+__consumer_offsets+auto+topic+creation On Mon, Jan 30, 2017 at 12:52 AM, Onur Karaman wrote: >

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

2017-01-30 Thread Jason Gustafson
+1 for transactional.id. -Jason On Mon, Jan 30, 2017 at 1:05 PM, Guozhang Wang wrote: > If I have to choose between app.id and transactional.instance.id, I'd > choose the latter. > > Renaming transactional.instance.id to transactional.id sounds even better. > > > Guozhang >

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

2017-01-30 Thread Guozhang Wang
If I have to choose between app.id and transactional.instance.id, I'd choose the latter. Renaming transactional.instance.id to transactional.id sounds even better. Guozhang On Mon, Jan 30, 2017 at 11:49 AM, Apurva Mehta wrote: > > Bumping one suggestion from Apurva

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

2017-01-30 Thread Apurva Mehta
> Bumping one suggestion from Apurva above. The name "AppID" has caused some > confusion. We're considering the following renaming: > > 1. AppID -> ProducerId (transaction.app.id -> producer.id) > 2. PID -> IPID (internal producer ID) > How about AppId -> TransactionalId (transaction.app.id ->

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

2017-01-30 Thread Apurva Mehta
> > Eugen, moving your email to the main thread so that it doesn't get split. >> >> The `transaction.app.id` is a prerequisite for using transactional APIs. >> And only messages wrapped inside transactions will enjoy idempotent >> guarantees across sessions, and that too only when they employ a >>

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

2017-01-30 Thread Jason Gustafson
Bumping one suggestion from Apurva above. The name "AppID" has caused some confusion. We're considering the following renaming: 1. AppID -> ProducerId (transaction.app.id -> producer.id) 2. PID -> IPID (internal producer ID) The main drawback of using ProducerId is that it fails to express the

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

2017-01-30 Thread Apurva Mehta
> 102. When there is a correlated hard failure (e.g., power outage), it's >> possible that an existing commit/abort marker is lost in all replicas. >> This may not be fixed by the transaction coordinator automatically and >> the >> consumer may get stuck on that incomplete transaction

[GitHub] kafka pull request #2467: MINOR: Use API hyperlinks in 'Kafka Protocol Guide...

2017-01-30 Thread vahidhashemian
GitHub user vahidhashemian opened a pull request: https://github.com/apache/kafka/pull/2467 MINOR: Use API hyperlinks in 'Kafka Protocol Guide' to facilitate navigation Finding the protocol associated with an API key can be a challenge in the lengthy web page. Adding hyperlinks

[GitHub] kafka pull request #2466: KAFKA-4144: Allow per stream/table timestamp extra...

2017-01-30 Thread jeyhunkarimov
GitHub user jeyhunkarimov opened a pull request: https://github.com/apache/kafka/pull/2466 KAFKA-4144: Allow per stream/table timestamp extractor You can merge this pull request into a Git repository by running: $ git pull https://github.com/jeyhunkarimov/kafka KAFKA-4144

Re: Request to be added as a contributor in Kafka JIRA board

2017-01-30 Thread Jason Gustafson
Done. Thanks for contributing! -Jason On Mon, Jan 30, 2017 at 8:28 AM, Sharad Gulati wrote: > Hi, > > Please add me as a contributor in Kafka JIRA board. > My JIRA Id is sharad.develop > > Thanks, > Sharad >

Re: Rewind Kafka Stream consumer offset by timestamp

2017-01-30 Thread Matthias J. Sax
It would be enough, IMHO :) However, we need to discuss some details about this. 1) we could extend the reset tool with an flag --start-from-offsets and the user can specify an offset per partition This would give the most flexibility, but it is hard to use. Especially if you have many

Fwd: Re: [DISCUSS] KIP-114: KTable materialization and improved semantics

2017-01-30 Thread Matthias J. Sax
cc from user list Forwarded Message Subject: Re: [DISCUSS] KIP-114: KTable materialization and improved semantics Date: Mon, 30 Jan 2017 09:23:04 -0800 From: Matthias J. Sax Organization: Confluent Inc To: us...@kafka.apache.org Hi, I think Eno's

Request to be added as a contributor in Kafka JIRA board

2017-01-30 Thread Sharad Gulati
Hi, Please add me as a contributor in Kafka JIRA board. My JIRA Id is sharad.develop Thanks, Sharad

Re: [DISCUSS] KIP-114: KTable materialization and improved semantics

2017-01-30 Thread Eno Thereska
So I think there are several important discussion threads that are emerging here. Let me try to tease them apart: 1. inconsistency in what is materialized and what is not, what is queryable and what is not. I think we all agree there is some inconsistency there and this will be addressed with

Re: [DISCUSS] KIP-114: KTable materialization and improved semantics

2017-01-30 Thread Eno Thereska
Hi Damian, I feel that if we go with the builder method as described below, we are moving away from a declarative DSL. Eno > On 30 Jan 2017, at 09:38, Damian Guy wrote: > > How about something a bit different? We could pass builders to all the > KTable methods. So we

Re: Rewind Kafka Stream consumer offset by timestamp

2017-01-30 Thread Jorge Esteban Quilcate Otoya
Thanks Eno and Matthias for your feedback! I've check KIP-95 and Matthias blog post ( https://www.confluent.io/blog/data-reprocessing-with-kafka-streams-resetting-a-streams-application/) and I have a clearer idea on how stream internals work. In a general use-case, following Application Reset

Re: [DISCUSS] KIP-114: KTable materialization and improved semantics

2017-01-30 Thread Eno Thereska
Hi there, The inconsistency will be resolved, whether with materialize or overloaded methods. With the discussion on the DSL & stores I feel we've gone in a slightly different tangent, which is worth discussing nonetheless. We have entered into an argument around the scope of the DSL. The

Re: Re: [DISCUSS] KIP-114: KTable materialization and improved semantics

2017-01-30 Thread Damian Guy
How about something a bit different? We could pass builders to all the KTable methods. So we have something like this: ktable.filter(new FilteredTableBuilder(predicate) .materializeAs(someName)) ktable.join(new TableJoinBuilder(otherTable, valueJoiner)

Re: [DISCUSS] KIP-115: Enforce offsets.topic.replication.factor

2017-01-30 Thread Onur Karaman
Regarding Joel's comment: > On Jan 25, 2017, at 9:26 PM, Joel Koshy wrote: > > already voted, but one thing worth considering (since this KIP speaks of > *enforcement*) is desired behavior if the topic already exists and the > config != existing RF. > The short answer: The

[GitHub] kafka pull request #2465: KAFKA-4710: Interpolate log4j's logging source int...

2017-01-30 Thread kawamuray
GitHub user kawamuray opened a pull request: https://github.com/apache/kafka/pull/2465 KAFKA-4710: Interpolate log4j's logging source interpretation to correct loation info of logs written through trait methods Issue: https://issues.apache.org/jira/browse/KAFKA-4710 This

Fwd: Re: [DISCUSS] KIP-114: KTable materialization and improved semantics

2017-01-30 Thread Matthias J. Sax
cc from user list Forwarded Message Subject: Re: [DISCUSS] KIP-114: KTable materialization and improved semantics Date: Mon, 30 Jan 2017 00:06:37 -0800 From: Matthias J. Sax Organization: Confluent Inc To: us...@kafka.apache.org I understand point (1)