Re: [VOTE] KIP-48 Support for delegation tokens as an authentication mechanism

2017-02-10 Thread Manikumar
Yes, owners and the renewers can always describe their own tokens. Updated the KIP. On Sat, Feb 11, 2017 at 3:12 AM, Jun Rao wrote: > Hi, Mani, > > Thanks for the update. Just a minor comment below. Otherwise, +1 from me. > > > > > > > > > > 116. Could you document the ACL rules associated with

Re: [VOTE] 0.10.2.0 RC1

2017-02-10 Thread Matthias J. Sax
Hi Ian, thanks for reporting this. I had a look at the stack trace and code and the whole situation is quite confusing. The exception itself is expected but we have a try-catch-block that should swallow the exception and it should never bubble up: In AbstractTaskCreator.retryWithBackoff a call

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

2017-02-10 Thread Gwen Shapira
+1 (binding) Thank you. This is truly incredible design and very well written paper. On Wed, Feb 1, 2017 at 8:13 PM, Guozhang Wang wrote: > Hi all, > > We would like to start the voting process for KIP-98. The KIP can be found > at > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-98+-+E

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

2017-02-10 Thread Gwen Shapira
Understood, thank you! On Fri, Feb 10, 2017 at 2:54 PM, Apurva Mehta wrote: > The point about management of transactionalId was raised before, and Jason > tackled it here (note it was called 'AppId' back then): > https://www.mail-archive.com/dev@kafka.apache.org/msg61200.html > > 1. Essentially,

Jenkins build is back to normal : kafka-0.10.2-jdk7 #76

2017-02-10 Thread Apache Jenkins Server
See

[jira] [Commented] (KAFKA-4756) The auto-generated broker id should be passed to MetricReporter.configure

2017-02-10 Thread Colin P. McCabe (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4756?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15862112#comment-15862112 ] Colin P. McCabe commented on KAFKA-4756: I put up a patch for this that fixes the

Re: [DISCUSS] KIP-117: Add a public AdministrativeClient API for Kafka admin operations

2017-02-10 Thread Dong Lin
Hi Jun, Currently KIP-107 uses this API: Future> purgeDataBefore(Map offsetForPartition) Are you suggesting that we should provide this: Future purgeDataBefore(TopicPartition, Long) I think the second solution works and the resulting implementation of KIP-107 will be simpler. The only concern

[jira] [Commented] (KAFKA-4756) The auto-generated broker id should be passed to MetricReporter.configure

2017-02-10 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4756?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15862105#comment-15862105 ] ASF GitHub Bot commented on KAFKA-4756: --- GitHub user cmccabe opened a pull request:

[GitHub] kafka pull request #2540: KAFKA-4756: The auto-generated broker id should be...

2017-02-10 Thread cmccabe
GitHub user cmccabe opened a pull request: https://github.com/apache/kafka/pull/2540 KAFKA-4756: The auto-generated broker id should be passed to MetricRe… …porter.configure You can merge this pull request into a Git repository by running: $ git pull https://github.com/cmcc

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

2017-02-10 Thread Apache Jenkins Server
See Changes: [me] MINOR: Fix quickstart in docs -- [...truncated 8408 lines...] kafka.security.auth.SimpleAclAuthorizerTest > testLoadCache STARTED kafka.security.auth.SimpleAclAuthorizerTest

Re: [DISCUSS] KIP-112: Handle disk failure for JBOD

2017-02-10 Thread Dong Lin
Hi Jun, Thanks for the detailed comments. Please see answers inline: On Fri, Feb 10, 2017 at 3:08 PM, Jun Rao wrote: > Hi, Dong, > > Thanks for the updated wiki. A few comments below. > > 1. Topics get created > 1.1 Instead of storing successfully created replicas in ZK, could we store > unsucc

Re: [DISCUSS] KIP-117: Add a public AdministrativeClient API for Kafka admin operations

2017-02-10 Thread Jun Rao
Hi, Dong, For KIP-107, the purgeDataBefore() api will eventually be added to the AdminClient too, right? It would be useful to make the apis consistent. Currently, in KIP-107, we do batching in purgeDataBefore(). In Colin's current proposal, there is no batching. Thanks, Jun On Thu, Feb 9, 2017

Re: [DISCUSS] KIP-112: Handle disk failure for JBOD

2017-02-10 Thread Jun Rao
Hi, Dong, Thanks for the updated wiki. A few comments below. 1. Topics get created 1.1 Instead of storing successfully created replicas in ZK, could we store unsuccessfully created replicas in ZK? Since the latter is less common, it probably reduces the load on ZK. 1.2 If an error is received for

[GitHub] kafka pull request #2539: MINOR: Fix quickstart in docs

2017-02-10 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2539 --- 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 enab

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

2017-02-10 Thread Apurva Mehta
The point about management of transactionalId was raised before, and Jason tackled it here (note it was called 'AppId' back then): https://www.mail-archive.com/dev@kafka.apache.org/msg61200.html 1. Essentially, the full transactional functionality really only makes sense for a stateful application

[jira] [Assigned] (KAFKA-4732) Unstable test: KStreamKTableJoinIntegrationTest.shouldCountClicksPerRegion[1]

2017-02-10 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4732?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax reassigned KAFKA-4732: -- Assignee: Matthias J. Sax > Unstable test: KStreamKTableJoinIntegrationTest.shouldCount

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

2017-02-10 Thread Gwen Shapira
Thank you so much for the detailed explanation! I got the READ COMMITTED behavior. It is very odd that we will routinely see parts of a committed transaction but not other parts. But I understand the reasons. Regarding transactional.id: I read the document but I'm still not 100% clear on how uniqu

[jira] [Updated] (KAFKA-4144) Allow per stream/table timestamp extractor

2017-02-10 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4144?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax updated KAFKA-4144: --- Labels: api needs-kip (was: api) > Allow per stream/table timestamp extractor > -

Re: [VOTE] KIP-48 Support for delegation tokens as an authentication mechanism

2017-02-10 Thread Jun Rao
Hi, Mani, Thanks for the update. Just a minor comment below. Otherwise, +1 from me. > > > > > 116. Could you document the ACL rules associated with those new requests? > > For example, do we allow any one to create, delete, describe delegation > > tokens? > > > > > Currently we only allow a owne

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

2017-02-10 Thread Apache Jenkins Server
See

Re: [DISCUSS] KIP-116 - Add State Store Checkpoint Interval Configuration

2017-02-10 Thread Damian Guy
I'm fine with that. Gouzhang? On Fri, 10 Feb 2017 at 19:45, Matthias J. Sax wrote: > I am actually supporting Eno's view: checkpoint on every commit. > > @Dhwani: I understand your view and did raise the same question about > performance trade-off with checkpoiting enabled/disabled etc. However,

[GitHub] kafka pull request #2539: MINOR: Fix quickstart in docs

2017-02-10 Thread vahidhashemian
GitHub user vahidhashemian opened a pull request: https://github.com/apache/kafka/pull/2539 MINOR: Fix quickstart in docs You can merge this pull request into a Git repository by running: $ git pull https://github.com/vahidhashemian/kafka doc/fix_quickstart_issues Alternative

[jira] [Commented] (KAFKA-2857) ConsumerGroupCommand throws GroupCoordinatorNotAvailableException when describing a non-existent group before the offset topic is created

2017-02-10 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2857?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15861796#comment-15861796 ] ASF GitHub Bot commented on KAFKA-2857: --- GitHub user vahidhashemian opened a pull re

[GitHub] kafka pull request #2538: KAFKA-2857: Retry querying the consumer group whil...

2017-02-10 Thread vahidhashemian
GitHub user vahidhashemian opened a pull request: https://github.com/apache/kafka/pull/2538 KAFKA-2857: Retry querying the consumer group while initializing This applies to new-consumer based groups and would avoid scenarios in which user issues a `--describe` query while the group

[jira] [Resolved] (KAFKA-4430) Broker logging "Topic and partition to exceptions: [topic,6] -> kafka.common.MessageSizeTooLargeException"

2017-02-10 Thread Srinivas Dhruvakumar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4430?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Srinivas Dhruvakumar resolved KAFKA-4430. - Resolution: Done > Broker logging "Topic and partition to exceptions: [topic,6] ->

Re: [VOTE] KIP-48 Support for delegation tokens as an authentication mechanism

2017-02-10 Thread Manikumar
Hi Jun, Thanks for the detailed review. Pl see the replies inline > 101.2/101.3. Could we just remove owner and renewer from > DelegationTokenResponse if we don't have a use case? > > Removed owner/renewer fileds from DelegationTokenResponse. Updated the KIP. > 111. ExpireTokenResponse: Should

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

2017-02-10 Thread Matthias J. Sax
+1 On 2/10/17 9:09 AM, Eno Thereska wrote: > +1 (non-binding). > > > >> On 10 Feb 2017, at 13:40, Tom Crayford wrote: >> >> I'm +0 (non-binding) on this now. I think the proposal here adds quite a >> lot of complexity to Kafka, makes building client libraries much harder, >> and is going to co

Re: [DISCUSS] KIP-116 - Add State Store Checkpoint Interval Configuration

2017-02-10 Thread Matthias J. Sax
I am actually supporting Eno's view: checkpoint on every commit. @Dhwani: I understand your view and did raise the same question about performance trade-off with checkpoiting enabled/disabled etc. However, it seems that writing the checkpoint file is super cheap -- thus, there is nothing to gain p

[jira] [Commented] (KAFKA-4739) KafkaConsumer poll going into an infinite loop

2017-02-10 Thread Sagar Sadashiv Patwardhan (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4739?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15861745#comment-15861745 ] Sagar Sadashiv Patwardhan commented on KAFKA-4739: -- [~hachikuji] [~huxi_2

[GitHub] kafka pull request #2537: MINOR: Improve LogCleaner buffer size change loggi...

2017-02-10 Thread cotedm
GitHub user cotedm opened a pull request: https://github.com/apache/kafka/pull/2537 MINOR: Improve LogCleaner buffer size change logging Right now it looks like the buffer size only grows if you are watching the cleaner logs. It would be nice if we also logged when the buffer size w

Build failed in Jenkins: kafka-0.10.2-jdk7 #75

2017-02-10 Thread Apache Jenkins Server
See Changes: [wangguoz] MINOR: add GlobalKTable doc to streams.html -- [...truncated 15853 lines...] org.apache.kafka.common.security.scram.ScramCredentialUtilsTest > generateCredential PASSED

[jira] [Updated] (KAFKA-4714) Implement remaining KIP-66 SMTs

2017-02-10 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4714?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ewen Cheslack-Postava updated KAFKA-4714: - Component/s: KafkaConnect > Implement remaining KIP-66 SMTs >

[jira] [Created] (KAFKA-4757) NetworkClient trace level logging regressions

2017-02-10 Thread Onur Karaman (JIRA)
Onur Karaman created KAFKA-4757: --- Summary: NetworkClient trace level logging regressions Key: KAFKA-4757 URL: https://issues.apache.org/jira/browse/KAFKA-4757 Project: Kafka Issue Type: Bug

Is it possible to receive duplicate messages from Kafka when...

2017-02-10 Thread Michaud, Ben A
Is it possible to receive duplicate messages from Kafka 0.9.0.1 or 0.10.1.0 when you have a topic with three partitions, and one consumer group with three consumer clients. One client stops consuming and is taken offline. These clients do not commit offset immediately, but the offsets are commit

[GitHub] kafka pull request #2536: MINOR: Move compression stream construction into C...

2017-02-10 Thread hachikuji
GitHub user hachikuji opened a pull request: https://github.com/apache/kafka/pull/2536 MINOR: Move compression stream construction into CompressionType You can merge this pull request into a Git repository by running: $ git pull https://github.com/hachikuji/kafka minor-move-c

[GitHub] kafka pull request #2535: MINOR: don't throw CommitFailedException during su...

2017-02-10 Thread dguy
GitHub user dguy opened a pull request: https://github.com/apache/kafka/pull/2535 MINOR: don't throw CommitFailedException during suspendTasksAndState You can merge this pull request into a Git repository by running: $ git pull https://github.com/dguy/kafka minor-commit-failed

Re: [DISCUSS] KIP-112: Handle disk failure for JBOD

2017-02-10 Thread Dong Lin
Hi Jun, Can I replace zookeeper access with direct RPC for both ISR notification and disk failure notification in a future KIP, or do you feel we should do it in this KIP? Hi Eno, Grant and everyone, Is there further improvement you would like to see with this KIP? Thanks you all for the commen

Re: [VOTE] 0.10.2.0 RC1

2017-02-10 Thread Ian Duffy
Seeing the following failure when using multi-threaded streams Feb 10 17:21:15 ip-172-31-137-57 docker/43e65fe123cd[826]: org.apache.kafka.streams.errors.LockException: task [0_21] Failed to lock the state directory: /tmp/kafka-streams/text_pipeline_id/0_21 Feb 10 17:21:15 ip-172-31-137-57 docker/

[GitHub] kafka pull request #2516: MINOR: add GlobalKTable doc to streams.html

2017-02-10 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2516 --- 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 enab

Re: [DISCUSS] KIP-116 - Add State Store Checkpoint Interval Configuration

2017-02-10 Thread Damian Guy
Gouzhang, Thanks for the clarification. Understood. Eno, you are correct if we just used commit interval then we wouldn't need a KIP. But, then we'd have no way of turning it off. On Fri, 10 Feb 2017 at 17:14 Eno Thereska wrote: > A quick check: the checkpoint file is not new, we're just exposi

[jira] [Commented] (KAFKA-4753) KafkaConsumer susceptible to FetchResponse starvation

2017-02-10 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4753?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15861571#comment-15861571 ] Jason Gustafson commented on KAFKA-4753: I guess the background thread unintention

[GitHub] kafka pull request #2532: MINOR: update KafkaStreams.metadataForKey(...) jav...

2017-02-10 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2532 --- 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 enab

[GitHub] kafka pull request #2527: MINOR: update README with how to run code coverage...

2017-02-10 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2527 --- 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 enab

Re: [DISCUSS] KIP-116 - Add State Store Checkpoint Interval Configuration

2017-02-10 Thread Eno Thereska
A quick check: the checkpoint file is not new, we're just exposing a knob on when to set it, right? Would turning if off still do what it does today (i.e., write the checkpoint at the end when the user quits?) So it's not a new feature as such, I was only recommending we dial up the frequency by

[jira] [Created] (KAFKA-4756) The auto-generated broker id should be passed to MetricReporter.configure

2017-02-10 Thread Ismael Juma (JIRA)
Ismael Juma created KAFKA-4756: -- Summary: The auto-generated broker id should be passed to MetricReporter.configure Key: KAFKA-4756 URL: https://issues.apache.org/jira/browse/KAFKA-4756 Project: Kafka

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

2017-02-10 Thread Eno Thereska
+1 (non-binding). > On 10 Feb 2017, at 13:40, Tom Crayford wrote: > > I'm +0 (non-binding) on this now. I think the proposal here adds quite a > lot of complexity to Kafka, makes building client libraries much harder, > and is going to confuse users a lot. I'm also worried about this feature >

Re: [DISCUSS] KIP-116 - Add State Store Checkpoint Interval Configuration

2017-02-10 Thread Guozhang Wang
Damian, I was thinking if it is a new failure scenarios but as Eno pointed out it was not. Another thing I was considering is if it has any impact for incorporating KIP-98 to avoid duplicates: if there is a failure in the middle of a transaction, then upon recovery we cannot rely on the local sta

[VOTE] 0.10.2.0 RC1

2017-02-10 Thread Ewen Cheslack-Postava
Hello Kafka users, developers and client-developers, This is RC1 for release of Apache Kafka 0.10.2.0. This is a minor version release of Apache Kafka. It includes 19 new KIPs. See the release notes and release plan (https://cwiki.apache.org/ confluence/display/KAFKA/Release+Plan+0.10.2.0) for mo

[jira] [Commented] (KAFKA-4755) SimpleBenchmark consume test fails for streams

2017-02-10 Thread Eno Thereska (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4755?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15861469#comment-15861469 ] Eno Thereska commented on KAFKA-4755: - Logs did not reveal anything suspicious, might

Re: [DISCUSS] KIP-116 - Add State Store Checkpoint Interval Configuration

2017-02-10 Thread Eno Thereska
The overhead of writing to the checkpoint file should be much, much smaller than the overall overhead of doing a commit, so I think tuning the commit time is sufficient to guide performance tradeoffs. Eno > On 10 Feb 2017, at 13:08, Dhwani Katagade > wrote: > > May be for fine tuning the pe

[GitHub] kafka pull request #2534: Update SimpleAclAuthorizer.scala

2017-02-10 Thread cotedm
GitHub user cotedm opened a pull request: https://github.com/apache/kafka/pull/2534 Update SimpleAclAuthorizer.scala Comments had the wrong name for the Group znode, updating ConsumerGroup-->Group in the comments to match what's actually coded. @ijuma if you can review, it'

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

2017-02-10 Thread Jozef Koval (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1420?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15861359#comment-15861359 ] Jozef Koval commented on KAFKA-1420: I see the patch is available for this issue, woul

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

2017-02-10 Thread Apache Jenkins Server
See Changes: [ismael] KAFKA-4617; Improve configuration of Gradle’s eclipse task [ismael] KAFKA-4525; Kafka should not require SSL truststore password [me] MINOR: add architecture section and configure / execution for streams [jason

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

2017-02-10 Thread Tom Crayford
I'm +0 (non-binding) on this now. I think the proposal here adds quite a lot of complexity to Kafka, makes building client libraries much harder, and is going to confuse users a lot. I'm also worried about this feature being hard to implement, and/or testing not being rigorous enough - Aphyr's Jeps

Re: [DISCUSS] KIP-116 - Add State Store Checkpoint Interval Configuration

2017-02-10 Thread Dhwani Katagade
May be for fine tuning the performance. Say we don't need the checkpointing and would like to gain the lil bit of performance improvement by turning it off. The trade off is between giving people control knobs vs complicating the complete set of knobs. -dk On Friday 10 February 2017 04:05 PM,

[jira] [Commented] (KAFKA-4430) Broker logging "Topic and partition to exceptions: [topic,6] -> kafka.common.MessageSizeTooLargeException"

2017-02-10 Thread Jozef Koval (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4430?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15861174#comment-15861174 ] Jozef Koval commented on KAFKA-4430: I think this issue has been resolved. > Broker l

[GitHub] kafka pull request #2533: Checking the Timestamp is insufficient

2017-02-10 Thread rnpridgeon
GitHub user rnpridgeon opened a pull request: https://github.com/apache/kafka/pull/2533 Checking the Timestamp is insufficient My understanding is that the record format allots 1 bit for the timestamp type field. As a result it can be either 1 or 0 and never -1. This means Connect'

[GitHub] kafka pull request #2532: MINOR: update KafkaStreams.metadataForKey(...) jav...

2017-02-10 Thread dguy
GitHub user dguy opened a pull request: https://github.com/apache/kafka/pull/2532 MINOR: update KafkaStreams.metadataForKey(...) javadoc Add a note to `KafkaStreams.metadataForKey(String, K, Serializer)` to point out that in the case of a Window Store the Serializer should still be

[jira] [Updated] (KAFKA-4755) SimpleBenchmark consume test fails for streams

2017-02-10 Thread Eno Thereska (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4755?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Eno Thereska updated KAFKA-4755: Description: This occurred Feb 10th 2017: kafkatest.benchmarks.streams.streams_simple_benchmark_tes

[jira] [Created] (KAFKA-4755) SimpleBenchmark consume test fails for streams

2017-02-10 Thread Eno Thereska (JIRA)
Eno Thereska created KAFKA-4755: --- Summary: SimpleBenchmark consume test fails for streams Key: KAFKA-4755 URL: https://issues.apache.org/jira/browse/KAFKA-4755 Project: Kafka Issue Type: Bug

[GitHub] kafka pull request #2531: WIP: KIP-1610: Review usages of Map#mapValues.

2017-02-10 Thread jozi-k
GitHub user jozi-k opened a pull request: https://github.com/apache/kafka/pull/2531 WIP: KIP-1610: Review usages of Map#mapValues. KAFKA-1610: Local modifications to collections generated from mapValues will be lost. You can merge this pull request into a Git repository by running:

Re: [DISCUSS] KIP-116 - Add State Store Checkpoint Interval Configuration

2017-02-10 Thread Eno Thereska
I can't see why users would care to turn it off. Eno > On 10 Feb 2017, at 10:29, Damian Guy wrote: > > Hi Eno, > > Sounds good to me. The only reason i can think of is if we want to be able > to turn it off. > Gouzhang - thoughts? > > On Fri, 10 Feb 2017 at 10:28 Eno Thereska wrote: > >> Qu

[jira] [Commented] (KAFKA-4715) Consumer/Producer config does not work with related enums

2017-02-10 Thread Mathias Kub (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4715?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15861061#comment-15861061 ] Mathias Kub commented on KAFKA-4715: I thought of `configValue`, because it better des

Re: [DISCUSS] KIP-116 - Add State Store Checkpoint Interval Configuration

2017-02-10 Thread Damian Guy
Hi Eno, Sounds good to me. The only reason i can think of is if we want to be able to turn it off. Gouzhang - thoughts? On Fri, 10 Feb 2017 at 10:28 Eno Thereska wrote: > Question: if checkpointing is so cheap why not do it every commit > interval? That way we can get rid of this extra config v

Re: [DISCUSS] KIP-116 - Add State Store Checkpoint Interval Configuration

2017-02-10 Thread Eno Thereska
Question: if checkpointing is so cheap why not do it every commit interval? That way we can get rid of this extra config variable and just use the existing commit interval. Less tuning knobs. Eno > On 10 Feb 2017, at 09:27, Damian Guy wrote: > > Gouzhang, > > You've confused me. The failure

[jira] [Resolved] (KAFKA-4525) Kafka should not require SSL trust store password

2017-02-10 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4525?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma resolved KAFKA-4525. Resolution: Fixed Fix Version/s: 0.10.3.0 Issue resolved by pull request 2246 [https://github

Messages not received (C/C++ client)

2017-02-10 Thread ????
hi?? Messages are not received when the partition is not consumed for the first time. The auto.offset.reset is set to be largest. Repetition of steps are as follows?? 1.create a group; 2.start a producer, and send messages to a fixed partition, such as partition 0; 3.start a consumer 4.cl

Re: [DISCUSS] KIP-116 - Add State Store Checkpoint Interval Configuration

2017-02-10 Thread Damian Guy
Gouzhang, You've confused me. The failure scenarios you have described are the same as they are today. With the checkpoint files in place less data will be replayed, so there will be fewer duplicates. Are you saying you'd like the option to turn checkpointing off? Thanks, Damian On Thu, 9 Feb 2

[jira] [Commented] (KAFKA-2729) Cached zkVersion not equal to that in zookeeper, broker not recovering.

2017-02-10 Thread JIRA
[ https://issues.apache.org/jira/browse/KAFKA-2729?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15860948#comment-15860948 ] Sinóros-Szabó Péter commented on KAFKA-2729: Do you have any plan to resolve t

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

2017-02-10 Thread Jason Gustafson
Hey Gwen, Thanks for the questions and comments. Responses below: I not sure I'm clear on the expected behavior of READ_COMMITTED in > some interleaved cases: > * If a transaction starts, sends few messages and someone writes > non-transactional event into the same topic/partition, few more event

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

2017-02-10 Thread Apache Jenkins Server
See Changes: [wangguoz] KAFKA-4654: Improve test coverage for MemoryLRUCacheStore -- [...truncated 19919 lines...] org.apache.kafka.streams.KafkaStreamsTest > testCloseIsIdempotent STARTED org.

[GitHub] kafka pull request #2530: MINOR: Replacing for with foreach loop in common m...

2017-02-10 Thread PKOfficial
GitHub user PKOfficial opened a pull request: https://github.com/apache/kafka/pull/2530 MINOR: Replacing for with foreach loop in common module You can merge this pull request into a Git repository by running: $ git pull https://github.com/PKOfficial/kafka refactored-code Alt