Re: [DISCUSS] KIP-59 : Proposal for a kafka broker command

2017-10-16 Thread Tom Bentley
Hi Jayesh, Thanks, for the KIP. I few questions/points: 1. Could you elaborate on the motivation a little? Currently it seems to boil down to "Kafka doesn't have this, yet", but that's not, in itself, a reason to add it. What can't be done without this change? 2. The second bullet in the "Broker

Re: [VOTE] 1.0.0 RC1

2017-10-16 Thread Thomas Crayford
Hi Guozhang, This says the due date on the testing is October 13th, which was the day this email was sent. Is that accurate, or is it meant to read October 17th, which is next Tuesday? I feel like this short a testing window for a 1.0 RC is a little low, as 1.0 is clearly a big announcement of st

[GitHub] kafka pull request #4074: MINOR: add equals to SessionWindows

2017-10-16 Thread dguy
GitHub user dguy opened a pull request: https://github.com/apache/kafka/pull/4074 MINOR: add equals to SessionWindows You can merge this pull request into a Git repository by running: $ git pull https://github.com/dguy/kafka minor-session-window-equals Alternatively you can r

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

2017-10-16 Thread Jakub Scholz
I was having some more thoughts about it. We can simply take over what Kafka broker implements for the listeners: - We can take over the "listener" and "listener.security.protocol.map" options to define multiple REST listeners and the security protocol they should use - The HTTPS interface will by

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

2017-10-16 Thread Ted Yu
+1 to this proposal. On Mon, Oct 16, 2017 at 7:49 AM, Jakub Scholz wrote: > I was having some more thoughts about it. We can simply take over what > Kafka broker implements for the listeners: > - We can take over the "listener" and "listener.security.protocol.map" > options to define multiple RE

Re: [DISCUSS]: KIP-159: Introducing Rich functions to Streams

2017-10-16 Thread Jeyhun Karimov
Hi, Thanks for the feedback. 0. RichInitializer definition seems missing. - Fixed. I'd suggest moving the key parameter in the RichValueXX and RichReducer > after the value parameters, as well as in the templates; e.g. > public interface RichValueJoiner { > VR apply(final V1 value1, fi

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

2017-10-16 Thread Randall Hauch
The broker's configuration options are "listeners" (plural) and "listeners.security.protocol.map". I agree that following the pattern set by the broker is better, so these are really good ideas. However, at this point I don't see a need for the "listeners.security.procotol.map", which for the broke

Re: [VOTE] 1.0.0 RC1

2017-10-16 Thread Guozhang Wang
Hi Tom, Thanks for pointing it out. I meant to say Oct. 17th, Tuesday, for a 72 hours period. That being said, we need to have a lazy majority to accept a release RC according to our bylaws ( https://cwiki.apache.org/confluence/display/KAFKA/Bylaws). And if we cannot achieve that via thorough tes

[GitHub] kafka pull request #4075: MINOR: reduce partition state machine debug loggin...

2017-10-16 Thread onurkaraman
GitHub user onurkaraman opened a pull request: https://github.com/apache/kafka/pull/4075 MINOR: reduce partition state machine debug logging PartitionStateMachine.electLeaderForPartition logs all partition states in the cluster. This leads to quadratic logging behavior since Partit

Re: [DISCUSS] KIP-205: Add getAllKeys() API to ReadOnlyWindowStore

2017-10-16 Thread Xavier Léauté
Thank you Richard! Do you or Guozhang have any thoughts on my suggestions to use fetchAll() and fetchAll(timeFrom, timeTo) and reserve the "range" keyword for when we query a specific range of keys? Xavier On Sat, Oct 14, 2017 at 2:32 PM Richard Yu wrote: > Thanks for the clarifications, Xavier

[jira] [Created] (KAFKA-6064) Cluster hung when the controller tried to delete a bunch of topics

2017-10-16 Thread Chaitanya GSK (JIRA)
Chaitanya GSK created KAFKA-6064: Summary: Cluster hung when the controller tried to delete a bunch of topics Key: KAFKA-6064 URL: https://issues.apache.org/jira/browse/KAFKA-6064 Project: Kafka

Re: [VOTE] KIP-171 - Extend Consumer Group Reset Offset for Stream Application

2017-10-16 Thread Bill Bejeck
+1 Thanks, Bill On Fri, Oct 13, 2017 at 6:36 PM, Ted Yu wrote: > +1 > > On Fri, Oct 13, 2017 at 3:32 PM, Matthias J. Sax > wrote: > > > +1 > > > > > > > > On 9/11/17 3:04 PM, Jorge Esteban Quilcate Otoya wrote: > > > Hi All, > > > > > > It seems that there is no further concern with the KIP-17

Re: [kafka-clients] Re: [VOTE] 1.0.0 RC1

2017-10-16 Thread Jun Rao
Hi, Guozhang, Onur found an existing performance bug in the controller when there are lots of partitions. The fix is simple ( https://github.com/apache/kafka/pull/4075) and reduces the controlled shutdown time from 6.5 mins to 30 secs, with 25K partitions, RF=2 and 5 brokers. It would be useful t

[GitHub] kafka pull request #4075: MINOR: reduce partition state machine debug loggin...

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

[DISCUSS] KIP-209 Connection String Support

2017-10-16 Thread Clebert Suconic
I would like to start a discussion about KIP-209 (https://cwiki.apache.org/confluence/display/KAFKA/KIP-209+-+Connection+String+Support) This is an extension of my previous thread: http://mail-archives.apache.org/mod_mbox/kafka-dev/201710.mbox/%3cCAKF+bsoFbN13D-u20tUsP6G+aHX4BUNk=s8m4kyjxat_oyv...

Re: [DISCUSS] KIP-209 Connection String Support

2017-10-16 Thread Ted Yu
Please update link for Discussion thread and JIRA There're two TBD's for Invalid conversion and parameters. Can you fill them out ? Thanks On Mon, Oct 16, 2017 at 2:30 PM, Clebert Suconic wrote: > I would like to start a discussion about KIP-209 > (https://cwiki.apache.org/confluence/display/K

Re: [DISCUSS] KIP-209 Connection String Support

2017-10-16 Thread Clebert Suconic
On Mon, Oct 16, 2017 at 5:41 PM, Ted Yu wrote: > Please update link for Discussion thread and JIRA sure thing... I was waiting my email to go through the servers... so I could get the link. > > There're two TBD's for Invalid conversion and parameters. Can you fill them > out ? will do.. I was

Re: [DISCUSS] KIP-209 Connection String Support

2017-10-16 Thread Ted Yu
bq. I was waiting my email to go through the servers http://search-hadoop.com/ indexes mailing lists actively. The delay is very short. FYI On Mon, Oct 16, 2017 at 2:44 PM, Clebert Suconic wrote: > On Mon, Oct 16, 2017 at 5:41 PM, Ted Yu wrote: > > Please update link for Discussion thread and

Re: [DISCUSS] KIP-209 Connection String Support

2017-10-16 Thread Jakub Scholz
Hi Clebert, I think it would be good if this could cover not only KafkaConsumer and KafkaProducer but also the AdminClient. So that all three can be configured the same way. The bootstrap servers are a list - you can provide multiple bootstrap servers. Maybe you add an example of how that will be

Re: [DISCUSS] KIP-209 Connection String Support

2017-10-16 Thread Clebert Suconic
I updated the wiki with the list on the proposed arguments. I also changed it to include a new Exception class that would be named InvalidParameterException (since I couldn't find an existing Exception class that I could reuse into this). (I could review the name or the exception of course.. just

Jenkins build is back to normal : kafka-1.0-jdk7 #37

2017-10-16 Thread Apache Jenkins Server
See

[GitHub] kafka pull request #4076: MINOR: A few javadoc fixes

2017-10-16 Thread hachikuji
GitHub user hachikuji opened a pull request: https://github.com/apache/kafka/pull/4076 MINOR: A few javadoc fixes You can merge this pull request into a Git repository by running: $ git pull https://github.com/hachikuji/kafka javadoc-fixes Alternatively you can review and app

Re: [DISCUSS] KIP-209 Connection String Support

2017-10-16 Thread Ted Yu
Can't you use IllegalArgumentException ? Some example in current code base: clients/src/main/java/org/apache/kafka/clients/Metadata.java: throw new IllegalArgumentException("Max time to wait for metadata updates should not be < 0 milliseconds"); On Mon, Oct 16, 2017 at 3:06 PM, Clebert Suconic

[GitHub] kafka pull request #4077: KAFKA-5142: Added support for record headers, reus...

2017-10-16 Thread rhauch
GitHub user rhauch opened a pull request: https://github.com/apache/kafka/pull/4077 KAFKA-5142: Added support for record headers, reusing Kafka client's interfaces *This is still a work in progress and should not be merged.* This is a proposed PR that implements most of [K

Re: [DISCUSS] KIP-209 Connection String Support

2017-10-16 Thread Clebert Suconic
That works. On Mon, Oct 16, 2017 at 6:59 PM Ted Yu wrote: > Can't you use IllegalArgumentException ? > > Some example in current code base: > > clients/src/main/java/org/apache/kafka/clients/Metadata.java: > throw new IllegalArgumentException("Max time to wait for metadata updates > should not

Re: Possible Feature: Topic Retention Policy

2017-10-16 Thread Guozhang Wang
Hello Alexei, Thanks for bringing up this question. Just my 2 cents: 1. For request-response messaging, I think an alternative approach is to use a single topic for request queue, and use one temporary topic for response queue. I.e. everyone sends their request to a single topic, and wait for its

[GitHub] kafka pull request #4071: MINOR: a few web doc and javadoc fixes

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

Re: [kafka-clients] Re: [VOTE] 1.0.0 RC1

2017-10-16 Thread Guozhang Wang
Thanks Jun, I will roll out a new RC for this PR then. Guozhang On Mon, Oct 16, 2017 at 2:19 PM, Jun Rao wrote: > Hi, Guozhang, > > Onur found an existing performance bug in the controller when there are > lots of partitions. The fix is simple (https://github.com/apache/ > kafka/pull/4075) an

[GitHub] kafka pull request #4078: MINOR: update exception message for KIP-120

2017-10-16 Thread mjsax
GitHub user mjsax opened a pull request: https://github.com/apache/kafka/pull/4078 MINOR: update exception message for KIP-120 You can merge this pull request into a Git repository by running: $ git pull https://github.com/mjsax/kafka hotfix-streams Alternatively you can revi

Re: [VOTE] 1.0.0 RC1

2017-10-16 Thread Vahid S Hashemian
Hi Guozhang, I'm not sure if this should be covered by "Java 9 support" in the RC note, but when I try to build jars from source using Java 9 (./gradlew -PscalaVersion=2.12 jar) even though the build reports as succeeded, it doesn't seem to have been successful: $ bin/zookeeper-server-start.sh

[GitHub] kafka pull request #4078: MINOR: update exception message for KIP-120

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

Re: [VOTE] 1.0.0 RC1

2017-10-16 Thread Ted Yu
After specifying the location of zookeeper jar: export CLASSPATH= The command can be executed successfully: bin/zookeeper-server-start.sh config/zookeeper.properties This doesn't seem to be Java 9 specific issue. On Mon, Oct 16, 2017 at 5:30 PM, Vahid S Hashemian < vahidhashem...@us.ibm.com> w

[GitHub] kafka pull request #4076: MINOR: A few javadoc fixes

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

[GitHub] kafka pull request #4074: MINOR: add equals to SessionWindows

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

Re: [DISCUSS]: KIP-159: Introducing Rich functions to Streams

2017-10-16 Thread Guozhang Wang
Regarding #6 above, I'm still not clear why we would need `commit()` in both ProcessorContext and RecordContext, could you elaborate a bit more? To me `commit()` is really a processor context not a record context logically: when you call that function, it means we would commit the state of the who

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

2017-10-16 Thread Apache Jenkins Server
See Changes: [wangguoz] MINOR: a few web doc and javadoc fixes -- [...truncated 1.82 MB...] org.apache.kafka.streams.state.internals.RocksDBSegmentedBytesStoreTest > shou

Jenkins build is back to normal : kafka-trunk-jdk9 #128

2017-10-16 Thread Apache Jenkins Server
See

Re: [VOTE] 1.0.0 RC1

2017-10-16 Thread Ismael Juma
If you don't use the default Scala version, you have to set the SCALA_VERSION environment variable for the bin scripts to work. Ismael On 17 Oct 2017 1:30 am, "Vahid S Hashemian" wrote: Hi Guozhang, I'm not sure if this should be covered by "Java 9 support" in the RC note, but when I try to bu

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

2017-10-16 Thread Dong Lin
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 any further discussion around this KIP, so I'd like to start > > the vote for it. > > > > Just for reference :

Re: [DISCUSS] KIP-205: Add getAllKeys() API to ReadOnlyWindowStore

2017-10-16 Thread Richard Yu
As Guozhang Wang mentioned earlier, we want to mirror the structure of similar Store class (namely KTable). The WindowedStore class might be unique in itself as it uses fetch() methods, but in my opinion, uniformity should be better suited for simplicity. On Mon, Oct 16, 2017 at 11:54 AM, Xavier L

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

2017-10-16 Thread Apache Jenkins Server
See Changes: [wangguoz] MINOR: update exception message for KIP-120 [jason] MINOR: A few javadoc fixes [wangguoz] MINOR: add equals to SessionWindows -- [...truncated 1.

[jira] [Created] (KAFKA-6065) Add zookeeper metrics to ZookeeperClient as in KIP-188

2017-10-16 Thread Onur Karaman (JIRA)
Onur Karaman created KAFKA-6065: --- Summary: Add zookeeper metrics to ZookeeperClient as in KIP-188 Key: KAFKA-6065 URL: https://issues.apache.org/jira/browse/KAFKA-6065 Project: Kafka Issue Type

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

2017-10-16 Thread Becket Qin
Hi Paolo, Thanks for the KIP and sorry for being late on the thread. I am wondering what is the KafkaFuture returned by all() call? Should it be a Map instead? Thanks, Jiangjie (Becket) QIn On Thu, Sep 28, 2017 at 3:48 AM, Paolo Patierno wrote: > Hi, > > > maybe we want to start without the d

[jira] [Created] (KAFKA-6066) Use of SimpleDateFormat in RocksDBWindowStore may not be Threadsafe

2017-10-16 Thread Srikanth Sundarrajan (JIRA)
Srikanth Sundarrajan created KAFKA-6066: --- Summary: Use of SimpleDateFormat in RocksDBWindowStore may not be Threadsafe Key: KAFKA-6066 URL: https://issues.apache.org/jira/browse/KAFKA-6066 Proje