[jira] [Created] (KAFKA-6077) Let SimpleConsumer support Kerberos authentication

2017-10-17 Thread huangjianan (JIRA)
huangjianan created KAFKA-6077: -- Summary: Let SimpleConsumer support Kerberos authentication Key: KAFKA-6077 URL: https://issues.apache.org/jira/browse/KAFKA-6077 Project: Kafka Issue Type: Impr

Re: [VOTE] KIP-204 : adding records deletion operation to the new Admin Client API

2017-10-17 Thread Manikumar
+ (non-binding) Thanks, Manikumar On Tue, Oct 17, 2017 at 7:42 AM, Dong Lin wrote: > Thanks for the KIP. +1 (non-binding) > > On Wed, Oct 11, 2017 at 2:27 AM, Ted Yu wrote: > > > +1 > > > > On Mon, Oct 2, 2017 at 10:51 PM, Paolo Patierno > > wrote: > > > > > Hi all, > > > > > > I didn't see

[jira] [Created] (KAFKA-6076) Using new producer api of transaction twice failed when server run on Windows OS

2017-10-17 Thread Orwen Xiang (JIRA)
Orwen Xiang created KAFKA-6076: -- Summary: Using new producer api of transaction twice failed when server run on Windows OS Key: KAFKA-6076 URL: https://issues.apache.org/jira/browse/KAFKA-6076 Project: K

Re: [VOTE] KIP-207:The Offsets which ListOffsetsResponse returns should monotonically increase even during a partition leader change

2017-10-17 Thread Satish Duggana
+1 (non binding) Thanks, Satish. On Wed, Oct 18, 2017 at 1:56 AM, Jun Rao wrote: > Hi, Colin, > > Thanks for the KIP. +1. Just a minor comment. For the old client requests, > would it be better to return a LEADER_NOT_AVAILABLE error instead? > > Jun > > On Tue, Oct 17, 2017 at 11:11 AM, Colin M

[GitHub] kafka pull request #3782: KAFKA-5829; Speedup broker startup after unclean s...

2017-10-17 Thread lindong28
Github user lindong28 closed the pull request at: https://github.com/apache/kafka/pull/3782 ---

[GitHub] kafka pull request #4086: HOTFIX: Normal poll with zero during restoration

2017-10-17 Thread guozhangwang
GitHub user guozhangwang opened a pull request: https://github.com/apache/kafka/pull/4086 HOTFIX: Normal poll with zero during restoration Mirror of #4085 against trunk. You can merge this pull request into a Git repository by running: $ git pull https://github.com/guozhangwang

[jira] [Created] (KAFKA-6075) Kafka cannot recover after an unclean shutdown on Windows

2017-10-17 Thread Vahid Hashemian (JIRA)
Vahid Hashemian created KAFKA-6075: -- Summary: Kafka cannot recover after an unclean shutdown on Windows Key: KAFKA-6075 URL: https://issues.apache.org/jira/browse/KAFKA-6075 Project: Kafka I

Re: [DISCUSS] KIP-204 : adding records deletion operation to the new Admin Client API

2017-10-17 Thread Dong Lin
Hey Colin, I have also thought about deleteRecordsBeforeOffset so that we can keep the name consistent with the existing API in the Scala AdminClient. But then I think it may be better to be able to specify in DeleteRecordsOptions whether the deletion is before/after timestamp or offset. By doing

[jira] [Created] (KAFKA-6074) Use ZookeeperClient in ReplicaManager and Partition

2017-10-17 Thread Jun Rao (JIRA)
Jun Rao created KAFKA-6074: -- Summary: Use ZookeeperClient in ReplicaManager and Partition Key: KAFKA-6074 URL: https://issues.apache.org/jira/browse/KAFKA-6074 Project: Kafka Issue Type: Sub-task

[jira] [Created] (KAFKA-6073) Use ZookeeperClient in KafkaApis

2017-10-17 Thread Jun Rao (JIRA)
Jun Rao created KAFKA-6073: -- Summary: Use ZookeeperClient in KafkaApis Key: KAFKA-6073 URL: https://issues.apache.org/jira/browse/KAFKA-6073 Project: Kafka Issue Type: Sub-task Components:

[jira] [Created] (KAFKA-6072) Use ZookeeperClient in GroupCoordinator

2017-10-17 Thread Jun Rao (JIRA)
Jun Rao created KAFKA-6072: -- Summary: Use ZookeeperClient in GroupCoordinator Key: KAFKA-6072 URL: https://issues.apache.org/jira/browse/KAFKA-6072 Project: Kafka Issue Type: Sub-task Comp

[jira] [Created] (KAFKA-6071) Use ZookeeperClient in LogManager

2017-10-17 Thread Jun Rao (JIRA)
Jun Rao created KAFKA-6071: -- Summary: Use ZookeeperClient in LogManager Key: KAFKA-6071 URL: https://issues.apache.org/jira/browse/KAFKA-6071 Project: Kafka Issue Type: Sub-task Component

[GitHub] kafka pull request #4085: HOTFIX: poll with zero millis during restoration

2017-10-17 Thread guozhangwang
GitHub user guozhangwang opened a pull request: https://github.com/apache/kafka/pull/4085 HOTFIX: poll with zero millis during restoration You can merge this pull request into a Git repository by running: $ git pull https://github.com/guozhangwang/kafka KHotfix-0110-restore-on

[GitHub] kafka pull request #4084: KAFKA-6070: add ipaddress and enum34 dependencies ...

2017-10-17 Thread cmccabe
GitHub user cmccabe opened a pull request: https://github.com/apache/kafka/pull/4084 KAFKA-6070: add ipaddress and enum34 dependencies to docker image You can merge this pull request into a Git repository by running: $ git pull https://github.com/cmccabe/kafka KAFKA-6070 Alte

[jira] [Created] (KAFKA-6070) ducker-ak: add ipaddress and enum34 dependencies to docker image

2017-10-17 Thread Colin P. McCabe (JIRA)
Colin P. McCabe created KAFKA-6070: -- Summary: ducker-ak: add ipaddress and enum34 dependencies to docker image Key: KAFKA-6070 URL: https://issues.apache.org/jira/browse/KAFKA-6070 Project: Kafka

[GitHub] kafka pull request #4083: MINOR: Improve a Windows quickstart instruction

2017-10-17 Thread vahidhashemian
GitHub user vahidhashemian opened a pull request: https://github.com/apache/kafka/pull/4083 MINOR: Improve a Windows quickstart instruction The output of `wmic` can be very long and could truncate the search keywords in the existing command. If those keywords are truncated no proces

Re: [VOTE] KIP-207:The Offsets which ListOffsetsResponse returns should monotonically increase even during a partition leader change

2017-10-17 Thread Jun Rao
Hi, Colin, Thanks for the KIP. +1. Just a minor comment. For the old client requests, would it be better to return a LEADER_NOT_AVAILABLE error instead? Jun On Tue, Oct 17, 2017 at 11:11 AM, Colin McCabe wrote: > Hi all, > > I'd like to start the voting process for KIP-207:The Offsets which >

[GitHub] kafka-site pull request #98: Added Pinterest logo to streams page

2017-10-17 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka-site/pull/98 ---

[GitHub] kafka-site issue #98: Added Pinterest logo to streams page

2017-10-17 Thread guozhangwang
Github user guozhangwang commented on the issue: https://github.com/apache/kafka-site/pull/98 LGTM. Merged to asf-site. ---

Re: [VOTE] 1.0.0 RC1

2017-10-17 Thread Vahid S Hashemian
Thanks Ismael for the tip. I missed it in the Readme page ( https://github.com/apache/kafka#running-a-task-on-a-particular-version-of-scala-either-211x-or-212x ) --Vahid From: Ismael Juma To: dev@kafka.apache.org Cc: Kafka Users Date: 10/16/2017 06:50 PM Subject:Re: [VOTE]

Re: [DISCUSS] KIP-204 : adding records deletion operation to the new Admin Client API

2017-10-17 Thread Colin McCabe
Hi Paolo, This is a nice improvement. I agree that the discussion of creating a DeleteTopicPolicy can wait until later. Perhaps we can do it in a follow-on KIP. However, we do need to specify what ACL permissions are needed to invoke this API. That should be in the JavaDoc comments as well. B

Re: [VOTE] KIP-207:The Offsets which ListOffsetsResponse returns should monotonically increase even during a partition leader change

2017-10-17 Thread Ted Yu
+1 On Tue, Oct 17, 2017 at 11:23 AM, Apurva Mehta wrote: > +1 (non-binding) > > On Tue, Oct 17, 2017 at 11:11 AM, Colin McCabe wrote: > > > Hi all, > > > > I'd like to start the voting process for KIP-207:The Offsets which > > ListOffsetsResponse returns should monotonically increase even duri

Re: [VOTE] KIP-207:The Offsets which ListOffsetsResponse returns should monotonically increase even during a partition leader change

2017-10-17 Thread Apurva Mehta
+1 (non-binding) On Tue, Oct 17, 2017 at 11:11 AM, Colin McCabe wrote: > Hi all, > > I'd like to start the voting process for KIP-207:The Offsets which > ListOffsetsResponse returns should monotonically increase even during a > partition leader change. > > See > https://cwiki.apache.org/conflue

[VOTE] KIP-207:The Offsets which ListOffsetsResponse returns should monotonically increase even during a partition leader change

2017-10-17 Thread Colin McCabe
Hi all, I'd like to start the voting process for KIP-207:The Offsets which ListOffsetsResponse returns should monotonically increase even during a partition leader change. See https://cwiki.apache.org/confluence/display/KAFKA/KIP-207%3A+Offsets+returned+by+ListOffsetsResponse+should+be+monotonic

Can you please subscribe me in this project

2017-10-17 Thread Nikhil Deore
Hi, I want to learn and contribute to this project, Please subscribe me in. Thanks, Nikhil

Re: [DISCUSS] KIP-209 Connection String Support

2017-10-17 Thread Clebert Suconic
I had these updates in already... you just changed the names at the string.. but it was pretty much the same thing I think... I had taken you suggestions though. The Exceptions.. these would be implementation details... all I wanted to make sure is that users would get the name of the invalid par

Re: [DISCUSS] KIP-209 Connection String Support

2017-10-17 Thread Clebert Suconic
I had tweaked that section a bit.. although I though it was clear what the benefit. Since it was a bit obvious I thought describing the feature and the API simplification would been enough.. I am hoping it would be clearer now. On Tue, Oct 17, 2017 at 4:37 AM, Tom Bentley wrote: > Hi Clebert, >

[VOTE] 1.0.0 RC2

2017-10-17 Thread Guozhang Wang
Hello Kafka users, developers and client-developers, This is the third candidate for release of Apache Kafka 1.0.0. The main PRs that gets merged in after RC1 are the following: https://github.com/apache/kafka/commit/dc6bfa553e73ffccd1e604963e076c78d8ddcd69 It's worth noting that starting in thi

Build failed in Jenkins: kafka-trunk-jdk9 #131

2017-10-17 Thread Apache Jenkins Server
See Changes: [junrao] KAFKA-5163; Support replicas movement between log directories (KIP-113) -- [...truncated 1.79 MB...] org.apache.kafka.streams.kstream.internals.KGrou

[GitHub] kafka-site pull request #98: Added Pinterest logo to streams page

2017-10-17 Thread manjuapu
GitHub user manjuapu opened a pull request: https://github.com/apache/kafka-site/pull/98 Added Pinterest logo to streams page @guozhangwang Please review. You can merge this pull request into a Git repository by running: $ git pull https://github.com/manjuapu/kafka-site asf-sit

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

2017-10-17 Thread Apache Jenkins Server
See Changes: [junrao] KAFKA-5163; Support replicas movement between log directories (KIP-113) -- [...truncated 461.57 KB...] kafka.tools.ConsoleConsumerTest > shouldPars

[GitHub] kafka pull request #3874: KAFKA-5163; Support replicas movement between log ...

2017-10-17 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/3874 ---

Re: [DISCUSS] KIP-208: Add SSL support to Kafka Connect REST interface

2017-10-17 Thread Jakub Scholz
Ok, so I updated the KIP according to what we discussed. Please have a look at the updates. Two points I'm not 100% sure about: 1) Should we mark the rest.host.name and rest.port options as deprecated? 2) I needed to also address the advertised hostname / port. With multiple listeners it is not c

[GitHub] kafka pull request #4082: MINOR: Adds an option to consume continuously

2017-10-17 Thread astubbs
GitHub user astubbs opened a pull request: https://github.com/apache/kafka/pull/4082 MINOR: Adds an option to consume continuously You can merge this pull request into a Git repository by running: $ git pull https://github.com/astubbs/kafka continuous-poll Alternatively you c

[GitHub] kafka pull request #4081: KAFKA-6069: Properly tag KafkaStreams metrics with...

2017-10-17 Thread twbecker
GitHub user twbecker opened a pull request: https://github.com/apache/kafka/pull/4081 KAFKA-6069: Properly tag KafkaStreams metrics with the client id. You can merge this pull request into a Git repository by running: $ git pull https://github.com/twbecker/kafka KAFKA-6069 Al

[jira] [Created] (KAFKA-6069) Streams metrics tagged incorrectly

2017-10-17 Thread Tommy Becker (JIRA)
Tommy Becker created KAFKA-6069: --- Summary: Streams metrics tagged incorrectly Key: KAFKA-6069 URL: https://issues.apache.org/jira/browse/KAFKA-6069 Project: Kafka Issue Type: Bug Comp

[GitHub] kafka pull request #4080: Minor: Print units for the performance consumer

2017-10-17 Thread astubbs
GitHub user astubbs opened a pull request: https://github.com/apache/kafka/pull/4080 Minor: Print units for the performance consumer You can merge this pull request into a Git repository by running: $ git pull https://github.com/astubbs/kafka perfcons Alternatively you can re

Re: [VOTE] 1.0.0 RC1

2017-10-17 Thread Thomas Crayford
Hi Ghouzang, We have indeed started our performance testing at Heroku for RC1. However, we are more than happy to retest once RC2 is available, especially given larger amounts of time to do so. Thanks Tom Crayford Heroku Kafka On Tue, Oct 17, 2017 at 2:50 AM, Ismael Juma wrote: > If you don't

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

2017-10-17 Thread Apache Jenkins Server
See

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

2017-10-17 Thread Apache Jenkins Server
See Changes: [ismael] KAFKA-4504; Clarify that retention.bytes is a partition level config -- [...truncated 369.10 KB...] kafka.security.auth.SimpleAclAuthorizerTest > t

[GitHub] kafka pull request #4079: MINOR: JavaDoc improvements for RangeAssignor

2017-10-17 Thread astubbs
GitHub user astubbs opened a pull request: https://github.com/apache/kafka/pull/4079 MINOR: JavaDoc improvements for RangeAssignor You can merge this pull request into a Git repository by running: $ git pull https://github.com/astubbs/kafka trunk Alternatively you can review

[jira] [Resolved] (KAFKA-4504) Details of retention.bytes property at Topic level are not clear on how they impact partition size

2017-10-17 Thread Manikumar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4504?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar resolved KAFKA-4504. -- Resolution: Fixed Assignee: Manikumar Fix Version/s: 1.0.0 > Details of retention.bytes

[GitHub] kafka pull request #3814: KAFKA-4504: update retention.bytes config descript...

2017-10-17 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/3814 ---

[jira] [Created] (KAFKA-6068) kafka-topic.sh alter replication-factor raise broker failure

2017-10-17 Thread haiyangyu (JIRA)
haiyangyu created KAFKA-6068: Summary: kafka-topic.sh alter replication-factor raise broker failure Key: KAFKA-6068 URL: https://issues.apache.org/jira/browse/KAFKA-6068 Project: Kafka Issue Typ

Re: [DISCUSS] KIP-209 Connection String Support

2017-10-17 Thread Tom Bentley
Hi Clebert, The motivation section is written as more of a summary and doesn't really give any motivation for this change. Can you explain why it would be beneficial for Kafka to have this change? For example, if you have use cases where the current way of instantiating a producer, consumer or adm

Re: [DISCUSS] KIP-209 Connection String Support

2017-10-17 Thread Satish Duggana
You may need to update KIP with the details discussed in this thread in proposed changes section. >>My proposed format for the connection string would be: >>IP1:host1,IP2:host2,...IPN:hostn;parameterName=value1;parameterName2=value2;... parameterNameN=valueN Format should be host1:port1,host2:por

[jira] [Created] (KAFKA-6067) how to process when all isr crashed

2017-10-17 Thread haiyangyu (JIRA)
haiyangyu created KAFKA-6067: Summary: how to process when all isr crashed Key: KAFKA-6067 URL: https://issues.apache.org/jira/browse/KAFKA-6067 Project: Kafka Issue Type: New Feature