Hi David,

Thank you for the comments, they were helpful. Here are the changes I made to 
the KIP: 
https://cwiki.apache.org/confluence/pages/diffpagesbyversion.action?pageId=158869615&selectedPageVersions=12&selectedPageVersions=11

> 1. Once dynamic configs have been loaded and resolved, how can a client
> know what values are selected? Will we log the actual resolved values once
> they are loaded?

Yes, they will be logged if the client determines that they are valid.

> 2. Do we want to support the case that we load dynamic configs but also
> allow client overrides?

I think this would mean that enable.dynamic.config would need to be able to 
take three values instead of just true or false. Since there are so few 
configurations that this KIP provides support for it may make more sense to 
just disable the feature if the user wants the user-provided client configs to 
take precedence. 

> 3. Are there other configs we'd like to allow the broker to push up to the
> clients? Did we consider making this mechanism generic so the broker could
> push any consumer/producer config up to the clients via dynamic configs?

Yes, the mechanism is generic. All that needs to be done to add support for an 
additional dynamic client config is to add that config to this ConfigDef 
https://github.com/apache/kafka/blob/b9cd0fd52741f99fd8f5912435bd2deaca5e5bab/core/src/main/scala/kafka/server/DynamicConfig.scala#L121

Once that is done, kafka-configs.sh will accept that new configuration, the 
broker will store it, and the client(s) will receive it when they pull their 
dynamic configs. Logic will also have to be added to the client to actually use 
this value. 



On 2020/08/03 17:47:19, David Arthur <mum...@gmail.com> wrote: 
> Hey Ryan, thanks for the KIP. This will be a really useful feature. Few
> questions
> 
> 1. Once dynamic configs have been loaded and resolved, how can a client
> know what values are selected? Will we log the actual resolved values once
> they are loaded?
> 
> 2. Do we want to support the case that we load dynamic configs but also
> allow client overrides?
> 
> 3. Are there other configs we'd like to allow the broker to push up to the
> clients? Did we consider making this mechanism generic so the broker could
> push any consumer/producer config up to the clients via dynamic configs?
> 
> Also, minor note, in ConfigResource#Type, I think the byte values are used
> for bit masking somewhere, so 9 won't work -- you'd need to bump up to 16
> 
> -David
> 
> On Fri, Jul 31, 2020 at 6:06 PM Ryan Dielhenn <rdielh...@confluent.io>
> wrote:
> 
> > Thanks Jose,
> > Here are the changes to the KIP:
> > https://cwiki.apache.org/confluence/pages/diffpagesbyversion.action?pageId=158869615&selectedPageVersions=10&selectedPageVersions=9
> >
> > 1. Done.
> > 2. Done.
> > 3. Done.
> > 4. Yes metadata.max.age.ms will be use, I updated the KIP with this.
> >
> > > 5.
> > > The Consumer Changes section mentions that the consumer would ask for
> > > the dynamic configuration from the broker before joining the group
> > > coordinator. This makes sense to me. How about the producer? Should
> > > the producer also describe the dynamic configuration before sending
> > > acks for the "produce" messages?
> >
> >
> > I do think that the producer should do one synchronous
> > DescribeConfigsRequest before producing messages, just for different
> > reasons than the consumer. The decision to do this in the consumer was to
> > avoid an expensive rebalance operation associated with sending an extra
> > join group request.
> >
> > If this were done for the producer it would be to maintain durability
> > semantics by ensuring that the correct dynamic acks value is being used
> > before producing messages, however it would not improve performance like it
> > does for consumer groups.
> >
> > > 6.
> > > For the Admin Client Changes section, how are DescribeConfigs and
> > > IncrementalAlterConfig requests going to get routed by the client to
> > > the different brokers in the cluster?
> >
> > They will all be routed to the least loaded node for the CLIENT resource
> > type. I update the KIP with this.
> >
> > > 7.
> > > You mentioned that the producer and the consumer will validate the
> > > keys and values received from the broker through DescribeConfigs. Will
> > > the ConfigCommand validate any of the keys or values specified in
> > > --add-config and --delete-config? Will the broker validate any of the
> > > keys or values received in the IncrementalAlterConfigs?
> >
> > Yes, ConfigCommand will only allow keys that we are providing support for
> > to be sent to the broker. The broker will validate the keys as well as the
> > values and return an InvalidRequest error code if any key-value pairs are
> > invalid.
> >
> > 8. I removed this line.
> >
> > On 2020/07/31 18:27:38, Jose Garcia Sancio <jsan...@confluent.io> wrote:
> > > Thanks for the KIP Ryan. Here are some of my observations.
> > >
> > > 1.
> > > > We will also be adding a new client configuration
> > enable.dynamic.config to both the producer and consumer that will be true
> > by default so that the user has the option to disable this feature.
> > >
> > > How about?
> > > "The Java producer and consumer clients will have a new configuration
> > > property `enable.dynamic.config` with a default value of `true`. When
> > > this configuration property is true the proposed producer and consumer
> > > changes in this KIP are enabled."
> > >
> > > You can go into more details in the producer and consumer changes
> > section.
> > >
> > > 2.
> > > > ... the following order of precedence:
> > >
> > > I would be a bit more specific here. For example, when the broker
> > > handles `DescribeConfigRequest`, it will first use the client config
> > > key-values store in `/config/clients/<default>`. If there is a
> > > matching `config/clients/<client-id>` then any client config key-value
> > > in the matching znode will override the key-values found in
> > > `config/clients/<default>`.
> > >
> > > 3.
> > > > These dynamic configs will be stored in zookeeper as the children of
> > the ZNode /config/clients
> > >
> > > How are the client dynamic config key-values stored here? I assume
> > > that they are updated based on the content of
> > > `IncrementalAlterConfigsRequest`. When is `.../<default>` updated?
> > > When is `.../<client-id>` updated?
> > >
> > > 4.
> > > > The interval on which dynamic configs are fetched will be the same
> > amount of time as the interval for MetadataRequest which is currently five
> > minutes.
> > >
> > > Will this be hard-coded to 5 minutes? Or is this KIP going to use the
> > > same frequency as the producer config `metadata.max.age.ms`? Same
> > > question for the "Consumer Changes" section.
> > >
> > > 5.
> > > The Consumer Changes section mentions that the consumer would ask for
> > > the dynamic configuration from the broker before joining the group
> > > coordinator. This makes sense to me. How about the producer? Should
> > > the producer also describe the dynamic configuration before sending
> > > acks for the "produce" messages?
> > >
> > > 6.
> > > For the Admin Client Changes section, how are DescribeConfigs and
> > > IncrementalAlterConfig requests going to get routed by the client to
> > > the different brokers in the cluster?
> > >
> > > 7.
> > > You mentioned that the producer and the consumer will validate the
> > > keys and values received from the broker through DescribeConfigs. Will
> > > the ConfigCommand validate any of the keys or values specified in
> > > --add-config and --delete-config? Will the broker validate any of the
> > > keys or values received in the IncrementalAlterConfigs?
> > >
> > > 8.
> > > In rejected ideas the KIP says:
> > > > This might make sense for certain configurations such as acks, but
> > does not for others such as timeouts.
> > >
> > > I don't think it makes sense even for acks since the clients of the
> > > Java Producer assume that all of the produce messages are sent with
> > > the same ack value.
> > >
> > > --
> > > -Jose
> > >
> >
> 
> 
> -- 
> David Arthur
> 

Reply via email to