Re: Issue in using kafka SimpleConsumer code

2014-08-07 Thread Harsha
Hi Pankaj, Do you notice any errors in kafka logs when your consumer stops reading data. Attaching those logs would be helpful in finding the issue. -Harsha On Thu, Aug 7, 2014, at 06:12 AM, pankaj ojha wrote: Hi Team, Can you please provide any information

Re: Issue in using kafka SimpleConsumer code

2014-08-07 Thread Harsha
might have experience this issue before. On Thu, Aug 7, 2014, at 06:54 AM, pankaj ojha wrote: Hi Harsha, Thank you for replying. There is no error in the kafka logs. The output I had pasted in my previous mail is from kafka logs only. But when I restart the kafka process after incrementing

kafka contrib/hadoop-consumer,producer

2014-08-18 Thread Harsha
. If there is any interest in the above projects from the community I am hoping to work on documentation and the code as there isn't much activity happened on these projects. Thanks, Harsha

Re: Kafka Security

2014-09-16 Thread Harsha
Hi Joe, I am interested in joining the efforts. I went through apache storm security with kerberos so I can bring some of that experience into the discussion. Thanks, Harsha On Tue, Sep 16, 2014, at 10:37 AM, Joe Stein wrote: Hi Andrew, yes the meeting took place and we

Re: [DISCUSS] 0.8.2 release branch, unofficial release candidates(s), 0.8.1.2 release

2014-09-16 Thread Harsha
Hi All, Do we have any ballpark date on the release of 0.8.1.2 or 0.8.2. Thanks, Harsha On Thu, Sep 11, 2014, at 03:53 PM, Jay Kreps wrote: I agree that a beta for 0.8.2 would be useful. It would also be good to get it in production at LinkedIn before the final version. Sorry about

Re: Kafka Command Line Shell

2014-10-19 Thread Harsha
+1 for Web Api On Sat, Oct 18, 2014, at 11:48 PM, Glen Mazza wrote: Apache Karaf has been doing this for quite a few years, albeit in Java not Scala. Still, their coding approach to creating a CLI probably captures many lessons learned over that time. Glen On 10/17/2014 08:03 PM, Joe

Re: Welcome Kafka's newest committer

2014-11-19 Thread Harsha
Congrats Guozhang On Wed, Nov 19, 2014, at 04:31 PM, Joe Stein wrote: Congrats!!! On Wed, Nov 19, 2014 at 7:05 PM, Neha Narkhede neha.narkh...@gmail.com wrote: Hi everyone, I'm very happy to announce that the Kafka PMC has invited Guozhang Wang to become a committer. Guozhang has

Re: Does Kafka Producer service ?

2014-11-25 Thread Harsha
If you want to separate your main application from producer logic you can use or write something similar to https://github.com/mozilla-metrics/bagheera . Basically have a service that provides an REST api which your main application can call and has a Kafkaproducer to write to kafka. -Harsha

Re: Cannot stop Kafka server if zookeeper is shutdown first

2015-02-03 Thread Harsha
Any reason not to go with apache curator http://curator.apache.org/ . -Harsha On Tue, Feb 3, 2015, at 09:55 PM, Guozhang Wang wrote: I am also +1 on Neha's suggestion that At some point, if we find ourselves fiddling too much with ZkClient, it wouldn't hurt to write our own little zookeeper

Re: Cannot stop Kafka server if zookeeper is shutdown first

2015-02-04 Thread Harsha
curator handles sasl connection https://issues.apache.org/jira/browse/KAFKA-1695 On Wed, Feb 4, 2015, at 06:10 AM, Jaikiran Pai wrote: FWIW - the ZkClient project team have merged the pull request that I had submitted to allow for timeouts to operations

Re: [DISCUSS] KIPs

2015-02-05 Thread Harsha
for contributors if its lazy as in no strong objections . Just to make sure this is only for KIPs not for regular bug fixes right? Thanks, Harsha On Thu, Feb 5, 2015, at 05:59 PM, Jiangjie Qin wrote: I¹m having an impression that KIP is mostly for new features but not for bug fixes. But I

Re: ProducerFailureHandlingTest.testCannotSendToInternalTopic is failing

2015-01-17 Thread Harsha
I don't see any failures in tests with the latest trunk or 0.8.2. I ran it few times in a loop. -Harsha On Sat, Jan 17, 2015, at 08:38 AM, Manikumar Reddy wrote: ProducerFailureHandlingTest.testCannotSendToInternalTopic is failing on both 0.8.2 and trunk. Error on 0.8.2

Re: ProducerFailureHandlingTest.testCannotSendToInternalTopic is failing

2015-01-18 Thread Harsha
of these test failure cases. If you can reproduce this or even intermittent test failure can you please open up a new JIRA and attach your patch there. Your review patch is attached KAFKA-1867 which is a different issue. Thanks, Harsha On Sun, Jan 18, 2015, at 07:16 AM, Jaikiran Pai wrote: I could reproduce

Fwd: [DISCUSS] KIP-12 - Kafka Sasl/Kerberos implementation

2015-02-11 Thread Harsha
the work is in progress. I'll update the thread with a initial prototype patch. Thanks, Harsha

[DISCUSS] KIP-12 - Kafka Sasl/Kerberos implementation

2015-02-11 Thread Harsha
the work is in progress. I'll update the thread with a initial prototype patch. Thanks, Harsha

Re: [DISCUSS] KIP-11- Authorization design for kafka security

2015-03-06 Thread Harsha
Hi Parth, Thanks for putting this together. Overall it looks good to me. Although AdminUtils is a concern KIP-4 can probably fix that part. Thanks, Harsha On Thu, Mar 5, 2015, at 10:39 AM, Parth Brahmbhatt wrote: Forgot to add links to wiki and jira. Link

Re: 0.8.3 release plan

2015-03-12 Thread Harsha
I would like to include ssl/sasl 1) Kafka-1684 (Patch posted for a review) 2) Kafka-1686 (Patch depends on kafka-1684) 3) Kafka-1688 (work is in progress) Thanks, Harsha On Thu, Mar 12, 2015, at 04:35 AM, Guozhang Wang wrote: Gwen, Just for clarification, you were suggesting we should

Re: Plan of Controlled Shutdown

2015-03-25 Thread Harsha
improvement to the controlled shutdown especially in case of having high partitions per broker which might degrade the performance as per the JIRA. Thanks, Harsha On March 25, 2015 at 11:15:21 AM, Mingjie Lai (m...@apache.org) wrote: Hi. I've been trying to figure out the best way to do kafka broker

Re: [DISCUSS] KIP-11- Authorization design for kafka security

2015-03-31 Thread Harsha
if a producer doesn’t want to provide client auth and just needs wire encryption there won’t be any identity , in this case and we won’t be able to enforce an authorizer as the client will be anonymous. --  Harsha On March 31, 2015 at 10:29:33 AM, Don Bosco Durai (bo...@apache.org) wrote: Related

Re: kafka system tests

2015-04-01 Thread Harsha
on the framework side than on the system tests but in evaluating frameworks can we consider windows as another option too?. Thanks, Harsha On Wed, Mar 25, 2015, at 01:02 PM, Geoffrey Anderson wrote: Hi Gwen, Sorry about that, the ducttape repository was not yet public, but now

Re: ConsumerTest

2015-02-27 Thread Harsha
+1 On Fri, Feb 27, 2015, at 01:37 PM, Neha Narkhede wrote: Would anyone object if I commented out the kafka.api.ConsumerTest out until it is fixed? It hangs and is making accepting patches very time-consuming. -- Thanks, Neha

Re: [VOTE] KIP-2 - Refactor brokers to allow listening on multiple ports and IPs

2015-03-03 Thread Harsha
+1 non-binding On Tue, Mar 3, 2015, at 12:39 PM, Jeff Holoman wrote: +1 non-binding of course On Tue, Mar 3, 2015 at 3:18 PM, Joe Stein joe.st...@stealth.ly wrote: +1 ~ Joe Stein - - - - - - - - - - - - - - - - - http://www.stealth.ly - - - - - - - - - - - - - - - - - On

Re: [kafka-clients] Re: [VOTE] 0.8.2.1 Candidate 2

2015-03-05 Thread Harsha
+1 unit tests and ran tests on 3-node cluster. On Wed, Mar 4, 2015, at 09:01 PM, Neha Narkhede wrote: +1. Verified quick start, unit tests. On Tue, Mar 3, 2015 at 12:09 PM, Joe Stein joe.st...@stealth.ly wrote: Ok, lets fix the transient test failure on trunk agreed not a blocker. +1

Re: Announcing the Confluent Platform built on Apache Kafka

2015-02-25 Thread Harsha
Congrats! -Harsha On Wed, Feb 25, 2015, at 11:20 AM, Sriram Subramanian wrote: Congratulations! On 2/25/15 11:15 AM, Joe Stein joe.st...@stealth.ly wrote: Awesome! The future of schema management, has arrived ~ Joestein On Wed, Feb 25, 2015 at 1:34 PM, Gwen Shapira gshap

Re: KAFKA-1615: patch submitted

2015-02-20 Thread Harsha
Hi, Reviewboard looks good to me. Can you paste the link to reviewboard on the JIRA. -Harsha On Fri, Feb 20, 2015, at 07:11 AM, Jonathan Rafalski wrote: Hello, I am working through getting the python script for the review process working but wanted to get this first patch up

Re: [DISCUSS] New partitioning for better load balancing

2015-04-03 Thread Harsha
Gianmarco,                  I am coming from storm community. I think PKG is a very interesting and we can provide an implementation of Partitioner for PKG. Can you open a JIRA for this. --  Harsha Sent with Airmail On April 3, 2015 at 4:49:15 AM, Gianmarco De Francisci Morales (g

Re: [KIP-DISCUSSION] KIP-22 Expose a Partitioner interface in the new producer

2015-05-06 Thread Harsha
Thanks for the review Joel. I agree don't need a init method we can use configure. I'll update the KIP. -Harsha On Wed, May 6, 2015, at 04:45 PM, Joel Koshy wrote: +1 with a minor comment: do we need an init method given it extends Configurable? Also, can you move this wiki out of drafts

Re: [Vote] KIP-11 Authorization design for kafka security

2015-05-15 Thread Harsha
+1 non-binding On Fri, May 15, 2015 at 9:18 AM -0700, Parth Brahmbhatt pbrahmbh...@hortonworks.com wrote: Hi, Opening the voting thread for KIP-11. Link to the KIP: https://cwiki.apache.org/confluence/display/KAFKA/KIP-11+-+Authorization+Interface Link to Jira:

EOL JDK 1.6 for Kafka

2015-07-01 Thread Harsha
potential MITM. This api doesn't exist in Java 1.6. Are there any users still want 1.6 support or can we stop supporting 1.6 from next release on wards. Thanks, Harsha

Re: [jira] [Commented] (KAFKA-1690) new java producer needs ssl support as a client

2015-08-10 Thread Harsha
- 20150711_01 based on Oracle jdk8u51-b15 Thanks, Harsha On Mon, Aug 10, 2015, at 01:52 PM, Rajini Sivaram wrote: Harsha, I am using the code from https://github.com/harshach/kafka/tree/KAFKA-1690-V1 with the latest commit on 25 July which I think corresponds to the latest patch in KAFKA- 1690

Re: Official Kafka Gitter Room?

2015-07-19 Thread Harsha
one of the benefits of keeping conversations in mailing lists is it can be searchable later. It will benefit users who might seen the same error which was answered on the mailing lists before. slack/glitter can do that? On Sat, Jul 18, 2015, at 09:59 PM, Gwen Shapira wrote: FYI: Joining Slack

Re: (User) Document/guide on new Kafka features on security in upcoming 0.9.0?

2015-10-23 Thread Harsha
Jaikiran, Here is the SSL setup doc https://cwiki.apache.org/confluence/display/KAFKA/Deploying+SSL+for+Kafka . I am going to write one for SASL ( kerberos ) soon and Parth is working on pushing one for Authorizer as well. Thanks, Harsha On Fri, Oct 23, 2015, at 04:42

Re: [VOTE] 0.9.0.0 Candiate 2

2015-11-14 Thread Harsha
+1 (binding) 1. ran unit tests On Fri, Nov 13, 2015, at 06:28 PM, Jun Rao wrote: > This is the second candidate for release of Apache Kafka 0.9.0.0. This a > major release that includes (1) authentication (through SSL and SASL) and > authorization, (2) a new java consumer, (3) a Kafka connect

Re: [VOTE] 0.9.0.0 Candiate 2

2015-11-14 Thread Harsha
+1 (binding) 1. ran unit tests 2. ran simple tests (producer, consumer) with sasl, ssl and non-secure. Thanks, Harsha On Sat, Nov 14, 2015, at 02:42 PM, Harsha wrote: > +1 (binding) > 1. ran unit tests > > On Fri, Nov 13, 2015, at 06:28 PM, Jun Rao wrote: > > This is th

Re: [ANNOUNCE] New Kafka Committer Ewen Cheslack-Postava

2015-12-08 Thread Harsha
Congrats Ewen. -Harsha On Tue, Dec 8, 2015, at 12:08 PM, Ashish Singh wrote: > Congrats Ewen! > > On Tuesday, December 8, 2015, Joe Stein <joe.st...@stealth.ly> wrote: > > > Ewen, > > > > Congrats! > > > > ~ Joestein > > > > On Tu

Re: 0.9.0.0 RC4

2015-11-23 Thread Harsha
+1 (binding). 1. Ran unit tests 2. Ran SSL & SASL tests using vagrant cluster. Thanks, Harsha On Mon, Nov 23, 2015, at 09:34 AM, Neha Narkhede wrote: > +1 (binding). > > Verified source and binary artifacts, ran unit tests. > > On Mon, Nov 23, 2015 at 9:32 AM, Jun Rao

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

2016-06-09 Thread Harsha
Jun & Ismael, Unfortunately I couldn't attend the KIP meeting when delegation tokens discussed. Appreciate if you can update the thread if you have any further questions. Thanks, Harsha On Tue,

Re: [VOTE] KIP-55: Secure quotas for authenticated users

2016-06-15 Thread Harsha
Rajini, How does sub-quotas works in case of authenticated users. Where are we maintaining the relation between users and their client Ids. Can you add an example of zk data under /users. Thanks, Harsha On Mon, Jun 13, 2016, at 05:01 AM, Rajini Sivaram wrote: >

Re: KAFKA-3722 : Discussion about custom PrincipalBuilder and Authorizer configs

2016-05-27 Thread Harsha
Mayuresh & Ismael, Agree on not breaking interfaces on public API. +1 on option 2. Thanks, Harsha On Mon, May 23, 2016, at 10:30 AM, Mayuresh Gharat wrote: > Hi Harsha and Ismael, > > Option 2 sounds like a good idea if we want to ma

Re: [VOTE] KIP-4 Create Topics Schema

2016-06-16 Thread Harsha
+1 (binding) Thanks, Harsha On Thu, Jun 16, 2016, at 04:15 PM, Guozhang Wang wrote: > +1. > > On Thu, Jun 16, 2016 at 3:47 PM, Ismael Juma <ism...@juma.me.uk> wrote: > > > +1 (binding) > > > > On Thu, Jun 16, 2016 at 11:50 PM, Grant Henke <ghe...@clou

Re: [VOTE] KIP-62: Allow consumer to send heartbeats from a background thread

2016-06-16 Thread Harsha
+1 (binding) Thanks, Harsha On Thu, Jun 16, 2016, at 05:46 PM, Henry Cai wrote: > +1 > > On Thu, Jun 16, 2016 at 3:46 PM, Ismael Juma <ism...@juma.me.uk> wrote: > > > +1 (binding) > > > > On Fri, Jun 17, 2016 at 12:44 AM, Guozhang Wang <wang

Re: [VOTE] KIP-4 Create Topics Schema

2016-06-20 Thread Harsha
+1 (binding) -Harsha On Mon, Jun 20, 2016, at 11:33 AM, Ismael Juma wrote: > +1 (binding) > > On Mon, Jun 20, 2016 at 8:27 PM, Dana Powers <dana.pow...@gmail.com> > wrote: > > > +1 -- thanks for the update > > > > On Mon, Jun 20, 2016 at 10:49 AM,

Re: [DISCUSS] KIP-44 - Allow Kafka to have a customized security protocol

2016-01-26 Thread Harsha
SASL itself can provide pluggable authentication , why not extend there. There is also proposal for SASL/PLAIN which does extend the current authentication options. I think thats what Rajini is also talking about. -Harsha On Tue, Jan 26, 2016, at 01:56 AM, tao xiao wrote: > Hi Rajini, >

Re: [DISCUSS] KIP-43: Kafka SASL enhancements

2016-02-03 Thread Harsha
=PLAIN. Client can send its sasl mechanism before the handshake starts and if the broker accepts that particular mechanism than it can go ahead with handshake otherwise return a error saying that the mechanism not allowed. Thanks, Harsha On Wed, Feb 3, 2016, at 04:58 AM, Rajini Sivaram wrote

Re: [DISCUSS] KIP-43: Kafka SASL enhancements

2016-01-30 Thread Harsha
on the JAAS file. -Harsha On Fri, Jan 29, 2016, at 02:34 AM, Rajini Sivaram wrote: > Ismael, > > The first packet from the client is deliberately empty to distinguish > between non-negotiating GSSAPI packet and a negotiation packet. If this > packet contained mechanisms, then the cod

Re: [VOTE] Make next Kafka release 0.10.0.0 instead of 0.9.1.0

2016-02-23 Thread Harsha
+1 On Tue, Feb 23, 2016, at 02:25 PM, Christian Posta wrote: > +1 non binding > > On Tue, Feb 23, 2016 at 3:18 PM, Gwen Shapira wrote: > > > +1 > > > > On Tue, Feb 23, 2016 at 1:58 PM, Jun Rao wrote: > > > > > +1. > > > > > > Thanks, > > > > > > Jun > > >

Re: [DISCUSS] KIP-43: Kafka SASL enhancements

2016-02-29 Thread Harsha
Rajini, Thanks for the changes to the KIP. It looks good to me. I think we can move to voting. Thanks, Harsha On Mon, Feb 29, 2016, at 12:43 AM, Rajini Sivaram wrote: > I have added some more detail to the KIP based on the discussion in the > last KIP m

Re: [DISCUSS] KIP-43: Kafka SASL enhancements

2016-03-13 Thread Harsha
haven't seen anyone asking for this yet. Thanks, Harsha On Thu, Mar 10, 2016, at 01:44 AM, Rajini Sivaram wrote: > Gwen, > > Just to be clear, the alternative would be: > > *jaas.conf:* > > GssapiKafkaServer { > > com.ibm.security.auth.module.Krb5LoginModule required &

Re: [DISCUSS] KIP-50 - Enhance Authorizer interface to be aware of supported Principal Types

2016-03-09 Thread Harsha
Why we need to add this additional method just for validation. This will invalidate all the existing authorizer implementations. Why can't we add the logic for validation and pass it as authorizer config. -Harsha On Mon, Mar 7, 2016, at 04:33 PM, Ashish Singh wrote: > + Parth, Har

Re: [VOTE] KIP-43: Kafka SASL enhancements

2016-03-09 Thread Harsha
+1 (binding) On Tue, Mar 8, 2016, at 02:37 AM, tao xiao wrote: > +1 (non-binding) > > On Tue, 8 Mar 2016 at 05:33 Andrew Schofield < > andrew_schofield_j...@outlook.com> wrote: > > > +1 (non-binding) > > > > > > > From: ism...@juma.me.uk > > > Date: Mon,

Re: [DISCUSS] KIP-50 - Enhance Authorizer interface to be aware of supported Principal Types

2016-04-07 Thread Harsha
Gwen, Ashish, Rolling upgrade should be fine in this case. Users can bring down single broker at a time and upgrade their kafka binaries along with new authorizer implementation . This looks fine to me. Thanks, Harsha

Re: [DISCUSS] KIP-50 - Enhance Authorizer interface to be aware of supported Principal Types

2016-04-07 Thread Harsha
g to break the rolling upgrade? and also moving to Java , a authorizer implementation going to run inside a KafkaBroker and I don't see why this is necessary to move to clients package. Are we planning on introducing common module to have it independent of broker and client code? -Harsha On Thu, A

Re: [DISCUSS] KIP-43: Kafka SASL enhancements

2016-04-07 Thread Harsha
right. -Harsha On Thu, Apr 7, 2016, at 05:46 AM, Rajini Sivaram wrote: > Magnus, > > > > > *"Why would it be harder to achieve? And is it harder for the Kafka > developeror the user?Wouldnt it be possible to accept both non-SSL-auth > and > SSL-auth clients on

Re: [DISCUSS] KIP-50 - Enhance Authorizer interface to be aware of supported Principal Types

2016-04-07 Thread Harsha
Ashish, Thanks for the details. We are not changing any of the zookeeper data format for acls right? Thanks, Harsha On Thu, Apr 7, 2016, at 11:25 AM, Gwen Shapira wrote: > Can you guys go into details on what will happen during a rolling upgrade > exactly? > > Gwen >

Re: [VOTE] KIP-43: Kafka SASL enhancements

2016-03-23 Thread Harsha
Any update on this. Gwen since the KIP is adjusted to address the pluggable classes we should make a move on this. Rajini, Can you restart the voting thread. Thanks, Harsha On Wed, Mar 16, 2016, at 06:42 AM, Rajini Sivaram wrote: > As discussed in the KIP meeting yesterday, the sc

Re: [VOTE] Release plan - Kafka 0.10.0

2016-03-07 Thread Harsha
+1 Thanks, Harsha On Mon, Mar 7, 2016, at 09:49 PM, Jun Rao wrote: > +1 > > Thanks, > > Jun > > On Mon, Mar 7, 2016 at 9:27 AM, Gwen Shapira <g...@confluent.io> wrote: > > > Greetings Kafka Developer Community, > > > > As you all know, w

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

2016-03-07 Thread Harsha
It doesn't need any release vehicle but still the work can move forward. If anyone is interested in the KIP please do the review and provide the comments. -Harsha On Mon, Mar 7, 2016, at 04:59 PM, Ismael Juma wrote: > I agree that it would be good to have more time to review and discuss >

Re: [VOTE] KIP-56 Allow cross origin HTTP requests on all HTTP methods

2016-04-27 Thread Harsha
+1 -Harsha On Wed, Apr 27, 2016, at 01:29 PM, Guozhang Wang wrote: > +1 > > On Wed, Apr 27, 2016 at 1:21 PM, Ewen Cheslack-Postava > <e...@confluent.io> > wrote: > > > +1 > > > > On Thu, Apr 21, 2016 at 10:30 AM, Liquan Pei <liquan...@gmail.com&

Re: [VOTE] KIP-50: Move Authorizer to o.a.k.common package - Round 2

2016-04-26 Thread Harsha
+1 -Harsha On Mon, Apr 25, 2016, at 10:34 AM, Grant Henke wrote: > +1 (non-binding) > > On Mon, Apr 25, 2016 at 12:30 PM, Gwen Shapira <g...@confluent.io> wrote: > > > +1 > > > > On Mon, Apr 25, 2016 at 10:04 AM, Ashish Singh <asi...@cloudera.com> >

Re: [VOTE] KIP-45: Standardize KafkaConsumer API to use Collection

2016-04-29 Thread Harsha
they are stable and thats not way to vet new apis. -Harsha On Fri, Apr 29, 2016, at 10:39 AM, Grant Henke wrote: > If anyone wants to review the KIP call discussion we had on this just > before the vote, here is a link to the relevant session (6 minutes in): > https://youtu.be/Hcjur17TjBE?t

Re: [VOTE] KIP-45: Standardize KafkaConsumer API to use Collection

2016-04-30 Thread Harsha
Hi Jason, Yes I am in favor removing them 0.11 and it definitely gives everyone one major version to update their clients to remove the deprecated commands. Thanks, Harsha On Fri, Apr 29, 2016, at 11:02 PM, Ewen Cheslack-Postava wrote: > I agree with Gr

Re: [VOTE] KIP-45: Standardize KafkaConsumer API to use Collection

2016-04-28 Thread Harsha
in having these methods with deprecated tag? we keep the interface as it is. Thanks, Harsha On Thu, Apr 28, 2016, at 01:27 PM, Ismael Juma wrote: > Hi Harsha, > > What is the aim of the PR, is it to fix binary compatibility, source > compatibility or both? I think it only

Re: [VOTE] KIP-45: Standardize KafkaConsumer API to use Collection

2016-04-28 Thread Harsha
ng is to give these users time to move onto the new api thats added and keep the old methods with deprecated tag for atleast one version. Thanks, Harsha On Thu, Apr 28, 2016, at 04:41 PM, Grant Henke wrote: > FYI. I have attached a sample clients API change/compatibility report to > KAFKA-1880

Re: [DISCUSSION] KAFKA-3633. Kafka Consumer API breaking backward compatibility

2016-04-28 Thread Harsha
Hi Jason, Agree and its my bad to not to raise this during the voting of KIP-45. Which I missed it. We can definitely start a vote around it. Thanks, Harsha On Thu, Apr 28, 2016, at 12:45 PM, Jason Gustafson wrote: > Hey Harsha, > > Probably the mai

Re: [VOTE] KIP-45: Standardize KafkaConsumer API to use Collection

2016-04-28 Thread Harsha
Hi, We missed this vote earlier and realized thats its breaking the 0.9.x client api compatibility. I opened a JIRA here https://issues.apache.org/jira/browse/KAFKA-3633 . Can we keep the old methods with deprecated tag in 0.10 release. Thanks, Harsha On Fri, Mar 18

Re: [ANNOUNCE] New committer: Ismael Juma

2016-04-26 Thread Harsha
Congrats, Ismael -Harsha On Tue, Apr 26, 2016, at 08:01 AM, Jun Rao wrote: > Congratulations, Ismael! > > Jun > > On Mon, Apr 25, 2016 at 10:52 PM, Neha Narkhede <n...@confluent.io> > wrote: > > > The PMC for Apache Kafka has invited Ismael Juma to join as a

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

2016-04-26 Thread Harsha
Parth, Overall current design looks good to me. I am +1 on the KIP. Gwen , Jun can you review this as well. -Harsha On Tue, Apr 19, 2016, at 09:57 AM, parth brahmbhatt wrote: > Thanks for review Jitendra. > > I don't like the idea of infinite lifetime but I see the Stre

Re: Apache Kafka JIRA Worflow: Add Closed -> Reopen transition

2016-05-20 Thread Harsha
Manikumar, Any reason for this. Before the workflow is to open a new JIRA if a JIRA closed. -Harsha On Fri, May 20, 2016, at 08:54 PM, Manikumar Reddy wrote: > Jun/Ismail, > > I requested Apache Infra to change JIRA workflow to add Closed -&

Re: KAFKA-3722 : Discussion about custom PrincipalBuilder and Authorizer configs

2016-05-20 Thread Harsha
eems to be better even though it breaks the interface. Thanks, Harsha On Fri, May 20, 2016, at 05:00 PM, Mayuresh Gharat wrote: > Hi All, > > I came across an issue with plugging in a custom PrincipalBuilder class > using the config "principal.builder.class" along with a custo

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

2016-05-19 Thread Harsha
Hi All, Can we have a KIP meeting around this. The KIP is up for sometime and if there are any questions lets quickly hash out details. Thanks, Harsha On Thu, May 19, 2016, at 08:40 AM, parth brahmbhatt wrote: > That is what the hadoop echo system uses

Re: [VOTE] 0.10.0.0 RC6

2016-05-20 Thread Harsha
+1 . Ran a 3-node cluster with few system tests on our side. Looks good. -Harsha On Thu, May 19, 2016, at 07:47 PM, Jun Rao wrote: > Thanks for running the release. +1 from me. Verified the quickstart. > > Jun > > On Tue, May 17, 2016 at 10:00 PM, Gwen Shapira <g...@c

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

2016-05-12 Thread Harsha
Hi Gwen, Can you look at Parth's last reply. Does it answer your concerns. Thanks, Harsha On Wed, May 4, 2016, at 09:25 AM, parth brahmbhatt wrote: > Thanks for reviewing Gwen. The wiki already has details on token > expiration > under token acquisition process

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

2016-05-03 Thread Harsha
sharing a single instance of producer. Also in this case other clients have access all the tokens no? Thanks, Harsha On Tue, May 3, 2016, at 11:49 AM, Gwen Shapira wrote: > Sorry for the delay: > > Two questions that we didn't see in the wiki: > 1. Is there a

Re: [VOTE] 0.10.0.0 RC2

2016-05-02 Thread Harsha
I would also like to get a decision on KIP-45. From the discussions it seems we are ok to include the changes proposed. -Harsha On Mon, May 2, 2016, at 09:15 AM, Gwen Shapira wrote: > Makes sense. > I'll try to get one out later today. > > > > On Mon, May 2, 2016 at 8:

Re: [RELEASE] Anyone minds if we push the next RC another week away?

2016-04-15 Thread Harsha
+1 On Fri, Apr 15, 2016, at 08:06 AM, Grant Henke wrote: > +1 > > On Fri, Apr 15, 2016 at 10:05 AM, Ashish Singh > wrote: > > > Good idea. Thanks! > > > > On Friday, April 15, 2016, Ismael Juma wrote: > > > > > +1 > > > > > > On Fri, Apr 15, 2016 at

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

2016-04-18 Thread Harsha
> On Mon, Mar 7, 2016 at 9:25 PM, Harsha <ka...@harsha.io> wrote: > > It doesn't need any release vehicle but still the work can move forward. > > If anyone is interested in the KIP please do the review and provide the > > comments. > > > > -Harsha > > &g

Re: Reg: Kafka With Kerberos/SSL [Enhancement to add option, Need suggestions on this]

2016-04-16 Thread Harsha
Bharat, Here is the jira https://issues.apache.org/jira/browse/KAFKA-3567 -Harsha On Thu, Apr 14, 2016, at 10:29 AM, BigData dev wrote: > Hi All, > When Kafka is running on kerberoized cluster/ SSL. Can we add an option > security.protocol. So, that user

Re: Reg: Issue with Kafka Kerberos (Kafka Version 0.9.0.1)

2016-04-16 Thread Harsha
Bharat, Did you add super.users=user:kafka to your server.properties. -Harsha On Tue, Apr 12, 2016, at 05:07 PM, BigData dev wrote: > Hi All, > I am facing issue with kafka kerberoized cluster. > > After following the steps how to enables SASL on kafka by

[DISCUSSION] KAFKA-3633. Kafka Consumer API breaking backward compatibility

2016-04-28 Thread Harsha
used new consumer API from 0.9 release. Anyone has any concerns having this patch in. Thanks, -Harsha

Re: [VOTE] KIP-45: Standardize KafkaConsumer API to use Collection

2016-05-07 Thread Harsha
Ismael, Do we need to add old assign and subscribe that accepts List. It will get implicitly cast to collection with the new methods. The only problem comes from the methods that accepts varargs. -Harsha On Sat, May 7, 2016, at 05:53 PM, Mark Grover wrote: > Thanks Ismael, I ag

Re: [DISCUSS] KIP-308: Support dynamic update of max.connections.per.ip/max.connections.per.ip.overrides configs

2018-06-26 Thread Harsha
This is very useful. LGTM. Thanks, Harsha On Mon, Jun 25th, 2018 at 10:20 AM, Dong Lin wrote: > > > > Hey Manikumar, > > Thanks much for the KIP. It looks pretty good. > > Thanks, > Dong > > On Thu, Jun 21, 2018 at 11:38 PM, Manikumar < manikumar.re..

Re: [DISCUSS] KIP-291: Have separate queues for control requests and data requests

2018-06-26 Thread Harsha
Thanks for the pointer. Will take a look might suit our requirements better. Thanks, Harsha On Mon, Jun 25th, 2018 at 2:52 PM, Lucas Wang wrote: > > > > Hi Harsha, > > If I understand correctly, the replication quota mechanism proposed in > KIP-73 can be helpful in

Re: [VOTE] KIP-324: Add method to get metrics() in AdminClient

2018-06-27 Thread Harsha
+1 (binding) Thanks, Harsha On Wed, Jun 27th, 2018 at 10:56 AM, Damian Guy wrote: > > > > +1 (binding) > > Thanks > > > > On Wed, 27 Jun 2018 at 18:50 Bill Bejeck < bbej...@gmail.com > wrote: > > > +1 > > > > -Bill > >

Re: [VOTE] KIP-291: Have separate queues for control requests and data requests

2018-06-20 Thread Harsha
+1 -Harsha On Wed, Jun 20, 2018, at 5:15 AM, Thomas Crayford wrote: > +1 (non-binding) > > On Tue, Jun 19, 2018 at 8:20 PM, Lucas Wang wrote: > > > Hi Jun, Ismael, > > > > Can you please take a look when you get a chance? Thanks! > > > > Lucas >

Re: [VOTE] 2.0.0 RC1

2018-07-02 Thread Harsha
+1.  1) Ran unit tests  2) 3 node cluster , tested basic operations.  Thanks, Harsha On Mon, Jul 2nd, 2018 at 11:13 AM, "Vahid S Hashemian" wrote: > > > > +1 (non-binding) > > Built from source and ran quickstart successfully on Ubuntu (with Java 8). >

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

2018-07-02 Thread Harsha
+1.      1) Ran unit tests 2) 3 node cluster , tested basic operations. Thanks, Harsha On Mon, Jul 2nd, 2018 at 11:57 AM, Jun Rao wrote: > > > > Hi, Matthias, > > Thanks for the running the release. Verified quickstart on scala 2.12 > binary. +1 > > Jun > &

Re: [VOTE] KIP-322: Return new error code for DeleteTopics API when topic deletion disabled.

2018-07-03 Thread Harsha
+1. Thanks, Harsha On Tue, Jul 3rd, 2018 at 9:22 AM, Ted Yu wrote: > > > > +1 > > On Tue, Jul 3, 2018 at 9:05 AM, Mickael Maison < mickael.mai...@gmail.com > > > wrote: > > > +1 (non binding) > > Thanks for the KIP > > &g

Re: [VOTE] KIP-346 - Improve LogCleaner behavior on error

2018-08-07 Thread Harsha
+1 (binding) Thanks, Harsha On Tue, Aug 7, 2018, at 10:22 AM, Manikumar wrote: > +1 (non-binding) > > Thanks for the KIP. > > On Tue, Aug 7, 2018 at 10:42 PM Ray Chiang wrote: > > > +1 (non-binding) > > > > -Ray > > > > On 8/7/18 9:26 AM,

Re: [DISCUSS] KIP-357: Add support to list ACLs per principal

2018-08-23 Thread Harsha
+1 (binding) Thanks, Harsha On Wed, Aug 22, 2018, at 9:15 AM, Manikumar wrote: > Hi Viktor, > We already have a method in Authorizer interface to get acls for a given > principal. > We will use this method to fetch acls and filter the results for requested > Resources. > Aut

Re: [VOTE] KIP-357: Add support to list ACLs per principal

2018-08-27 Thread Harsha
+1 (binding) -Harsha On Mon, Aug 27, 2018, at 12:46 PM, Jakub Scholz wrote: > +1 (non-binding) > > On Mon, Aug 27, 2018 at 6:24 PM Manikumar wrote: > > > Hi All, > > > > I would like to start voting on KIP-357 which allows to list ACLs per > > princ

Re: [VOTE] KIP-336: Consolidate ExtendedSerializer/Serializer and ExtendedDeserializer/Deserializer

2018-08-30 Thread Harsha
+1. Thanks, Harsha On Thu, Aug 30, 2018, at 4:19 AM, Attila Sasvári wrote: > Thanks for the KIP and the updates Viktor! > > +1 (non-binding) > > > > On Wed, Aug 29, 2018 at 10:44 AM Manikumar > wrote: > > > +1 (non-binding) > > > > Thanks for the

Re: [VOTE] KIP-231: Improve the Required ACL of ListGroups API

2018-07-06 Thread Harsha
+1. Thanks, Harsha On Fri, Jun 1st, 2018 at 10:21 AM, "Vahid S Hashemian" wrote: > > > > I'm bumping this vote thread up as the KIP requires only one binding +1 to > > pass. > The KIP is very similar in nature to the recently approved KIP-277 ( > h

Re: [VOTE] 2.0.0 RC2

2018-07-12 Thread Harsha
+1 1. Ran unit tests 2. Tested few use cases through 3-node cluster. Thanks, Harsha On Thu, Jul 12, 2018, at 9:33 AM, Mickael Maison wrote: > +1 non-binding > Built from source, ran tests, ran quickstart and check signatures > > Thanks! > > > On Wed, Jul 11, 2018 at 1

Re: [VOTE] 1.1.1 RC3

2018-07-12 Thread Harsha
+1. 1. Ran unit tests 2. Ran 3 node cluster to run few tests. Thanks, Harsha On Thu, Jul 12, 2018, at 7:29 AM, Manikumar wrote: > +1 (non-binding) Ran tests, Verified quick start, producer/consumer perf > tests > > > > On Thu, Jul 12, 2018 at 11:06 AM Brett Rann >

Re: KIP-327: Add describe all topics API to AdminClient

2018-07-12 Thread Harsha
Very useful. LGTM. Thanks, Harsha On Thu, Jul 12, 2018, at 9:56 AM, Manikumar wrote: > Hi all, > > I have created a KIP to add describe all topics API to AdminClient . > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-327%3A+Add+describe+all+topics+API+to+AdminClient &

Re: [VOTE] 1.1.1 RC3

2018-07-09 Thread Harsha
+1. * Ran unit tests * Installed in a cluster and ran simple tests Thanks, Harsha On Mon, Jul 9th, 2018 at 6:38 AM, Ted Yu wrote: > > > > +1 > > Ran test suite. > > Checked signatures. > > > > On Sun, Jul 8, 2018 at 3:36 PM Dong Lin < lindon..

Re: [VOTE] 1.1.0 RC2

2018-03-14 Thread Harsha
+1 Ran tests Ran a 3 node cluster to test basic operations. Thanks, Harsha On Wed, Mar 14, 2018, at 9:04 AM, Ted Yu wrote: > +1 > > Ran test suite - passed (apart from testMetricsLeak which is flaky). > > On Wed, Mar 14, 2018 at 3:30 AM, Damian Guy <damian@gmail.com>

Re: [DISCUSS] KIP-291: Have separate queues for control requests and data requests

2018-06-24 Thread Harsha
.               Thanks, Harsha On Fri, Jun 22nd, 2018 at 11:35 AM Lucas Wang wrote: > > > > Hi Eno, > > Sorry for the delayed response. > - I haven't implemented the feature yet, so no experimental results so > far. > And I plan to test in out in the following days. >

Re: [ANNOUNCE] New Committer: Manikumar Reddy

2018-10-17 Thread Harsha
Congrats Mani!! Very well deserved. --Harsha On Tue, Oct 16, 2018, at 5:20 PM, Attila Sasvari wrote: > Congratulations Manikumar! Keep up the good work. > > On Tue, Oct 16, 2018 at 12:30 AM Jungtaek Lim wrote: > > > Congrats Mani! > > On Tue, 16 Oct 2018 at 1

Re: [DISCUSS] KIP-382: MirrorMaker 2.0

2018-10-17 Thread Harsha
which can deployed in mesos or container orchestrations. If possible can you document in the rejected alternatives what are missing parts that made you to consider a new design from ground up. Thanks, Harsha On Wed, Oct 17, 2018, at 8:34 AM, Ryanne Dolan wrote: > Jan, these are two separ

  1   2   3   >