+1.
On Tue, Oct 25, 2016 at 2:58 PM, Jay Kreps wrote:
> -1
>
> I think the REST server for Kafka that already exists is quite good and
> getting contributions. Moving this into the core project doesn't solve a
> problem that I see.
>
> -Jay
>
> On Tue, Oct 25, 2016 at 2:16 PM, Harsha Chintalapan
[
https://issues.apache.org/jira/browse/KAFKA-2700?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15493958#comment-15493958
]
Parth Brahmbhatt commented on KAFKA-2700:
-
all yours.
> delete topic
>> 1. Who / how are tokens renewed? By original requester only? or
> using
> > > >> Kerberos auth only?
> > > >> 2. Are tokens stored on each broker or in ZK?
> > > >> 3. How are tokens invalidated / expired?
> > > >> 4. Which e
PM, Harsha wrote:
> >> > Jun & Ismael,
> >> > Unfortunately I couldn't attend the KIP
> meeting
> >> > when delegation tokens discussed. Appreciate
> if
> >> >
gt; https://tools.ietf.org/html/rfc7677 describes the protocol for
> > > > SCRAM-SHA-256.
> > > >
> > > > On Tue, May 24, 2016 at 2:37 AM, Jun Rao wrote:
> > > >
> > > > > Parth,
> > > > >
> > > > &
meeting invite. We can discuss this in the meeting
> > tomorrow.
> >
> > Thanks,
> >
> > Jun
> >
> > On Thu, May 19, 2016 at 8:47 AM, Harsha wrote:
> >
> >> Hi All,
> >>Can we have a KIP meeting around this. The KIP is u
the wiki. I think that we
> > >> can decouple the problem of "token distribution" from "shared secret
> > >> distribution" and use the controller as the only token generator to
> > >> solve the second issue, while still using ZK async to distr
wondering if there are alternatives in other ecosystems
> (Mesos? Tachyon? Cassandra?) and whether there are some advantages
> there.
>
> Gwen
>
> On Thu, May 12, 2016 at 1:05 PM, Harsha wrote:
> > Hi Gwen,
> >Can you look at Parth's last reply. Does i
or
>
> Anythoughts on this.
>
>
> Regards,
> Bharat
>
>
>
> On Thu, May 5, 2016 at 4:19 PM, parth brahmbhatt <
> brahmbhatt.pa...@gmail.com> wrote:
>
>> Acls will be written in zookeeper but you are using getAcl , what you need
>> is get
Acls will be written in zookeeper but you are using getAcl , what you need
is get /kafka-acl/Topic/permissiontopic
Thanks
Parth
On Thu, May 5, 2016 at 3:28 PM, BigData dev wrote:
> Hi,
> When I run the command
> /bin/kafka-acls.sh --topic permissiontopic --add --allow-host {host}
> --allow-p
it Storm
> >> job as my user), we will need a producer for every job (we can't share
> >> them between multiple jobs running on same node), since we only
> >> authenticate when connecting. Is there a plan to change this for
> >> delegation tokens, in order to allow m
, 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 Streaming use
> > case. Even for Streaming use case I was hoping there will be some notion
> > of
> > maste
[
https://issues.apache.org/jira/browse/KAFKA-3294?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15261562#comment-15261562
]
Parth Brahmbhatt commented on KAFKA-3294:
-
Here is a version we are workin
gt; >3. In catastrophic failures where all brokers go down, the
> > tokens will
> > > >> > >be lost even if servers are restarted as tokens are not
> > persisted
> > > >> > anywhere.
> > > >> > >If this happens, then the
+1
Thanks
Parth
On Wed, Mar 30, 2016 at 8:22 PM, Ewen Cheslack-Postava
wrote:
> +1
>
> -Ewen
>
> On Wed, Mar 30, 2016 at 7:20 PM, Gwen Shapira wrote:
>
> > So my +1 is back :)
> >
> > On Wed, Mar 30, 2016 at 4:51 PM, Ashish Singh
> wrote:
> >
> > > My bad, I moved the config option to rejecte
Hi,
I have filed KIP-48 so we can offer hadoop like delegation tokens in kafka. You
can review the design
https://cwiki.apache.org/confluence/display/KAFKA/KIP-48+Delegation+token+support+for+Kafka.
This KIP depends on KIP-43 and we have also discussed an alternative to
proposed design
here
GitHub user Parth-Brahmbhatt opened a pull request:
https://github.com/apache/kafka/pull/975
KAFKA-3291: DumpLogSegment tool should also provide an option to onlyâ¦
⦠verify index sanity.
You can merge this pull request into a Git repository by running:
$ git pull https
Parth Brahmbhatt created KAFKA-3291:
---
Summary: DumpLogSegment tool should also provide an option to only
verify index sanity.
Key: KAFKA-3291
URL: https://issues.apache.org/jira/browse/KAFKA-3291
[
https://issues.apache.org/jira/browse/KAFKA-1696?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15168010#comment-15168010
]
Parth Brahmbhatt commented on KAFKA-1696:
-
[~gwenshap] I think that discussio
[
https://issues.apache.org/jira/browse/KAFKA-1696?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15167928#comment-15167928
]
Parth Brahmbhatt commented on KAFKA-1696:
-
[~gwenshap] [~sriharsha] Please s
[
https://issues.apache.org/jira/browse/KAFKA-1696?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15167800#comment-15167800
]
Parth Brahmbhatt edited comment on KAFKA-1696 at 2/25/16 8:2
[
https://issues.apache.org/jira/browse/KAFKA-1696?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15167800#comment-15167800
]
Parth Brahmbhatt commented on KAFKA-1696:
-
So here is how that request path w
[
https://issues.apache.org/jira/browse/KAFKA-1696?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15167696#comment-15167696
]
Parth Brahmbhatt commented on KAFKA-1696:
-
[~singhashish] Thanks for taking
[
https://issues.apache.org/jira/browse/KAFKA-1696?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15149659#comment-15149659
]
Parth Brahmbhatt commented on KAFKA-1696:
-
[~gwenshap] [~harsha_ch] [~singhas
[
https://issues.apache.org/jira/browse/KAFKA-1696?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Parth Brahmbhatt reassigned KAFKA-1696:
---
Assignee: Parth Brahmbhatt (was: Sriharsha Chintalapani)
> Kafka should be able
[
https://issues.apache.org/jira/browse/KAFKA-1696?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15113295#comment-15113295
]
Parth Brahmbhatt commented on KAFKA-1696:
-
I will assign it to my self and fi
[
https://issues.apache.org/jira/browse/KAFKA-1696?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15106023#comment-15106023
]
Parth Brahmbhatt commented on KAFKA-1696:
-
[~gwenshap] Are you still workin
GitHub user Parth-Brahmbhatt opened a pull request:
https://github.com/apache/kafka/pull/704
KAFKA-2000: Delete topic should also delete consumer offsets.
You can merge this pull request into a Git repository by running:
$ git pull https://github.com/Parth-Brahmbhatt/kafka
[
https://issues.apache.org/jira/browse/KAFKA-2000?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15064564#comment-15064564
]
Parth Brahmbhatt commented on KAFKA-2000:
-
Talked to [~harsha_ch] , I will su
[
https://issues.apache.org/jira/browse/KAFKA-2547?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Work on KAFKA-2547 started by Parth Brahmbhatt.
---
> Make DynamicConfigManager to use the ZkNodeChangeNotificationListe
GitHub user Parth-Brahmbhatt opened a pull request:
https://github.com/apache/kafka/pull/679
KAFKA-2547: Make DynamicConfigManager to use the ZkNodeChangeNotificaâ¦
â¦tionListener introduced as part of KAFKA-2211
You can merge this pull request into a Git repository by running
[
https://issues.apache.org/jira/browse/KAFKA-2854?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Work on KAFKA-2854 started by Parth Brahmbhatt.
---
> Make KerberosName implement PrincipalToLocal plugin so authorizer
[
https://issues.apache.org/jira/browse/KAFKA-2854?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15056930#comment-15056930
]
Parth Brahmbhatt commented on KAFKA-2854:
-
[~ijuma][~harsha_ch] Can you re
[
https://issues.apache.org/jira/browse/KAFKA-2794?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15056934#comment-15056934
]
Parth Brahmbhatt commented on KAFKA-2794:
-
https://github.com/apache/kafka/
[
https://issues.apache.org/jira/browse/KAFKA-2794?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Work on KAFKA-2794 started by Parth Brahmbhatt.
---
> Add group support for authorizer a
GitHub user Parth-Brahmbhatt opened a pull request:
https://github.com/apache/kafka/pull/547
Kafka-2854: Making KerberosShortNamer implement an interface and making it
pluggable.
You can merge this pull request into a Git repository by running:
$ git pull https://github.com
Parth Brahmbhatt created KAFKA-2854:
---
Summary: Make KerberosName implement PrincipalToLocal plugin so
authorizer and authenticator can share this.
Key: KAFKA-2854
URL: https://issues.apache.org/jira/browse
GitHub user Parth-Brahmbhatt opened a pull request:
https://github.com/apache/kafka/pull/545
Kafka-2852:Updating the Authorizer CLI to use a consistent way to specify a
list of values for a config options.
â¦ecify a list of values for a config options.
You can merge this pull
Parth Brahmbhatt created KAFKA-2852:
---
Summary: Kafka Authroizer CLI should use consistent way to specify
multiple values for all config options.
Key: KAFKA-2852
URL: https://issues.apache.org/jira/browse/KAFKA
GitHub user Parth-Brahmbhatt opened a pull request:
https://github.com/apache/kafka/pull/529
KAFKA-2838: Allow comma in super users, allow comma in CLI authz propâ¦
â¦erties.
You can merge this pull request into a Git repository by running:
$ git pull https://github.com
[
https://issues.apache.org/jira/browse/KAFKA-2808?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15000985#comment-15000985
]
Parth Brahmbhatt commented on KAFKA-2808:
-
[~tgraves] your last comment is wh
[
https://issues.apache.org/jira/browse/KAFKA-2808?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15000865#comment-15000865
]
Parth Brahmbhatt commented on KAFKA-2808:
-
Ideally we would always give a
GitHub user Parth-Brahmbhatt opened a pull request:
https://github.com/apache/kafka/pull/489
KAFKA-2788: Allow specifying principals with comman in ACL CLI.
You can merge this pull request into a Git repository by running:
$ git pull https://github.com/Parth-Brahmbhatt/kafka
[
https://issues.apache.org/jira/browse/KAFKA-2788?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14998905#comment-14998905
]
Parth Brahmbhatt commented on KAFKA-2788:
-
I have started working on it,
[
https://issues.apache.org/jira/browse/KAFKA-2788?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14998035#comment-14998035
]
Parth Brahmbhatt commented on KAFKA-2788:
-
Looks like we already have a valid
GitHub user Parth-Brahmbhatt opened a pull request:
https://github.com/apache/kafka/pull/483
Kafka-2794: Added group support to authorizer.
You can merge this pull request into a Git repository by running:
$ git pull https://github.com/Parth-Brahmbhatt/kafka KAFKA-2794
Parth Brahmbhatt created KAFKA-2794:
---
Summary: Add group support for authorizer acls
Key: KAFKA-2794
URL: https://issues.apache.org/jira/browse/KAFKA-2794
Project: Kafka
Issue Type: Sub
[
https://issues.apache.org/jira/browse/KAFKA-2754?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Parth Brahmbhatt reassigned KAFKA-2754:
---
Assignee: Parth Brahmbhatt
> Support defining ACLs at topic creation t
[
https://issues.apache.org/jira/browse/KAFKA-2700?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Parth Brahmbhatt reassigned KAFKA-2700:
---
Assignee: Parth Brahmbhatt
> delete topic should remove the corresponding ACL
[
https://issues.apache.org/jira/browse/KAFKA-2682?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Parth Brahmbhatt reassigned KAFKA-2682:
---
Assignee: Parth Brahmbhatt
> Authorization section in official d
I have 2 suggestions:
1) We need to document how does one move from secure to non secure
environment:
1) change the config on all brokers to zookeeper.set.acl = false and do
a
rolling upgrade.
2) Run the migration script with the jass config file so it is sasl
authenticated with
[
https://issues.apache.org/jira/browse/KAFKA-2212?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14959503#comment-14959503
]
Parth Brahmbhatt commented on KAFKA-2212:
-
[~junrao] [~ijuma]
# we already
GitHub user Parth-Brahmbhatt opened a pull request:
https://github.com/apache/kafka/pull/300
KAFKA-2598: Adding integration test for the authorizer at API level. â¦
â¦Some bug fixes that I encountered while running the tests.
You can merge this pull request into a Git repository
[
https://issues.apache.org/jira/browse/KAFKA-2598?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14947417#comment-14947417
]
Parth Brahmbhatt commented on KAFKA-2598:
-
I just started yesterday, should
[
https://issues.apache.org/jira/browse/KAFKA-2587?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14943792#comment-14943792
]
Parth Brahmbhatt commented on KAFKA-2587:
-
https://github.com/apache/kafka/
GitHub user Parth-Brahmbhatt opened a pull request:
https://github.com/apache/kafka/pull/277
Kafka-2587: Only notification handler will update the cache and all
verifications will use waitUntilTrue.
You can merge this pull request into a Git repository by running:
$ git
[
https://issues.apache.org/jira/browse/KAFKA-2587?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14943764#comment-14943764
]
Parth Brahmbhatt commented on KAFKA-2587:
-
I am fine with changing the code s
[
https://issues.apache.org/jira/browse/KAFKA-2587?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14941533#comment-14941533
]
Parth Brahmbhatt commented on KAFKA-2587:
-
Hey Ismael,
As mentioned above
[
https://issues.apache.org/jira/browse/KAFKA-2587?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14941474#comment-14941474
]
Parth Brahmbhatt commented on KAFKA-2587:
-
I looked at the code to reason ar
[
https://issues.apache.org/jira/browse/KAFKA-2598?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14935589#comment-14935589
]
Parth Brahmbhatt commented on KAFKA-2598:
-
I should be able to pick this on
Parth Brahmbhatt created KAFKA-2598:
---
Summary: Add Test with authorizer for producer and consumer
Key: KAFKA-2598
URL: https://issues.apache.org/jira/browse/KAFKA-2598
Project: Kafka
Issue
[
https://issues.apache.org/jira/browse/KAFKA-2587?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Work on KAFKA-2587 started by Parth Brahmbhatt.
---
> Transient test failure: `SimpleAclAuthorizerT
GitHub user Parth-Brahmbhatt opened a pull request:
https://github.com/apache/kafka/pull/260
KAFKA-2587:Increasing timeout for the test verification.
You can merge this pull request into a Git repository by running:
$ git pull https://github.com/Parth-Brahmbhatt/kafka KAFKA
[
https://issues.apache.org/jira/browse/KAFKA-2587?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14934498#comment-14934498
]
Parth Brahmbhatt commented on KAFKA-2587:
-
[~ijuma] I tried to reproduce
[
https://issues.apache.org/jira/browse/KAFKA-2212?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Parth Brahmbhatt updated KAFKA-2212:
Status: In Progress (was: Patch Available)
> KafkaAuthorizer: Add CLI for Acl managem
GitHub user Parth-Brahmbhatt opened a pull request:
https://github.com/apache/kafka/pull/230
KAFKA-2212: Authorizer CLI implementation.
You can merge this pull request into a Git repository by running:
$ git pull https://github.com/Parth-Brahmbhatt/kafka KAFKA-2212
[
https://issues.apache.org/jira/browse/KAFKA-2549?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14744368#comment-14744368
]
Parth Brahmbhatt commented on KAFKA-2549:
-
[~ijuma] You are right, my bad, ed
[
https://issues.apache.org/jira/browse/KAFKA-2549?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Parth Brahmbhatt updated KAFKA-2549:
Description: Introduced in
https://github.com/apache/kafka/commit
GitHub user Parth-Brahmbhatt opened a pull request:
https://github.com/apache/kafka/pull/215
KAFKA-2549: Fixing checkstyle failure resulting due to unused importsâ¦
⦠in Selector.
You can merge this pull request into a Git repository by running:
$ git pull https
Parth Brahmbhatt created KAFKA-2549:
---
Summary: Checkstyle reporting failure in trunk due to unused
imports in Selector.java
Key: KAFKA-2549
URL: https://issues.apache.org/jira/browse/KAFKA-2549
Parth Brahmbhatt created KAFKA-2547:
---
Summary: Make DynamicConfigManager to use the
ZkNodeChangeNotificationListener introduced as part of KAFKA-2211
Key: KAFKA-2547
URL: https://issues.apache.org/jira/browse
GitHub user Parth-Brahmbhatt opened a pull request:
https://github.com/apache/kafka/pull/208
KAFKA-2538: Fixing a compilation error in trunk.
You can merge this pull request into a Git repository by running:
$ git pull https://github.com/Parth-Brahmbhatt/kafka KAFKA-2538
Parth Brahmbhatt created KAFKA-2538:
---
Summary: Compilation in trunk is failing due to
https://github.com/apache/kafka/commit/845514d62329be8382e6d02b8041fc858718d534
Key: KAFKA-2538
URL: https
[
https://issues.apache.org/jira/browse/KAFKA-2538?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Parth Brahmbhatt reassigned KAFKA-2538:
---
Assignee: Parth Brahmbhatt
> Compilation in trunk is failing due to
>
t; > KAFKA-2212 has some similar issues too, it may be worth taking a pass there
> > too.
> >
> > I will look at these two patches again early next week.
>
> Parth Brahmbhatt wrote:
> Hey , thanks for reviewing this. This patch needs to be updated with all
&
tchers so I don't completely rely on the watchers always
firing. If that understading is incorrect i can remove the scheduler thread and
change the code to only read from zookeeper as part of handling the watcher
notification.
- Parth
--
[
https://issues.apache.org/jira/browse/KAFKA-2211?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Work on KAFKA-2211 started by Parth Brahmbhatt.
---
> KafkaAuthorizer: Add simpleACLAuthorizer implementat
[
https://issues.apache.org/jira/browse/KAFKA-2211?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Parth Brahmbhatt updated KAFKA-2211:
Status: Patch Available (was: In Progress)
> KafkaAuthorizer: Add simpleACLAuthori
GitHub user Parth-Brahmbhatt opened a pull request:
https://github.com/apache/kafka/pull/195
KAFKA-2211: Adding simpleAclAuthorizer implementation and test cases.
You can merge this pull request into a Git repository by running:
$ git pull https://github.com/Parth-Brahmbhatt
--
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/34492/#review97466
---
On Sept. 3, 2015, 12:36 a.m., Parth Brahmbhatt wrote:
>
> ---
where we don't seem to use the data produced by the
> > `partition` call.
Cherry picked. The FIXME does not need any change if you see
https://github.com/Parth-Brahmbhatt/kafka/blob/az/core/src/main/scala/kafka/server/KafkaApis.scala#L195
it uses unauthorized partiton in constructing re
[
https://issues.apache.org/jira/browse/KAFKA-2210?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Parth Brahmbhatt updated KAFKA-2210:
Attachment: KAFKA-2210_2015-09-02_17:36:47.patch
> KafkaAuthorizer: Add all public entit
[
https://issues.apache.org/jira/browse/KAFKA-2210?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14728299#comment-14728299
]
Parth Brahmbhatt commented on KAFKA-2210:
-
Updated reviewboard h
rated e-mail. To reply, visit:
https://reviews.apache.org/r/34492/#review97554
---
On Sept. 3, 2015, 12:36 a.m., Parth Brahmbhatt wrote:
>
> ---
> This is an automatical
core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala
3da666f73227fc7ef7093e3790546344065f6825
Diff: https://reviews.apache.org/r/34492/diff/
Testing
---
Thanks,
Parth Brahmbhatt
[
https://issues.apache.org/jira/browse/KAFKA-2210?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Parth Brahmbhatt updated KAFKA-2210:
Attachment: KAFKA-2210_2015-09-02_17:32:06.patch
> KafkaAuthorizer: Add all public entit
[
https://issues.apache.org/jira/browse/KAFKA-2210?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14728293#comment-14728293
]
Parth Brahmbhatt commented on KAFKA-2210:
-
Updated reviewboard h
gTest.scala
3da666f73227fc7ef7093e3790546344065f6825
Diff: https://reviews.apache.org/r/34492/diff/
Testing
---
Thanks,
Parth Brahmbhatt
[
https://issues.apache.org/jira/browse/KAFKA-2210?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14728093#comment-14728093
]
Parth Brahmbhatt commented on KAFKA-2210:
-
Updated reviewboard h
[
https://issues.apache.org/jira/browse/KAFKA-2210?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Parth Brahmbhatt updated KAFKA-2210:
Attachment: KAFKA-2210_2015-09-02_14:50:29.patch
> KafkaAuthorizer: Add all public entit
tps://reviews.apache.org/r/34492/#comment153374>
nope. reverted.
- Parth Brahmbhatt
On Sept. 2, 2015, 9:50 p.m., Parth Brahmbhatt wrote:
>
> ---
> This is an automatically generated e-mail. To reply, visit:
> https://re
test/scala/unit/kafka/server/KafkaConfigTest.scala
3da666f73227fc7ef7093e3790546344065f6825
Diff: https://reviews.apache.org/r/34492/diff/
Testing
---
Thanks,
Parth Brahmbhatt
xception in the
> > client should be of ApiException.
fixed.
- Parth
-------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/34492/#review96909
---
[
https://issues.apache.org/jira/browse/KAFKA-2210?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Parth Brahmbhatt updated KAFKA-2210:
Attachment: KAFKA-2210_2015-09-01_15:36:02.patch
> KafkaAuthorizer: Add all public entit
[
https://issues.apache.org/jira/browse/KAFKA-2210?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14726341#comment-14726341
]
Parth Brahmbhatt commented on KAFKA-2210:
-
Updated reviewboard h
https://reviews.apache.org/r/34492/diff/
Testing
---
Thanks,
Parth Brahmbhatt
[
https://issues.apache.org/jira/browse/KAFKA-2210?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14723866#comment-14723866
]
Parth Brahmbhatt commented on KAFKA-2210:
-
[~junrao] yes that is cor
[
https://issues.apache.org/jira/browse/KAFKA-2210?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Parth Brahmbhatt updated KAFKA-2210:
Attachment: KAFKA-2210_2015-08-26_14:29:02.patch
> KafkaAuthorizer: Add all public entit
[
https://issues.apache.org/jira/browse/KAFKA-2210?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14715553#comment-14715553
]
Parth Brahmbhatt commented on KAFKA-2210:
-
Updated reviewboard h
core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala
3da666f73227fc7ef7093e3790546344065f6825
Diff: https://reviews.apache.org/r/34492/diff/
Testing
---
Thanks,
Parth Brahmbhatt
1 - 100 of 407 matches
Mail list logo