[GitHub] kafka pull request #2477: MINOR: Change rocksdb logging to error level

2017-02-01 Thread dguy
GitHub user dguy opened a pull request: https://github.com/apache/kafka/pull/2477 MINOR: Change rocksdb logging to error level According to the java-doc: https://github.com/facebook/rocksdb/blob/master/java/src/main/java/org/rocksdb/Logger.java#L31 the rocksdb logging level should

Re: [DISCUSS] KIP-113: Support replicas movement between log directories

2017-02-01 Thread Alexey Ozeritsky
24.01.2017, 22:03, "Dong Lin" : > Hey Alexey, > > Thanks. I think we agreed that the suggested solution doesn't work in > general for kafka users. To answer your questions: > > 1. I agree we need quota to rate limit replica movement when a broker is > moving a "leader" replica. I will come up wit

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

2017-02-01 Thread Damian Guy
Hi all, I would like to start the discussion on KIP-116: https://cwiki.apache.org/confluence/display/KAFKA/KIP-116+-+Add+State+Store+Checkpoint+Interval+Configuration Thanks, Damian

Re: [DISCUS] consuming messages is polling - is there a reason? new KIP for poll?

2017-02-01 Thread Alexander Binzberger
Yes I have seen fetch.max.wait.ms - you would not need this parameter with push. The broker would have time to collect a batch while the last push gets processed or lets say until the broker gets a ack with the new offset for the last pushed message. But of course you could use this param (brok

[GitHub] kafka pull request #2478: KAFKA-4702: Parametrize streams benchmarks to run ...

2017-02-01 Thread enothereska
GitHub user enothereska opened a pull request: https://github.com/apache/kafka/pull/2478 KAFKA-4702: Parametrize streams benchmarks to run at scale You can merge this pull request into a Git repository by running: $ git pull https://github.com/enothereska/kafka minor-benchmark

[jira] [Commented] (KAFKA-4702) Parametrize streams benchmarks to run at scale

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

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

2017-02-01 Thread Eno Thereska
Thanks Damian, this is a good idea and will reduce the restore time. Looking forward, with exactly once and support for transactions in Kafka, I believe we'll have to add some support for rolling back checkpoints, e.g., when a transaction is aborted. We need to be aware of that and ideally antic

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

2017-02-01 Thread Eno Thereska
As a follow up to my previous comment, have you thought about writing the checkpoint to a topic instead of a local file? That would have the advantage that all metadata continues to be managed by Kafka, as well as fit with EoS. The potential disadvantage would be a slower latency, however if it

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

2017-02-01 Thread Eno Thereska
I'm coming somewhat late to the discussion, apologies for that. I'm worried about this proposal. It's moving Kafka to a world where it manages disks. So in a sense, the scope of the KIP is limited, but the direction it sets for Kafka is quite a big step change. Fundamentally this is about balan

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

2017-02-01 Thread Damian Guy
Thanks for the comments Eno. As for exactly once, i don't believe this matters as we are just restoring the change-log, i.e, the result of the aggregations that previously ran etc. So once initialized the state store will be in the same state as it was before. Having the checkpoint in a kafka topic

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

2017-02-01 Thread Ismael Juma
Hi all, A few comments follow: 1. The document states "inter-broker communications will be increased by M * N * P round trips per sec. We need to conduct some system performance test to make sure this additional inter-broker traffic would not largely impact the broker cluster". Has this testing b

[jira] [Created] (KAFKA-4719) Timed out waiting for consumption in OffsetValidationTest.test_broker_failure

2017-02-01 Thread Jason Gustafson (JIRA)
Jason Gustafson created KAFKA-4719: -- Summary: Timed out waiting for consumption in OffsetValidationTest.test_broker_failure Key: KAFKA-4719 URL: https://issues.apache.org/jira/browse/KAFKA-4719 Proje

[jira] [Commented] (KAFKA-4689) OffsetValidationTest fails validation with "Current position greater than the total number of consumed records"

2017-02-01 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4689?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15848602#comment-15848602 ] Jason Gustafson commented on KAFKA-4689: Note I created KAFKA-4719 for the Timeout

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

2017-02-01 Thread Eno Thereska
Thanks everyone! I think it's time to do a V2 on the KIP so I'll do that and we can see how it looks and continue the discussion from there. Stay tuned. Thanks Eno > On 30 Jan 2017, at 17:23, Matthias J. Sax wrote: > > Hi, > > I think Eno's separation is very clear and helpful. In order to >

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

2017-02-01 Thread Dong Lin
Hey Eno, Thanks much for the review. I think your suggestion is to split disks of a machine into multiple disk sets and run one broker per disk set. Yeah this is similar to Colin's suggestion of one-broker-per-disk, which we have evaluated at LinkedIn and considered it to be a good short term app

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

2017-02-01 Thread Grant Henke
Hi Dong, Thanks for putting this together. Since we are discussing alternative/simplified options. Have you considered handling the disk failures broker side to prevent a crash, marking the disk as "bad" to that individual broker, and continuing as normal? I imagine the broker would then fall out

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

2017-02-01 Thread Dong Lin
Hey Grant, Yes, this KIP does exactly what you described:) Thanks, Dong On Wed, Feb 1, 2017 at 9:45 AM, Grant Henke wrote: > Hi Dong, > > Thanks for putting this together. > > Since we are discussing alternative/simplified options. Have you considered > handling the disk failures broker side t

[GitHub] kafka pull request #2479: KAFKA-4719: Consumption timeout should take into a...

2017-02-01 Thread hachikuji
GitHub user hachikuji opened a pull request: https://github.com/apache/kafka/pull/2479 KAFKA-4719: Consumption timeout should take into account producer request timeout You can merge this pull request into a Git repository by running: $ git pull https://github.com/hachikuji/k

[jira] [Commented] (KAFKA-4719) Timed out waiting for consumption in OffsetValidationTest.test_broker_failure

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

[GitHub] kafka pull request #2480: MINOR: Update docs for version 0.10.2.0

2017-02-01 Thread ewencp
GitHub user ewencp opened a pull request: https://github.com/apache/kafka/pull/2480 MINOR: Update docs for version 0.10.2.0 You can merge this pull request into a Git repository by running: $ git pull https://github.com/ewencp/kafka docs-version-0.10.2.0 Alternatively you can

[GitHub] kafka pull request #2480: MINOR: Update docs for version 0.10.2.0

2017-02-01 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2480 --- 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-112: Handle disk failure for JBOD

2017-02-01 Thread Colin McCabe
Hmm. Maybe I misinterpreted, but I got the impression that Grant was suggesting that we avoid introducing this concept of "offline replicas" for now. Is that feasible? What is the strategy for declaring a log directory bad? Is it an administrative action? Or is the broker itself going to be re

[jira] [Updated] (KAFKA-4719) Timed out waiting for consumption in OffsetValidationTest.test_broker_failure

2017-02-01 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4719?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson updated KAFKA-4719: --- Fix Version/s: (was: 0.10.2.0) 0.10.3.0 > Timed out waiting for consump

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

2017-02-01 Thread Eno Thereska
Hi Dong, Would it make sense to do a discussion over video/voice about this? I think it's sufficiently complex that we can probably make quicker progress that way? So shall we do a KIP meeting soon? I can do this week (Thu/Fri) or next week. Thanks Eno > On 1 Feb 2017, at 18:29, Colin McCabe w

[jira] [Created] (KAFKA-4720) Add KStream.peek(ForeachAction)

2017-02-01 Thread Steven Schlansker (JIRA)
Steven Schlansker created KAFKA-4720: Summary: Add KStream.peek(ForeachAction) Key: KAFKA-4720 URL: https://issues.apache.org/jira/browse/KAFKA-4720 Project: Kafka Issue Type: New Feature

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

2017-02-01 Thread Dong Lin
Hey Grant, Colin, My bad, I misunderstood Grant's suggestion initially. Indeed this is a very interesting idea to just wait for replica.max.lag.ms for the replica on the bad disk to drop out of ISR instead of having broker actively reporting this to the controller. I have several concerns with th

[jira] [Commented] (KAFKA-3729) Auto-configure non-default SerDes passed alongside the topology builder

2017-02-01 Thread Roy Majumdar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3729?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15848875#comment-15848875 ] Roy Majumdar commented on KAFKA-3729: - Hi, with all due respect has this issue been fi

[jira] [Commented] (KAFKA-3729) Auto-configure non-default SerDes passed alongside the topology builder

2017-02-01 Thread Bharat Viswanadham (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3729?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15848880#comment-15848880 ] Bharat Viswanadham commented on KAFKA-3729: --- [~Roy19] I have not started working

[jira] [Comment Edited] (KAFKA-3729) Auto-configure non-default SerDes passed alongside the topology builder

2017-02-01 Thread Bharat Viswanadham (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3729?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15848880#comment-15848880 ] Bharat Viswanadham edited comment on KAFKA-3729 at 2/1/17 8:08 PM: -

[jira] [Created] (KAFKA-4721) KafkaStreams (and possibly others) should inherit Closeable

2017-02-01 Thread Steven Schlansker (JIRA)
Steven Schlansker created KAFKA-4721: Summary: KafkaStreams (and possibly others) should inherit Closeable Key: KAFKA-4721 URL: https://issues.apache.org/jira/browse/KAFKA-4721 Project: Kafka

[GitHub] kafka pull request #2477: MINOR: Change rocksdb logging to error level

2017-02-01 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2477 --- 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 #2481: Kafka 4706 :Unify StreamsKafkaClient instances

2017-02-01 Thread sharad-develop
GitHub user sharad-develop opened a pull request: https://github.com/apache/kafka/pull/2481 Kafka 4706 :Unify StreamsKafkaClient instances Kafka 4706 :Unify StreamsKafkaClient instances You can merge this pull request into a Git repository by running: $ git pull https://github.

[jira] [Resolved] (KAFKA-2172) Round-robin partition assignment strategy too restrictive

2017-02-01 Thread Andrew Olson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2172?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Andrew Olson resolved KAFKA-2172. - Resolution: Fixed Assignee: Andrew Olson Fix Version/s: 0.10.2.0 Fixed for new con

[jira] [Commented] (KAFKA-3729) Auto-configure non-default SerDes passed alongside the topology builder

2017-02-01 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3729?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15848947#comment-15848947 ] Matthias J. Sax commented on KAFKA-3729: Code freeze for {{0.10.2}} was last Frida

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

2017-02-01 Thread Vahid Hashemian (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2857?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15848972#comment-15848972 ] Vahid Hashemian commented on KAFKA-2857: [~hachikuji] I am thinking about the two

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

2017-02-01 Thread Colin McCabe
On Wed, Feb 1, 2017, at 11:35, Dong Lin wrote: > Hey Grant, Colin, > > My bad, I misunderstood Grant's suggestion initially. Indeed this is a > very > interesting idea to just wait for replica.max.lag.ms for the replica on > the > bad disk to drop out of ISR instead of having broker actively repor

[jira] [Created] (KAFKA-4722) StreamThread should allow customization of thread prefix

2017-02-01 Thread Steven Schlansker (JIRA)
Steven Schlansker created KAFKA-4722: Summary: StreamThread should allow customization of thread prefix Key: KAFKA-4722 URL: https://issues.apache.org/jira/browse/KAFKA-4722 Project: Kafka

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

2017-02-01 Thread Apache Jenkins Server
See Changes: [wangguoz] MINOR: Change rocksdb logging to error level -- [...truncated 4107 lines...] kafka.integration.SaslPlaintextTopicMetadataTest > testIsrAfterBrokerShutDownAndJoinsBack S

[jira] [Commented] (KAFKA-4722) StreamThread should allow customization of thread prefix

2017-02-01 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4722?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15849057#comment-15849057 ] Matthias J. Sax commented on KAFKA-4722: How does this relate to multiple KStream

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

2017-02-01 Thread Apache Jenkins Server
See Changes: [wangguoz] MINOR: Change rocksdb logging to error level -- [...truncated 8379 lines...] kafka.integration.SslTopicMetadataTest > testGetAllTopicMetadata STARTED kafka.integration.

Re: [DISCUSS] KIP-111 : Kafka should preserve the Principal generated by the PrincipalBuilder while processing the request received on socket channel, on the broker.

2017-02-01 Thread Mayuresh Gharat
Hi All, I have updated the KIP as per our discussion here. It would be great if you can take another look and let me know if there are any concerns. Thanks, Mayuresh On Sat, Jan 28, 2017 at 6:10 PM, Mayuresh Gharat wrote: > I had offline discussions with Joel, Dong and Radai. > > I agree that

[jira] [Commented] (KAFKA-4722) StreamThread should allow customization of thread prefix

2017-02-01 Thread Steven Schlansker (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4722?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15849098#comment-15849098 ] Steven Schlansker commented on KAFKA-4722: -- I have multiple different functional

[VOTE] 0.10.2.0 RC0

2017-02-01 Thread Ewen Cheslack-Postava
Hello Kafka users, developers and client-developers, This is the first candidate 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+

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

2017-02-01 Thread Matthias J. Sax
Thanks for the KIP Damian. I am wondering about two things: 1. what should be the default value for the new parameter? 2. why is the new parameter provided in ms? About (2): because "the minimum checkpoint interval will be the value of commit.interval.ms. In effect the actual checkpoint interva

Re: [VOTE] 0.10.2.0 RC0

2017-02-01 Thread Vahid S Hashemian
Ewen, Thanks a lot for running the release. I checked some of the links in the email and noticed a couple of issues: When I go to the documentation link ( http://kafka.apache.org/0102/documentation.html) and scroll down I get a second left-nav when I reach the end of list of content (try this l

Re: [VOTE] 0.10.2.0 RC0

2017-02-01 Thread Ewen Cheslack-Postava
Thanks for testing Vahid. Let me check in with Derrick, who made some adjustments to our handling of docs and other versions. I'm wondering if this could only affecting this page due to changes that were made in kafka-site and not included in the main Kafka repo. I found some other changes where th

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

2017-02-01 Thread Guozhang Wang
Ismael, thanks for your feedbacks. Replied inline. On Wed, Feb 1, 2017 at 8:03 AM, Ismael Juma wrote: > Hi all, > > A few comments follow: > > 1. The document states "inter-broker communications will be increased by M > * N * P round trips per sec. We need to conduct some system performance > te

[jira] [Commented] (KAFKA-4722) StreamThread should allow customization of thread prefix

2017-02-01 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4722?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15849195#comment-15849195 ] Matthias J. Sax commented on KAFKA-4722: I see. I guess, we could add the {{applic

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

2017-02-01 Thread Jason Gustafson
Ismael, Thanks for the comments. A few responses below: > 2. `ProducerAppId` is a new authorization resource type. This introduces a > compatibility issue with regards to existing third-party authorizers. It > would be good to highlight this in the migration/compatibility section. Ack. I added

Re: [VOTE] 0.10.2.0 RC0

2017-02-01 Thread Gwen Shapira
Note that doc fixes do not require new RC (we fix docs between releases). So unless bugs are found, you won't need to do another. On Wed, Feb 1, 2017 at 4:10 PM, Ewen Cheslack-Postava wrote: > Thanks for testing Vahid. Let me check in with Derrick, who made some > adjustments to our handling of d

[GitHub] kafka pull request #2482: MINOR: Update Streams docs: quickstart and concept...

2017-02-01 Thread guozhangwang
GitHub user guozhangwang opened a pull request: https://github.com/apache/kafka/pull/2482 MINOR: Update Streams docs: quickstart and concepts Added figures for topology and table-stream duality; added sections about aggregations; misc code fixes. You can merge this pull request int

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

2017-02-01 Thread Dong Lin
Hey Colin, Thanks much for the comment. Please see my reply inline. On Wed, Feb 1, 2017 at 1:54 PM, Colin McCabe wrote: > On Wed, Feb 1, 2017, at 11:35, Dong Lin wrote: > > Hey Grant, Colin, > > > > My bad, I misunderstood Grant's suggestion initially. Indeed this is a > > very > > interesting

Re: [VOTE] 0.10.2.0 RC0

2017-02-01 Thread Ewen Cheslack-Postava
Huh, that's surprising to me given that site-docs are part of the artifacts voted on. Having a way to update docs between releases is actually different from considering a known docs bug an issue w/ the release if someone cares about the site-docs artifacts being correct. But the site-docs seem som

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

2017-02-01 Thread Dong Lin
Hey Eno, Colin, Would you have time next Tuesday morning to discuss the KIP? How about 10 - 11 am? To make best use of our time, can you please invite one or more committer from Confluent to join the meeting? I hope the KIP can receive one or more +1 from committer at Confluent if we have no conc

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

2017-02-01 Thread Dong Lin
Sorry for the typo. I mean that before the KIP meeting, please free feel to provide comment in this email thread so that discussion in the KIP meeting can be more efficient. On Wed, Feb 1, 2017 at 6:53 PM, Dong Lin wrote: > Hey Eno, Colin, > > Would you have time next Tuesday morning to discuss

[jira] [Updated] (KAFKA-3959) __consumer_offsets wrong number of replicas at startup

2017-02-01 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3959?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ewen Cheslack-Postava updated KAFKA-3959: - Resolution: Fixed Status: Resolved (was: Patch Available) Issue resolved

[GitHub] kafka pull request #2177: KAFKA-3959: enforce offsets.topic.replication.fact...

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

[jira] [Commented] (KAFKA-3959) __consumer_offsets wrong number of replicas at startup

2017-02-01 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3959?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15849363#comment-15849363 ] ASF GitHub Bot commented on KAFKA-3959: --- Github user asfgit closed the pull request

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

2017-02-01 Thread Guozhang Wang
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+-+Exactly+Once+Delivery+and+Transactional+Messaging Discussion thread can be found here: http://search-hadoop.com/m/Kafka/uyzND1jwZrr7HRHf?subj=+DISCUSS

[GitHub] kafka pull request #2442: KAFKA-4642: Improve test coverage of ProcessorStat...

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

[jira] [Updated] (KAFKA-4642) Improve test coverage of ProcessorStateManager

2017-02-01 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4642?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang updated KAFKA-4642: - Resolution: Fixed Status: Resolved (was: Patch Available) Issue resolved by pull request

[jira] [Commented] (KAFKA-4642) Improve test coverage of ProcessorStateManager

2017-02-01 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4642?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15849389#comment-15849389 ] ASF GitHub Bot commented on KAFKA-4642: --- Github user asfgit closed the pull request

[jira] [Updated] (KAFKA-4647) Improve test coverage of GlobalStreamThread

2017-02-01 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4647?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang updated KAFKA-4647: - Resolution: Fixed Status: Resolved (was: Patch Available) Issue resolved by pull request

[GitHub] kafka pull request #2450: KAFKA-4647: Improve test coverage of GlobalStreamT...

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

[jira] [Commented] (KAFKA-4647) Improve test coverage of GlobalStreamThread

2017-02-01 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4647?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15849393#comment-15849393 ] ASF GitHub Bot commented on KAFKA-4647: --- Github user asfgit closed the pull request

[jira] [Updated] (KAFKA-4645) Improve test coverage of ProcessorTopology

2017-02-01 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4645?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang updated KAFKA-4645: - Resolution: Fixed Status: Resolved (was: Patch Available) Issue resolved by pull request

[jira] [Commented] (KAFKA-4645) Improve test coverage of ProcessorTopology

2017-02-01 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4645?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15849406#comment-15849406 ] ASF GitHub Bot commented on KAFKA-4645: --- Github user asfgit closed the pull request

[GitHub] kafka pull request #2444: KAFKA-4645: Improve test coverage of ProcessorTopo...

2017-02-01 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2444 --- 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-111 : Kafka should preserve the Principal generated by the PrincipalBuilder while processing the request received on socket channel, on the broker.

2017-02-01 Thread Dong Lin
The last paragraph of the motivation section is a bit confusing. I guess you want to say "This issue can be addressed if the Session class stores the Principal object extracted from a request". I like the approach of changing Session class to be case class *Session(principal: KafkaPrincipal, clien

Re: [DISCUSS] KIP-111 : Kafka should preserve the Principal generated by the PrincipalBuilder while processing the request received on socket channel, on the broker.

2017-02-01 Thread Mayuresh Gharat
Hi Dong, What I meant was "Right now Kafka just extracts the name out of the Principal that is generated by the PrincipalBuilder. Instead of doing that if it preserves the Principal itself, this issue can be addressed". May be I should have used the word "preserve" instead of "stores". I have upd

[GitHub] kafka pull request #2483: MINOR: Ensure timestamp type is provided when upco...

2017-02-01 Thread hachikuji
GitHub user hachikuji opened a pull request: https://github.com/apache/kafka/pull/2483 MINOR: Ensure timestamp type is provided when upconverting messages You can merge this pull request into a Git repository by running: $ git pull https://github.com/hachikuji/kafka minor-upc

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

2017-02-01 Thread Apache Jenkins Server
See Changes: [me] KAFKA-3959: enforce offsets.topic.replication.factor upon -- [...truncated 5667 lines...] kafka.api.PlaintextConsumerTest > testRoundRobinAssignment STARTED kafka.api.Plainte

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

2017-02-01 Thread Apache Jenkins Server
See

[GitHub] kafka pull request #2484: KAFKA-3959: Follow-up; move upgrade notes to 0.10....

2017-02-01 Thread ewencp
GitHub user ewencp opened a pull request: https://github.com/apache/kafka/pull/2484 KAFKA-3959: Follow-up; move upgrade notes to 0.10.3 upgrade section. You can merge this pull request into a Git repository by running: $ git pull https://github.com/ewencp/kafka kafka-3959-foll

[jira] [Reopened] (KAFKA-3959) __consumer_offsets wrong number of replicas at startup

2017-02-01 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3959?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ewen Cheslack-Postava reopened KAFKA-3959: -- Assignee: Ewen Cheslack-Postava (was: Onur Karaman) > __consumer_offsets wron

[jira] [Commented] (KAFKA-3959) __consumer_offsets wrong number of replicas at startup

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

[jira] [Updated] (KAFKA-3959) __consumer_offsets wrong number of replicas at startup

2017-02-01 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3959?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ewen Cheslack-Postava updated KAFKA-3959: - Status: Patch Available (was: Reopened) > __consumer_offsets wrong number of repl

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

2017-02-01 Thread Apache Jenkins Server
See Changes: [wangguoz] KAFKA-4642: Improve test coverage of ProcessorStateManager [wangguoz] KAFKA-4647: Improve test coverage of GlobalStreamThread [wangguoz] KAFKA-4645: Improve test coverage of ProcessorTopology

[GitHub] kafka pull request #2485: KAFKA-3896: Fix KStream-KStream leftJoin

2017-02-01 Thread guozhangwang
GitHub user guozhangwang opened a pull request: https://github.com/apache/kafka/pull/2485 KAFKA-3896: Fix KStream-KStream leftJoin The issue of transiently having duplicates is due to the bad design of the left join itself: in order to ignore the partial joined results such as `A:n

[jira] [Commented] (KAFKA-3896) Unstable test KStreamRepartitionJoinTest.shouldCorrectlyRepartitionOnJoinOperations

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