Re: [DISCUSS] KIP-290: Support for wildcard suffixed ACLs

2018-05-10 Thread Andy Coates
/kafka/common/acl/AclBindingFilter.java#L39 >>> and >>> >>> https://github.com/apache/kafka/blob/trunk/clients/src/ >> main/java/org/apache/kafka/common/resource/ResourceFilter.java#L37 >>> ). >>> - However, the above two changes are breaking

Re: [DISCUSS] KIP-290: Support for wildcard suffixed ACLs

2018-05-11 Thread Andy Coates
fields. The important thing is that there's a new > static function, or new constructor overload, etc. that works for patterns > rather than literal strings. > > On Thu, May 10, 2018, at 03:30, Andy Coates wrote: > > Rather than having name and pattern fields on the ResourceFilter,

Re: [DISCUSS] KIP-292: Add transformValues() method to KTable

2018-05-11 Thread Andy Coates
r at least not > > > > logically materialized and not be queryable. What if there is at > least > > > one > > > > state store provided? Will any of them be provided as the > materialized > > > > store, or should we still add a Materialize

Re: [DISCUSS] KIP-292: Add transformValues() method to KTable

2018-05-11 Thread Andy Coates
ansformValues should also not allow calls to getStateStore() where the name is the materialzied.name(). If that's the case, then sure that shouldn't be an issue. On 11 May 2018 at 09:42, Andy Coates wrote: > Sorry for my lack of response - I've been out of action with a bad back

Re: [DISCUSS] KIP-292: Add transformValues() method to KTable

2018-05-11 Thread Andy Coates
the overloaded value functions "with key" only, then we > should not add the ones without keys any more in new KIPs. > > WDYT? > > > Guozhang > > > On Fri, May 11, 2018 at 9:42 AM, Andy Coates wrote: > > > Sorry for my lack of response - I've bee

Re: [DISCUSS] KIP-292: Add transformValues() method to KTable

2018-05-11 Thread Andy Coates
calls to getStateStore where the store name matches the materialized result of the call. I'll sort out the PR next. On 11 May 2018 at 10:26, Damian Guy wrote: > I'm a +1 for Guozhang's suggestion > > On Fri, 11 May 2018 at 10:20 Andy Coates wrote: > > > Ma

Re: [DISCUSS] KIP-292: Add transformValues() method to KTable

2018-05-11 Thread Andy Coates
Just a thought - but on the subject of disallowing access to the materialized state store from within the transformer's init method... might this not be overly restrictive? Could there be valid uses where read-only access would be useful / valid. On 11 May 2018 at 10:35, Andy Coates wrote:

Re: [DISCUSS] KIP-292: Add transformValues() method to KTable

2018-05-11 Thread Andy Coates
10:42, Andy Coates wrote: > Just a thought - but on the subject of disallowing access to the > materialized state store from within the transformer's init method... might > this not be overly restrictive? Could there be valid uses where read-only > access would be useful / valid. &g

Re: [DISCUSS] KIP-292: Add transformValues() method to KTable

2018-05-11 Thread Andy Coates
)` with the correct internal store name? If not the > > semantics is a bit complex, if yes we are breaking the protocol to not > > expose internal store. > > > > > > Guozhang > > > > On Fri, May 11, 2018 at 10:42 AM, Andy Coates wrote: > > >

Re: [DISCUSS] KIP-292: Add transformValues() method to KTable

2018-05-11 Thread Andy Coates
e(): should we return the internal > store, > > even if user provided the right internal store name? > > > > 2) For stores used in materialization, should we allow users to write to > > them? > > > > > > Guozhang > > > > > > On Fri, May 11,

[VOTE] KIP-292: Add transformValues() method to KTable

2018-05-11 Thread Andy Coates
Hi all, I would like to start the vote on KIP-292: Add transformValues() method to KTable The link to this KIP is here: https://cwiki.apache.org/confluence/display/KAFKA/KIP-292%3A+Add+transformValues%28%29+method+to+KTable The discussion thread is here: http://mail-archives.apache.org/mod_mbox/

Re: [VOTE] KIP-292: Add KTable#transformValues() method in Kafka Streams DSL

2018-05-14 Thread Andy Coates
>> there is a voting started already. > >> > >> > >> Guozhang > >> > >> > >> On Fri, May 11, 2018 at 2:07 PM, Andy Coates wrote: > >> > >>> Hi all, > >>> > >>> I would like to st

Re: [DISCUSS] KIP-290: Support for wildcard suffixed ACLs

2018-05-14 Thread Andy Coates
er the wire (just the string + version number) > > cheers, > Colin > > > On Fri, May 11, 2018, at 09:39, Andy Coates wrote: > > i think I'm agreeing with you. I was merely suggesting that having an > > additional field that controls how the current field is interprete

Re: [VOTE] KIP-292: Add KTable#transformValues() method in Kafka Streams DSL

2018-05-14 Thread Andy Coates
Hey all, With 3 binding votes and 1 non-binding, this KIP has been accepted. Thanks for everyones input and time. Andy On 14 May 2018 at 09:42, Guozhang Wang wrote: > Thanks Andy. That's fine, we can continue the vote thread. > > On Mon, May 14, 2018 at 9:25 AM, Andy

Re: [DISCUSS] KIP-290: Support for wildcard suffixed ACLs

2018-05-17 Thread Andy Coates
e this: > > > > > > > >> > > AclBindingFilter filter = new AclBindingFiler(new > > > > > > > >> > ResourceFilter(ResourceType.GROUP, "foo*")) > > > > > > > >&

Re: [DISCUSS] KIP-290: Support for wildcard suffixed ACLs

2018-05-17 Thread Andy Coates
Hey Piyush - my bad. Sorry. On 17 May 2018 at 13:23, Piyush Vijay wrote: > It's still not complete. I'll drop a message here when I'm done with the > updates. > > Thanks > > > Piyush Vijay > > On Thu, May 17, 2018 at 12:04 PM, Andy Coates wrote: >

Re: [DISCUSS] KIP-290: Support for wildcard suffixed ACLs

2018-05-18 Thread Andy Coates
things today. But that > would still give us 72 hours before the deadline :) > > Thanks > > > Piyush Vijay > > On Thu, May 17, 2018 at 1:27 PM, Andy Coates wrote: > > > Hey Piyush - my bad. Sorry. > > > > On 17 May 2018 at 13:23, Piyush Vijay wrote: &g

Re: [DISCUSS] KIP-290: Support for wildcard suffixed ACLs

2018-05-18 Thread Andy Coates
to > > support snowflakes but I will this better. > > > > I'm giving it a final read. I'll update here once I think it's ready. > > > > Thanks > > > > > > Piyush Vijay > > > > On Fri, May 18, 2018 at 12:18 PM, Piyush Vijay > > wr

Re: [DISCUSS] KIP-290: Support for wildcard suffixed ACLs

2018-05-18 Thread Andy Coates
aulting to wildcard-suffix being on, (better experience going forward), though off is more backwards compatible. 3. Again, examples of how to store ACLs for specific resources that *end* with an asterisk and wildcard-suffix ACLs, with any escaping would really help. On 18 May 2018 at 13:55, Andy C

Re: [DISCUSS] KIP-290: Support for wildcard suffixed ACLs

2018-05-18 Thread Andy Coates
Hi Piyush, We're fast approaching the KIP deadline. Are you actively working on this? If you're not I can take over. Thanks, Andy On 18 May 2018 at 14:25, Andy Coates wrote: > OK I've read it now. > > 1. I see you have an example: > > For example: If I wan

Re: [DISCUSS] KIP-290: Support for wildcard suffixed ACLs

2018-05-20 Thread Andy Coates
; everyone for the valuable feedback. > > > > > Piyush Vijay > > > Piyush Vijay > >> On Fri, May 18, 2018 at 6:07 PM, Andy Coates wrote: >> >> Hi Piyush, >> >> We're fast approaching the KIP deadline. Are you actively working on t

Re: [DISCUSS] KIP-290: Support for wildcard suffixed ACLs

2018-05-21 Thread Andy Coates
+1 (binding) > > Regards, > > Rajini > > On Sun, May 20, 2018 at 2:53 PM, Andy Coates wrote: > > > Awesome last minute effort Piyush. > > > > Really appreciate your time and input, > > > > Andy > > > > Sent from my iPhone > >

Re: [DISCUSS] KIP-290: Support for wildcard suffixed ACLs

2018-05-25 Thread Andy Coates
ementations, e.g. new CaseInsenisticeResourceFilter(new PrefixedResourceFilter("/foo")) to get a case-insensitive prefixed filter. On 22 May 2018 at 05:15, Colin McCabe wrote: > On Mon, May 21, 2018, at 04:53, Andy Coates wrote: > > Hey Piyush, > > > > Thanks

Re: [DISCUSS] KIP-290: Support for wildcard suffixed ACLs

2018-06-13 Thread Andy Coates
Hi All, Just a note to say the KIP documentation has been updated inline with the current implementation. https://cwiki.apache.org/confluence/display/KAFKA/KIP-290%3A+Support+for+Prefixed+ACLs Thanks, Andy On 25 May 2018 at 17:44, Andy Coates wrote: > > Since Resource is a concrete

Permission to create KIP

2018-05-01 Thread Andy Coates
Hi, Can I get permission to add a KIP in Confluence please? My Wiki Id is: bigandy Thanks! Andy

Re: [DISCUSS] KIP-290: Support for wildcard suffixed ACLs

2018-05-01 Thread Andy Coates
Hi Piyush, Thanks for raising this KIP - it's very much appreciated. I've not had chance to digest it yet, but... 1. you might want to add details of how the internals of the `getMatchingAcls` is implemented. We'd want to make sure the complexity of the operation isn't adversely affected. 2. You

Re: [DISCUSS] KIP-290: Support for wildcard suffixed ACLs

2018-05-01 Thread Andy Coates
t comes to making this initiative for users. On 1 May 2018 at 19:57, Andy Coates wrote: > Hi Piyush, > > Thanks for raising this KIP - it's very much appreciated. > > I've not had chance to digest it yet, but... > > 1. you might want to add details of how the internals

[DISCUSS] KIP-292: Add transformValues() method to KTable

2018-05-02 Thread Andy Coates
Hi everyone, I would like to start a discussion for KIP 292. I would appreciate it if you could review and provide feedback. KIP: KIP-292: Add transformValues() method to KTable Jira: KAFKA-6

Re: [DISCUSS] KIP-290: Support for wildcard suffixed ACLs

2018-05-02 Thread Andy Coates
uthorizer instead of replacing the > > > > implementation for simple ACL authorizer with adding the wildcard > > > > capability? > > > > > > > > 2) is there an impact to performance as now we're evaluating more > > rules ? A > > >

Re: [VOTE] KIP-421: Support resolving externalized secrets in AbstractConfig

2019-01-24 Thread Andy Coates
I'm wondering why we're rejected changing AbstractConfig to automatically resolve the variables? > 1. Change AbstractConfig to *automatically* resolve variables of the form specified in KIP-297. This was rejected because it would change the behavior of existing code and might cause unexpected effe

Re: [VOTE] KIP-421: Support resolving externalized secrets in AbstractConfig

2019-01-25 Thread Andy Coates
onnect). > > > > As a practical suggestion, while it doesn't give you the update for free, > > we could consider also deprecating the existing constructor to encourage > > people to update. Further, if we're worried about confusion about how to > > load t

Re: [DISCUSS] KIP-464 Default Replication Factor for AdminClient#createTopic

2019-05-02 Thread Andy Coates
Rather than adding overloaded constructors, which can lead to API bloat, how about using a builder pattern? I see it’s already got some constructor overloading, but we could add a single new constructor that takes just the name, and support everything else being set via builder methods. This w

[DISCUSS] KIP-476: Add Java AdminClient Interface

2019-05-31 Thread Andy Coates
Hi folks, I'd like to start a discussion thread for KIP-476: https://cwiki.apache.org/confluence/display/KAFKA/KIP-476%3A+Add+Java+AdminClient+Interface Thanks, Andy

[VOTE] KIP-476: Add Java AdminClient interface

2019-06-04 Thread Andy Coates
Hi folks As there's been no chatter on this KIP I'm assuming it's non-contentious, (or just boring), hence I'd like to call a vote for KIP-476: https://cwiki.apache.org/confluence/display/KAFKA/KIP-476%3A+Add+Java+AdminClient+Interface Thanks, Andy

Re: [VOTE] KIP-476: Add Java AdminClient interface

2019-06-10 Thread Andy Coates
p the existing public API intact if it's not hurting > anything. > > Also, what will AdminClient.create() return? Would it be a breaking change? > > Ryanne > > On Tue, Jun 4, 2019, 11:17 AM Andy Coates wrote: > >> Hi folks >> >> As there's been

Re: [VOTE] KIP-476: Add Java AdminClient interface

2019-06-11 Thread Andy Coates
that it > breaks every time the upstream project adds an API. In order to support > this, we could have an official DelegatingAdminClient base class that > forwarded every method to an underlying AdminClient instance. Then the > client code could override the methods they needed, li

Re: [VOTE] KIP-476: Add Java AdminClient interface

2019-06-12 Thread Andy Coates
Thanks Great. Do that mean you're a +1? On Tue, 11 Jun 2019 at 21:46, Colin McCabe wrote: > On Tue, Jun 11, 2019, at 12:12, Andy Coates wrote: > > Thanks for the response Colin, > > > > > What specific benefits do we get from transitioning to using an > interfa

Re: [VOTE] KIP-476: Add Java AdminClient interface

2019-06-12 Thread Andy Coates
With 3.0 not imminent I would prefer to make this change soon, rather than later. On Tue, 11 Jun 2019 at 21:46, Colin McCabe wrote: > On Tue, Jun 11, 2019, at 12:12, Andy Coates wrote: > > Thanks for the response Colin, > > > > > What specific benefits do we get from t

Re: [VOTE] KIP-476: Add Java AdminClient interface

2019-06-12 Thread Andy Coates
out classes that implement the > interface? > > > -Matthias > > On 6/10/19 11:22 AM, Andy Coates wrote: > > `AdminClient` would be deprecated purely because it would no longer serve > > any purpose and would be virtually empty, getting all of its > implementation >

Re: [VOTE] KIP-476: Add Java AdminClient interface

2019-06-17 Thread Andy Coates
usecase using dynamic proxies with Admin client > interface. Dynamic proxy can have performance penalty if it is used in > critical path. Is that the primary motivation for creating the KIP? > > Thanks, > Satish. > > On Wed, Jun 12, 2019 at 8:43 PM Andy Coates wrote: > >

Re: [VOTE] KIP-476: Add Java AdminClient interface

2019-06-17 Thread Andy Coates
ry motivation for creating the KIP? > > Thanks, > Satish. > > On Wed, Jun 12, 2019 at 8:43 PM Andy Coates wrote: > > > I'm not married to that part. That was only done to keep it more or less > > inline with what's already there, (an abstract class that

Re: [VOTE] KIP-476: Add Java AdminClient interface

2019-06-18 Thread Andy Coates
yanne > > On Mon, Jun 17, 2019 at 12:09 PM Andy Coates wrote: > > > Hi all, > > > > I think I've addressed all concerns. Let me know if I've not. Can I call > > another round of votes please? > > > > Thanks, > > > > Andy > >

Re: [VOTE] KIP-476: Add Java AdminClient interface

2019-06-21 Thread Andy Coates
uma wrote: > > I don't agree with this change. The idea that an interface cannot have a > default implementation is outdated in my view. Can someone provide any > benefit to introducing a separate class for the factory method? > > Ismael > > On Mon, Jun 17, 201

Re: [VOTE] KIP-476: Add Java AdminClient interface

2019-06-21 Thread Andy Coates
. Andy > On 18 Jun 2019, at 16:09, Ismael Juma wrote: > > I agree with Ryanne, I think we should avoid deprecating AdminClient and > causing so much churn for users who don't actually care about this niche > use case. > > Ismael > > On Tue, Jun 18, 2019 at 6:43

Re: [VOTE] KIP-476: Add Java AdminClient interface

2019-06-24 Thread Andy Coates
low users to specify the kafka clients > version and hence avoid using new classes/interfaces for some time. They > would get a bunch of warnings they cannot do anything about apart from > suppressing. > > Ismael > > On Fri, Jun 21, 2019 at 4:00 AM Andy Coates wrote: > > &

Re: [VOTE] KIP-476: Add Java AdminClient interface

2019-06-24 Thread Andy Coates
on and hence avoid using new classes/interfaces for some time. They > > would get a bunch of warnings they cannot do anything about apart from > > suppressing. > > > > Ismael > > > > On Fri, Jun 21, 2019 at 4:00 AM Andy Coates wrote: > > > >> Hi Is

Re: [VOTE] KIP-476: Add Java AdminClient interface

2019-06-24 Thread Andy Coates
Hi all, KIP updated: - No deprecation - Factory method back onto Admin interface I'd like to kick off another round of voting please. Thanks, Andy On Mon, 24 Jun 2019 at 16:03, Andy Coates wrote: > I agree Matthias. > > (In Scala, such factory methods are on a companion o

Re: [VOTE] KIP-476: Add Java AdminClient interface

2019-07-01 Thread Andy Coates
should not have/expose any > >>> implementation. I am fine with either way as it is more of taste or > >>> preference. > >>> > >>> Agree with Ismael/Colin/Ryanne on not deprecating for good reasons. > >>> > >>> > >>

Re: [VOTE] KIP-476: Add Java AdminClient interface

2019-07-01 Thread Andy Coates
>>> > >>> +1 Matthias/Andy. > >>> IMHO, interface is about the contract, it should not have/expose any > >>> implementation. I am fine with either way as it is more of taste or > >>> preference. > >>> > >>> Agree

Re: [VOTE] KIP-476: Add Java AdminClient interface

2019-07-03 Thread Andy Coates
; > > > > > > > > Can you elaborate? > > > > > > IMHO, just adding a statement to JavaDocs is a little weak, and at some > > > point, we need to deprecate those methods anyway if we ever want to > > > remove them. The earlier we deprecate them, the earlier we can r

Re: [VOTE] KIP-476: Add Java AdminClient interface

2019-07-11 Thread Andy Coates
hods either. > > > +1 (binding) on the current proposal > > > > -Matthias > > On 7/3/19 5:03 AM, Andy Coates wrote: > > Matthias, > > > > I was referring to platforms such as spark or flink that support multiple > > versions of the Kafka clients. Isma

Re: [VOTE] KIP-476: Add Java AdminClient interface

2019-07-12 Thread Andy Coates
wrote: > +1 (binding) > > Thanks for the KIP, Andy! > > Regards, > > Rajini > > > On Thu, Jul 11, 2019 at 1:18 PM Gwen Shapira wrote: > > > +1 (binding) > > > > Thank you for the improvement. > > > > On Thu, Jul 11, 2019, 3:53 AM An

Re: [DISCUSS] KIP-481: SerDe Improvements for Connect Decimal type in JSON

2019-07-25 Thread Andy Coates
Hi Almog, Like the KIP - I think being able to support decimals in JSON in the same way most other systems do is a great improvement. It's not 100% clear to me from the KIP what the current format is. Is the text decimal a base16 encoded number, or is it base16 encoded binary form of the number?

Re: [DISCUSS] KIP-481: SerDe Improvements for Connect Decimal type in JSON

2019-07-29 Thread Andy Coates
schema/JsonSchemaConverter.java#L251-L261 > > >. > > Not everyone is going to have JSONSchemas available when converting, but > if > > you do, it is an easy way to support JSON numbers as decimals. > > > > Carry on! :) > > > > On Thu, Jul 25, 2019

Re: [VOTE] KIP-476: Add Java AdminClient interface

2019-08-06 Thread Andy Coates
://github.com/apache/kafka/pull/7162 Let me know if anyone has any issues / suggestions, etc. Andy On Fri, 12 Jul 2019 at 20:35, Andy Coates wrote: > Awesome sauce - so I'd like to close the voting. final vote was: > > 4 for binding, none against > 3 non-binding, none against. > >

[DISCUSS] KIP-594: Expose output topic names from TopologyTestDriver

2020-04-14 Thread Andy Coates
Hey all, I would like to start off the discussion for KIP-594: https://cwiki.apache.org/confluence/display/KAFKA/KIP-594%3A+Expose+output+topic+names+from+TopologyTestDriver This KIP proposes to expose the names of the topics a topology produces records during a test run from the TopologyTestDrive

[VOTE] KIP-594: Expose output topic names from TopologyTestDriver

2020-04-15 Thread Andy Coates
Hey all, I would like to start the vote for KIP- 594 . Thanks, Andy

Re: [DISCUSS] KIP-594: Expose output topic names from TopologyTestDriver

2020-04-20 Thread Andy Coates
gt; > Anyway, thanks again for the KIP. It does seem useful to me, and I > hope my feedback helps speed you to a successful vote! > > > > > > Thanks, > > > John > > > > > > On Tue, Apr 14, 2020, at 19:49, Matthias J. Sax wrote: > > >> Andy

Re: [VOTE] KIP-594: Expose output topic names from TopologyTestDriver

2020-04-23 Thread Andy Coates
Dump On Thu, 16 Apr 2020 at 00:48, Andy Coates wrote: > Hey all, > > I would like to start the vote for KIP- > <https://cwiki.apache.org/confluence/display/KAFKA/KIP-594%3A+Expose+output+topic+names+from+TopologyTestDriver>594 > > <https://cwiki.apache.org/c

Re: [VOTE] KIP-594: Expose output topic names from TopologyTestDriver

2020-04-23 Thread Andy Coates
inding) for me. > > > > -Bill > > > > On Thu, Apr 23, 2020 at 9:45 AM John Roesler > wrote: > > > > > Hi Andy, > > > > > > I just took another look, and it looks good to me! Thanks for the > > > contribution. > >

Re: [DISCUSS] KIP-598: Augment TopologyDescription with store and source / sink serde information

2020-05-11 Thread Andy Coates
Hi Guozhang, Thanks for writing this up. I’m very interested to see this, so I hope you don’t mind me commenting. I’ve only really one comment to make, and that’s on the text printed for the serde classes: As I understand it, the name will either come from the passed in config, or may default

[jira] [Created] (KAFKA-7005) Remove duplicate Java Resource class.

2018-06-06 Thread Andy Coates (JIRA)
Andy Coates created KAFKA-7005: -- Summary: Remove duplicate Java Resource class. Key: KAFKA-7005 URL: https://issues.apache.org/jira/browse/KAFKA-7005 Project: Kafka Issue Type: Sub-task

[jira] [Created] (KAFKA-7006) Remove duplicate Scala ResourceNameType class.

2018-06-06 Thread Andy Coates (JIRA)
Andy Coates created KAFKA-7006: -- Summary: Remove duplicate Scala ResourceNameType class. Key: KAFKA-7006 URL: https://issues.apache.org/jira/browse/KAFKA-7006 Project: Kafka Issue Type: Sub

[jira] [Created] (KAFKA-7007) All ACL changes should use single /kafka-acl-changes path

2018-06-06 Thread Andy Coates (JIRA)
Andy Coates created KAFKA-7007: -- Summary: All ACL changes should use single /kafka-acl-changes path Key: KAFKA-7007 URL: https://issues.apache.org/jira/browse/KAFKA-7007 Project: Kafka Issue

[jira] [Created] (KAFKA-7008) Consider replacing the Resource field in AclBinding with a ResourceFilter or ResourceMatcher.

2018-06-06 Thread Andy Coates (JIRA)
Andy Coates created KAFKA-7008: -- Summary: Consider replacing the Resource field in AclBinding with a ResourceFilter or ResourceMatcher. Key: KAFKA-7008 URL: https://issues.apache.org/jira/browse/KAFKA-7008

[jira] [Created] (KAFKA-7010) Rename ResourceNameType.ANY to MATCH

2018-06-06 Thread Andy Coates (JIRA)
Andy Coates created KAFKA-7010: -- Summary: Rename ResourceNameType.ANY to MATCH Key: KAFKA-7010 URL: https://issues.apache.org/jira/browse/KAFKA-7010 Project: Kafka Issue Type: Sub-task

[jira] [Created] (KAFKA-7011) Investigate if its possible to drop the ResourceNameType field from Java Resource class.

2018-06-06 Thread Andy Coates (JIRA)
Andy Coates created KAFKA-7011: -- Summary: Investigate if its possible to drop the ResourceNameType field from Java Resource class. Key: KAFKA-7011 URL: https://issues.apache.org/jira/browse/KAFKA-7011

[jira] [Resolved] (KAFKA-7008) Consider replacing the Resource field in AclBinding with a ResourceFilter or ResourceMatcher

2018-06-07 Thread Andy Coates (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-7008?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Andy Coates resolved KAFKA-7008. Resolution: Won't Fix > Consider replacing the Resource field in AclBinding with a Resour

[jira] [Created] (KAFKA-7069) AclCommand does not allow 'create' operation on 'topic'

2018-06-18 Thread Andy Coates (JIRA)
Andy Coates created KAFKA-7069: -- Summary: AclCommand does not allow 'create' operation on 'topic' Key: KAFKA-7069 URL: https://issues.apache.org/jira/browse/KAFKA-7069 Project: K

[jira] [Resolved] (KAFKA-7069) AclCommand does not allow 'create' operation on 'topic'

2018-06-18 Thread Andy Coates (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-7069?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Andy Coates resolved KAFKA-7069. Resolution: Invalid > AclCommand does not allow 'create' operat

[jira] [Created] (KAFKA-6727) org.apache.kafka.clients.admin.Config has broken equals and hashCode method.

2018-03-29 Thread Andy Coates (JIRA)
Andy Coates created KAFKA-6727: -- Summary: org.apache.kafka.clients.admin.Config has broken equals and hashCode method. Key: KAFKA-6727 URL: https://issues.apache.org/jira/browse/KAFKA-6727 Project

[jira] [Created] (KAFKA-6849) Add transformValues() method to KTable

2018-05-02 Thread Andy Coates (JIRA)
Andy Coates created KAFKA-6849: -- Summary: Add transformValues() method to KTable Key: KAFKA-6849 URL: https://issues.apache.org/jira/browse/KAFKA-6849 Project: Kafka Issue Type: Improvement

[jira] [Created] (KAFKA-8454) Add Java AdminClient interface

2019-05-31 Thread Andy Coates (JIRA)
Andy Coates created KAFKA-8454: -- Summary: Add Java AdminClient interface Key: KAFKA-8454 URL: https://issues.apache.org/jira/browse/KAFKA-8454 Project: Kafka Issue Type: Bug

[jira] [Created] (KAFKA-3919) Broker faills to start after ungraceful shutdown due to non-monotonically incrementing offsets in logs

2016-06-29 Thread Andy Coates (JIRA)
Andy Coates created KAFKA-3919: -- Summary: Broker faills to start after ungraceful shutdown due to non-monotonically incrementing offsets in logs Key: KAFKA-3919 URL: https://issues.apache.org/jira/browse/KAFKA-3919

[jira] [Updated] (KAFKA-3919) Broker faills to start after ungraceful shutdown due to non-monotonically incrementing offsets in logs

2016-06-29 Thread Andy Coates (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3919?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Andy Coates updated KAFKA-3919: --- Description: Hi All, I encountered an issue with Kafka following a power outage that saw a

[jira] [Created] (KAFKA-3918) Broker faills to start after ungraceful shutdown due to non-monotonically incrementing offsets in logs

2016-06-29 Thread Andy Coates (JIRA)
Andy Coates created KAFKA-3918: -- Summary: Broker faills to start after ungraceful shutdown due to non-monotonically incrementing offsets in logs Key: KAFKA-3918 URL: https://issues.apache.org/jira/browse/KAFKA-3918

[jira] [Resolved] (KAFKA-3918) Broker faills to start after ungraceful shutdown due to non-monotonically incrementing offsets in logs

2016-06-29 Thread Andy Coates (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3918?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Andy Coates resolved KAFKA-3918. Resolution: Duplicate > Broker faills to start after ungraceful shutdown due to non-monotonica

[jira] [Commented] (KAFKA-3919) Broker faills to start after ungraceful shutdown due to non-monotonically incrementing offsets in logs

2016-06-30 Thread Andy Coates (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3919?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15356679#comment-15356679 ] Andy Coates commented on KAFKA-3919: [~ijuma] - duplicate submission not intenti

[jira] [Updated] (KAFKA-3919) Broker faills to start after ungraceful shutdown due to non-monotonically incrementing offsets in logs

2016-07-01 Thread Andy Coates (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3919?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Andy Coates updated KAFKA-3919: --- Description: Hi All, I encountered an issue with Kafka following a power outage that saw a

[jira] [Commented] (KAFKA-3919) Broker faills to start after ungraceful shutdown due to non-monotonically incrementing offsets in logs

2016-07-01 Thread Andy Coates (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3919?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15358704#comment-15358704 ] Andy Coates commented on KAFKA-3919: Hi [~junrao], thanks for taking the time to

[jira] [Issue Comment Deleted] (KAFKA-3919) Broker faills to start after ungraceful shutdown due to non-monotonically incrementing offsets in logs

2016-07-01 Thread Andy Coates (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3919?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Andy Coates updated KAFKA-3919: --- Comment: was deleted (was: Hi [~junrao], thanks for taking the time to look at this. Note: I&#x

[jira] [Updated] (KAFKA-3919) Broker faills to start after ungraceful shutdown due to non-monotonically incrementing offsets in logs

2016-07-01 Thread Andy Coates (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3919?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Andy Coates updated KAFKA-3919: --- Description: Hi All, I encountered an issue with Kafka following a power outage that saw a

[jira] [Commented] (KAFKA-3919) Broker faills to start after ungraceful shutdown due to non-monotonically incrementing offsets in logs

2016-07-05 Thread Andy Coates (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3919?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15362604#comment-15362604 ] Andy Coates commented on KAFKA-3919: Hi [~junrao], thanks for taking the time to

[jira] [Updated] (KAFKA-3919) Broker faills to start after ungraceful shutdown due to non-monotonically incrementing offsets in logs

2016-07-05 Thread Andy Coates (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3919?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Andy Coates updated KAFKA-3919: --- Description: Hi All, I encountered an issue with Kafka following a power outage that saw a

[jira] [Updated] (KAFKA-3919) Broker faills to start after ungraceful shutdown due to non-monotonically incrementing offsets in logs

2016-07-05 Thread Andy Coates (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3919?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Andy Coates updated KAFKA-3919: --- Description: Hi All, I encountered an issue with Kafka following a power outage that saw a

[jira] [Commented] (KAFKA-3919) Broker faills to start after ungraceful shutdown due to non-monotonically incrementing offsets in logs

2016-07-06 Thread Andy Coates (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3919?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15364141#comment-15364141 ] Andy Coates commented on KAFKA-3919: [~junrao] My understanding was that the of

[jira] [Commented] (KAFKA-3919) Broker faills to start after ungraceful shutdown due to non-monotonically incrementing offsets in logs

2016-07-08 Thread Andy Coates (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3919?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15367489#comment-15367489 ] Andy Coates commented on KAFKA-3919: [~junrao] Yes, we lost a good number of bro

[jira] [Commented] (KAFKA-3919) Broker faills to start after ungraceful shutdown due to non-monotonically incrementing offsets in logs

2016-07-12 Thread Andy Coates (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3919?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15372565#comment-15372565 ] Andy Coates commented on KAFKA-3919: [~junrao] Good stuff. Look forward to hea

[jira] [Commented] (KAFKA-3919) Broker faills to start after ungraceful shutdown due to non-monotonically incrementing offsets in logs

2016-09-26 Thread Andy Coates (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3919?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15522583#comment-15522583 ] Andy Coates commented on KAFKA-3919: We experienced a similar incident again rece

[jira] [Commented] (KAFKA-3919) Broker faills to start after ungraceful shutdown due to non-monotonically incrementing offsets in logs

2016-09-26 Thread Andy Coates (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3919?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15522746#comment-15522746 ] Andy Coates commented on KAFKA-3919: Thanks [~ijuma]. Given that the problems

[jira] [Created] (KAFKA-10494) Streams: enableSendingOldValues should not call parent if node is itself materialized

2020-09-17 Thread Andy Coates (Jira)
Andy Coates created KAFKA-10494: --- Summary: Streams: enableSendingOldValues should not call parent if node is itself materialized Key: KAFKA-10494 URL: https://issues.apache.org/jira/browse/KAFKA-10494

[jira] [Created] (KAFKA-17120) Race condition in KafkaStreams.close can result in StreamsException "Failed to shut down while in state"

2024-07-11 Thread Andy Coates (Jira)
Andy Coates created KAFKA-17120: --- Summary: Race condition in KafkaStreams.close can result in StreamsException "Failed to shut down while in state" Key: KAFKA-17120 URL: https://issues.apache.org/jira/br

[jira] [Created] (KAFKA-9865) Expose output topic names from TopologyTestDriver

2020-04-14 Thread Andy Coates (Jira)
Andy Coates created KAFKA-9865: -- Summary: Expose output topic names from TopologyTestDriver Key: KAFKA-9865 URL: https://issues.apache.org/jira/browse/KAFKA-9865 Project: Kafka Issue Type: Bug

[jira] [Created] (KAFKA-10077) Filter downstream of state-store results in suprious tombstones

2020-06-01 Thread Andy Coates (Jira)
Andy Coates created KAFKA-10077: --- Summary: Filter downstream of state-store results in suprious tombstones Key: KAFKA-10077 URL: https://issues.apache.org/jira/browse/KAFKA-10077 Project: Kafka

[jira] [Created] (KAFKA-14660) Divide by zero security vulnerability

2023-01-30 Thread Andy Coates (Jira)
Andy Coates created KAFKA-14660: --- Summary: Divide by zero security vulnerability Key: KAFKA-14660 URL: https://issues.apache.org/jira/browse/KAFKA-14660 Project: Kafka Issue Type: Bug

[jira] [Reopened] (KAFKA-14660) Divide by zero security vulnerability (sonatype-2019-0422)

2023-02-02 Thread Andy Coates (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-14660?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Andy Coates reopened KAFKA-14660: - The issue here is more the SonaType security vulnerability report than any impossible to reach

[jira] [Resolved] (KAFKA-14660) Divide by zero security vulnerability (sonatype-2019-0422)

2023-02-04 Thread Andy Coates (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-14660?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Andy Coates resolved KAFKA-14660. - Resolution: Fixed > Divide by zero security vulnerability (sonatype-2019-0

[jira] [Created] (KAFKA-9416) Streams get stuck in `PENDING_SHUTDOWN` is underlying topics deleted.

2020-01-13 Thread Andy Coates (Jira)
Andy Coates created KAFKA-9416: -- Summary: Streams get stuck in `PENDING_SHUTDOWN` is underlying topics deleted. Key: KAFKA-9416 URL: https://issues.apache.org/jira/browse/KAFKA-9416 Project: Kafka

[jira] [Created] (KAFKA-9667) Connect JSON serde strip trailing zeros

2020-03-05 Thread Andy Coates (Jira)
Andy Coates created KAFKA-9667: -- Summary: Connect JSON serde strip trailing zeros Key: KAFKA-9667 URL: https://issues.apache.org/jira/browse/KAFKA-9667 Project: Kafka Issue Type: Bug

[jira] [Created] (KAFKA-9668) Iterating over KafkaStreams.getAllMetadata() results in ConcurrentModificationException

2020-03-05 Thread Andy Coates (Jira)
Andy Coates created KAFKA-9668: -- Summary: Iterating over KafkaStreams.getAllMetadata() results in ConcurrentModificationException Key: KAFKA-9668 URL: https://issues.apache.org/jira/browse/KAFKA-9668

  1   2   >