Thanks for updating and improving this KIP, Konstantine! Overall I'm pretty
happy with the KIP, but I have a few fairly nit-picky questions/comments.

1) Will connectors see these configuration properties when Connect calls
Connector.start(...)? We've added a number of configs to connectors that
are handled by the framework, and it'd be good to follow the same
precedence and mention in this KIP what that behavior is.
2) Is it worth mentioning that the replication factor must be no larger
than the number of brokers in the cluster? I'm not sure it is, but it is a
common problem for folks that are less familiar with Kafka semantics. And
because the connector would fail only after trying to create
3) It may be worth mentioning *when* the connector might fail due to
problems with topic creation. It won't fail upon initial deployment, but
rather only if/when the connector writes to a topic that does not exist and
Connect attempts to create that topic.
4) Fourth, what do you think about adding an example that uses Connect's
client override feature by specifying `producer.override.` or
`admin.override.` properties. Using `producer.override.` properties is
sufficient, but `admin.override.` properties will take precedence for the
AdminClient use. (This might make more sense if "Configuration Examples"
appeared after the "Security" section.) Also, the last paragraph in the
"Compatibility" section mentions:

"If ACLs are used, the Kafka principal used in the Connect worker's
producer.* settings is assumed to have privilege..."


This should probably be corrected, since this is not quite the whole story.

5) The "Compatibility" section mentions:

This feature will do nothing if the broker does not support the Admin API
methods, which is equivalent to relying upon auto-topic creation.


Is this really what we want? Should the connector fail since earlier we
mention that we don't want to silently ignore properties?



Best regards,

Randall

On Thu, Dec 12, 2019 at 2:00 PM Konstantine Karantasis <
konstant...@confluent.io> wrote:

> I've taken a second look to KIP-158 after syncing with Randall Hauch, who
> was the original author of the proposal, and I have updated the KIP in
> place.
>
> The main new features of this updated KIP-158 is the introduction of groups
> of configs that can be composed and the ability to match topics to these
> groups via the use of regex. The design builds on top of the existing
> definition of config groups used in single message transformations (SMT)
> and therefore I'm hoping that the approach fits well in Kafka Connect's
> current configuration capabilities.
>
> The new proposal aims to strike a good balance between requiring to
> explicitly set the configs for each possible topic or having a
> one-size-fits-all default set of properties for all the topics a connector
> may create during runtime.
>
>
> The updated KIP-158 can be found under the same page as the old one:
>
>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-158%3A+Kafka+Connect+should+allow+source+connectors+to+set+topic-specific+settings+for+new+topics
>
> I've intentionally changed the title here in this thread to avoid confusion
> with the threads that discussed KIP-158 previously.
> Looking forward to your comments and hoping we can pick up this work from
> the very good starting point that was reached in the previous discussions.
>
>
> Best,
> Konstantine
>

Reply via email to