Hi again,

First of all, I apologize for the duplicate email I did not mean to send 2.

> I added the / dynamic / config `supported.configs` to the KIP. This is a map 
> of client software name and version to a list of supported configs registered 
> with that software name and version. 
> 
> e.g.
> {
> 'ClientInformation(softwareName=apache-kafka-java, 
> softwareVersion=x.y.a-SNAPSHOT)': 'acks',
> 'ClientInformation(softwareName=apache-kafka-java, 
> softwareVersion=x.y.b-SNAPSHOT)': 'acks, enable.idempotence'
> }  

There could be an internal topic for this instead of a dynamic config so that 
ad hoc aggregations can be performed on the config registrations. Would it make 
sense to key config registration by <user, client-id>? This would allow 
compatibility information and the current config values for each client that 
registered with a particular <user, client-id> entity to be aggregated and 
returned to the user in the DescribeClientConfigsResponse. Could clients 
register their configs every time that they update their configs? This way a 
log retention time could be set on the internal topic so that only data from 
active clients is kept.

Best,
Ryan

On 2020/09/08 07:10:52, Ryan Dielher <dielhe...@gmail.com> wrote: 
> Hi Jason,
> 
> Thank you again for all of your feedback, it is greatly appreciated.
> Here are some more changes to the KIP: 
> https://cwiki.apache.org/confluence/pages/diffpagesbyversion.action?pageId=158869615&selectedPageVersions=21&selectedPageVersions=17
> 
> > 1. I wonder if we need to bother with `enable.dynamic.config`, especially
> > if the default is going to be true anyway.
> 
> I removed this / static / config in favor of a / dynamic / config 
> `supported.configs`. This config is explained in more detail below.
> 
> > 3. I'm trying to understand the contract between brokers and clients to
> > support dynamic configurations. I imagine that once this is available,
> > users will have a hard time telling which applications support the
> > capability and which do not. Also, we would likely add new dynamic config
> > support over time which would make this even harder since we cannot
> > retroactively change clients to add support for new dynamic configs. I'm
> > wondering if there is anything we can do to make it easier for users to
> > tell which dynamic configs are available for each application.
> 
> 
> I added the / dynamic / config `supported.configs` to the KIP. This is a map 
> of client software name and version to a list of supported configs registered 
> with that software name and version. 
> 
> e.g.
> {
> 'ClientInformation(softwareName=apache-kafka-java, 
> softwareVersion=x.y.a-SNAPSHOT)': 'acks',
> 'ClientInformation(softwareName=apache-kafka-java, 
> softwareVersion=x.y.b-SNAPSHOT)': 'acks, enable.idempotence'
> }  
> 
> The changes propose that `supported.configs` be serialized as a json and 
> stored in an entity config alongside dynamic configs and quotas for an 
> entity. This config is updated for an entity when producers and consumers 
> register the dynamic configurations that they support with the entity. This 
> registration happens every time that they request configs.
> 
> This makes the information refreshable since `supported.configs` can be 
> deleted from an entity. Clients register configs every time they request 
> configs, so the compatibility information will eventually be added again and 
> continuously updated as new clients associate themselves with the entity.
> 
> > 2. Tying dynamic configurations to clientId has some downsides.
> 
> The new protocols in the changes to the KIP are based on the protocols for 
> client quotas. 
> They allow dynamic configs to be tied to a user principal and optionally to a 
> client-id without trying to fit multiple entity names and types into the 
> {Describe, IncrementalAlter}Configs APIs. They also provide a more expressive 
> and extensible interface for dynamic client config entity names and types.
> 
> > It is common for users to use a different clientId for every application in 
> > a
> > consumer group so that it is easier to tie group members back to where
> > the client is running. This makes setting configurations at an application
> > level cumbersome. The alternative is to use the default, but that means
> > hitting /all/ applications which I think is probably not a good idea. A
> > convenient alternative for consumers would be to use group.id, but we don't
> > have anything similar for the producer. I am wondering if we need to give
> > the clients a separate config label of some kind so that there is a
> > convenient way to group configurations. For example `config.group`. Note
> > that this would be another way to opt into dynamic config support.
> 
> Would it be reasonable to treat consumer entities slightly differently than 
> producer entities since they are different types of clients with different 
> needs?
> Since the design pattern of this new protocol is built in part for 
> extensibility of entity names and types, could consumers be associated with 
> an additional entity name and type? 
> 
> e.g. Consumer configs could be set at the user, group-id and client-id 
> levels. They could be resolved with the following order of precedence from 
> most precedent to least precedent.
> 
> /config/users/<user>/groups/<group-id>/clients/<client-id>
> /config/users/<user>/groups/<group-id>
> /config/users/<user>
> 
> This doesn’t introduce a new label but it would instead allow dynamic 
> consumer configs to be set at the group-id level.
> 
> > 4. In the case of `session.timeout.ms`, even if the config is updated, the
> > group will need to be rebalanced for it to take effect. This is because the
> > session timeout is sent to the group coordinator in the JoinGroup request.
> > I'm wondering if we need to change the JoinGroup behavior so that it can be
> > used to update the session timeout without triggering a rebalance.
> 
> I updated the KIP and PR so that a JoinGroupRequest will not rebalance the 
> group if the session timeout in the JoinGroupRequest is different from the 
> session timeout in the GroupCoordinator’s metadata and if the group member 
> that is rejoining is in a stable state. In practice this just changes the 
> behavior of the leader of the group when the timeout is updated since the 
> leader is the only one that will trigger a rebalance when rejoining while in 
> a stable state. 
> 
> I would greatly appreciate any more feedback from you about these ideas.
> 
> Best,
> Ryan
> 
> On 2020/08/06 00:23:28, Jason Gustafson <ja...@confluent.io> wrote: 
> > Hi Ryan,
> > 
> > Thanks for the proposal. Just a few quick questions:
> > 
> > 1. I wonder if we need to bother with `enable.dynamic.config`, especially
> > if the default is going to be true anyway. I think users who don't want to
> > use this capability can just not set dynamic configs. The only case I can
> > see an explicit opt-out being useful is when users are trying to avoid
> > getting affected by dynamic defaults. And on that note, is there a strong
> > case for supporting default overrides? Many client configs are tied closely
> > to application behavior, so it feels a bit dangerous to give users the
> > ability to override the configuration for all applications.
> > 
> > 2. Tying dynamic configurations to clientId has some downsides. It is
> > common for users to use a different clientId for every application in a
> > consumer group so that it is easier to tie group members back to where
> > the client is running. This makes setting configurations at an application
> > level cumbersome. The alternative is to use the default, but that means
> > hitting /all/ applications which I think is probably not a good idea. A
> > convenient alternative for consumers would be to use group.id, but we don't
> > have anything similar for the producer. I am wondering if we need to give
> > the clients a separate config label of some kind so that there is a
> > convenient way to group configurations. For example `config.group`. Note
> > that this would be another way to opt into dynamic config support.
> > 
> > 3. I'm trying to understand the contract between brokers and clients to
> > support dynamic configurations. I imagine that once this is available,
> > users will have a hard time telling which applications support the
> > capability and which do not. Also, we would likely add new dynamic config
> > support over time which would make this even harder since we cannot
> > retroactively change clients to add support for new dynamic configs. I'm
> > wondering if there is anything we can do to make it easier for users to
> > tell which dynamic configs are available for each application.
> > 
> > 4. In the case of `session.timeout.ms`, even if the config is updated, the
> > group will need to be rebalanced for it to take effect. This is because the
> > session timeout is sent to the group coordinator in the JoinGroup request.
> > I'm wondering if we need to change the JoinGroup behavior so that it can be
> > used to update the session timeout without triggering a rebalance.
> > 
> > Thanks,
> > Jason
> > 
> > 
> > 
> > 
> > On Mon, Aug 3, 2020 at 3:10 PM Ryan Dielhenn <rdielh...@confluent.io> wrote:
> > 
> > > Hi David,
> > >
> > > Here are some additional thoughts...
> > >
> > > > 1. Once dynamic configs have been loaded and resolved, how can a client
> > > > know what values are selected?
> > >
> > > A copy of the original user-provided configs is kept by the client.
> > > Currently these are used to revert to the user-provided config if a 
> > > dynamic
> > > config is deleted. However, they can also be used to distinguish between
> > > dynamic and user-provided configs.
> > >
> > > > 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?
> > >
> > > Rephrasing my answer to this question:
> > >
> > > The mechanism for sending and altering configs is rather generic. However,
> > > the client-side handling of these configs is not. The reason for this is
> > > that configs affect the behavior of the clients in specific ways, so the
> > > client must reconfigure itself in a specific way for each different 
> > > config.
> > >
> > > An example of this is that when session.timeout.ms is dynamically
> > > configured, the consumer must rejoin the group by sending a
> > > JoinGroupRequest. This is because the session timeout is sent in the
> > > initial JoinGroupRequest to the coordinator and stored with the rest of 
> > > the
> > > group member's metadata. To reconfigure the client, the value in the
> > > coordinator must also be changed. This does not need to be done for
> > > heartbeat.interval.ms.
> > >
> > >
> > > 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