[GitHub] kafka pull request #2620: MINOR: Doc change related to ZK sasl configs

2017-02-28 Thread omkreddy
GitHub user omkreddy opened a pull request: https://github.com/apache/kafka/pull/2620 MINOR: Doc change related to ZK sasl configs You can merge this pull request into a Git repository by running: $ git pull https://github.com/omkreddy/kafka MINOR-ZK-CHANGE Alternatively you

[jira] [Updated] (KAFKA-4822) Kafka producer implementation without additional threads, similar to sync producer of 0.8.

2017-02-28 Thread Giri (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4822?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Giri updated KAFKA-4822: Affects Version/s: 0.9.0.0 0.9.0.1 0.10.0.0 0.10

Re: [DISCUSS] KIP-128: Add ByteArrayConverter for Kafka Connect

2017-02-28 Thread Guozhang Wang
Generally I'd prefer not duplicating functional logic as sometimes you may miss to sync one of them when you change the other. I understand for this specific case such scenario may never happen as the logic is quite simple and static, but still sounds a good coding practice to me? Guozhang On T

[jira] [Created] (KAFKA-4822) Kafka producer implementation without additional threads, similar to sync producer of 0.8.

2017-02-28 Thread Giri (JIRA)
Giri created KAFKA-4822: --- Summary: Kafka producer implementation without additional threads, similar to sync producer of 0.8. Key: KAFKA-4822 URL: https://issues.apache.org/jira/browse/KAFKA-4822 Project: Kafka

Re: [DISCUSS] KIP-82 - Add Record Headers

2017-02-28 Thread Michael Pearce
Sent to early: Hi Radai: RE: Header header(String key) - returns JUST ONE (the very last) value given a key Iterable headers(String key) - returns ALL under a key Iterable headers() - returns all, period. maybe allow null as key to prev method instead? void add(Header header) - appends a header

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

2017-02-28 Thread Apache Jenkins Server
See Changes: [ismael] KAFKA-4773; The Kafka build should run findbugs -- [...truncated 159.37 KB...] kafka.utils.timer.TimerTest > testTaskExpiration STARTED kafka.utils

Re: [DISCUSS] KIP-82 - Add Record Headers

2017-02-28 Thread Michael Pearce
Hi Radai: RE: Header header(String key) - returns JUST ONE (the very last) value given a key Iterable headers(String key) - returns ALL under a key void add(Header header) - appends a header (key inside). void remove(String key) - removes ALL HEADERS under a key. I don't think this one is neede

[jira] [Commented] (KAFKA-4722) Add application.id to StreamThread name

2017-02-28 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4722?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15889540#comment-15889540 ] Matthias J. Sax commented on KAFKA-4722: Thanks for rebasing. I left a comment on

[jira] [Created] (KAFKA-4821) 9244L

2017-02-28 Thread Vamsi Jakkula (JIRA)
Vamsi Jakkula created KAFKA-4821: Summary: 9244L Key: KAFKA-4821 URL: https://issues.apache.org/jira/browse/KAFKA-4821 Project: Kafka Issue Type: Task Reporter: Vamsi Jakkula Cr

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

2017-02-28 Thread Apache Jenkins Server
See Changes: [ismael] MINOR: Add code quality checks (and suppressions) to checkstyle.xml -- [...truncated 693.73 KB...] org.apache.kafka.streams.state.internals.ChangeLo

[jira] [Commented] (KAFKA-4722) Add application.id to StreamThread name

2017-02-28 Thread Sharad (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4722?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15889483#comment-15889483 ] Sharad commented on KAFKA-4722: --- PR submitted: https://github.com/apache/kafka/pull/2617 Th

[jira] [Issue Comment Deleted] (KAFKA-4722) Add application.id to StreamThread name

2017-02-28 Thread Sharad (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4722?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Sharad updated KAFKA-4722: -- Comment: was deleted (was: PR submitted: https://github.com/apache/kafka/pull/2487) > Add application.id to Str

[jira] [Commented] (KAFKA-4820) ConsumerNetworkClient.send() should not require global lock

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

[GitHub] kafka pull request #2619: KAFKA-4820; ConsumerNetworkClient.send() should no...

2017-02-28 Thread lindong28
GitHub user lindong28 opened a pull request: https://github.com/apache/kafka/pull/2619 KAFKA-4820; ConsumerNetworkClient.send() should not require global lock You can merge this pull request into a Git repository by running: $ git pull https://github.com/lindong28/kafka KAFKA-

[jira] [Updated] (KAFKA-4820) ConsumerNetworkClient.send() should not require global lock

2017-02-28 Thread Dong Lin (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4820?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Dong Lin updated KAFKA-4820: Description: Currently `ConsumerNetworkClient.send()` needs to acquire global lock of `ConumserNetworkClien

[jira] [Created] (KAFKA-4820) ConsumerNetworkClient.send() should not require global lock

2017-02-28 Thread Dong Lin (JIRA)
Dong Lin created KAFKA-4820: --- Summary: ConsumerNetworkClient.send() should not require global lock Key: KAFKA-4820 URL: https://issues.apache.org/jira/browse/KAFKA-4820 Project: Kafka Issue Type:

[jira] [Commented] (KAFKA-4738) Remove generic type of class ClientState

2017-02-28 Thread Sharad (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4738?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15889417#comment-15889417 ] Sharad commented on KAFKA-4738: --- PR submitted: https://github.com/apache/kafka/pull/2616 T

[jira] [Issue Comment Deleted] (KAFKA-4738) Remove generic type of class ClientState

2017-02-28 Thread Sharad (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4738?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Sharad updated KAFKA-4738: -- Comment: was deleted (was: Yes, its done. PR submitted: https://github.com/apache/kafka/pull/2605) > Remove ge

[GitHub] kafka pull request #2616: KAFKA:4738 - Remove generic type of class ClientSt...

2017-02-28 Thread sharad-develop
Github user sharad-develop closed the pull request at: https://github.com/apache/kafka/pull/2616 --- 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

Re: [DISCUSS] KIP-82 - Add Record Headers

2017-02-28 Thread Becket Qin
Hi Ismael, Yes, there is a difference between Batch and Headers. I was just trying to see if that would work. Good point about sending the same ProducerRecord twice, but in fact in that case any reuse of objects would cause problem. As you can imagine if the ProducerRecord has a value as a List a

[jira] [Updated] (KAFKA-4791) Kafka Streams - unable to add state stores when using wildcard topics on the source

2017-02-28 Thread Bill Bejeck (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4791?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Bill Bejeck updated KAFKA-4791: --- Status: Patch Available (was: In Progress) > Kafka Streams - unable to add state stores when using wi

[jira] [Commented] (KAFKA-4791) Kafka Streams - unable to add state stores when using wildcard topics on the source

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

[GitHub] kafka pull request #2618: KAFKA-4791: unable to add state store with regex m...

2017-02-28 Thread bbejeck
GitHub user bbejeck opened a pull request: https://github.com/apache/kafka/pull/2618 KAFKA-4791: unable to add state store with regex matched topics Fix for adding state stores with regex defined sources You can merge this pull request into a Git repository by running: $ git pu

[GitHub] kafka-site pull request #49: Fix bad kafka stream link on use cases page

2017-02-28 Thread haoch
GitHub user haoch opened a pull request: https://github.com/apache/kafka-site/pull/49 Fix bad kafka stream link on use cases page When clicking [Kafka Streams](https://kafka.apache.org/%7B%7Bversion%7D%7D/documentation/streams) link on [Use Cases](https://kafka.apache.org/uses) pag

[GitHub] kafka pull request #2617: KAFKA:4722 - Add application.id to StreamThread na...

2017-02-28 Thread sharad-develop
GitHub user sharad-develop opened a pull request: https://github.com/apache/kafka/pull/2617 KAFKA:4722 - Add application.id to StreamThread name Add application.id to StreamThread name You can merge this pull request into a Git repository by running: $ git pull https://github.c

[GitHub] kafka pull request #2487: Kafka-4722 : Add application.id to StreamThread na...

2017-02-28 Thread sharad-develop
Github user sharad-develop closed the pull request at: https://github.com/apache/kafka/pull/2487 --- 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

[GitHub] kafka pull request #2616: KAFKA:4738 - Remove generic type of class ClientSt...

2017-02-28 Thread sharad-develop
GitHub user sharad-develop opened a pull request: https://github.com/apache/kafka/pull/2616 KAFKA:4738 - Remove generic type of class ClientState Remove generic type of class ClientState and generic T inTaskAssignor. You can merge this pull request into a Git repository by running:

[GitHub] kafka pull request #2605: Kafka 4738:Remove generic type of class ClientStat...

2017-02-28 Thread sharad-develop
Github user sharad-develop closed the pull request at: https://github.com/apache/kafka/pull/2605 --- 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

Re: [DISCUSS] KIP-128: Add ByteArrayConverter for Kafka Connect

2017-02-28 Thread Ewen Cheslack-Postava
Guozhang, Did you look at the PR? I'm fine with doing that if we really think it's better, but since this is a config-less passthrough, it's actually just adding overhead to do that... -Ewen On Mon, Feb 27, 2017 at 11:47 AM, Guozhang Wang wrote: > Thanks Ewen, > > "use the corresponding serial

[GitHub] kafka pull request #2615: Add a consumer offset migration tool

2017-02-28 Thread jeffwidman
GitHub user jeffwidman opened a pull request: https://github.com/apache/kafka/pull/2615 Add a consumer offset migration tool Extends #1715 to support renaming the consumer group as part of the migration. I tested this pretty thoroughly and seems to be working perfectly.

[jira] [Commented] (KAFKA-1712) Excessive storage usage on newly added node

2017-02-28 Thread Alan Braithwaite (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1712?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15889137#comment-15889137 ] Alan Braithwaite commented on KAFKA-1712: - Has this been looked at recently? We'v

[jira] [Commented] (KAFKA-4816) Message format changes for idempotent/transactional producer

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

[GitHub] kafka pull request #2614: KAFKA-4816: Message format changes for idempotent/...

2017-02-28 Thread hachikuji
GitHub user hachikuji opened a pull request: https://github.com/apache/kafka/pull/2614 KAFKA-4816: Message format changes for idempotent/transactional producer You can merge this pull request into a Git repository by running: $ git pull https://github.com/confluentinc/kafka ex

[jira] [Commented] (KAFKA-4095) When a topic is deleted and then created with the same name, 'committed' offsets are not reset

2017-02-28 Thread Vahid Hashemian (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4095?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15889127#comment-15889127 ] Vahid Hashemian commented on KAFKA-4095: [~jeffwidman] This JIRA applies to the ol

[jira] [Commented] (KAFKA-4773) The Kafka build should run findbugs

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

[jira] [Resolved] (KAFKA-4773) The Kafka build should run findbugs

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

[jira] [Commented] (KAFKA-4095) When a topic is deleted and then created with the same name, 'committed' offsets are not reset

2017-02-28 Thread Jeff Widman (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4095?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15889093#comment-15889093 ] Jeff Widman commented on KAFKA-4095: Wasn't this solved by KAFKA-2000? At least for t

[GitHub] kafka pull request #2557: KAFKA-4773: The Kafka build should run findbugs

2017-02-28 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2557 --- 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 #2594: MINOR: add code quality checks to checkstyle.xml. ...

2017-02-28 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2594 --- 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] [Assigned] (KAFKA-4819) Expose states of active tasks to public API

2017-02-28 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4819?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax reassigned KAFKA-4819: -- Assignee: Florian Hussonnois > Expose states of active tasks to public API > --

[jira] [Commented] (KAFKA-4819) Expose states of active tasks to public API

2017-02-28 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4819?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15889006#comment-15889006 ] Matthias J. Sax commented on KAFKA-4819: [~fhussonnois] Thanks for the JIRA an PR.

[jira] [Updated] (KAFKA-4819) Expose states of active tasks to public API

2017-02-28 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4819?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax updated KAFKA-4819: --- Labels: needs-kip (was: ) > Expose states of active tasks to public API > ---

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

2017-02-28 Thread Dong Lin
Hey Jun, I just realized that StopReplicaRequest itself doesn't specify the replicaId in the wire protocol. Thus controller would need to log the brokerId with StopReplicaRequest in the log. Thus it may be reasonable for controller to do the same with LeaderAndIsrRequest and only specify the isNew

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

2017-02-28 Thread Mayuresh Gharat
Hi Jun, Sure. I had an offline discussion with Joel on how we can deprecate the KafkaPrincipal from Session and Authorizer. I will update the KIP to see if we can address all the concerns here. If not we can keep the KafkaPrincipal. Thanks, Mayuresh On Tue, Feb 28, 2017 at 1:53 PM, Jun Rao wr

[GitHub] kafka pull request #2613: MINOR. Fix tests/docker/Dockerfile

2017-02-28 Thread cmccabe
GitHub user cmccabe opened a pull request: https://github.com/apache/kafka/pull/2613 MINOR. Fix tests/docker/Dockerfile Fix tests/docker/Dockerfile to put the old Kafka distributions in the correct spot for tests. Also, run_tests.sh should exit with an error code if image re

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

2017-02-28 Thread Dong Lin
Hi Jun, Yeah there is tradeoff between controller's implementation complexity vs. wire-protocol complexity. I personally think it is more important to keep wire-protocol concise and only add information in wire-protocol if necessary. It seems fine to add a little bit complexity to controller's imp

[jira] [Commented] (KAFKA-4738) Remove generic type of class ClientState

2017-02-28 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4738?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15888950#comment-15888950 ] Matthias J. Sax commented on KAFKA-4738: [~sharad.develop] Your PR shows 81 commit

[jira] [Commented] (KAFKA-4789) ProcessorTopologyTestDriver does not forward extracted timestamps to internal topics

2017-02-28 Thread Hamidreza Afzali (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4789?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15888941#comment-15888941 ] Hamidreza Afzali commented on KAFKA-4789: - Thanks! > ProcessorTopologyTestDriver

[DISCUSS] KIP-129: Kafka Streams Exactly-Once Semantics

2017-02-28 Thread Guozhang Wang
Hi all, I have just created KIP-129 to leverage KIP-98 in Kafka Streams and provide exactly-once processing semantics: https://cwiki.apache.org/confluence/display/KAFKA/KIP-129%3A+Streams+Exactly-Once+Semantics This KIP enables Streams users to optionally turn on exactly-once processing semantic

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

2017-02-28 Thread Jun Rao
Hi, Joel, Good point on the getAcls() method. KafkaPrincipal is also tied to ACL, which is used in pretty much every method in Authorizer. Now, I am not sure if it's easy to deprecate KafkaPrincipal. Hi, Mayuresh, Given the above, it seems that the easiest thing is to add a new Principal field i

[jira] [Commented] (KAFKA-4819) Expose states of active tasks to public API

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

[GitHub] kafka pull request #2612: KAFKA-4819: Expose states for active tasks to publ...

2017-02-28 Thread fhussonnois
GitHub user fhussonnois opened a pull request: https://github.com/apache/kafka/pull/2612 KAFKA-4819: Expose states for active tasks to public API Simple implementation of the feature : [KAFKA-4819](https://issues.apache.org/jira/browse/KAFKA-4819) KAFKA-4819 This PR ad

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

2017-02-28 Thread Jun Rao
Hi, Dong, 52. What you suggested would work. However, I am thinking that it's probably simpler to just set isNewReplica at the replica level. That way, the LeaderAndIsrRequest can be created a bit simpler. When reading a LeaderAndIsrRequest in the controller log, it's easier to see which replicas

[jira] [Commented] (KAFKA-4789) ProcessorTopologyTestDriver does not forward extracted timestamps to internal topics

2017-02-28 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4789?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15888903#comment-15888903 ] Guozhang Wang commented on KAFKA-4789: -- Thanks [~hrafzali] for the patch! I have adde

[jira] [Assigned] (KAFKA-4789) ProcessorTopologyTestDriver does not forward extracted timestamps to internal topics

2017-02-28 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4789?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang reassigned KAFKA-4789: Assignee: Hamidreza Afzali > ProcessorTopologyTestDriver does not forward extracted timesta

[jira] [Commented] (KAFKA-4789) ProcessorTopologyTestDriver does not forward extracted timestamps to internal topics

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

[jira] [Resolved] (KAFKA-4789) ProcessorTopologyTestDriver does not forward extracted timestamps to internal topics

2017-02-28 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4789?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang resolved KAFKA-4789. -- Resolution: Fixed Fix Version/s: 0.10.3.0 Issue resolved by pull request 2590 [https://gi

[GitHub] kafka pull request #2590: KAFKA-4789: Added support to ProcessorTopologyTest...

2017-02-28 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2590 --- 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 #2607: MINOR: Fix typo in javadoc of `flatMapValues`

2017-02-28 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2607 --- 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 #2303: MINOR: improve license header check by providing h...

2017-02-28 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2303 --- 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] [Created] (KAFKA-4819) Expose states of active tasks to public API

2017-02-28 Thread Florian Hussonnois (JIRA)
Florian Hussonnois created KAFKA-4819: - Summary: Expose states of active tasks to public API Key: KAFKA-4819 URL: https://issues.apache.org/jira/browse/KAFKA-4819 Project: Kafka Issue Typ

[GitHub] kafka pull request #2487: Kafka-4722 : Add application.id to StreamThread na...

2017-02-28 Thread sharad-develop
GitHub user sharad-develop reopened a pull request: https://github.com/apache/kafka/pull/2487 Kafka-4722 : Add application.id to StreamThread name Kafka-4722 : Add application.id to StreamThread name You can merge this pull request into a Git repository by running: $ git pull h

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

2017-02-28 Thread Apache Jenkins Server
See Changes: [me] KAFKA-4809: docker/run_tests.sh should set up /opt/kafka-dev to be the [me] MINOR: improve license header check by providing head file instead of -- Sta

[jira] [Assigned] (KAFKA-4815) Idempotent/transactional Producer Checklist (KIP-98)

2017-02-28 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4815?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax reassigned KAFKA-4815: -- Assignee: Jason Gustafson > Idempotent/transactional Producer Checklist (KIP-98) >

[jira] [Updated] (KAFKA-4815) Idempotent/transactional Producer Checklist (KIP-98)

2017-02-28 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4815?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax updated KAFKA-4815: --- Labels: kip (was: ) > Idempotent/transactional Producer Checklist (KIP-98) >

[jira] [Assigned] (KAFKA-4815) Idempotent/transactional Producer Checklist (KIP-98)

2017-02-28 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4815?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax reassigned KAFKA-4815: -- Assignee: (was: Matthias J. Sax) > Idempotent/transactional Producer Checklist (KIP

[jira] [Assigned] (KAFKA-4815) Idempotent/transactional Producer Checklist (KIP-98)

2017-02-28 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4815?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax reassigned KAFKA-4815: -- Assignee: Matthias J. Sax (was: Jason Gustafson) > Idempotent/transactional Producer C

[jira] [Updated] (KAFKA-4817) Implement idempotent producer

2017-02-28 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4817?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson updated KAFKA-4817: --- Summary: Implement idempotent producer (was: Basic idempotent producer implementation) > Imp

[jira] [Created] (KAFKA-4818) Implement transactional producer

2017-02-28 Thread Jason Gustafson (JIRA)
Jason Gustafson created KAFKA-4818: -- Summary: Implement transactional producer Key: KAFKA-4818 URL: https://issues.apache.org/jira/browse/KAFKA-4818 Project: Kafka Issue Type: Sub-task

[GitHub] kafka pull request #2487: Kafka-4722 : Add application.id to StreamThread na...

2017-02-28 Thread sharad-develop
Github user sharad-develop closed the pull request at: https://github.com/apache/kafka/pull/2487 --- 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

[jira] [Commented] (KAFKA-4738) Remove generic type of class ClientState

2017-02-28 Thread Sharad (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4738?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=1561#comment-1561 ] Sharad commented on KAFKA-4738: --- Yes, its done. PR submitted: https://github.com/apache/kaf

[jira] [Created] (KAFKA-4817) Basic idempotent producer implementation

2017-02-28 Thread Jason Gustafson (JIRA)
Jason Gustafson created KAFKA-4817: -- Summary: Basic idempotent producer implementation Key: KAFKA-4817 URL: https://issues.apache.org/jira/browse/KAFKA-4817 Project: Kafka Issue Type: Sub-ta

[jira] [Created] (KAFKA-4816) Message format changes for idempotent/transactional producer

2017-02-28 Thread Jason Gustafson (JIRA)
Jason Gustafson created KAFKA-4816: -- Summary: Message format changes for idempotent/transactional producer Key: KAFKA-4816 URL: https://issues.apache.org/jira/browse/KAFKA-4816 Project: Kafka

Re: Improve License Header Check

2017-02-28 Thread Matthias J. Sax
Just a reminder. This PR got merged today. -Matthias On 1/20/17 9:02 AM, Matthias J. Sax wrote: > Hi, > > I opened an PR to improve the check for file license header (the check > is currently quite weak and it's possible to have files with an invalid > header). > > https://github.com/apache/ka

[jira] [Created] (KAFKA-4815) Idempotent/transactional Producer Checklist (KIP-98)

2017-02-28 Thread Jason Gustafson (JIRA)
Jason Gustafson created KAFKA-4815: -- Summary: Idempotent/transactional Producer Checklist (KIP-98) Key: KAFKA-4815 URL: https://issues.apache.org/jira/browse/KAFKA-4815 Project: Kafka Issue

Re: [DISCUSS] KIP-82 - Add Record Headers

2017-02-28 Thread radai
I will settle for any API really, but just wanted to point out that as it stands right now the API targets the most "advanced" (hence obscure and rare) use cases, at the expense of the simple and common ones. i'd suggest (as the minimal set): Header header(String key) - returns JUST ONE (the very

[jira] [Resolved] (KAFKA-4809) docker/run_tests.sh should set up /opt/kafka-dev to be the source directory

2017-02-28 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4809?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ewen Cheslack-Postava resolved KAFKA-4809. -- Resolution: Fixed Reviewer: Ewen Cheslack-Postava Fix Version/s:

[GitHub] kafka pull request #2602: KAFKA-4809: docker/run_tests.sh should set up /opt...

2017-02-28 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2602 --- 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-4809) docker/run_tests.sh should set up /opt/kafka-dev to be the source directory

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

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

2017-02-28 Thread Apache Jenkins Server
See Changes: [me] MINOR: Make asJsonSchema() and asConnectSchema() methods public -- [...truncated 7.43 KB...]

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

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

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

2017-02-28 Thread ewencp
Github user ewencp closed the pull request at: https://github.com/apache/kafka/pull/2484 --- 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 #2610: MINOR: Make asJsonSchema() and asConnectSchema() m...

2017-02-28 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2610 --- 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] 0.10.3.0/0.11.0.0 release planning

2017-02-28 Thread Apurva Mehta
+1 (non-binding) for 0.11.0 I do agree with Ismael's point that exactly-once should go through one release of stabilization before bumping the version to 1.0. Thanks, Apurva On Mon, Feb 27, 2017 at 7:47 PM, Ismael Juma wrote: > Hi all, > > With 0.10.2.0 out of the way, I would like to voluntee

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

2017-02-28 Thread Joel Koshy
If we deprecate KafkaPrincipal, then the Authorizer interface will also need to change - i.e., deprecate the getAcls(KafkaPrincipal) method. On Tue, Feb 28, 2017 at 10:11 AM, Mayuresh Gharat < gharatmayures...@gmail.com> wrote: > Hi Jun/Ismael, > > Thanks for the comments. > > I agree. > What I w

[GitHub] kafka pull request #2611: MINOR: improve MinTimestampTrackerTest and fix NPE...

2017-02-28 Thread dguy
GitHub user dguy opened a pull request: https://github.com/apache/kafka/pull/2611 MINOR: improve MinTimestampTrackerTest and fix NPE when null element removed You can merge this pull request into a Git repository by running: $ git pull https://github.com/dguy/kafka testing Al

Re: [VOTE] KIP-107: Add purgeDataBefore() API in AdminClient

2017-02-28 Thread Dong Lin
Thanks Jun. I have updated the KIP to reflect this change. On Tue, Feb 28, 2017 at 9:44 AM, Jun Rao wrote: > Hi, Dong, > > Yes, this change makes sense to me. > > Thanks, > > Jun > > On Mon, Feb 27, 2017 at 8:26 PM, Dong Lin wrote: > > > Hi Jun and everyone, > > > > I would like to change the K

Re: KIP-122: Add a tool to Reset Consumer Group Offsets

2017-02-28 Thread Vahid S Hashemian
Thanks Jorge for addressing my suggestions. Looks good to me. --Vahid From: Jorge Esteban Quilcate Otoya To: dev@kafka.apache.org Date: 02/27/2017 01:57 AM Subject:Re: KIP-122: Add a tool to Reset Consumer Group Offsets @Vahid: make sense to add "new lag" info IMO, I will up

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

2017-02-28 Thread Eno Thereska
Thanks Todd for the explanation. Eno > On 28 Feb 2017, at 18:15, Todd Palino wrote: > > We have tested RAID 5/6 in the past (and recently) and found it to be > lacking. So, as noted, rebuild takes more time than RAID 10 because all the > disks need to be accessed to recalculate parity. In additi

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

2017-02-28 Thread Dong Lin
Hey Jun, Certainly, I have added Todd to reply to the thread. And I have updated the item to in the wiki. 50. The full statement is "Broker assumes a log directory to be good after it starts, and mark log directory as bad once there is IOException when broker attempts to access (i.e. read or writ

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

2017-02-28 Thread Todd Palino
We have tested RAID 5/6 in the past (and recently) and found it to be lacking. So, as noted, rebuild takes more time than RAID 10 because all the disks need to be accessed to recalculate parity. In addition, there’s a significant performance loss just in normal operations. It’s been a while since I

Re: [DISCUSS] 0.10.3.0/0.11.0.0 release planning

2017-02-28 Thread Bill Bejeck
+1 Sounds good to me. Thanks, Ismael. -Bill On Tue, Feb 28, 2017 at 1:01 PM, Colin McCabe wrote: > +1 (non-binding). > > Thanks, Ismael. > > cheers, > Colin > > > On Mon, Feb 27, 2017, at 19:47, Ismael Juma wrote: > > Hi all, > > > > With 0.10.2.0 out of the way, I would like to volunteer to b

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

2017-02-28 Thread Mayuresh Gharat
Hi Jun/Ismael, Thanks for the comments. I agree. What I was thinking was, we get the KIP passed now and wait till major kafka version release. We can then make this change, but for now we can wait. Does that work? If there are concerns, we can make the addition of extra field of type Principal t

Re: [VOTE] KIP-81: Bound Fetch memory usage in the consumer

2017-02-28 Thread Mickael Maison
Yes I agree, having a generic flag is more future proof. I'll update the KIP in the coming days. Thanks On Tue, Feb 28, 2017 at 5:08 AM, Jason Gustafson wrote: > Hey Mickael, > > The suggestion to add something to Node makes sense. I could imagine for > example adding a flag to indicate that the

Re: [DISCUSS] 0.10.3.0/0.11.0.0 release planning

2017-02-28 Thread Colin McCabe
+1 (non-binding). Thanks, Ismael. cheers, Colin On Mon, Feb 27, 2017, at 19:47, Ismael Juma wrote: > Hi all, > > With 0.10.2.0 out of the way, I would like to volunteer to be the release > manager for our next time-based release. See https://cwiki.apache.org/c > onfluence/display/KAFKA/Time+Ba

[GitHub] kafka pull request #2610: MINOR: Make asJsonSchema() and asConnectSchema() m...

2017-02-28 Thread C0urante
GitHub user C0urante opened a pull request: https://github.com/apache/kafka/pull/2610 MINOR: Make asJsonSchema() and asConnectSchema() methods public Want to use these methods in an external project. You can merge this pull request into a Git repository by running: $ git pull h

Re: [DISCUSS] KIP-124: Request rate quotas

2017-02-28 Thread Colin McCabe
I noticed that the throttle_time_ms added to all the message responses is in milliseconds. Does it make sense to express this in microseconds in case we start doing more fine-grained CPU throttling later on? An int32 should still be more than enough if using microseconds. best, Colin On Fri, F

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

2017-02-28 Thread Jun Rao
Hi, Ismael, Good point on compatibility. Hi, Mayuresh, Given that, it seems that it's better to just add the raw principal as a new field in Session for now and deprecate the KafkaPrincipal field in the future if needed? Thanks, Jun On Mon, Feb 27, 2017 at 5:05 PM, Ismael Juma wrote: > Brea

Re: [VOTE] KIP-107: Add purgeDataBefore() API in AdminClient

2017-02-28 Thread Jun Rao
Hi, Dong, Yes, this change makes sense to me. Thanks, Jun On Mon, Feb 27, 2017 at 8:26 PM, Dong Lin wrote: > Hi Jun and everyone, > > I would like to change the KIP in the following way. Currently, if any > replica if offline, the purge result for a partition will > be NotEnoughReplicasExcept

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

2017-02-28 Thread Jun Rao
Hi, Dong, RAID6 is an improvement over RAID5 and can tolerate 2 disks failure. Eno's point is that the rebuild of RAID5/RAID6 requires reading more data compared with RAID10, which increases the probability of error during rebuild. This makes sense. In any case, do you think you could ask the SREs

  1   2   >