Re: [VOTE] #2 KIP-248: Create New ConfigCommand That Uses The New AdminClient

2018-05-20 Thread Viktor Somogyi
Hi All,

I'd like to ask the community to please vote for this as the KIP
freeze is tomorrow.

Thank you very much,
Viktor

On Mon, May 21, 2018 at 9:39 AM, Viktor Somogyi  wrote:
> Hi Colin,
>
> Sure, I'll add a note.
> Thanks for your vote.
>
> Viktor
>
> On Sat, May 19, 2018 at 1:01 AM, Colin McCabe  wrote:
>> Hi Viktor,
>>
>> Thanks, this looks good.
>>
>> The boolean should default to false if not set, to ensure that existing 
>> clients continue to work as-is, right?  Might be good to add a note 
>> specifying that.
>>
>> +1 (non-binding)
>>
>> best,
>> Colin
>>
>> On Fri, May 18, 2018, at 08:16, Viktor Somogyi wrote:
>>> Updated KIP-248:
>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-248+-+Create+New+ConfigCommand+That+Uses+The+New+AdminClient
>>>
>>> I'd like to ask project members, committers and contributors to vote
>>> as this would be a useful improvement in Kafka.
>>>
>>> Sections changed:
>>> - Public interfaces: added the bin/scram-credentials.sh command that
>>> we discussed with Colin.
>>> - Wire format types: removed AlterOperations. As discussed with Colin,
>>> we don't actually need this: we should behave in an incremental way in
>>> AlterQuotas. For AlterConfig we'll implement this behavior with an
>>> extra flag on the protocol (and incrementing the version).
>>> - AlterQuotas protocol: removed AlterOperations. Added some more
>>> description to the behavior of the protocol. Removing quotas will
>>> happen by sending a NaN instead of the AlterOperations. (Since IEEE
>>> 754 covers NaNs and it is not a valid config for any quota, I think it
>>> is a good notation.)
>>> - SCRAM: so it will be done by the scram-credentials command that uses
>>> direct zookeeper connection. I think further modes, like doing it
>>> through the broker is not necessary. The idea here is that zookeeper
>>> in this case acts as a credentials store. This should be decoupled
>>> from the broker as we manage broker credentials as well. The new
>>> command acts as a client to the store.
>>> - AlterConfigs will have an incremental_update flag as discussed. By
>>> default it is false to provide the backward compatible behavior. When
>>> it is true it will merge the configs with what's there in the node.
>>> Deletion in incremental mode is done by sending an empty string as
>>> config value.
>>> - Other compatibility changes: this KIP doesn't scope listing multiple
>>> users and client's quotas. As per a conversation with Rajini, it is
>>> not a common use case and we can add it back later if it is needed. If
>>> this functionality is needed, the old code should be still available
>>> through run-kafka-class. (Removed the USE_OLD_KAFKA_CONFIG_COMMAND as
>>> it doesn't make sense anymore.)
>>>
>>> On Fri, May 18, 2018 at 12:33 PM, Viktor Somogyi
>>>  wrote:
>>> > Ok, ignore my previous mail (except the last sentence), gmail didn't
>>> > update me about your last email :/.
>>> >
>>> >> I think we should probably just create a flag for alterConfigs which 
>>> >> marks it as incremental, like we discussed earlier, and do this as a 
>>> >> compatible change that is needed for the shell command.
>>> >
>>> > Alright, I missed that about the sensitive configs too, so in this
>>> > case I can agree with this. I'll update the KIP this afternoon and
>>> > update this thread.
>>> > Thanks again for your contribution.
>>> >
>>> > Viktor
>>> >
>>> > On Fri, May 18, 2018 at 2:34 AM, Colin McCabe  wrote:
>>> >> Actually, I just realized that this won't work.  The AlterConfigs API is 
>>> >> kind of broken right now.  DescribeConfigs won't return the "sensitive" 
>>> >> configurations like passwords.  So doing describe + edit + alter will 
>>> >> wipe out all sensitive configs. :(
>>> >>
>>> >> I think we should probably just create a flag for alterConfigs which 
>>> >> marks it as incremental, like we discussed earlier, and do this as a 
>>> >> compatible change that is needed for the shell command.
>>> >>
>>> >> best,
>>> >> Colin
>>> >>
>>> >>
>>> >> On Thu, May 17, 2018, at 09:32, Colin McCabe wrote:
>>> >>> Hi Viktor,
>>> >>>
>>> >>> Since the KIP freeze is coming up really soon, maybe we should just drop
>>> >>> the section about changes to AlterConfigs from KIP-248.  We don't really
>>> >>> need it here, since ConfigCommand can use AlterConfigs as-is.
>>> >>>
>>> >>> We can pick up the discussion about improving AlterConfigs in a future 
>>> >>> KIP.
>>> >>>
>>> >>> cheers,
>>> >>> Colin
>>> >>>
>>> >>> On Wed, May 16, 2018, at 22:06, Colin McCabe wrote:
>>> >>> > Hi Viktor,
>>> >>> >
>>> >>> > The shell command isn’t that easy to integrate into applications.
>>> >>> > AdminClient will get integrated  into a lot more stuff, which
>>> >>> > increases the potential for conflicts.  I would argue that we should
>>> >>> > fix this soon.
>>> >>> > If we do want to reduce the scope in this KIP, we could do the merge 
>>> >>> > in
>>> >>> > the ConfigCommand  tool for now, and leave AC unchanged.
>>> >>> > Best,
>>> >>> > C

Re: [VOTE] #2 KIP-248: Create New ConfigCommand That Uses The New AdminClient

2018-05-20 Thread Viktor Somogyi
Hi Colin,

Sure, I'll add a note.
Thanks for your vote.

Viktor

On Sat, May 19, 2018 at 1:01 AM, Colin McCabe  wrote:
> Hi Viktor,
>
> Thanks, this looks good.
>
> The boolean should default to false if not set, to ensure that existing 
> clients continue to work as-is, right?  Might be good to add a note 
> specifying that.
>
> +1 (non-binding)
>
> best,
> Colin
>
> On Fri, May 18, 2018, at 08:16, Viktor Somogyi wrote:
>> Updated KIP-248:
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-248+-+Create+New+ConfigCommand+That+Uses+The+New+AdminClient
>>
>> I'd like to ask project members, committers and contributors to vote
>> as this would be a useful improvement in Kafka.
>>
>> Sections changed:
>> - Public interfaces: added the bin/scram-credentials.sh command that
>> we discussed with Colin.
>> - Wire format types: removed AlterOperations. As discussed with Colin,
>> we don't actually need this: we should behave in an incremental way in
>> AlterQuotas. For AlterConfig we'll implement this behavior with an
>> extra flag on the protocol (and incrementing the version).
>> - AlterQuotas protocol: removed AlterOperations. Added some more
>> description to the behavior of the protocol. Removing quotas will
>> happen by sending a NaN instead of the AlterOperations. (Since IEEE
>> 754 covers NaNs and it is not a valid config for any quota, I think it
>> is a good notation.)
>> - SCRAM: so it will be done by the scram-credentials command that uses
>> direct zookeeper connection. I think further modes, like doing it
>> through the broker is not necessary. The idea here is that zookeeper
>> in this case acts as a credentials store. This should be decoupled
>> from the broker as we manage broker credentials as well. The new
>> command acts as a client to the store.
>> - AlterConfigs will have an incremental_update flag as discussed. By
>> default it is false to provide the backward compatible behavior. When
>> it is true it will merge the configs with what's there in the node.
>> Deletion in incremental mode is done by sending an empty string as
>> config value.
>> - Other compatibility changes: this KIP doesn't scope listing multiple
>> users and client's quotas. As per a conversation with Rajini, it is
>> not a common use case and we can add it back later if it is needed. If
>> this functionality is needed, the old code should be still available
>> through run-kafka-class. (Removed the USE_OLD_KAFKA_CONFIG_COMMAND as
>> it doesn't make sense anymore.)
>>
>> On Fri, May 18, 2018 at 12:33 PM, Viktor Somogyi
>>  wrote:
>> > Ok, ignore my previous mail (except the last sentence), gmail didn't
>> > update me about your last email :/.
>> >
>> >> I think we should probably just create a flag for alterConfigs which 
>> >> marks it as incremental, like we discussed earlier, and do this as a 
>> >> compatible change that is needed for the shell command.
>> >
>> > Alright, I missed that about the sensitive configs too, so in this
>> > case I can agree with this. I'll update the KIP this afternoon and
>> > update this thread.
>> > Thanks again for your contribution.
>> >
>> > Viktor
>> >
>> > On Fri, May 18, 2018 at 2:34 AM, Colin McCabe  wrote:
>> >> Actually, I just realized that this won't work.  The AlterConfigs API is 
>> >> kind of broken right now.  DescribeConfigs won't return the "sensitive" 
>> >> configurations like passwords.  So doing describe + edit + alter will 
>> >> wipe out all sensitive configs. :(
>> >>
>> >> I think we should probably just create a flag for alterConfigs which 
>> >> marks it as incremental, like we discussed earlier, and do this as a 
>> >> compatible change that is needed for the shell command.
>> >>
>> >> best,
>> >> Colin
>> >>
>> >>
>> >> On Thu, May 17, 2018, at 09:32, Colin McCabe wrote:
>> >>> Hi Viktor,
>> >>>
>> >>> Since the KIP freeze is coming up really soon, maybe we should just drop
>> >>> the section about changes to AlterConfigs from KIP-248.  We don't really
>> >>> need it here, since ConfigCommand can use AlterConfigs as-is.
>> >>>
>> >>> We can pick up the discussion about improving AlterConfigs in a future 
>> >>> KIP.
>> >>>
>> >>> cheers,
>> >>> Colin
>> >>>
>> >>> On Wed, May 16, 2018, at 22:06, Colin McCabe wrote:
>> >>> > Hi Viktor,
>> >>> >
>> >>> > The shell command isn’t that easy to integrate into applications.
>> >>> > AdminClient will get integrated  into a lot more stuff, which
>> >>> > increases the potential for conflicts.  I would argue that we should
>> >>> > fix this soon.
>> >>> > If we do want to reduce the scope in this KIP, we could do the merge in
>> >>> > the ConfigCommand  tool for now, and leave AC unchanged.
>> >>> > Best,
>> >>> > Colin
>> >>> >
>> >>> >
>> >>> > On Wed, May 16, 2018, at 04:57, Viktor Somogyi wrote:
>> >>> > > Hi Colin,
>> >>> > >
>> >>> > > > Doing get-merge-set is buggy, though.  If someone else does 
>> >>> > > > get-merge-
>> >>> > > > set at the same time as you, you might overwrite that person's
>> >>> > > > cha

[jira] [Created] (KAFKA-6924) Making state store queryable on replicas

2018-05-20 Thread Navinder Brar (JIRA)
Navinder Brar created KAFKA-6924:


 Summary: Making state store queryable on replicas
 Key: KAFKA-6924
 URL: https://issues.apache.org/jira/browse/KAFKA-6924
 Project: Kafka
  Issue Type: New Feature
  Components: streams
Reporter: Navinder Brar


State store in Kafka streams are currently only queryable when StreamTask is in 
RUNNING state. The idea is to make it queryable even for StandbyTasks to 
decrease the downtime if client is not able to fetch data from Active machine.

Suppose the coordinator is not able to connect to machine which had active 
partition due to some reason. So, rather than failing that request we could 
serve request from replica which could be on some other machine. Although state 
on replica might be little behind the active but it could still be beneficial 
in some cases to serve request from replica than failing the request.

It's very important improvement as it could simply improve the availability of 
microservices developed using kafka streams.

I am working on a patch for this change. Any feedback or comments are welcome.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


Re: [VOTE] KIP-255: OAuth Authentication via SASL/OAUTHBEARER

2018-05-20 Thread Ron Dagostino
Hi Committers.  One more binding affirmative vote is required if KIP 255 is to 
have a chance of being included in the 2.0.0 release.  Please vote today.

Ron

> On May 18, 2018, at 9:27 PM, Ron Dagostino  wrote:
> 
> Hi committers.  KIP 255 still needs 1 more binding vote.  Currently there are 
> two binding + 1 votes, from Rajini and Jun, and three non-binding +1 votes, 
> from Mickael, Manikumar, and myself.
> 
> Please vote by the Monday deadline.
> 
> Ron
> 
>> On Thu, May 17, 2018 at 10:59 AM, Ron Dagostino  wrote:
>> Hi Jun.  I've updated the KIP to add a new section titled "Summary for 
>> Production Use" that includes this information along with a consolidated set 
>> of references to the applicable specifications.  Thanks for the questions.
>> 
>> *We still need another binding vote* (currently there are two binding + 1 
>> votes, from Rajini and Jun, and three non-binding +1 votes, from Mickael, 
>> Manikumar, and myself).
>> 
>> Please vote before the May 22nd KIP Freeze deadline so this KIP can be 
>> included in the 2.0.0 release.
>> 
>> A pull request is available and includes additional commits reflecting 
>> initial review comments: https://github.com/apache/kafka/pull/4994
>> 
>> Ron
>> 
>>> On Wed, May 16, 2018 at 8:09 PM, Jun Rao  wrote:
>>> Hi, Ron,
>>> 
>>> Thanks. I understand now. It may be useful to add a reference to JWT in the
>>> KIP.
>>> 
>>> Jun
>>> 
>>> On Tue, May 15, 2018 at 6:51 PM, Ron Dagostino  wrote:
>>> 
>>> > Hi Jun.  I think you are getting at the fact that OAuth 2 is a flexible
>>> > framework that allows different installations to do things differently.  
>>> > It
>>> > is true that the principal name in Kafka could come from any claim in the
>>> > token.  Most of the time it would come from the 'sub' claim, but it could
>>> > certainly come from another claim, or it could be only indirectly based on
>>> > a claim value (maybe certain text would be trimmed or prefixed/suffixed).
>>> > The point, which I think you are getting at, is that because the framework
>>> > is flexible we need to accommodate that flexibility.  The callback handler
>>> > class defined by the listener.name.sasl_ssl.oauthbearer.sasl.server.
>>> > callback.handler.class configuration value gives us the required
>>> > flexibility.  As an example, I have an implementation that leverages a
>>> > popular open source JOSE library to parse the compact serialization,
>>> > retrieve the public key if it has not yet been retrieved, verify the
>>> > digital signature, and map the 'sub' claim to the OAuthBearerToken's
>>> > principal name (which becomes the SASL authorization ID, which becomes the
>>> > Kafka principal name).  I could just as easily have mapped a different
>>> > claim to the OAuthBearerToken's principal name, manipulated the 'sub' 
>>> > claim
>>> > value in some way, etc.  I write the callback handler code, so I complete
>>> > flexibility to do whatever my OAuth 2 installation requires me to do.
>>> >
>>> > Ron
>>> >
>>> > On Tue, May 15, 2018 at 1:39 PM, Jun Rao  wrote:
>>> >
>>> > > Hi, Ron,
>>> > >
>>> > > Thanks for the reply. I understood your answers to #2 and #3.
>>> > >
>>> > > For #1, will the server map all clients' principal name to the value
>>> > > associated with "sub" claim? How do we support mapping different clients
>>> > to
>>> > > different principal names?
>>> > >
>>> > > Jun
>>> > >
>>> > > On Mon, May 14, 2018 at 7:02 PM, Ron Dagostino 
>>> > wrote:
>>> > >
>>> > > > Hi Jun.  Thanks for the +1 vote.
>>> > > >
>>> > > > Regarding the first question about token claims, yes, you have it
>>> > correct
>>> > > > about translating the OAuth token to a principle name via a JAAS 
>>> > > > module
>>> > > > option in the default unsecured case.  Specifically, the OAuth SASL
>>> > > Server
>>> > > > implementation is responsible for setting the authorization ID, and it
>>> > > gets
>>> > > > the authorization ID from the OAuthBearerToken's principalName()
>>> > method.
>>> > > > The listener.name.sasl_ssl.oauthbearer.sasl.server.
>>> > > callback.handler.class
>>> > > > is responsible for handling an instance of 
>>> > > > OAuthBearerValidatorCallback
>>> > > to
>>> > > > accept a token compact serialization from the client and return an
>>> > > instance
>>> > > > of OAuthBearerToken (assuming the compact serialization validates), 
>>> > > > and
>>> > > in
>>> > > > the default unsecured case the builtin unsecured validator callback
>>> > > handler
>>> > > > defines the OAuthBearerToken.principalName() method to return the
>>> > 'sub'
>>> > > > claim value by default (with the actual claim it uses being
>>> > configurable
>>> > > > via the unsecuredValidatorPrincipalClaimName JAAS module option).  So
>>> > > that
>>> > > > is how we translate from a token to a principal name in the default
>>> > > > unsecured (out-of-the-box) case.
>>> > > >
>>> > > > For production use cases, the implementation associated with
>>> > > > listener.name.sasl_ssl.oauthbearer.sasl.server.callback

Build failed in Jenkins: kafka-1.1-jdk7 #135

2018-05-20 Thread Apache Jenkins Server
See 


Changes:

[wangguoz] MINOR: Do not start processor for bounce-at-start (#4639)

--
[...truncated 420.87 KB...]
kafka.tools.ConsoleConsumerTest > 
shouldParseValidNewConsumerConfigWithAutoOffsetResetAndMatchingFromBeginning 
PASSED

kafka.tools.ConsoleConsumerTest > shouldStopWhenOutputCheckErrorFails STARTED

kafka.tools.ConsoleConsumerTest > shouldStopWhenOutputCheckErrorFails PASSED

kafka.tools.ConsoleConsumerTest > 
shouldExitOnInvalidConfigWithAutoOffsetResetAndConflictingFromBeginningNewConsumer
 STARTED

kafka.tools.ConsoleConsumerTest > 
shouldExitOnInvalidConfigWithAutoOffsetResetAndConflictingFromBeginningNewConsumer
 PASSED

kafka.tools.ConsoleConsumerTest > 
shouldSetAutoResetToSmallestWhenFromBeginningConfigured STARTED

kafka.tools.ConsoleConsumerTest > 
shouldSetAutoResetToSmallestWhenFromBeginningConfigured PASSED

kafka.tools.ConsoleConsumerTest > 
shouldParseValidNewConsumerConfigWithAutoOffsetResetEarliest STARTED

kafka.tools.ConsoleConsumerTest > 
shouldParseValidNewConsumerConfigWithAutoOffsetResetEarliest PASSED

kafka.tools.ConsoleConsumerTest > 
shouldParseValidNewSimpleConsumerValidConfigWithStringOffset STARTED

kafka.tools.ConsoleConsumerTest > 
shouldParseValidNewSimpleConsumerValidConfigWithStringOffset PASSED

kafka.tools.ConsoleConsumerTest > 
shouldParseValidOldConsumerConfigWithAutoOffsetResetLargest STARTED

kafka.tools.ConsoleConsumerTest > 
shouldParseValidOldConsumerConfigWithAutoOffsetResetLargest PASSED

kafka.tools.ConsoleConsumerTest > shouldParseConfigsFromFile STARTED

kafka.tools.ConsoleConsumerTest > shouldParseConfigsFromFile PASSED

kafka.tools.ConsoleConsumerTest > groupIdsProvidedInDifferentPlacesMustMatch 
STARTED

kafka.tools.ConsoleConsumerTest > groupIdsProvidedInDifferentPlacesMustMatch 
PASSED

kafka.tools.ConsoleConsumerTest > 
shouldExitOnInvalidConfigWithAutoOffsetResetAndConflictingFromBeginningOldConsumer
 STARTED

kafka.tools.ConsoleConsumerTest > 
shouldExitOnInvalidConfigWithAutoOffsetResetAndConflictingFromBeginningOldConsumer
 PASSED

kafka.tools.ConsoleConsumerTest > 
shouldParseValidNewSimpleConsumerValidConfigWithNumericOffset STARTED

kafka.tools.ConsoleConsumerTest > 
shouldParseValidNewSimpleConsumerValidConfigWithNumericOffset PASSED

kafka.tools.ConsoleConsumerTest > testDefaultConsumer STARTED

kafka.tools.ConsoleConsumerTest > testDefaultConsumer PASSED

kafka.tools.ConsoleConsumerTest > 
shouldParseValidNewConsumerConfigWithAutoOffsetResetLatest STARTED

kafka.tools.ConsoleConsumerTest > 
shouldParseValidNewConsumerConfigWithAutoOffsetResetLatest PASSED

kafka.tools.ConsoleConsumerTest > shouldParseValidOldConsumerValidConfig STARTED

kafka.tools.ConsoleConsumerTest > shouldParseValidOldConsumerValidConfig PASSED

kafka.message.MessageCompressionTest > testCompressSize STARTED

kafka.message.MessageCompressionTest > testCompressSize PASSED

kafka.message.MessageCompressionTest > testSimpleCompressDecompress STARTED

kafka.message.MessageCompressionTest > testSimpleCompressDecompress PASSED

kafka.message.MessageTest > testChecksum STARTED

kafka.message.MessageTest > testChecksum PASSED

kafka.message.MessageTest > testInvalidTimestamp STARTED

kafka.message.MessageTest > testInvalidTimestamp PASSED

kafka.message.MessageTest > testIsHashable STARTED

kafka.message.MessageTest > testIsHashable PASSED

kafka.message.MessageTest > testInvalidTimestampAndMagicValueCombination STARTED

kafka.message.MessageTest > testInvalidTimestampAndMagicValueCombination PASSED

kafka.message.MessageTest > testExceptionMapping STARTED

kafka.message.MessageTest > testExceptionMapping PASSED

kafka.message.MessageTest > testFieldValues STARTED

kafka.message.MessageTest > testFieldValues PASSED

kafka.message.MessageTest > testInvalidMagicByte STARTED

kafka.message.MessageTest > testInvalidMagicByte PASSED

kafka.message.MessageTest > testEquality STARTED

kafka.message.MessageTest > testEquality PASSED

kafka.message.ByteBufferMessageSetTest > testMessageWithProvidedOffsetSeq 
STARTED

kafka.message.ByteBufferMessageSetTest > testMessageWithProvidedOffsetSeq PASSED

kafka.message.ByteBufferMessageSetTest > testValidBytes STARTED

kafka.message.ByteBufferMessageSetTest > testValidBytes PASSED

kafka.message.ByteBufferMessageSetTest > testValidBytesWithCompression STARTED

kafka.message.ByteBufferMessageSetTest > testValidBytesWithCompression PASSED

kafka.message.ByteBufferMessageSetTest > 
testWriteToChannelThatConsumesPartially STARTED

kafka.message.ByteBufferMessageSetTest > 
testWriteToChannelThatConsumesPartially PASSED

kafka.message.ByteBufferMessageSetTest > testIteratorIsConsistent STARTED

kafka.message.ByteBufferMessageSetTest > testIteratorIsConsistent PASSED

kafka.message.ByteBufferMessageSetTest > testWrittenEqualsRead STARTED

kafka.message.ByteBufferMessageSetTest > testWrittenEqualsRead PASSED

kafka.mess

[jira] [Created] (KAFKA-6923) Deprecate ExtendedSerializer/Serializer and ExtendedDeserializer/Deserializer

2018-05-20 Thread Ismael Juma (JIRA)
Ismael Juma created KAFKA-6923:
--

 Summary: Deprecate ExtendedSerializer/Serializer and 
ExtendedDeserializer/Deserializer
 Key: KAFKA-6923
 URL: https://issues.apache.org/jira/browse/KAFKA-6923
 Project: Kafka
  Issue Type: Improvement
  Components: clients
Reporter: Ismael Juma
 Fix For: 2.1.0


The Javadoc of ExtendedDeserializer states:

{code}
 * Prefer {@link Deserializer} if access to the headers is not required. Once 
Kafka drops support for Java 7, the
 * {@code deserialize()} method introduced by this interface will be added to 
Deserializer with a default implementation
 * so that backwards compatibility is maintained. This interface may be 
deprecated once that happens.
{code}

Since we have dropped Java 7 support, we should figure out how to do this. 
There are compatibility implications, so a KIP is needed.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


Re: [DISCUSS] KIP-285: Connect Rest Extension Plugin

2018-05-20 Thread Magesh Nandakumar
Hi Jason,

Thanks a lot for your feedback. The health interface does provide the same
information that's already exposed in the Connect Rest endpoints. Where it
would be useful is if we need to infer certain kind of health status based
on the connect and task status. One good example would be Liveliness probe
for kubernetes. Depending on how critical the connector is one could decide
the liveliness based on various task status. This is something very
specific to SLA requirements for the connector for a user. Hence exposing
this to Extension does allow users to implement healthchecks based on their
requirements. Let me know your thoughts.

Thanks
Magesh

On Fri, May 18, 2018 at 4:05 PM, Jason Gustafson  wrote:

> Hi Magesh,
>
> Thanks for the KIP. It's definitely useful to have a pluggable auth layer,
> as we have for the kafka brokers. I was a bit unclear why we needed to
> expose all this health information in the context though. Since it is the
> bigger part of the API, I was hoping to see a little more motivation for
> why a plugin would need this. Do the current status endpoints not integrate
> well with common alerting systems? Is that something that can be fixed?
>
> Thanks,
> Jason
>
> On Thu, May 17, 2018 at 10:05 PM, Ewen Cheslack-Postava  >
> wrote:
>
> > Yup, thanks for the changes. The 'health' package in particular feels
> like
> > a nice fit given the way we expect it to be used.
> >
> > -Ewen
> >
> > On Wed, May 16, 2018 at 7:02 PM Randall Hauch  wrote:
> >
> > > Looks good to me. Thanks for quickly making the changes! Great work!
> > >
> > > Best regards,
> > >
> > > Randall
> > >
> > > > On May 16, 2018, at 8:07 PM, Magesh Nandakumar  >
> > > wrote:
> > > >
> > > > Randall,
> > > >
> > > > I have adjusted the package names per Ewen's suggestions and also
> made
> > > some
> > > > minor edits per your suggestions. Since there are no major
> outstanding
> > > > issues, i'm moving this to vote.
> > > >
> > > > Thanks
> > > > Magesh
> > > >
> > > >> On Wed, May 16, 2018 at 4:38 PM, Randall Hauch 
> > > wrote:
> > > >>
> > > >> A few very minor suggestions:
> > > >>
> > > >>
> > > >>   1. There are a few formatting issues with paragraphs that use a
> > > >>   monospace font. Minor, but it would be nice to fix.
> > > >>   2. Would be nice to link to the PR
> > > >>   3. Do we need the org.apache.kafka.connect.rest.
> extension.entities
> > > >>   package? Could we just move the two classes into the parent
> > > >>   org.apache.kafka.connect.rest.extension package?
> > > >>   4. This sentence "The above approach helps alleviate any issues
> that
> > > >>   could arise if Extension accidentally reregister the" is cut off.
> > > >>   5. The "ConnectRestExtensionContext.configure(...)" method's
> > JavaDoc
> > > >>   should describe the behaviors that are mentioned in the "Rest
> > > Extension
> > > >>   Integration with Connect" section; e.g., behavior when an
> extension
> > > >> adds a
> > > >>   resource that is already registered, whether unregistering works,
> > etc.
> > > >>   Also, ideally the "close()" method would have JavaDoc that
> explained
> > > >> when
> > > >>   it is called (e.g., no other methods will be called on the
> extension
> > > >> after
> > > >>   this, etc.).
> > > >>   6. Packaging requirements are different for this component vs
> > > >>   connectors, transformations, and converters, since this now
> mandates
> > > the
> > > >>   Service Loader manifest file. This should be called out more
> > > explicitly.
> > > >>   7. It'd be nice if the example included how extension-specific
> > config
> > > >>   properties are to be defined in the worker configuration file.
> > > >>
> > > >> As I said, these are all minor suggestions that only affect the KIP
> > > >> document. Once these are fixed, I think this is ready to move to
> > voting.
> > > >>
> > > >> Best regards,
> > > >>
> > > >> Randall
> > > >>
> > > >> On Tue, May 15, 2018 at 11:30 AM, Magesh Nandakumar <
> > > mage...@confluent.io>
> > > >> wrote:
> > > >>
> > > >>> Randall- I think I have addressed all the comments. Let me know if
> we
> > > can
> > > >>> take this to Vote.
> > > >>>
> > > >>> Thanks
> > > >>> Magesh
> > > >>>
> > > >>> On Tue, May 8, 2018 at 10:12 PM, Magesh Nandakumar <
> > > mage...@confluent.io
> > > >>>
> > > >>> wrote:
> > > >>>
> > >  Hi All,
> > > 
> > >  I have updated the KIP to reflect changes based on the PR
> > >  https://github.com/apache/kafka/pull/4931. Its mostly has minor
> > > >> changes
> > >  to the interfaces and includes details on packages for the
> > interfaces
> > > >> and
> > >  the classes. Let me know your thoughts.
> > > 
> > >  Thanks
> > >  Magesh
> > > 
> > >  On Fri, Apr 27, 2018 at 12:03 PM, Randall Hauch  >
> > > >>> wrote:
> > > 
> > > > Great work, Magesh. I like the overall approach a lot, so I left
> > some
> > > > pretty nuanced comments about specific details.
> > > >
> > > > Best regards,
> > 

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

2018-05-20 Thread Apache Jenkins Server
See 


Changes:

[wangguoz] MINOR: Fix type inference on joins and aggregates (#5019)

--
[...truncated 3.13 MB...]

org.apache.kafka.common.config.ConfigDefTest > testInvalidDefaultString STARTED

org.apache.kafka.common.config.ConfigDefTest > testInvalidDefaultString PASSED

org.apache.kafka.common.config.ConfigDefTest > testSslPasswords STARTED

org.apache.kafka.common.config.ConfigDefTest > testSslPasswords PASSED

org.apache.kafka.common.config.ConfigDefTest > testDynamicUpdateModeInDocs 
STARTED

org.apache.kafka.common.config.ConfigDefTest > testDynamicUpdateModeInDocs 
PASSED

org.apache.kafka.common.config.ConfigDefTest > testBaseConfigDefDependents 
STARTED

org.apache.kafka.common.config.ConfigDefTest > testBaseConfigDefDependents 
PASSED

org.apache.kafka.common.config.ConfigDefTest > testConvertValueToStringClass 
STARTED

org.apache.kafka.common.config.ConfigDefTest > testConvertValueToStringClass 
PASSED

org.apache.kafka.common.config.ConfigDefTest > testConvertValueToStringShort 
STARTED

org.apache.kafka.common.config.ConfigDefTest > testConvertValueToStringShort 
PASSED

org.apache.kafka.common.config.ConfigDefTest > testInvalidDefault STARTED

org.apache.kafka.common.config.ConfigDefTest > testInvalidDefault PASSED

org.apache.kafka.common.config.ConfigDefTest > toRst STARTED

org.apache.kafka.common.config.ConfigDefTest > toRst PASSED

org.apache.kafka.common.config.ConfigDefTest > testCanAddInternalConfig STARTED

org.apache.kafka.common.config.ConfigDefTest > testCanAddInternalConfig PASSED

org.apache.kafka.common.config.ConfigDefTest > testMissingRequired STARTED

org.apache.kafka.common.config.ConfigDefTest > testMissingRequired PASSED

org.apache.kafka.common.config.ConfigDefTest > 
testParsingEmptyDefaultValueForStringFieldShouldSucceed STARTED

org.apache.kafka.common.config.ConfigDefTest > 
testParsingEmptyDefaultValueForStringFieldShouldSucceed PASSED

org.apache.kafka.common.config.ConfigDefTest > testConvertValueToStringPassword 
STARTED

org.apache.kafka.common.config.ConfigDefTest > testConvertValueToStringPassword 
PASSED

org.apache.kafka.common.config.ConfigDefTest > testConvertValueToStringList 
STARTED

org.apache.kafka.common.config.ConfigDefTest > testConvertValueToStringList 
PASSED

org.apache.kafka.common.config.ConfigDefTest > testConvertValueToStringLong 
STARTED

org.apache.kafka.common.config.ConfigDefTest > testConvertValueToStringLong 
PASSED

org.apache.kafka.common.config.ConfigDefTest > testConvertValueToStringBoolean 
STARTED

org.apache.kafka.common.config.ConfigDefTest > testConvertValueToStringBoolean 
PASSED

org.apache.kafka.common.config.ConfigDefTest > testNullDefaultWithValidator 
STARTED

org.apache.kafka.common.config.ConfigDefTest > testNullDefaultWithValidator 
PASSED

org.apache.kafka.common.config.ConfigDefTest > testMissingDependentConfigs 
STARTED

org.apache.kafka.common.config.ConfigDefTest > testMissingDependentConfigs 
PASSED

org.apache.kafka.common.config.ConfigDefTest > testConvertValueToStringDouble 
STARTED

org.apache.kafka.common.config.ConfigDefTest > testConvertValueToStringDouble 
PASSED

org.apache.kafka.common.config.ConfigDefTest > toEnrichedRst STARTED

org.apache.kafka.common.config.ConfigDefTest > toEnrichedRst PASSED

org.apache.kafka.common.config.ConfigDefTest > testDefinedTwice STARTED

org.apache.kafka.common.config.ConfigDefTest > testDefinedTwice PASSED

org.apache.kafka.common.config.ConfigDefTest > testConvertValueToStringString 
STARTED

org.apache.kafka.common.config.ConfigDefTest > testConvertValueToStringString 
PASSED

org.apache.kafka.common.config.ConfigDefTest > testNestedClass STARTED

org.apache.kafka.common.config.ConfigDefTest > testNestedClass PASSED

org.apache.kafka.common.config.ConfigDefTest > testBadInputs STARTED

org.apache.kafka.common.config.ConfigDefTest > testBadInputs PASSED

org.apache.kafka.common.config.ConfigDefTest > testValidateMissingConfigKey 
STARTED

org.apache.kafka.common.config.ConfigDefTest > testValidateMissingConfigKey 
PASSED

org.apache.kafka.common.config.AbstractConfigTest > testOriginalsWithPrefix 
STARTED

org.apache.kafka.common.config.AbstractConfigTest > testOriginalsWithPrefix 
PASSED

org.apache.kafka.common.config.AbstractConfigTest > testConfiguredInstances 
STARTED

org.apache.kafka.common.config.AbstractConfigTest > testConfiguredInstances 
PASSED

org.apache.kafka.common.config.AbstractConfigTest > testEmptyList STARTED

org.apache.kafka.common.config.AbstractConfigTest > testEmptyList PASSED

org.apache.kafka.common.config.AbstractConfigTest > 
testValuesWithSecondaryPrefix STARTED

org.apache.kafka.common.config.AbstractConfigTest > 
testValuesWithSecondaryPrefix PASSED

org.apache.kafka.common.config.AbstractConfigTest > 
testValuesWithPrefixOverride STARTED

org.apache.kafka.common.config.AbstractConfigTest > 
testValuesWithPre

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

2018-05-20 Thread Apache Jenkins Server
See 




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

2018-05-20 Thread Apache Jenkins Server
See 


Changes:

[github] KAFKA-6729: Follow up; disable logging for source KTable. (#5038)

--
[...truncated 424.70 KB...]
kafka.utils.json.JsonValueTest > testDecodeString STARTED

kafka.utils.json.JsonValueTest > testDecodeString PASSED

kafka.utils.json.JsonValueTest > testJsonValueToString STARTED

kafka.utils.json.JsonValueTest > testJsonValueToString PASSED

kafka.utils.json.JsonValueTest > testAsJsonObjectOption STARTED

kafka.utils.json.JsonValueTest > testAsJsonObjectOption PASSED

kafka.utils.json.JsonValueTest > testAsJsonArrayOption STARTED

kafka.utils.json.JsonValueTest > testAsJsonArrayOption PASSED

kafka.utils.json.JsonValueTest > testAsJsonArray STARTED

kafka.utils.json.JsonValueTest > testAsJsonArray PASSED

kafka.utils.json.JsonValueTest > testJsonValueHashCode STARTED

kafka.utils.json.JsonValueTest > testJsonValueHashCode PASSED

kafka.utils.json.JsonValueTest > testDecodeInt STARTED

kafka.utils.json.JsonValueTest > testDecodeInt PASSED

kafka.utils.json.JsonValueTest > testDecodeMap STARTED

kafka.utils.json.JsonValueTest > testDecodeMap PASSED

kafka.utils.json.JsonValueTest > testDecodeSeq STARTED

kafka.utils.json.JsonValueTest > testDecodeSeq PASSED

kafka.utils.json.JsonValueTest > testJsonObjectGet STARTED

kafka.utils.json.JsonValueTest > testJsonObjectGet PASSED

kafka.utils.json.JsonValueTest > testJsonValueEquals STARTED

kafka.utils.json.JsonValueTest > testJsonValueEquals PASSED

kafka.utils.json.JsonValueTest > testJsonArrayIterator STARTED

kafka.utils.json.JsonValueTest > testJsonArrayIterator PASSED

kafka.utils.json.JsonValueTest > testJsonObjectApply STARTED

kafka.utils.json.JsonValueTest > testJsonObjectApply PASSED

kafka.utils.json.JsonValueTest > testDecodeBoolean STARTED

kafka.utils.json.JsonValueTest > testDecodeBoolean PASSED

kafka.utils.PasswordEncoderTest > testEncoderConfigChange STARTED

kafka.utils.PasswordEncoderTest > testEncoderConfigChange PASSED

kafka.utils.PasswordEncoderTest > testEncodeDecodeAlgorithms STARTED

kafka.utils.PasswordEncoderTest > testEncodeDecodeAlgorithms PASSED

kafka.utils.PasswordEncoderTest > testEncodeDecode STARTED

kafka.utils.PasswordEncoderTest > testEncodeDecode PASSED

kafka.producer.AsyncProducerTest > testFailedSendRetryLogic STARTED

kafka.producer.AsyncProducerTest > testFailedSendRetryLogic PASSED

kafka.producer.AsyncProducerTest > testQueueTimeExpired STARTED

kafka.producer.AsyncProducerTest > testQueueTimeExpired PASSED

kafka.producer.AsyncProducerTest > testPartitionAndCollateEvents STARTED

kafka.producer.AsyncProducerTest > testPartitionAndCollateEvents PASSED

kafka.producer.AsyncProducerTest > testBatchSize STARTED

kafka.producer.AsyncProducerTest > testBatchSize PASSED

kafka.producer.AsyncProducerTest > testSerializeEvents STARTED

kafka.producer.AsyncProducerTest > testSerializeEvents PASSED

kafka.producer.AsyncProducerTest > testProducerQueueSize STARTED

kafka.producer.AsyncProducerTest > testProducerQueueSize PASSED

kafka.producer.AsyncProducerTest > testRandomPartitioner STARTED

kafka.producer.AsyncProducerTest > testRandomPartitioner PASSED

kafka.producer.AsyncProducerTest > testInvalidConfiguration STARTED

kafka.producer.AsyncProducerTest > testInvalidConfiguration PASSED

kafka.producer.AsyncProducerTest > testInvalidPartition STARTED

kafka.producer.AsyncProducerTest > testInvalidPartition PASSED

kafka.producer.AsyncProducerTest > testNoBroker STARTED

kafka.producer.AsyncProducerTest > testNoBroker PASSED

kafka.producer.AsyncProducerTest > testProduceAfterClosed STARTED

kafka.producer.AsyncProducerTest > testProduceAfterClosed PASSED

kafka.producer.AsyncProducerTest > testJavaProducer STARTED

kafka.producer.AsyncProducerTest > testJavaProducer PASSED

kafka.producer.AsyncProducerTest > testIncompatibleEncoder STARTED

kafka.producer.AsyncProducerTest > testIncompatibleEncoder PASSED

kafka.producer.SyncProducerTest > testReachableServer STARTED

kafka.producer.SyncProducerTest > testReachableServer PASSED

kafka.producer.SyncProducerTest > testMessageSizeTooLarge STARTED

kafka.producer.SyncProducerTest > testMessageSizeTooLarge PASSED

kafka.producer.SyncProducerTest > testNotEnoughReplicas STARTED

kafka.producer.SyncProducerTest > testNotEnoughReplicas PASSED

kafka.producer.SyncProducerTest > testMessageSizeTooLargeWithAckZero STARTED

kafka.producer.SyncProducerTest > testMessageSizeTooLargeWithAckZero PASSED

kafka.producer.SyncProducerTest > testProducerCanTimeout STARTED

kafka.producer.SyncProducerTest > testProducerCanTimeout PASSED

kafka.producer.SyncProducerTest > testProduceRequestWithNoResponse STARTED

kafka.producer.SyncProducerTest > testProduceRequestWithNoResponse PASSED

kafka.producer.SyncProducerTest > testEmptyProduceRequest STARTED

kafka.producer.SyncProducerTest > testEmptyProduceRequest PASSED

kafka.producer.SyncProducerTest >

Jenkins build is back to normal : kafka-trunk-jdk10 #118

2018-05-20 Thread Apache Jenkins Server
See 




Re: [VOTE] #2 KIP-248: Create New ConfigCommand That Uses The New AdminClient

2018-05-20 Thread Attila Sasvári
+1 (non-binding)

On Fri, May 18, 2018 at 5:16 PM, Viktor Somogyi 
wrote:

> Updated KIP-248:
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-248+-+Create+New+
> ConfigCommand+That+Uses+The+New+AdminClient
>
> I'd like to ask project members, committers and contributors to vote
> as this would be a useful improvement in Kafka.
>
> Sections changed:
> - Public interfaces: added the bin/scram-credentials.sh command that
> we discussed with Colin.
> - Wire format types: removed AlterOperations. As discussed with Colin,
> we don't actually need this: we should behave in an incremental way in
> AlterQuotas. For AlterConfig we'll implement this behavior with an
> extra flag on the protocol (and incrementing the version).
> - AlterQuotas protocol: removed AlterOperations. Added some more
> description to the behavior of the protocol. Removing quotas will
> happen by sending a NaN instead of the AlterOperations. (Since IEEE
> 754 covers NaNs and it is not a valid config for any quota, I think it
> is a good notation.)
> - SCRAM: so it will be done by the scram-credentials command that uses
> direct zookeeper connection. I think further modes, like doing it
> through the broker is not necessary. The idea here is that zookeeper
> in this case acts as a credentials store. This should be decoupled
> from the broker as we manage broker credentials as well. The new
> command acts as a client to the store.
> - AlterConfigs will have an incremental_update flag as discussed. By
> default it is false to provide the backward compatible behavior. When
> it is true it will merge the configs with what's there in the node.
> Deletion in incremental mode is done by sending an empty string as
> config value.
> - Other compatibility changes: this KIP doesn't scope listing multiple
> users and client's quotas. As per a conversation with Rajini, it is
> not a common use case and we can add it back later if it is needed. If
> this functionality is needed, the old code should be still available
> through run-kafka-class. (Removed the USE_OLD_KAFKA_CONFIG_COMMAND as
> it doesn't make sense anymore.)
>
> On Fri, May 18, 2018 at 12:33 PM, Viktor Somogyi
>  wrote:
> > Ok, ignore my previous mail (except the last sentence), gmail didn't
> > update me about your last email :/.
> >
> >> I think we should probably just create a flag for alterConfigs which
> marks it as incremental, like we discussed earlier, and do this as a
> compatible change that is needed for the shell command.
> >
> > Alright, I missed that about the sensitive configs too, so in this
> > case I can agree with this. I'll update the KIP this afternoon and
> > update this thread.
> > Thanks again for your contribution.
> >
> > Viktor
> >
> > On Fri, May 18, 2018 at 2:34 AM, Colin McCabe 
> wrote:
> >> Actually, I just realized that this won't work.  The AlterConfigs API
> is kind of broken right now.  DescribeConfigs won't return the "sensitive"
> configurations like passwords.  So doing describe + edit + alter will wipe
> out all sensitive configs. :(
> >>
> >> I think we should probably just create a flag for alterConfigs which
> marks it as incremental, like we discussed earlier, and do this as a
> compatible change that is needed for the shell command.
> >>
> >> best,
> >> Colin
> >>
> >>
> >> On Thu, May 17, 2018, at 09:32, Colin McCabe wrote:
> >>> Hi Viktor,
> >>>
> >>> Since the KIP freeze is coming up really soon, maybe we should just
> drop
> >>> the section about changes to AlterConfigs from KIP-248.  We don't
> really
> >>> need it here, since ConfigCommand can use AlterConfigs as-is.
> >>>
> >>> We can pick up the discussion about improving AlterConfigs in a future
> KIP.
> >>>
> >>> cheers,
> >>> Colin
> >>>
> >>> On Wed, May 16, 2018, at 22:06, Colin McCabe wrote:
> >>> > Hi Viktor,
> >>> >
> >>> > The shell command isn’t that easy to integrate into applications.
> >>> > AdminClient will get integrated  into a lot more stuff, which
> >>> > increases the potential for conflicts.  I would argue that we should
> >>> > fix this soon.
> >>> > If we do want to reduce the scope in this KIP, we could do the merge
> in
> >>> > the ConfigCommand  tool for now, and leave AC unchanged.
> >>> > Best,
> >>> > Colin
> >>> >
> >>> >
> >>> > On Wed, May 16, 2018, at 04:57, Viktor Somogyi wrote:
> >>> > > Hi Colin,
> >>> > >
> >>> > > > Doing get-merge-set is buggy, though.  If someone else does
> get-merge-
> >>> > > > set at the same time as you, you might overwrite that person's
> >>> > > > changes, or vice versa.  So I really don't think we should try
> to do
> >>> > > > this.  Also, having both an incremental and a full API is useful,
> >>> > > > and it's just a single boolean at the protocol and API level.>
> >>> > > Overwriting somebody's change is currently possible with the
> >>> > > ConfigCommand, as it will do this get-merge-set behavior on the
> client> side, in the command. From this perspective I think it's not much
> >>> > > different to do this with the 

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

2018-05-20 Thread Andy Coates
Awesome last minute effort Piyush.

Really appreciate your time and input,

Andy

Sent from my iPhone

> On 19 May 2018, at 03:43, Piyush Vijay  wrote:
> 
> Updated the KIP.
> 
> 1. New enum field 'ResourceNameType' in Resource and ResourceFilter classes.
> 2. modify getAcls() and rely on ResourceNameType' field in Resource to
> return either exact matches or all matches based on wildcard-suffix.
> 3. CLI changes to identify if resource name is literal or wildcard-suffix
> 4. Escaping doesn't work and isn't required if we're keeping a separate
> path on ZK (kafka-wildcard-acl) to store wildcard-suffix ACLs.
> 5. New API keys for Create / Delete / Describe Acls request with a new
> field in schemas for 'ResourceNameType'.
> 
> Looks ready to me for the vote, will start voting thread now. Thanks
> 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 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 want to fetch all ACLs that match ’topicA*’, it’s not
>>> possible without introducing new API AND maintaining backwards
>>> compatibility.
>>> getAcls takes a Resource, right, which would be either a full resource
>>> name or 'ALL', i.e. '*', right?  The point of the call is to get all ACLs
>>> relating to a specific resource, not a partial resource like 'topicA*'.
>>> Currently, I'm guessing / half-remembering that if you ask it for ACLs
>> for
>>> topic 'foo' it doesn't include global 'ALL' ACLs in the list - that would
>>> be a different call.  With the introduction of partial wildcards I think
>>> the _most_ backwards compatible change would be to have
>>> getAcls("topic:foo") to return all the ACLs, including that affect this
>>> topic. This could include any '*'/ALL Acls, (which would be a small
>>> backwards compatible change), or exclude them as it current does.
>>> Excluding any matching partial wildcard acl, e.g. 'f*' would break
>>> compatibility IMHO.
>>> 
>>> 2. Example command lines, showing how to add ACLs to specific resources
>>> that *end* with an asterisk char and adding wildcard-suffixed ACLs, would
>>> really help clarify the KIP. e.g.
>>> 
>>> bin/kafka-acls.sh --authorizer-properties zookeeper.connect=localhost:
>> 2181
>>> --add --allow-principal User:Bob --allow-principal User:Alice
>> --allow-host
>>> 198.51.100.0 --allow-host 198.51.100.1 --operation Read --group my-app-*
>>> 
>>> With the above command I can't see how the code can know if the user
>> means
>>> a literal group called 'my-app-*', or a wildcard suffix for any group
>>> starting with 'my-app-'. Escaping isn't enough as the escape char can
>> clash
>>> too, e.g. escaping a literal to 'my-app-\*' can still clash with someone
>>> wanting a wildcard sufiix matching any group starting with 'my-app-\'.
>>> 
>>> So there needs to be a syntax change here, I think.  Maybe some new
>>> command line switch to either explicitly enable or disable
>>> 'wildcard-suffix' support?  Probably defaulting 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 Coates  wrote:
 
 Hey Piyush,
 
 Thanks for getting this in! :D
 
 About to read now. But just quickly...
 
 1. I'll read up on the need for getMatchingAcls - but just playing
>> devils
 advocate for a moment - if a current caller of getAcls() expects it to
 return the full set of ACLs for a given resource, would post this change
 only returning a sub set and requiring them to return getMatchingAcls to
 get the full set not itself be a break in compatibility? I'm thinking
>> about
 any tooling / UI / etc people may have built on top of this.  If Im
>> missing
 the point, then maybe a concrete example, (if you've not already added
>> one
 to the doc), may help.
 
 2. Something must change on the command line, surely? e.g. as command
 line user how would the command differ if I wanted to add an ACL onto a
 group called 'foo*' as opposed to a all groups starting with 'foo'?
 
 3. Thinking this through, I actually bracktracked - I don't think it
>> will
 work due to path collisions, even with escaping - as the escaped version
 can still collide.
 
 Off to read the doc now.
 
> On 18 May 2018 at 13:33, Piyush Vijay  wrote:
> 
> Ready to review. Let me know if something looks missing or not clear.
> 
> Thanks
> 
> 
> Piyush Vijay
>