Re: [DISCUSS] KIP-411: Add option to make Kafka Connect task client ID values unique

2019-04-12 Thread Ryanne Dolan
Thanks for the reminder.

+1 (non-binding)

Ryanne

On Fri, Apr 12, 2019, 11:13 AM Paul Davidson
 wrote:

> Hi everyone. Just a reminder that KIP-411 is now open for voting. No votes
> received yet!
>
> Thanks,
>
> Paul
>
> On Thu, Apr 4, 2019 at 9:09 AM pdavidson  wrote:
>
> > Thanks Randall.  You're absolutely right that Worker creates the clients
> > before passing them to the tasks, so I'm very happy with your changes.
> >
> > Paul
> >
> > On Thu, Apr 4, 2019 at 8:02 AM Randall Hauch  wrote:
> >
> >> Sounds great.
> >>
> >> I did make a few minor grammatical edits to the "Proposed Changes"
> section
> >> to avoid the notion that the sink and source tasks create the consumers
> >> and
> >> producers, respectively. I think it's important to accurately denote
> that
> >> the framework creates the producers and consumers for the tasks. (This
> in
> >> no way changes the proposal at all, and feel free to roll back if you
> >> disagree with the changes. I felt it was easier to change than to
> >> explain.)
> >>
> >> Looking forward to a vote.
> >>
> >> Best regards,
> >>
> >> Randall
> >>
> >> On Wed, Apr 3, 2019 at 6:49 PM pdavidson  >> .invalid>
> >> wrote:
> >>
> >> > Thanks Randall, I updated the proposal as suggested. Let me know if
> any
> >> > other changes need to be made, otherwise I think the KIP-411 proposal
> is
> >> > ready to finalize.  I will aim to call a vote on Friday.
> >> >
> >> > On Mon, Mar 25, 2019 at 7:12 AM Ryanne Dolan 
> >> > wrote:
> >> >
> >> > > Randall, Paul, the proposal looks great, thanks.
> >> > >
> >> > > Ryanne
> >> > >
> >> > > On Mon, Mar 25, 2019, 9:03 AM Randall Hauch 
> wrote:
> >> > >
> >> > > > Paul,
> >> > > >
> >> > > > Thanks for updating the KIP with the proposal. I do think the KIP
> >> > should
> >> > > at
> >> > > > least mention that the prior behavior is to allow the worker to
> >> > override
> >> > > > the `producer.client.id` or `consumer.client.id`, which is
> entirely
> >> > > > possible (though unlikely since there would be an MBean conflict,
> as
> >> > > > pointed out in the discussion). It might be sufficient to just
> add a
> >> > > > sentence to the "Compatibility, Deprecation, and Migration Plan"
> >> > section,
> >> > > > like "Any client IDs specified in the worker configuration via `
> >> > > > producer.client.id` or `consumer.client.id` properties will be
> >> > > unchanged,
> >> > > > as those will take precedence." Thoughts?
> >> > > >
> >> > > > Ryanne,
> >> > > >
> >> > > > IIUC your last message, I think the latest KIP proposal will align
> >> > pretty
> >> > > > closely with your suggestion. Can you review and confirm?
> >> > > >
> >> > > > Best regards,
> >> > > >
> >> > > > Randall
> >> > > >
> >> > > > On Fri, Mar 1, 2019 at 3:04 PM Ryanne Dolan <
> ryannedo...@gmail.com>
> >> > > wrote:
> >> > > >
> >> > > > > Paul, Randall, I don't think most people will care to exercise
> so
> >> > much
> >> > > > > control over the client IDs, so long as they are filled in
> >> > > automatically
> >> > > > in
> >> > > > > a way that eliminates duplicate metrics and remains somewhat
> >> legible.
> >> > > If
> >> > > > we
> >> > > > > let the user specify a pattern or something, we're really just
> >> making
> >> > > the
> >> > > > > user worry about these requirements.
> >> > > > >
> >> > > > > For example, if they specify "foo" as the client.id, they'll
> get
> >> a
> >> > > bunch
> >> > > > > of
> >> > > > > exceptions about that MBean already existing. So they'll try
> >> > > > > "${connectorName}-foo", which won't work because connectors that
> >> get
> >> > > > > restarted will re-use the same client ID and the same MBean
> again.
> >> > And
> >> > > so
> >> > > > > on, until they end up solving the same problem we are trying to
> >> solve
> >> > > > here.
> >> > > > >
> >> > > > > I think you at least need something like
> >> > > "connect--producer-dlq"
> >> > > > to
> >> > > > > avoid MBeans being re-registered within the same JVM. I believe
> >> the
> >> > > task
> >> > > > ID
> >> > > > > is based on the connector name, so you'd get e.g.
> >> > > > > "connect-myconnector-1-producer".
> >> > > > >
> >> > > > > Ryanne
> >> > > > >
> >> > > > >
> >> > > > > On Fri, Mar 1, 2019 at 12:44 PM Paul Davidson
> >> > > > >  wrote:
> >> > > > >
> >> > > > > > Thanks Randall.  I like your suggestion: as you say, this
> would
> >> > make
> >> > > it
> >> > > > > > possible to usefully override the default client id
> properties.
> >> > > > > >
> >> > > > > > I'm not sure how we would handle the dead-letter queue case
> >> though
> >> > -
> >> > > > > maybe
> >> > > > > > we could automatically add a "dlq-" prefix to the producer
> >> client
> >> > id?
> >> > > > > >
> >> > > > > > If there is agreement on this change I will update the KIP and
> >> the
> >> > PR
> >> > > > > (when
> >> > > > > > I find some time).
> >> > > > > >
> >> > > > > >
> >> > > > > > On Thu, Feb 21, 2019 at 8:12 AM Randall Hauch <
> rha...@gmail.com
> >> >
> >> > > > wrote:
> 

Re: [DISCUSS] KIP-411: Add option to make Kafka Connect task client ID values unique

2019-04-12 Thread Paul Davidson
Hi everyone. Just a reminder that KIP-411 is now open for voting. No votes
received yet!

Thanks,

Paul

On Thu, Apr 4, 2019 at 9:09 AM pdavidson  wrote:

> Thanks Randall.  You're absolutely right that Worker creates the clients
> before passing them to the tasks, so I'm very happy with your changes.
>
> Paul
>
> On Thu, Apr 4, 2019 at 8:02 AM Randall Hauch  wrote:
>
>> Sounds great.
>>
>> I did make a few minor grammatical edits to the "Proposed Changes" section
>> to avoid the notion that the sink and source tasks create the consumers
>> and
>> producers, respectively. I think it's important to accurately denote that
>> the framework creates the producers and consumers for the tasks. (This in
>> no way changes the proposal at all, and feel free to roll back if you
>> disagree with the changes. I felt it was easier to change than to
>> explain.)
>>
>> Looking forward to a vote.
>>
>> Best regards,
>>
>> Randall
>>
>> On Wed, Apr 3, 2019 at 6:49 PM pdavidson > .invalid>
>> wrote:
>>
>> > Thanks Randall, I updated the proposal as suggested. Let me know if any
>> > other changes need to be made, otherwise I think the KIP-411 proposal is
>> > ready to finalize.  I will aim to call a vote on Friday.
>> >
>> > On Mon, Mar 25, 2019 at 7:12 AM Ryanne Dolan 
>> > wrote:
>> >
>> > > Randall, Paul, the proposal looks great, thanks.
>> > >
>> > > Ryanne
>> > >
>> > > On Mon, Mar 25, 2019, 9:03 AM Randall Hauch  wrote:
>> > >
>> > > > Paul,
>> > > >
>> > > > Thanks for updating the KIP with the proposal. I do think the KIP
>> > should
>> > > at
>> > > > least mention that the prior behavior is to allow the worker to
>> > override
>> > > > the `producer.client.id` or `consumer.client.id`, which is entirely
>> > > > possible (though unlikely since there would be an MBean conflict, as
>> > > > pointed out in the discussion). It might be sufficient to just add a
>> > > > sentence to the "Compatibility, Deprecation, and Migration Plan"
>> > section,
>> > > > like "Any client IDs specified in the worker configuration via `
>> > > > producer.client.id` or `consumer.client.id` properties will be
>> > > unchanged,
>> > > > as those will take precedence." Thoughts?
>> > > >
>> > > > Ryanne,
>> > > >
>> > > > IIUC your last message, I think the latest KIP proposal will align
>> > pretty
>> > > > closely with your suggestion. Can you review and confirm?
>> > > >
>> > > > Best regards,
>> > > >
>> > > > Randall
>> > > >
>> > > > On Fri, Mar 1, 2019 at 3:04 PM Ryanne Dolan 
>> > > wrote:
>> > > >
>> > > > > Paul, Randall, I don't think most people will care to exercise so
>> > much
>> > > > > control over the client IDs, so long as they are filled in
>> > > automatically
>> > > > in
>> > > > > a way that eliminates duplicate metrics and remains somewhat
>> legible.
>> > > If
>> > > > we
>> > > > > let the user specify a pattern or something, we're really just
>> making
>> > > the
>> > > > > user worry about these requirements.
>> > > > >
>> > > > > For example, if they specify "foo" as the client.id, they'll get
>> a
>> > > bunch
>> > > > > of
>> > > > > exceptions about that MBean already existing. So they'll try
>> > > > > "${connectorName}-foo", which won't work because connectors that
>> get
>> > > > > restarted will re-use the same client ID and the same MBean again.
>> > And
>> > > so
>> > > > > on, until they end up solving the same problem we are trying to
>> solve
>> > > > here.
>> > > > >
>> > > > > I think you at least need something like
>> > > "connect--producer-dlq"
>> > > > to
>> > > > > avoid MBeans being re-registered within the same JVM. I believe
>> the
>> > > task
>> > > > ID
>> > > > > is based on the connector name, so you'd get e.g.
>> > > > > "connect-myconnector-1-producer".
>> > > > >
>> > > > > Ryanne
>> > > > >
>> > > > >
>> > > > > On Fri, Mar 1, 2019 at 12:44 PM Paul Davidson
>> > > > >  wrote:
>> > > > >
>> > > > > > Thanks Randall.  I like your suggestion: as you say, this would
>> > make
>> > > it
>> > > > > > possible to usefully override the default client id properties.
>> > > > > >
>> > > > > > I'm not sure how we would handle the dead-letter queue case
>> though
>> > -
>> > > > > maybe
>> > > > > > we could automatically add a "dlq-" prefix to the producer
>> client
>> > id?
>> > > > > >
>> > > > > > If there is agreement on this change I will update the KIP and
>> the
>> > PR
>> > > > > (when
>> > > > > > I find some time).
>> > > > > >
>> > > > > >
>> > > > > > On Thu, Feb 21, 2019 at 8:12 AM Randall Hauch > >
>> > > > wrote:
>> > > > > >
>> > > > > > > Hi, Paul. Thanks for the update to KIP-411 to reflect adding
>> > > > defaults,
>> > > > > > and
>> > > > > > > creating/updating https://github.com/apache/kafka/pull/6097
>> to
>> > > > reflect
>> > > > > > > this
>> > > > > > > approach.
>> > > > > > >
>> > > > > > > Now that we've avoided adding a new config and have changed
>> the
>> > > > > default `
>> > > > > > > client.id` to include some context, the connector name, and
>> task
>> > 

Re: [DISCUSS] KIP-411: Add option to make Kafka Connect task client ID values unique

2019-04-04 Thread pdavidson
Thanks Randall.  You're absolutely right that Worker creates the clients
before passing them to the tasks, so I'm very happy with your changes.

Paul

On Thu, Apr 4, 2019 at 8:02 AM Randall Hauch  wrote:

> Sounds great.
>
> I did make a few minor grammatical edits to the "Proposed Changes" section
> to avoid the notion that the sink and source tasks create the consumers and
> producers, respectively. I think it's important to accurately denote that
> the framework creates the producers and consumers for the tasks. (This in
> no way changes the proposal at all, and feel free to roll back if you
> disagree with the changes. I felt it was easier to change than to explain.)
>
> Looking forward to a vote.
>
> Best regards,
>
> Randall
>
> On Wed, Apr 3, 2019 at 6:49 PM pdavidson  .invalid>
> wrote:
>
> > Thanks Randall, I updated the proposal as suggested. Let me know if any
> > other changes need to be made, otherwise I think the KIP-411 proposal is
> > ready to finalize.  I will aim to call a vote on Friday.
> >
> > On Mon, Mar 25, 2019 at 7:12 AM Ryanne Dolan 
> > wrote:
> >
> > > Randall, Paul, the proposal looks great, thanks.
> > >
> > > Ryanne
> > >
> > > On Mon, Mar 25, 2019, 9:03 AM Randall Hauch  wrote:
> > >
> > > > Paul,
> > > >
> > > > Thanks for updating the KIP with the proposal. I do think the KIP
> > should
> > > at
> > > > least mention that the prior behavior is to allow the worker to
> > override
> > > > the `producer.client.id` or `consumer.client.id`, which is entirely
> > > > possible (though unlikely since there would be an MBean conflict, as
> > > > pointed out in the discussion). It might be sufficient to just add a
> > > > sentence to the "Compatibility, Deprecation, and Migration Plan"
> > section,
> > > > like "Any client IDs specified in the worker configuration via `
> > > > producer.client.id` or `consumer.client.id` properties will be
> > > unchanged,
> > > > as those will take precedence." Thoughts?
> > > >
> > > > Ryanne,
> > > >
> > > > IIUC your last message, I think the latest KIP proposal will align
> > pretty
> > > > closely with your suggestion. Can you review and confirm?
> > > >
> > > > Best regards,
> > > >
> > > > Randall
> > > >
> > > > On Fri, Mar 1, 2019 at 3:04 PM Ryanne Dolan 
> > > wrote:
> > > >
> > > > > Paul, Randall, I don't think most people will care to exercise so
> > much
> > > > > control over the client IDs, so long as they are filled in
> > > automatically
> > > > in
> > > > > a way that eliminates duplicate metrics and remains somewhat
> legible.
> > > If
> > > > we
> > > > > let the user specify a pattern or something, we're really just
> making
> > > the
> > > > > user worry about these requirements.
> > > > >
> > > > > For example, if they specify "foo" as the client.id, they'll get a
> > > bunch
> > > > > of
> > > > > exceptions about that MBean already existing. So they'll try
> > > > > "${connectorName}-foo", which won't work because connectors that
> get
> > > > > restarted will re-use the same client ID and the same MBean again.
> > And
> > > so
> > > > > on, until they end up solving the same problem we are trying to
> solve
> > > > here.
> > > > >
> > > > > I think you at least need something like
> > > "connect--producer-dlq"
> > > > to
> > > > > avoid MBeans being re-registered within the same JVM. I believe the
> > > task
> > > > ID
> > > > > is based on the connector name, so you'd get e.g.
> > > > > "connect-myconnector-1-producer".
> > > > >
> > > > > Ryanne
> > > > >
> > > > >
> > > > > On Fri, Mar 1, 2019 at 12:44 PM Paul Davidson
> > > > >  wrote:
> > > > >
> > > > > > Thanks Randall.  I like your suggestion: as you say, this would
> > make
> > > it
> > > > > > possible to usefully override the default client id properties.
> > > > > >
> > > > > > I'm not sure how we would handle the dead-letter queue case
> though
> > -
> > > > > maybe
> > > > > > we could automatically add a "dlq-" prefix to the producer client
> > id?
> > > > > >
> > > > > > If there is agreement on this change I will update the KIP and
> the
> > PR
> > > > > (when
> > > > > > I find some time).
> > > > > >
> > > > > >
> > > > > > On Thu, Feb 21, 2019 at 8:12 AM Randall Hauch 
> > > > wrote:
> > > > > >
> > > > > > > Hi, Paul. Thanks for the update to KIP-411 to reflect adding
> > > > defaults,
> > > > > > and
> > > > > > > creating/updating https://github.com/apache/kafka/pull/6097 to
> > > > reflect
> > > > > > > this
> > > > > > > approach.
> > > > > > >
> > > > > > > Now that we've avoided adding a new config and have changed the
> > > > > default `
> > > > > > > client.id` to include some context, the connector name, and
> task
> > > > > > number, I
> > > > > > > think it makes overriding the client ID via worker config `
> > > > > > > producer.client.id` or `consumer.client.id` properties less
> > > valuable
> > > > > > > because those overridden client IDs will be exactly the same
> for
> > > all
> > > > > > > connectors and tasks.
> > > > > > >
> > > 

Re: [DISCUSS] KIP-411: Add option to make Kafka Connect task client ID values unique

2019-04-04 Thread Randall Hauch
Sounds great.

I did make a few minor grammatical edits to the "Proposed Changes" section
to avoid the notion that the sink and source tasks create the consumers and
producers, respectively. I think it's important to accurately denote that
the framework creates the producers and consumers for the tasks. (This in
no way changes the proposal at all, and feel free to roll back if you
disagree with the changes. I felt it was easier to change than to explain.)

Looking forward to a vote.

Best regards,

Randall

On Wed, Apr 3, 2019 at 6:49 PM pdavidson 
wrote:

> Thanks Randall, I updated the proposal as suggested. Let me know if any
> other changes need to be made, otherwise I think the KIP-411 proposal is
> ready to finalize.  I will aim to call a vote on Friday.
>
> On Mon, Mar 25, 2019 at 7:12 AM Ryanne Dolan 
> wrote:
>
> > Randall, Paul, the proposal looks great, thanks.
> >
> > Ryanne
> >
> > On Mon, Mar 25, 2019, 9:03 AM Randall Hauch  wrote:
> >
> > > Paul,
> > >
> > > Thanks for updating the KIP with the proposal. I do think the KIP
> should
> > at
> > > least mention that the prior behavior is to allow the worker to
> override
> > > the `producer.client.id` or `consumer.client.id`, which is entirely
> > > possible (though unlikely since there would be an MBean conflict, as
> > > pointed out in the discussion). It might be sufficient to just add a
> > > sentence to the "Compatibility, Deprecation, and Migration Plan"
> section,
> > > like "Any client IDs specified in the worker configuration via `
> > > producer.client.id` or `consumer.client.id` properties will be
> > unchanged,
> > > as those will take precedence." Thoughts?
> > >
> > > Ryanne,
> > >
> > > IIUC your last message, I think the latest KIP proposal will align
> pretty
> > > closely with your suggestion. Can you review and confirm?
> > >
> > > Best regards,
> > >
> > > Randall
> > >
> > > On Fri, Mar 1, 2019 at 3:04 PM Ryanne Dolan 
> > wrote:
> > >
> > > > Paul, Randall, I don't think most people will care to exercise so
> much
> > > > control over the client IDs, so long as they are filled in
> > automatically
> > > in
> > > > a way that eliminates duplicate metrics and remains somewhat legible.
> > If
> > > we
> > > > let the user specify a pattern or something, we're really just making
> > the
> > > > user worry about these requirements.
> > > >
> > > > For example, if they specify "foo" as the client.id, they'll get a
> > bunch
> > > > of
> > > > exceptions about that MBean already existing. So they'll try
> > > > "${connectorName}-foo", which won't work because connectors that get
> > > > restarted will re-use the same client ID and the same MBean again.
> And
> > so
> > > > on, until they end up solving the same problem we are trying to solve
> > > here.
> > > >
> > > > I think you at least need something like
> > "connect--producer-dlq"
> > > to
> > > > avoid MBeans being re-registered within the same JVM. I believe the
> > task
> > > ID
> > > > is based on the connector name, so you'd get e.g.
> > > > "connect-myconnector-1-producer".
> > > >
> > > > Ryanne
> > > >
> > > >
> > > > On Fri, Mar 1, 2019 at 12:44 PM Paul Davidson
> > > >  wrote:
> > > >
> > > > > Thanks Randall.  I like your suggestion: as you say, this would
> make
> > it
> > > > > possible to usefully override the default client id properties.
> > > > >
> > > > > I'm not sure how we would handle the dead-letter queue case though
> -
> > > > maybe
> > > > > we could automatically add a "dlq-" prefix to the producer client
> id?
> > > > >
> > > > > If there is agreement on this change I will update the KIP and the
> PR
> > > > (when
> > > > > I find some time).
> > > > >
> > > > >
> > > > > On Thu, Feb 21, 2019 at 8:12 AM Randall Hauch 
> > > wrote:
> > > > >
> > > > > > Hi, Paul. Thanks for the update to KIP-411 to reflect adding
> > > defaults,
> > > > > and
> > > > > > creating/updating https://github.com/apache/kafka/pull/6097 to
> > > reflect
> > > > > > this
> > > > > > approach.
> > > > > >
> > > > > > Now that we've avoided adding a new config and have changed the
> > > > default `
> > > > > > client.id` to include some context, the connector name, and task
> > > > > number, I
> > > > > > think it makes overriding the client ID via worker config `
> > > > > > producer.client.id` or `consumer.client.id` properties less
> > valuable
> > > > > > because those overridden client IDs will be exactly the same for
> > all
> > > > > > connectors and tasks.
> > > > > >
> > > > > > One one hand, we can leave this as-is, and any users that
> include `
> > > > > > producer.client.id` and `consumer.client.id` in their worker
> > configs
> > > > > keep
> > > > > > the same (sort of useless) behavior. In fact, most users would
> > > probably
> > > > > be
> > > > > > better off by removing these worker config properties and instead
> > > > relying
> > > > > > upon the defaults.
> > > > > >
> > > > > > On the other, similar to what Ewen suggested earlier (in a
> > different

Re: [DISCUSS] KIP-411: Add option to make Kafka Connect task client ID values unique

2019-04-03 Thread pdavidson
Thanks Randall, I updated the proposal as suggested. Let me know if any
other changes need to be made, otherwise I think the KIP-411 proposal is
ready to finalize.  I will aim to call a vote on Friday.

On Mon, Mar 25, 2019 at 7:12 AM Ryanne Dolan  wrote:

> Randall, Paul, the proposal looks great, thanks.
>
> Ryanne
>
> On Mon, Mar 25, 2019, 9:03 AM Randall Hauch  wrote:
>
> > Paul,
> >
> > Thanks for updating the KIP with the proposal. I do think the KIP should
> at
> > least mention that the prior behavior is to allow the worker to override
> > the `producer.client.id` or `consumer.client.id`, which is entirely
> > possible (though unlikely since there would be an MBean conflict, as
> > pointed out in the discussion). It might be sufficient to just add a
> > sentence to the "Compatibility, Deprecation, and Migration Plan" section,
> > like "Any client IDs specified in the worker configuration via `
> > producer.client.id` or `consumer.client.id` properties will be
> unchanged,
> > as those will take precedence." Thoughts?
> >
> > Ryanne,
> >
> > IIUC your last message, I think the latest KIP proposal will align pretty
> > closely with your suggestion. Can you review and confirm?
> >
> > Best regards,
> >
> > Randall
> >
> > On Fri, Mar 1, 2019 at 3:04 PM Ryanne Dolan 
> wrote:
> >
> > > Paul, Randall, I don't think most people will care to exercise so much
> > > control over the client IDs, so long as they are filled in
> automatically
> > in
> > > a way that eliminates duplicate metrics and remains somewhat legible.
> If
> > we
> > > let the user specify a pattern or something, we're really just making
> the
> > > user worry about these requirements.
> > >
> > > For example, if they specify "foo" as the client.id, they'll get a
> bunch
> > > of
> > > exceptions about that MBean already existing. So they'll try
> > > "${connectorName}-foo", which won't work because connectors that get
> > > restarted will re-use the same client ID and the same MBean again. And
> so
> > > on, until they end up solving the same problem we are trying to solve
> > here.
> > >
> > > I think you at least need something like
> "connect--producer-dlq"
> > to
> > > avoid MBeans being re-registered within the same JVM. I believe the
> task
> > ID
> > > is based on the connector name, so you'd get e.g.
> > > "connect-myconnector-1-producer".
> > >
> > > Ryanne
> > >
> > >
> > > On Fri, Mar 1, 2019 at 12:44 PM Paul Davidson
> > >  wrote:
> > >
> > > > Thanks Randall.  I like your suggestion: as you say, this would make
> it
> > > > possible to usefully override the default client id properties.
> > > >
> > > > I'm not sure how we would handle the dead-letter queue case though -
> > > maybe
> > > > we could automatically add a "dlq-" prefix to the producer client id?
> > > >
> > > > If there is agreement on this change I will update the KIP and the PR
> > > (when
> > > > I find some time).
> > > >
> > > >
> > > > On Thu, Feb 21, 2019 at 8:12 AM Randall Hauch 
> > wrote:
> > > >
> > > > > Hi, Paul. Thanks for the update to KIP-411 to reflect adding
> > defaults,
> > > > and
> > > > > creating/updating https://github.com/apache/kafka/pull/6097 to
> > reflect
> > > > > this
> > > > > approach.
> > > > >
> > > > > Now that we've avoided adding a new config and have changed the
> > > default `
> > > > > client.id` to include some context, the connector name, and task
> > > > number, I
> > > > > think it makes overriding the client ID via worker config `
> > > > > producer.client.id` or `consumer.client.id` properties less
> valuable
> > > > > because those overridden client IDs will be exactly the same for
> all
> > > > > connectors and tasks.
> > > > >
> > > > > One one hand, we can leave this as-is, and any users that include `
> > > > > producer.client.id` and `consumer.client.id` in their worker
> configs
> > > > keep
> > > > > the same (sort of useless) behavior. In fact, most users would
> > probably
> > > > be
> > > > > better off by removing these worker config properties and instead
> > > relying
> > > > > upon the defaults.
> > > > >
> > > > > On the other, similar to what Ewen suggested earlier (in a
> different
> > > > > context), we could add support for users to optionally use
> > > > > "${connectorName}" and ${task}" in their overridden client ID
> > property
> > > > and
> > > > > have Connect replace these (if found) with the connector name and
> > task
> > > > > number. Any existing properties that don't use these variables
> would
> > > > behave
> > > > > as-is, but this way the users could define their own client IDs yet
> > > still
> > > > > get the benefit of uniquely identifying each of the clients. For
> > > example,
> > > > > if my worker config contained the following:
> > > > >
> > > > > producer.client.id
> > > > =connect-cluster-A-${connectorName}-${task}-producer
> > > > > consumer.client.id
> > > > =connect-cluster-A-${connectorName}-${task}-consumer
> > > > >
> > > > > Thoughts?
> > > > >
> > > > > 

Re: [DISCUSS] KIP-411: Add option to make Kafka Connect task client ID values unique

2019-03-25 Thread Randall Hauch
Paul,

Thanks for updating the KIP with the proposal. I do think the KIP should at
least mention that the prior behavior is to allow the worker to override
the `producer.client.id` or `consumer.client.id`, which is entirely
possible (though unlikely since there would be an MBean conflict, as
pointed out in the discussion). It might be sufficient to just add a
sentence to the "Compatibility, Deprecation, and Migration Plan" section,
like "Any client IDs specified in the worker configuration via `
producer.client.id` or `consumer.client.id` properties will be unchanged,
as those will take precedence." Thoughts?

Ryanne,

IIUC your last message, I think the latest KIP proposal will align pretty
closely with your suggestion. Can you review and confirm?

Best regards,

Randall

On Fri, Mar 1, 2019 at 3:04 PM Ryanne Dolan  wrote:

> Paul, Randall, I don't think most people will care to exercise so much
> control over the client IDs, so long as they are filled in automatically in
> a way that eliminates duplicate metrics and remains somewhat legible. If we
> let the user specify a pattern or something, we're really just making the
> user worry about these requirements.
>
> For example, if they specify "foo" as the client.id, they'll get a bunch
> of
> exceptions about that MBean already existing. So they'll try
> "${connectorName}-foo", which won't work because connectors that get
> restarted will re-use the same client ID and the same MBean again. And so
> on, until they end up solving the same problem we are trying to solve here.
>
> I think you at least need something like "connect--producer-dlq" to
> avoid MBeans being re-registered within the same JVM. I believe the task ID
> is based on the connector name, so you'd get e.g.
> "connect-myconnector-1-producer".
>
> Ryanne
>
>
> On Fri, Mar 1, 2019 at 12:44 PM Paul Davidson
>  wrote:
>
> > Thanks Randall.  I like your suggestion: as you say, this would make it
> > possible to usefully override the default client id properties.
> >
> > I'm not sure how we would handle the dead-letter queue case though -
> maybe
> > we could automatically add a "dlq-" prefix to the producer client id?
> >
> > If there is agreement on this change I will update the KIP and the PR
> (when
> > I find some time).
> >
> >
> > On Thu, Feb 21, 2019 at 8:12 AM Randall Hauch  wrote:
> >
> > > Hi, Paul. Thanks for the update to KIP-411 to reflect adding defaults,
> > and
> > > creating/updating https://github.com/apache/kafka/pull/6097 to reflect
> > > this
> > > approach.
> > >
> > > Now that we've avoided adding a new config and have changed the
> default `
> > > client.id` to include some context, the connector name, and task
> > number, I
> > > think it makes overriding the client ID via worker config `
> > > producer.client.id` or `consumer.client.id` properties less valuable
> > > because those overridden client IDs will be exactly the same for all
> > > connectors and tasks.
> > >
> > > One one hand, we can leave this as-is, and any users that include `
> > > producer.client.id` and `consumer.client.id` in their worker configs
> > keep
> > > the same (sort of useless) behavior. In fact, most users would probably
> > be
> > > better off by removing these worker config properties and instead
> relying
> > > upon the defaults.
> > >
> > > On the other, similar to what Ewen suggested earlier (in a different
> > > context), we could add support for users to optionally use
> > > "${connectorName}" and ${task}" in their overridden client ID property
> > and
> > > have Connect replace these (if found) with the connector name and task
> > > number. Any existing properties that don't use these variables would
> > behave
> > > as-is, but this way the users could define their own client IDs yet
> still
> > > get the benefit of uniquely identifying each of the clients. For
> example,
> > > if my worker config contained the following:
> > >
> > > producer.client.id
> > =connect-cluster-A-${connectorName}-${task}-producer
> > > consumer.client.id
> > =connect-cluster-A-${connectorName}-${task}-consumer
> > >
> > > Thoughts?
> > >
> > > Randall
> > >
> > > On Wed, Feb 20, 2019 at 3:18 PM Ryanne Dolan 
> > > wrote:
> > >
> > > > Thanks Paul, this is great. This will make monitoring Connect a ton
> > > easier.
> > > >
> > > > Ryanne
> > > >
> > > > On Wed, Feb 20, 2019 at 1:24 PM Paul Davidson
> > > >  wrote:
> > > >
> > > > > I have updated KIP-411 to propose changing the default client id -
> > see:
> > > > >
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-411%3A+Make+default+Kafka+Connect+worker+task+client+IDs+distinct
> > > > >
> > > > >
> > > > > There is also an PR ready to go here:
> > > > > https://github.com/apache/kafka/pull/6097
> > > > >
> > > > > On Fri, Jan 11, 2019 at 3:39 PM Paul Davidson <
> > > pdavid...@salesforce.com>
> > > > > wrote:
> > > > >
> > > > > > Hi everyone.  We seem to have agreement that the ideal approach
> is
> > to
> > 

Re: [DISCUSS] KIP-411: Add option to make Kafka Connect task client ID values unique

2019-03-25 Thread Ryanne Dolan
Randall, Paul, the proposal looks great, thanks.

Ryanne

On Mon, Mar 25, 2019, 9:03 AM Randall Hauch  wrote:

> Paul,
>
> Thanks for updating the KIP with the proposal. I do think the KIP should at
> least mention that the prior behavior is to allow the worker to override
> the `producer.client.id` or `consumer.client.id`, which is entirely
> possible (though unlikely since there would be an MBean conflict, as
> pointed out in the discussion). It might be sufficient to just add a
> sentence to the "Compatibility, Deprecation, and Migration Plan" section,
> like "Any client IDs specified in the worker configuration via `
> producer.client.id` or `consumer.client.id` properties will be unchanged,
> as those will take precedence." Thoughts?
>
> Ryanne,
>
> IIUC your last message, I think the latest KIP proposal will align pretty
> closely with your suggestion. Can you review and confirm?
>
> Best regards,
>
> Randall
>
> On Fri, Mar 1, 2019 at 3:04 PM Ryanne Dolan  wrote:
>
> > Paul, Randall, I don't think most people will care to exercise so much
> > control over the client IDs, so long as they are filled in automatically
> in
> > a way that eliminates duplicate metrics and remains somewhat legible. If
> we
> > let the user specify a pattern or something, we're really just making the
> > user worry about these requirements.
> >
> > For example, if they specify "foo" as the client.id, they'll get a bunch
> > of
> > exceptions about that MBean already existing. So they'll try
> > "${connectorName}-foo", which won't work because connectors that get
> > restarted will re-use the same client ID and the same MBean again. And so
> > on, until they end up solving the same problem we are trying to solve
> here.
> >
> > I think you at least need something like "connect--producer-dlq"
> to
> > avoid MBeans being re-registered within the same JVM. I believe the task
> ID
> > is based on the connector name, so you'd get e.g.
> > "connect-myconnector-1-producer".
> >
> > Ryanne
> >
> >
> > On Fri, Mar 1, 2019 at 12:44 PM Paul Davidson
> >  wrote:
> >
> > > Thanks Randall.  I like your suggestion: as you say, this would make it
> > > possible to usefully override the default client id properties.
> > >
> > > I'm not sure how we would handle the dead-letter queue case though -
> > maybe
> > > we could automatically add a "dlq-" prefix to the producer client id?
> > >
> > > If there is agreement on this change I will update the KIP and the PR
> > (when
> > > I find some time).
> > >
> > >
> > > On Thu, Feb 21, 2019 at 8:12 AM Randall Hauch 
> wrote:
> > >
> > > > Hi, Paul. Thanks for the update to KIP-411 to reflect adding
> defaults,
> > > and
> > > > creating/updating https://github.com/apache/kafka/pull/6097 to
> reflect
> > > > this
> > > > approach.
> > > >
> > > > Now that we've avoided adding a new config and have changed the
> > default `
> > > > client.id` to include some context, the connector name, and task
> > > number, I
> > > > think it makes overriding the client ID via worker config `
> > > > producer.client.id` or `consumer.client.id` properties less valuable
> > > > because those overridden client IDs will be exactly the same for all
> > > > connectors and tasks.
> > > >
> > > > One one hand, we can leave this as-is, and any users that include `
> > > > producer.client.id` and `consumer.client.id` in their worker configs
> > > keep
> > > > the same (sort of useless) behavior. In fact, most users would
> probably
> > > be
> > > > better off by removing these worker config properties and instead
> > relying
> > > > upon the defaults.
> > > >
> > > > On the other, similar to what Ewen suggested earlier (in a different
> > > > context), we could add support for users to optionally use
> > > > "${connectorName}" and ${task}" in their overridden client ID
> property
> > > and
> > > > have Connect replace these (if found) with the connector name and
> task
> > > > number. Any existing properties that don't use these variables would
> > > behave
> > > > as-is, but this way the users could define their own client IDs yet
> > still
> > > > get the benefit of uniquely identifying each of the clients. For
> > example,
> > > > if my worker config contained the following:
> > > >
> > > > producer.client.id
> > > =connect-cluster-A-${connectorName}-${task}-producer
> > > > consumer.client.id
> > > =connect-cluster-A-${connectorName}-${task}-consumer
> > > >
> > > > Thoughts?
> > > >
> > > > Randall
> > > >
> > > > On Wed, Feb 20, 2019 at 3:18 PM Ryanne Dolan 
> > > > wrote:
> > > >
> > > > > Thanks Paul, this is great. This will make monitoring Connect a ton
> > > > easier.
> > > > >
> > > > > Ryanne
> > > > >
> > > > > On Wed, Feb 20, 2019 at 1:24 PM Paul Davidson
> > > > >  wrote:
> > > > >
> > > > > > I have updated KIP-411 to propose changing the default client id
> -
> > > see:
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> 

Re: [DISCUSS] KIP-411: Add option to make Kafka Connect task client ID values unique

2019-03-01 Thread Ryanne Dolan
Paul, Randall, I don't think most people will care to exercise so much
control over the client IDs, so long as they are filled in automatically in
a way that eliminates duplicate metrics and remains somewhat legible. If we
let the user specify a pattern or something, we're really just making the
user worry about these requirements.

For example, if they specify "foo" as the client.id, they'll get a bunch of
exceptions about that MBean already existing. So they'll try
"${connectorName}-foo", which won't work because connectors that get
restarted will re-use the same client ID and the same MBean again. And so
on, until they end up solving the same problem we are trying to solve here.

I think you at least need something like "connect--producer-dlq" to
avoid MBeans being re-registered within the same JVM. I believe the task ID
is based on the connector name, so you'd get e.g.
"connect-myconnector-1-producer".

Ryanne


On Fri, Mar 1, 2019 at 12:44 PM Paul Davidson
 wrote:

> Thanks Randall.  I like your suggestion: as you say, this would make it
> possible to usefully override the default client id properties.
>
> I'm not sure how we would handle the dead-letter queue case though - maybe
> we could automatically add a "dlq-" prefix to the producer client id?
>
> If there is agreement on this change I will update the KIP and the PR (when
> I find some time).
>
>
> On Thu, Feb 21, 2019 at 8:12 AM Randall Hauch  wrote:
>
> > Hi, Paul. Thanks for the update to KIP-411 to reflect adding defaults,
> and
> > creating/updating https://github.com/apache/kafka/pull/6097 to reflect
> > this
> > approach.
> >
> > Now that we've avoided adding a new config and have changed the default `
> > client.id` to include some context, the connector name, and task
> number, I
> > think it makes overriding the client ID via worker config `
> > producer.client.id` or `consumer.client.id` properties less valuable
> > because those overridden client IDs will be exactly the same for all
> > connectors and tasks.
> >
> > One one hand, we can leave this as-is, and any users that include `
> > producer.client.id` and `consumer.client.id` in their worker configs
> keep
> > the same (sort of useless) behavior. In fact, most users would probably
> be
> > better off by removing these worker config properties and instead relying
> > upon the defaults.
> >
> > On the other, similar to what Ewen suggested earlier (in a different
> > context), we could add support for users to optionally use
> > "${connectorName}" and ${task}" in their overridden client ID property
> and
> > have Connect replace these (if found) with the connector name and task
> > number. Any existing properties that don't use these variables would
> behave
> > as-is, but this way the users could define their own client IDs yet still
> > get the benefit of uniquely identifying each of the clients. For example,
> > if my worker config contained the following:
> >
> > producer.client.id
> =connect-cluster-A-${connectorName}-${task}-producer
> > consumer.client.id
> =connect-cluster-A-${connectorName}-${task}-consumer
> >
> > Thoughts?
> >
> > Randall
> >
> > On Wed, Feb 20, 2019 at 3:18 PM Ryanne Dolan 
> > wrote:
> >
> > > Thanks Paul, this is great. This will make monitoring Connect a ton
> > easier.
> > >
> > > Ryanne
> > >
> > > On Wed, Feb 20, 2019 at 1:24 PM Paul Davidson
> > >  wrote:
> > >
> > > > I have updated KIP-411 to propose changing the default client id -
> see:
> > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-411%3A+Make+default+Kafka+Connect+worker+task+client+IDs+distinct
> > > >
> > > >
> > > > There is also an PR ready to go here:
> > > > https://github.com/apache/kafka/pull/6097
> > > >
> > > > On Fri, Jan 11, 2019 at 3:39 PM Paul Davidson <
> > pdavid...@salesforce.com>
> > > > wrote:
> > > >
> > > > > Hi everyone.  We seem to have agreement that the ideal approach is
> to
> > > > > alter the default client ids. Now I'm wondering about the best
> > process
> > > to
> > > > > proceed. Will the change in default behaviour require a new KIP,
> > given
> > > it
> > > > > will affect existing deployments?  Would I be best to repurpose
> this
> > > > > KIP-411, or am I best to  create a new KIP? Thanks!
> > > > >
> > > > > Paul
> > > > >
> > > > > On Tue, Jan 8, 2019 at 7:16 PM Randall Hauch 
> > wrote:
> > > > >
> > > > >> Hi, Paul.
> > > > >>
> > > > >> I concur with the others, and I like the new approach that avoids
> a
> > > new
> > > > >> configuration, especially because it does not change the behavior
> > for
> > > > >> anyone already using `producer.client.id` and/or `
> > consumer.client.id
> > > `.
> > > > I
> > > > >> did leave a few comments on the PR. Perhaps the biggest one is
> > whether
> > > > the
> > > > >> producer used for the sink task error reporter (for dead letter
> > queue)
> > > > >> should be `connector-producer-`, and whether that is
> > > > >> distinct
> > > > >> enough from source tasks, which 

Re: [DISCUSS] KIP-411: Add option to make Kafka Connect task client ID values unique

2019-03-01 Thread Paul Davidson
Thanks Randall.  I like your suggestion: as you say, this would make it
possible to usefully override the default client id properties.

I'm not sure how we would handle the dead-letter queue case though - maybe
we could automatically add a "dlq-" prefix to the producer client id?

If there is agreement on this change I will update the KIP and the PR (when
I find some time).


On Thu, Feb 21, 2019 at 8:12 AM Randall Hauch  wrote:

> Hi, Paul. Thanks for the update to KIP-411 to reflect adding defaults, and
> creating/updating https://github.com/apache/kafka/pull/6097 to reflect
> this
> approach.
>
> Now that we've avoided adding a new config and have changed the default `
> client.id` to include some context, the connector name, and task number, I
> think it makes overriding the client ID via worker config `
> producer.client.id` or `consumer.client.id` properties less valuable
> because those overridden client IDs will be exactly the same for all
> connectors and tasks.
>
> One one hand, we can leave this as-is, and any users that include `
> producer.client.id` and `consumer.client.id` in their worker configs keep
> the same (sort of useless) behavior. In fact, most users would probably be
> better off by removing these worker config properties and instead relying
> upon the defaults.
>
> On the other, similar to what Ewen suggested earlier (in a different
> context), we could add support for users to optionally use
> "${connectorName}" and ${task}" in their overridden client ID property and
> have Connect replace these (if found) with the connector name and task
> number. Any existing properties that don't use these variables would behave
> as-is, but this way the users could define their own client IDs yet still
> get the benefit of uniquely identifying each of the clients. For example,
> if my worker config contained the following:
>
> producer.client.id=connect-cluster-A-${connectorName}-${task}-producer
> consumer.client.id=connect-cluster-A-${connectorName}-${task}-consumer
>
> Thoughts?
>
> Randall
>
> On Wed, Feb 20, 2019 at 3:18 PM Ryanne Dolan 
> wrote:
>
> > Thanks Paul, this is great. This will make monitoring Connect a ton
> easier.
> >
> > Ryanne
> >
> > On Wed, Feb 20, 2019 at 1:24 PM Paul Davidson
> >  wrote:
> >
> > > I have updated KIP-411 to propose changing the default client id - see:
> > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-411%3A+Make+default+Kafka+Connect+worker+task+client+IDs+distinct
> > >
> > >
> > > There is also an PR ready to go here:
> > > https://github.com/apache/kafka/pull/6097
> > >
> > > On Fri, Jan 11, 2019 at 3:39 PM Paul Davidson <
> pdavid...@salesforce.com>
> > > wrote:
> > >
> > > > Hi everyone.  We seem to have agreement that the ideal approach is to
> > > > alter the default client ids. Now I'm wondering about the best
> process
> > to
> > > > proceed. Will the change in default behaviour require a new KIP,
> given
> > it
> > > > will affect existing deployments?  Would I be best to repurpose this
> > > > KIP-411, or am I best to  create a new KIP? Thanks!
> > > >
> > > > Paul
> > > >
> > > > On Tue, Jan 8, 2019 at 7:16 PM Randall Hauch 
> wrote:
> > > >
> > > >> Hi, Paul.
> > > >>
> > > >> I concur with the others, and I like the new approach that avoids a
> > new
> > > >> configuration, especially because it does not change the behavior
> for
> > > >> anyone already using `producer.client.id` and/or `
> consumer.client.id
> > `.
> > > I
> > > >> did leave a few comments on the PR. Perhaps the biggest one is
> whether
> > > the
> > > >> producer used for the sink task error reporter (for dead letter
> queue)
> > > >> should be `connector-producer-`, and whether that is
> > > >> distinct
> > > >> enough from source tasks, which will be of the form
> > > >> `connector-producer-`. Maybe it is fine. (The other
> > > >> comments were minor.)
> > > >>
> > > >> Best regards,
> > > >>
> > > >> Randall
> > > >>
> > > >> On Mon, Jan 7, 2019 at 1:19 PM Paul Davidson <
> > pdavid...@salesforce.com>
> > > >> wrote:
> > > >>
> > > >> > Thanks all. I've submitted a new PR with a possible
> implementation:
> > > >> > https://github.com/apache/kafka/pull/6097. Note I did not include
> > the
> > > >> > group
> > > >> > ID as part of the default client ID, mainly to avoid the connector
> > > name
> > > >> > appearing twice by default. As noted in the original Jira (
> > > >> > https://issues.apache.org/jira/browse/KAFKA-5061), leaving out
> the
> > > >> group
> > > >> > ID
> > > >> > could lead to naming conflicts if multiple clusters run the same
> > Kafka
> > > >> > cluster. This would probably not be a problem for many (including
> > us)
> > > as
> > > >> > metrics exporters can usually be configured to include a cluster
> ID
> > > and
> > > >> > guarantee uniqueness. Will be interested to hear your thoughts on
> > > this.
> > > >> >
> > > >> > Paul
> > > >> >
> > > >> >
> > > >> >
> > > >> > On Mon, Jan 7, 2019 at 10:27 AM Ryanne 

Re: [DISCUSS] KIP-411: Add option to make Kafka Connect task client ID values unique

2019-02-21 Thread Randall Hauch
Hi, Paul. Thanks for the update to KIP-411 to reflect adding defaults, and
creating/updating https://github.com/apache/kafka/pull/6097 to reflect this
approach.

Now that we've avoided adding a new config and have changed the default `
client.id` to include some context, the connector name, and task number, I
think it makes overriding the client ID via worker config `
producer.client.id` or `consumer.client.id` properties less valuable
because those overridden client IDs will be exactly the same for all
connectors and tasks.

One one hand, we can leave this as-is, and any users that include `
producer.client.id` and `consumer.client.id` in their worker configs keep
the same (sort of useless) behavior. In fact, most users would probably be
better off by removing these worker config properties and instead relying
upon the defaults.

On the other, similar to what Ewen suggested earlier (in a different
context), we could add support for users to optionally use
"${connectorName}" and ${task}" in their overridden client ID property and
have Connect replace these (if found) with the connector name and task
number. Any existing properties that don't use these variables would behave
as-is, but this way the users could define their own client IDs yet still
get the benefit of uniquely identifying each of the clients. For example,
if my worker config contained the following:

producer.client.id=connect-cluster-A-${connectorName}-${task}-producer
consumer.client.id=connect-cluster-A-${connectorName}-${task}-consumer

Thoughts?

Randall

On Wed, Feb 20, 2019 at 3:18 PM Ryanne Dolan  wrote:

> Thanks Paul, this is great. This will make monitoring Connect a ton easier.
>
> Ryanne
>
> On Wed, Feb 20, 2019 at 1:24 PM Paul Davidson
>  wrote:
>
> > I have updated KIP-411 to propose changing the default client id - see:
> >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-411%3A+Make+default+Kafka+Connect+worker+task+client+IDs+distinct
> >
> >
> > There is also an PR ready to go here:
> > https://github.com/apache/kafka/pull/6097
> >
> > On Fri, Jan 11, 2019 at 3:39 PM Paul Davidson 
> > wrote:
> >
> > > Hi everyone.  We seem to have agreement that the ideal approach is to
> > > alter the default client ids. Now I'm wondering about the best process
> to
> > > proceed. Will the change in default behaviour require a new KIP, given
> it
> > > will affect existing deployments?  Would I be best to repurpose this
> > > KIP-411, or am I best to  create a new KIP? Thanks!
> > >
> > > Paul
> > >
> > > On Tue, Jan 8, 2019 at 7:16 PM Randall Hauch  wrote:
> > >
> > >> Hi, Paul.
> > >>
> > >> I concur with the others, and I like the new approach that avoids a
> new
> > >> configuration, especially because it does not change the behavior for
> > >> anyone already using `producer.client.id` and/or `consumer.client.id
> `.
> > I
> > >> did leave a few comments on the PR. Perhaps the biggest one is whether
> > the
> > >> producer used for the sink task error reporter (for dead letter queue)
> > >> should be `connector-producer-`, and whether that is
> > >> distinct
> > >> enough from source tasks, which will be of the form
> > >> `connector-producer-`. Maybe it is fine. (The other
> > >> comments were minor.)
> > >>
> > >> Best regards,
> > >>
> > >> Randall
> > >>
> > >> On Mon, Jan 7, 2019 at 1:19 PM Paul Davidson <
> pdavid...@salesforce.com>
> > >> wrote:
> > >>
> > >> > Thanks all. I've submitted a new PR with a possible implementation:
> > >> > https://github.com/apache/kafka/pull/6097. Note I did not include
> the
> > >> > group
> > >> > ID as part of the default client ID, mainly to avoid the connector
> > name
> > >> > appearing twice by default. As noted in the original Jira (
> > >> > https://issues.apache.org/jira/browse/KAFKA-5061), leaving out the
> > >> group
> > >> > ID
> > >> > could lead to naming conflicts if multiple clusters run the same
> Kafka
> > >> > cluster. This would probably not be a problem for many (including
> us)
> > as
> > >> > metrics exporters can usually be configured to include a cluster ID
> > and
> > >> > guarantee uniqueness. Will be interested to hear your thoughts on
> > this.
> > >> >
> > >> > Paul
> > >> >
> > >> >
> > >> >
> > >> > On Mon, Jan 7, 2019 at 10:27 AM Ryanne Dolan  >
> > >> > wrote:
> > >> >
> > >> > > I'd also prefer to avoid the new configuration property if
> possible.
> > >> > Seems
> > >> > > like a lighter touch without it.
> > >> > >
> > >> > > Ryanne
> > >> > >
> > >> > > On Sun, Jan 6, 2019 at 7:25 PM Paul Davidson <
> > >> pdavid...@salesforce.com>
> > >> > > wrote:
> > >> > >
> > >> > > > Hi Konstantine,
> > >> > > >
> > >> > > > Thanks for your feedback!  I think my reply to Ewen covers most
> of
> > >> your
> > >> > > > points, and I mostly agree.  If there is general agreement that
> > >> > changing
> > >> > > > the default behavior is preferable to a config change I will
> > update
> > >> my
> > >> > PR
> > >> > > > to use  that approach.
> > >> > > 

Re: [DISCUSS] KIP-411: Add option to make Kafka Connect task client ID values unique

2019-02-20 Thread Ryanne Dolan
Thanks Paul, this is great. This will make monitoring Connect a ton easier.

Ryanne

On Wed, Feb 20, 2019 at 1:24 PM Paul Davidson
 wrote:

> I have updated KIP-411 to propose changing the default client id - see:
>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-411%3A+Make+default+Kafka+Connect+worker+task+client+IDs+distinct
>
>
> There is also an PR ready to go here:
> https://github.com/apache/kafka/pull/6097
>
> On Fri, Jan 11, 2019 at 3:39 PM Paul Davidson 
> wrote:
>
> > Hi everyone.  We seem to have agreement that the ideal approach is to
> > alter the default client ids. Now I'm wondering about the best process to
> > proceed. Will the change in default behaviour require a new KIP, given it
> > will affect existing deployments?  Would I be best to repurpose this
> > KIP-411, or am I best to  create a new KIP? Thanks!
> >
> > Paul
> >
> > On Tue, Jan 8, 2019 at 7:16 PM Randall Hauch  wrote:
> >
> >> Hi, Paul.
> >>
> >> I concur with the others, and I like the new approach that avoids a new
> >> configuration, especially because it does not change the behavior for
> >> anyone already using `producer.client.id` and/or `consumer.client.id`.
> I
> >> did leave a few comments on the PR. Perhaps the biggest one is whether
> the
> >> producer used for the sink task error reporter (for dead letter queue)
> >> should be `connector-producer-`, and whether that is
> >> distinct
> >> enough from source tasks, which will be of the form
> >> `connector-producer-`. Maybe it is fine. (The other
> >> comments were minor.)
> >>
> >> Best regards,
> >>
> >> Randall
> >>
> >> On Mon, Jan 7, 2019 at 1:19 PM Paul Davidson 
> >> wrote:
> >>
> >> > Thanks all. I've submitted a new PR with a possible implementation:
> >> > https://github.com/apache/kafka/pull/6097. Note I did not include the
> >> > group
> >> > ID as part of the default client ID, mainly to avoid the connector
> name
> >> > appearing twice by default. As noted in the original Jira (
> >> > https://issues.apache.org/jira/browse/KAFKA-5061), leaving out the
> >> group
> >> > ID
> >> > could lead to naming conflicts if multiple clusters run the same Kafka
> >> > cluster. This would probably not be a problem for many (including us)
> as
> >> > metrics exporters can usually be configured to include a cluster ID
> and
> >> > guarantee uniqueness. Will be interested to hear your thoughts on
> this.
> >> >
> >> > Paul
> >> >
> >> >
> >> >
> >> > On Mon, Jan 7, 2019 at 10:27 AM Ryanne Dolan 
> >> > wrote:
> >> >
> >> > > I'd also prefer to avoid the new configuration property if possible.
> >> > Seems
> >> > > like a lighter touch without it.
> >> > >
> >> > > Ryanne
> >> > >
> >> > > On Sun, Jan 6, 2019 at 7:25 PM Paul Davidson <
> >> pdavid...@salesforce.com>
> >> > > wrote:
> >> > >
> >> > > > Hi Konstantine,
> >> > > >
> >> > > > Thanks for your feedback!  I think my reply to Ewen covers most of
> >> your
> >> > > > points, and I mostly agree.  If there is general agreement that
> >> > changing
> >> > > > the default behavior is preferable to a config change I will
> update
> >> my
> >> > PR
> >> > > > to use  that approach.
> >> > > >
> >> > > > Paul
> >> > > >
> >> > > > On Fri, Jan 4, 2019 at 5:55 PM Konstantine Karantasis <
> >> > > > konstant...@confluent.io> wrote:
> >> > > >
> >> > > > > Hi Paul.
> >> > > > >
> >> > > > > I second Ewen and I intended to give similar feedback:
> >> > > > >
> >> > > > > 1) Can we avoid a config altogether?
> >> > > > > 2) If we prefer to add a config anyways, can we use a set of
> >> allowed
> >> > > > values
> >> > > > > instead of a boolean, even if initially these values are only
> >> two? As
> >> > > the
> >> > > > > discussion on Jira highlights, there is a potential for more
> >> naming
> >> > > > > conventions in the future, even if now the extra functionality
> >> > doesn't
> >> > > > seem
> >> > > > > essential. It's not optimal to have to deprecate a config
> instead
> >> of
> >> > > just
> >> > > > > extending its set of values.
> >> > > > > 3) I agree, the config name sounds too general. How about
> >> > > > > "client.ids.naming.policy" or "client.ids.naming" if you want
> two
> >> > more
> >> > > > > options?
> >> > > > >
> >> > > > > Konstantine
> >> > > > >
> >> > > > > On Fri, Jan 4, 2019 at 7:38 AM Ewen Cheslack-Postava <
> >> > > e...@confluent.io>
> >> > > > > wrote:
> >> > > > >
> >> > > > > > Hi Paul,
> >> > > > > >
> >> > > > > > Thanks for the KIP. A few comments.
> >> > > > > >
> >> > > > > > To me, biggest question here is if we can fix this behavior
> >> without
> >> > > > > adding
> >> > > > > > a config. In particular, today, we don't even set the
> client.id
> >> > for
> >> > > > the
> >> > > > > > producer and consumer at all, right? The *only* way it is set
> >> is if
> >> > > you
> >> > > > > > include an override in the worker config, but in that case you
> >> need
> >> > > to
> >> > > > be
> >> > > > > > explicitly opting in with a `producer.` or `consumer.` prefix,
> >> i.e.
> 

Re: [DISCUSS] KIP-411: Add option to make Kafka Connect task client ID values unique

2019-02-20 Thread Paul Davidson
I have updated KIP-411 to propose changing the default client id - see:
https://cwiki.apache.org/confluence/display/KAFKA/KIP-411%3A+Make+default+Kafka+Connect+worker+task+client+IDs+distinct


There is also an PR ready to go here:
https://github.com/apache/kafka/pull/6097

On Fri, Jan 11, 2019 at 3:39 PM Paul Davidson 
wrote:

> Hi everyone.  We seem to have agreement that the ideal approach is to
> alter the default client ids. Now I'm wondering about the best process to
> proceed. Will the change in default behaviour require a new KIP, given it
> will affect existing deployments?  Would I be best to repurpose this
> KIP-411, or am I best to  create a new KIP? Thanks!
>
> Paul
>
> On Tue, Jan 8, 2019 at 7:16 PM Randall Hauch  wrote:
>
>> Hi, Paul.
>>
>> I concur with the others, and I like the new approach that avoids a new
>> configuration, especially because it does not change the behavior for
>> anyone already using `producer.client.id` and/or `consumer.client.id`. I
>> did leave a few comments on the PR. Perhaps the biggest one is whether the
>> producer used for the sink task error reporter (for dead letter queue)
>> should be `connector-producer-`, and whether that is
>> distinct
>> enough from source tasks, which will be of the form
>> `connector-producer-`. Maybe it is fine. (The other
>> comments were minor.)
>>
>> Best regards,
>>
>> Randall
>>
>> On Mon, Jan 7, 2019 at 1:19 PM Paul Davidson 
>> wrote:
>>
>> > Thanks all. I've submitted a new PR with a possible implementation:
>> > https://github.com/apache/kafka/pull/6097. Note I did not include the
>> > group
>> > ID as part of the default client ID, mainly to avoid the connector name
>> > appearing twice by default. As noted in the original Jira (
>> > https://issues.apache.org/jira/browse/KAFKA-5061), leaving out the
>> group
>> > ID
>> > could lead to naming conflicts if multiple clusters run the same Kafka
>> > cluster. This would probably not be a problem for many (including us) as
>> > metrics exporters can usually be configured to include a cluster ID and
>> > guarantee uniqueness. Will be interested to hear your thoughts on this.
>> >
>> > Paul
>> >
>> >
>> >
>> > On Mon, Jan 7, 2019 at 10:27 AM Ryanne Dolan 
>> > wrote:
>> >
>> > > I'd also prefer to avoid the new configuration property if possible.
>> > Seems
>> > > like a lighter touch without it.
>> > >
>> > > Ryanne
>> > >
>> > > On Sun, Jan 6, 2019 at 7:25 PM Paul Davidson <
>> pdavid...@salesforce.com>
>> > > wrote:
>> > >
>> > > > Hi Konstantine,
>> > > >
>> > > > Thanks for your feedback!  I think my reply to Ewen covers most of
>> your
>> > > > points, and I mostly agree.  If there is general agreement that
>> > changing
>> > > > the default behavior is preferable to a config change I will update
>> my
>> > PR
>> > > > to use  that approach.
>> > > >
>> > > > Paul
>> > > >
>> > > > On Fri, Jan 4, 2019 at 5:55 PM Konstantine Karantasis <
>> > > > konstant...@confluent.io> wrote:
>> > > >
>> > > > > Hi Paul.
>> > > > >
>> > > > > I second Ewen and I intended to give similar feedback:
>> > > > >
>> > > > > 1) Can we avoid a config altogether?
>> > > > > 2) If we prefer to add a config anyways, can we use a set of
>> allowed
>> > > > values
>> > > > > instead of a boolean, even if initially these values are only
>> two? As
>> > > the
>> > > > > discussion on Jira highlights, there is a potential for more
>> naming
>> > > > > conventions in the future, even if now the extra functionality
>> > doesn't
>> > > > seem
>> > > > > essential. It's not optimal to have to deprecate a config instead
>> of
>> > > just
>> > > > > extending its set of values.
>> > > > > 3) I agree, the config name sounds too general. How about
>> > > > > "client.ids.naming.policy" or "client.ids.naming" if you want two
>> > more
>> > > > > options?
>> > > > >
>> > > > > Konstantine
>> > > > >
>> > > > > On Fri, Jan 4, 2019 at 7:38 AM Ewen Cheslack-Postava <
>> > > e...@confluent.io>
>> > > > > wrote:
>> > > > >
>> > > > > > Hi Paul,
>> > > > > >
>> > > > > > Thanks for the KIP. A few comments.
>> > > > > >
>> > > > > > To me, biggest question here is if we can fix this behavior
>> without
>> > > > > adding
>> > > > > > a config. In particular, today, we don't even set the client.id
>> > for
>> > > > the
>> > > > > > producer and consumer at all, right? The *only* way it is set
>> is if
>> > > you
>> > > > > > include an override in the worker config, but in that case you
>> need
>> > > to
>> > > > be
>> > > > > > explicitly opting in with a `producer.` or `consumer.` prefix,
>> i.e.
>> > > the
>> > > > > > settings are `producer.client.id` and `consumer.client.id`.
>> > > > Otherwise, I
>> > > > > > think we're getting the default behavior where we generate
>> unique,
>> > > > > > per-process IDs, i.e. via this logic
>> > > > > >
>> > > > > >
>> > > > >
>> > > >
>> > >
>> >
>> https://github.com/apache/kafka/blob/trunk/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java#L662-L664
>> > 

Re: [DISCUSS] KIP-411: Add option to make Kafka Connect task client ID values unique

2019-01-11 Thread Paul Davidson
Hi everyone.  We seem to have agreement that the ideal approach is to alter
the default client ids. Now I'm wondering about the best process to
proceed. Will the change in default behaviour require a new KIP, given it
will affect existing deployments?  Would I be best to repurpose this
KIP-411, or am I best to  create a new KIP? Thanks!

Paul

On Tue, Jan 8, 2019 at 7:16 PM Randall Hauch  wrote:

> Hi, Paul.
>
> I concur with the others, and I like the new approach that avoids a new
> configuration, especially because it does not change the behavior for
> anyone already using `producer.client.id` and/or `consumer.client.id`. I
> did leave a few comments on the PR. Perhaps the biggest one is whether the
> producer used for the sink task error reporter (for dead letter queue)
> should be `connector-producer-`, and whether that is distinct
> enough from source tasks, which will be of the form
> `connector-producer-`. Maybe it is fine. (The other
> comments were minor.)
>
> Best regards,
>
> Randall
>
> On Mon, Jan 7, 2019 at 1:19 PM Paul Davidson 
> wrote:
>
> > Thanks all. I've submitted a new PR with a possible implementation:
> > https://github.com/apache/kafka/pull/6097. Note I did not include the
> > group
> > ID as part of the default client ID, mainly to avoid the connector name
> > appearing twice by default. As noted in the original Jira (
> > https://issues.apache.org/jira/browse/KAFKA-5061), leaving out the group
> > ID
> > could lead to naming conflicts if multiple clusters run the same Kafka
> > cluster. This would probably not be a problem for many (including us) as
> > metrics exporters can usually be configured to include a cluster ID and
> > guarantee uniqueness. Will be interested to hear your thoughts on this.
> >
> > Paul
> >
> >
> >
> > On Mon, Jan 7, 2019 at 10:27 AM Ryanne Dolan 
> > wrote:
> >
> > > I'd also prefer to avoid the new configuration property if possible.
> > Seems
> > > like a lighter touch without it.
> > >
> > > Ryanne
> > >
> > > On Sun, Jan 6, 2019 at 7:25 PM Paul Davidson  >
> > > wrote:
> > >
> > > > Hi Konstantine,
> > > >
> > > > Thanks for your feedback!  I think my reply to Ewen covers most of
> your
> > > > points, and I mostly agree.  If there is general agreement that
> > changing
> > > > the default behavior is preferable to a config change I will update
> my
> > PR
> > > > to use  that approach.
> > > >
> > > > Paul
> > > >
> > > > On Fri, Jan 4, 2019 at 5:55 PM Konstantine Karantasis <
> > > > konstant...@confluent.io> wrote:
> > > >
> > > > > Hi Paul.
> > > > >
> > > > > I second Ewen and I intended to give similar feedback:
> > > > >
> > > > > 1) Can we avoid a config altogether?
> > > > > 2) If we prefer to add a config anyways, can we use a set of
> allowed
> > > > values
> > > > > instead of a boolean, even if initially these values are only two?
> As
> > > the
> > > > > discussion on Jira highlights, there is a potential for more naming
> > > > > conventions in the future, even if now the extra functionality
> > doesn't
> > > > seem
> > > > > essential. It's not optimal to have to deprecate a config instead
> of
> > > just
> > > > > extending its set of values.
> > > > > 3) I agree, the config name sounds too general. How about
> > > > > "client.ids.naming.policy" or "client.ids.naming" if you want two
> > more
> > > > > options?
> > > > >
> > > > > Konstantine
> > > > >
> > > > > On Fri, Jan 4, 2019 at 7:38 AM Ewen Cheslack-Postava <
> > > e...@confluent.io>
> > > > > wrote:
> > > > >
> > > > > > Hi Paul,
> > > > > >
> > > > > > Thanks for the KIP. A few comments.
> > > > > >
> > > > > > To me, biggest question here is if we can fix this behavior
> without
> > > > > adding
> > > > > > a config. In particular, today, we don't even set the client.id
> > for
> > > > the
> > > > > > producer and consumer at all, right? The *only* way it is set is
> if
> > > you
> > > > > > include an override in the worker config, but in that case you
> need
> > > to
> > > > be
> > > > > > explicitly opting in with a `producer.` or `consumer.` prefix,
> i.e.
> > > the
> > > > > > settings are `producer.client.id` and `consumer.client.id`.
> > > > Otherwise, I
> > > > > > think we're getting the default behavior where we generate
> unique,
> > > > > > per-process IDs, i.e. via this logic
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> https://github.com/apache/kafka/blob/trunk/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java#L662-L664
> > > > > >
> > > > > > If that's the case, would it maybe be possible to compatibly
> change
> > > the
> > > > > > default to use task IDs in the client ID, but only if we don't
> see
> > an
> > > > > > existing override from the worker config? This would only change
> > the
> > > > > > behavior when someone is using the default, but since the default
> > > would
> > > > > > just use what is effectively a random ID that is useless for
> > > monitoring
> > > > > > metrics, presumably this wouldn't affect any 

Re: [DISCUSS] KIP-411: Add option to make Kafka Connect task client ID values unique

2019-01-08 Thread Randall Hauch
Hi, Paul.

I concur with the others, and I like the new approach that avoids a new
configuration, especially because it does not change the behavior for
anyone already using `producer.client.id` and/or `consumer.client.id`. I
did leave a few comments on the PR. Perhaps the biggest one is whether the
producer used for the sink task error reporter (for dead letter queue)
should be `connector-producer-`, and whether that is distinct
enough from source tasks, which will be of the form
`connector-producer-`. Maybe it is fine. (The other
comments were minor.)

Best regards,

Randall

On Mon, Jan 7, 2019 at 1:19 PM Paul Davidson 
wrote:

> Thanks all. I've submitted a new PR with a possible implementation:
> https://github.com/apache/kafka/pull/6097. Note I did not include the
> group
> ID as part of the default client ID, mainly to avoid the connector name
> appearing twice by default. As noted in the original Jira (
> https://issues.apache.org/jira/browse/KAFKA-5061), leaving out the group
> ID
> could lead to naming conflicts if multiple clusters run the same Kafka
> cluster. This would probably not be a problem for many (including us) as
> metrics exporters can usually be configured to include a cluster ID and
> guarantee uniqueness. Will be interested to hear your thoughts on this.
>
> Paul
>
>
>
> On Mon, Jan 7, 2019 at 10:27 AM Ryanne Dolan 
> wrote:
>
> > I'd also prefer to avoid the new configuration property if possible.
> Seems
> > like a lighter touch without it.
> >
> > Ryanne
> >
> > On Sun, Jan 6, 2019 at 7:25 PM Paul Davidson 
> > wrote:
> >
> > > Hi Konstantine,
> > >
> > > Thanks for your feedback!  I think my reply to Ewen covers most of your
> > > points, and I mostly agree.  If there is general agreement that
> changing
> > > the default behavior is preferable to a config change I will update my
> PR
> > > to use  that approach.
> > >
> > > Paul
> > >
> > > On Fri, Jan 4, 2019 at 5:55 PM Konstantine Karantasis <
> > > konstant...@confluent.io> wrote:
> > >
> > > > Hi Paul.
> > > >
> > > > I second Ewen and I intended to give similar feedback:
> > > >
> > > > 1) Can we avoid a config altogether?
> > > > 2) If we prefer to add a config anyways, can we use a set of allowed
> > > values
> > > > instead of a boolean, even if initially these values are only two? As
> > the
> > > > discussion on Jira highlights, there is a potential for more naming
> > > > conventions in the future, even if now the extra functionality
> doesn't
> > > seem
> > > > essential. It's not optimal to have to deprecate a config instead of
> > just
> > > > extending its set of values.
> > > > 3) I agree, the config name sounds too general. How about
> > > > "client.ids.naming.policy" or "client.ids.naming" if you want two
> more
> > > > options?
> > > >
> > > > Konstantine
> > > >
> > > > On Fri, Jan 4, 2019 at 7:38 AM Ewen Cheslack-Postava <
> > e...@confluent.io>
> > > > wrote:
> > > >
> > > > > Hi Paul,
> > > > >
> > > > > Thanks for the KIP. A few comments.
> > > > >
> > > > > To me, biggest question here is if we can fix this behavior without
> > > > adding
> > > > > a config. In particular, today, we don't even set the client.id
> for
> > > the
> > > > > producer and consumer at all, right? The *only* way it is set is if
> > you
> > > > > include an override in the worker config, but in that case you need
> > to
> > > be
> > > > > explicitly opting in with a `producer.` or `consumer.` prefix, i.e.
> > the
> > > > > settings are `producer.client.id` and `consumer.client.id`.
> > > Otherwise, I
> > > > > think we're getting the default behavior where we generate unique,
> > > > > per-process IDs, i.e. via this logic
> > > > >
> > > > >
> > > >
> > >
> >
> https://github.com/apache/kafka/blob/trunk/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java#L662-L664
> > > > >
> > > > > If that's the case, would it maybe be possible to compatibly change
> > the
> > > > > default to use task IDs in the client ID, but only if we don't see
> an
> > > > > existing override from the worker config? This would only change
> the
> > > > > behavior when someone is using the default, but since the default
> > would
> > > > > just use what is effectively a random ID that is useless for
> > monitoring
> > > > > metrics, presumably this wouldn't affect any existing users. I
> think
> > > that
> > > > > would avoid having to introduce the config, give better out of the
> > box
> > > > > behavior, and still be a safe, compatible change to make.
> > > > >
> > > > >
> > > > > Other than that, just two minor comments. On the config naming, not
> > > sure
> > > > > about a better name, but I think the config name could be a bit
> > clearer
> > > > if
> > > > > we need to have it. Maybe something including "task" like
> > > > > "task.based.client.ids" or something like that (or change the type
> to
> > > be
> > > > an
> > > > > enum and make it something like task.client.ids=[default|task] and
> > > leave
> > > > it
> > > > > 

Re: [DISCUSS] KIP-411: Add option to make Kafka Connect task client ID values unique

2019-01-07 Thread Paul Davidson
Thanks all. I've submitted a new PR with a possible implementation:
https://github.com/apache/kafka/pull/6097. Note I did not include the group
ID as part of the default client ID, mainly to avoid the connector name
appearing twice by default. As noted in the original Jira (
https://issues.apache.org/jira/browse/KAFKA-5061), leaving out the group ID
could lead to naming conflicts if multiple clusters run the same Kafka
cluster. This would probably not be a problem for many (including us) as
metrics exporters can usually be configured to include a cluster ID and
guarantee uniqueness. Will be interested to hear your thoughts on this.

Paul



On Mon, Jan 7, 2019 at 10:27 AM Ryanne Dolan  wrote:

> I'd also prefer to avoid the new configuration property if possible. Seems
> like a lighter touch without it.
>
> Ryanne
>
> On Sun, Jan 6, 2019 at 7:25 PM Paul Davidson 
> wrote:
>
> > Hi Konstantine,
> >
> > Thanks for your feedback!  I think my reply to Ewen covers most of your
> > points, and I mostly agree.  If there is general agreement that changing
> > the default behavior is preferable to a config change I will update my PR
> > to use  that approach.
> >
> > Paul
> >
> > On Fri, Jan 4, 2019 at 5:55 PM Konstantine Karantasis <
> > konstant...@confluent.io> wrote:
> >
> > > Hi Paul.
> > >
> > > I second Ewen and I intended to give similar feedback:
> > >
> > > 1) Can we avoid a config altogether?
> > > 2) If we prefer to add a config anyways, can we use a set of allowed
> > values
> > > instead of a boolean, even if initially these values are only two? As
> the
> > > discussion on Jira highlights, there is a potential for more naming
> > > conventions in the future, even if now the extra functionality doesn't
> > seem
> > > essential. It's not optimal to have to deprecate a config instead of
> just
> > > extending its set of values.
> > > 3) I agree, the config name sounds too general. How about
> > > "client.ids.naming.policy" or "client.ids.naming" if you want two more
> > > options?
> > >
> > > Konstantine
> > >
> > > On Fri, Jan 4, 2019 at 7:38 AM Ewen Cheslack-Postava <
> e...@confluent.io>
> > > wrote:
> > >
> > > > Hi Paul,
> > > >
> > > > Thanks for the KIP. A few comments.
> > > >
> > > > To me, biggest question here is if we can fix this behavior without
> > > adding
> > > > a config. In particular, today, we don't even set the client.id for
> > the
> > > > producer and consumer at all, right? The *only* way it is set is if
> you
> > > > include an override in the worker config, but in that case you need
> to
> > be
> > > > explicitly opting in with a `producer.` or `consumer.` prefix, i.e.
> the
> > > > settings are `producer.client.id` and `consumer.client.id`.
> > Otherwise, I
> > > > think we're getting the default behavior where we generate unique,
> > > > per-process IDs, i.e. via this logic
> > > >
> > > >
> > >
> >
> https://github.com/apache/kafka/blob/trunk/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java#L662-L664
> > > >
> > > > If that's the case, would it maybe be possible to compatibly change
> the
> > > > default to use task IDs in the client ID, but only if we don't see an
> > > > existing override from the worker config? This would only change the
> > > > behavior when someone is using the default, but since the default
> would
> > > > just use what is effectively a random ID that is useless for
> monitoring
> > > > metrics, presumably this wouldn't affect any existing users. I think
> > that
> > > > would avoid having to introduce the config, give better out of the
> box
> > > > behavior, and still be a safe, compatible change to make.
> > > >
> > > >
> > > > Other than that, just two minor comments. On the config naming, not
> > sure
> > > > about a better name, but I think the config name could be a bit
> clearer
> > > if
> > > > we need to have it. Maybe something including "task" like
> > > > "task.based.client.ids" or something like that (or change the type to
> > be
> > > an
> > > > enum and make it something like task.client.ids=[default|task] and
> > leave
> > > it
> > > > open for extension in the future if needed).
> > > >
> > > > Finally, you have this:
> > > >
> > > > *"Allow overriding client.id  on a per-connector
> > > > basis"*
> > > > >
> > > > > This is a much more complex change, and would require individual
> > > > > connectors to be updated to support the change. In contrast, the
> > > proposed
> > > > > approach would immediately allow detailed consumer/producer
> > monitoring
> > > > for
> > > > > all existing connectors.
> > > > >
> > > >
> > > > I don't think this is quite accurate. I think the reason to reject is
> > > that
> > > > for your particular requirement for metrics, it simply doesn't give
> > > enough
> > > > granularity (there's only one value per entire connector), but it
> > doesn't
> > > > require any changes to connectors. The framework allocates all of
> these
> > > and
> > > > there are 

Re: [DISCUSS] KIP-411: Add option to make Kafka Connect task client ID values unique

2019-01-07 Thread Ryanne Dolan
I'd also prefer to avoid the new configuration property if possible. Seems
like a lighter touch without it.

Ryanne

On Sun, Jan 6, 2019 at 7:25 PM Paul Davidson 
wrote:

> Hi Konstantine,
>
> Thanks for your feedback!  I think my reply to Ewen covers most of your
> points, and I mostly agree.  If there is general agreement that changing
> the default behavior is preferable to a config change I will update my PR
> to use  that approach.
>
> Paul
>
> On Fri, Jan 4, 2019 at 5:55 PM Konstantine Karantasis <
> konstant...@confluent.io> wrote:
>
> > Hi Paul.
> >
> > I second Ewen and I intended to give similar feedback:
> >
> > 1) Can we avoid a config altogether?
> > 2) If we prefer to add a config anyways, can we use a set of allowed
> values
> > instead of a boolean, even if initially these values are only two? As the
> > discussion on Jira highlights, there is a potential for more naming
> > conventions in the future, even if now the extra functionality doesn't
> seem
> > essential. It's not optimal to have to deprecate a config instead of just
> > extending its set of values.
> > 3) I agree, the config name sounds too general. How about
> > "client.ids.naming.policy" or "client.ids.naming" if you want two more
> > options?
> >
> > Konstantine
> >
> > On Fri, Jan 4, 2019 at 7:38 AM Ewen Cheslack-Postava 
> > wrote:
> >
> > > Hi Paul,
> > >
> > > Thanks for the KIP. A few comments.
> > >
> > > To me, biggest question here is if we can fix this behavior without
> > adding
> > > a config. In particular, today, we don't even set the client.id for
> the
> > > producer and consumer at all, right? The *only* way it is set is if you
> > > include an override in the worker config, but in that case you need to
> be
> > > explicitly opting in with a `producer.` or `consumer.` prefix, i.e. the
> > > settings are `producer.client.id` and `consumer.client.id`.
> Otherwise, I
> > > think we're getting the default behavior where we generate unique,
> > > per-process IDs, i.e. via this logic
> > >
> > >
> >
> https://github.com/apache/kafka/blob/trunk/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java#L662-L664
> > >
> > > If that's the case, would it maybe be possible to compatibly change the
> > > default to use task IDs in the client ID, but only if we don't see an
> > > existing override from the worker config? This would only change the
> > > behavior when someone is using the default, but since the default would
> > > just use what is effectively a random ID that is useless for monitoring
> > > metrics, presumably this wouldn't affect any existing users. I think
> that
> > > would avoid having to introduce the config, give better out of the box
> > > behavior, and still be a safe, compatible change to make.
> > >
> > >
> > > Other than that, just two minor comments. On the config naming, not
> sure
> > > about a better name, but I think the config name could be a bit clearer
> > if
> > > we need to have it. Maybe something including "task" like
> > > "task.based.client.ids" or something like that (or change the type to
> be
> > an
> > > enum and make it something like task.client.ids=[default|task] and
> leave
> > it
> > > open for extension in the future if needed).
> > >
> > > Finally, you have this:
> > >
> > > *"Allow overriding client.id  on a per-connector
> > > basis"*
> > > >
> > > > This is a much more complex change, and would require individual
> > > > connectors to be updated to support the change. In contrast, the
> > proposed
> > > > approach would immediately allow detailed consumer/producer
> monitoring
> > > for
> > > > all existing connectors.
> > > >
> > >
> > > I don't think this is quite accurate. I think the reason to reject is
> > that
> > > for your particular requirement for metrics, it simply doesn't give
> > enough
> > > granularity (there's only one value per entire connector), but it
> doesn't
> > > require any changes to connectors. The framework allocates all of these
> > and
> > > there are already framework-defined config values that all connectors
> > share
> > > (some for only sinks or sources), so the framework can handle all of
> this
> > > without changes to connectors. Further, with connector-specific
> > overrides,
> > > you could get task-specific values if interpolation were supported in
> the
> > > config value (as we now do with managed secrets). For example, it could
> > > support something like client.id=connector-${taskId} and the task ID
> > would
> > > be substituted automatically into the string.
> > >
> > > I don't necessarily like that solution (seems complicated and not a
> great
> > > user experience), but it could work.
> > >
> > > -Ewen
> > >
> > >
> > >
> > >
> > > On Thu, Dec 20, 2018 at 5:05 PM Paul Davidson <
> pdavid...@salesforce.com>
> > > wrote:
> > >
> > > > Hi everyone,
> > > >
> > > > I would like to start a discussion around the following KIP:
> > > > *
> > > >
> > >
> >
> 

Re: [DISCUSS] KIP-411: Add option to make Kafka Connect task client ID values unique

2019-01-06 Thread Paul Davidson
Hi Konstantine,

Thanks for your feedback!  I think my reply to Ewen covers most of your
points, and I mostly agree.  If there is general agreement that changing
the default behavior is preferable to a config change I will update my PR
to use  that approach.

Paul

On Fri, Jan 4, 2019 at 5:55 PM Konstantine Karantasis <
konstant...@confluent.io> wrote:

> Hi Paul.
>
> I second Ewen and I intended to give similar feedback:
>
> 1) Can we avoid a config altogether?
> 2) If we prefer to add a config anyways, can we use a set of allowed values
> instead of a boolean, even if initially these values are only two? As the
> discussion on Jira highlights, there is a potential for more naming
> conventions in the future, even if now the extra functionality doesn't seem
> essential. It's not optimal to have to deprecate a config instead of just
> extending its set of values.
> 3) I agree, the config name sounds too general. How about
> "client.ids.naming.policy" or "client.ids.naming" if you want two more
> options?
>
> Konstantine
>
> On Fri, Jan 4, 2019 at 7:38 AM Ewen Cheslack-Postava 
> wrote:
>
> > Hi Paul,
> >
> > Thanks for the KIP. A few comments.
> >
> > To me, biggest question here is if we can fix this behavior without
> adding
> > a config. In particular, today, we don't even set the client.id for the
> > producer and consumer at all, right? The *only* way it is set is if you
> > include an override in the worker config, but in that case you need to be
> > explicitly opting in with a `producer.` or `consumer.` prefix, i.e. the
> > settings are `producer.client.id` and `consumer.client.id`. Otherwise, I
> > think we're getting the default behavior where we generate unique,
> > per-process IDs, i.e. via this logic
> >
> >
> https://github.com/apache/kafka/blob/trunk/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java#L662-L664
> >
> > If that's the case, would it maybe be possible to compatibly change the
> > default to use task IDs in the client ID, but only if we don't see an
> > existing override from the worker config? This would only change the
> > behavior when someone is using the default, but since the default would
> > just use what is effectively a random ID that is useless for monitoring
> > metrics, presumably this wouldn't affect any existing users. I think that
> > would avoid having to introduce the config, give better out of the box
> > behavior, and still be a safe, compatible change to make.
> >
> >
> > Other than that, just two minor comments. On the config naming, not sure
> > about a better name, but I think the config name could be a bit clearer
> if
> > we need to have it. Maybe something including "task" like
> > "task.based.client.ids" or something like that (or change the type to be
> an
> > enum and make it something like task.client.ids=[default|task] and leave
> it
> > open for extension in the future if needed).
> >
> > Finally, you have this:
> >
> > *"Allow overriding client.id  on a per-connector
> > basis"*
> > >
> > > This is a much more complex change, and would require individual
> > > connectors to be updated to support the change. In contrast, the
> proposed
> > > approach would immediately allow detailed consumer/producer monitoring
> > for
> > > all existing connectors.
> > >
> >
> > I don't think this is quite accurate. I think the reason to reject is
> that
> > for your particular requirement for metrics, it simply doesn't give
> enough
> > granularity (there's only one value per entire connector), but it doesn't
> > require any changes to connectors. The framework allocates all of these
> and
> > there are already framework-defined config values that all connectors
> share
> > (some for only sinks or sources), so the framework can handle all of this
> > without changes to connectors. Further, with connector-specific
> overrides,
> > you could get task-specific values if interpolation were supported in the
> > config value (as we now do with managed secrets). For example, it could
> > support something like client.id=connector-${taskId} and the task ID
> would
> > be substituted automatically into the string.
> >
> > I don't necessarily like that solution (seems complicated and not a great
> > user experience), but it could work.
> >
> > -Ewen
> >
> >
> >
> >
> > On Thu, Dec 20, 2018 at 5:05 PM Paul Davidson 
> > wrote:
> >
> > > Hi everyone,
> > >
> > > I would like to start a discussion around the following KIP:
> > > *
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-411%3A+Add+option+to+make+Kafka+Connect+task+client+ID+values+unique
> > > <
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-411%3A+Add+option+to+make+Kafka+Connect+task+client+ID+values+unique
> > > >*
> > >
> > > This proposes a small change to allow Kafka Connect the option to
> > > auto-generate unique client IDs for each task. This enables granular
> > > monitoring of the producer / consumer client in each task.
> > >

Re: [DISCUSS] KIP-411: Add option to make Kafka Connect task client ID values unique

2019-01-06 Thread Paul Davidson
Hi Ewen,

Thanks for the feedback!

If that's the case, would it maybe be possible to compatibly change the
> default to use task IDs in the client ID, but only if we don't see an
> existing override from the worker config? This would only change the
> behavior when someone is using the default, but since the default would
> just use what is effectively a random ID that is useless for monitoring
> metrics, presumably this wouldn't affect any existing users. I think that
> would avoid having to introduce the config, give better out of the box
> behavior, and still be a safe, compatible change to make.


Yes, it would be possible to simply change the default.

There were two reasons I went with a config option in the proposal. I was
primarily trying to avoid giving users a surprise by altering existing
behavior. But, as you said, people are unlikely be relying on the current
random default for monitoring so this might be fine. Another minor
advantage of my proposal is it keeps open the option of setting arbitrary
custom client IDs. If we only append the task ID to the default client ID
then setting the "producer.clientid" or "consumer.clientid" will always
result in a name conflict and will almost always need to be avoided.

If a change in the default behavior is acceptable, and we're OK with
advising people to avoid setting "producer.clientid" or "consumer.clientid"
in future, then I'm happy to go with your suggested approach of changing
the default.

... or change the type to be an
> enum and make it something like task.client.ids=[default|task] and leave it
> open for extension in the future if needed).


If we were going with new config, I like the enum option.


> *"Allow overriding client.id  on a per-connector
> basis"*
> >
> > This is a much more complex change, and would require individual
> > connectors to be updated to support the change. In contrast, the proposed
> > approach would immediately allow detailed consumer/producer monitoring
> for
> > all existing connectors.
> >
> I don't think this is quite accurate. I think the reason to reject is that
> for your particular requirement for metrics, it simply doesn't give enough
> granularity (there's only one value per entire connector), but it doesn't
> ...
> config value (as we now do with managed secrets). For example, it could
> support something like client.id=connector-${taskId} and the task ID would
> be substituted automatically into the string.
>

Yes, I think I was misinterpreting "on a per-connector basis" to mean
giving each connector a way to override client properties on a per-task
basis: a simple per-connector override clearly wouldn't work.  I was
imagining adding a way to intercept and modify the client properties before
they're used by the Worker to construct the client.  You're right that this
wouldn't necessarily need to live in the connector. Config interpolation is
an option, as you mentioned.  Either way, this is certainly a more complex
change.

Paul


Re: [DISCUSS] KIP-411: Add option to make Kafka Connect task client ID values unique

2019-01-04 Thread Konstantine Karantasis
Hi Paul.

I second Ewen and I intended to give similar feedback:

1) Can we avoid a config altogether?
2) If we prefer to add a config anyways, can we use a set of allowed values
instead of a boolean, even if initially these values are only two? As the
discussion on Jira highlights, there is a potential for more naming
conventions in the future, even if now the extra functionality doesn't seem
essential. It's not optimal to have to deprecate a config instead of just
extending its set of values.
3) I agree, the config name sounds too general. How about
"client.ids.naming.policy" or "client.ids.naming" if you want two more
options?

Konstantine

On Fri, Jan 4, 2019 at 7:38 AM Ewen Cheslack-Postava 
wrote:

> Hi Paul,
>
> Thanks for the KIP. A few comments.
>
> To me, biggest question here is if we can fix this behavior without adding
> a config. In particular, today, we don't even set the client.id for the
> producer and consumer at all, right? The *only* way it is set is if you
> include an override in the worker config, but in that case you need to be
> explicitly opting in with a `producer.` or `consumer.` prefix, i.e. the
> settings are `producer.client.id` and `consumer.client.id`. Otherwise, I
> think we're getting the default behavior where we generate unique,
> per-process IDs, i.e. via this logic
>
> https://github.com/apache/kafka/blob/trunk/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java#L662-L664
>
> If that's the case, would it maybe be possible to compatibly change the
> default to use task IDs in the client ID, but only if we don't see an
> existing override from the worker config? This would only change the
> behavior when someone is using the default, but since the default would
> just use what is effectively a random ID that is useless for monitoring
> metrics, presumably this wouldn't affect any existing users. I think that
> would avoid having to introduce the config, give better out of the box
> behavior, and still be a safe, compatible change to make.
>
>
> Other than that, just two minor comments. On the config naming, not sure
> about a better name, but I think the config name could be a bit clearer if
> we need to have it. Maybe something including "task" like
> "task.based.client.ids" or something like that (or change the type to be an
> enum and make it something like task.client.ids=[default|task] and leave it
> open for extension in the future if needed).
>
> Finally, you have this:
>
> *"Allow overriding client.id  on a per-connector
> basis"*
> >
> > This is a much more complex change, and would require individual
> > connectors to be updated to support the change. In contrast, the proposed
> > approach would immediately allow detailed consumer/producer monitoring
> for
> > all existing connectors.
> >
>
> I don't think this is quite accurate. I think the reason to reject is that
> for your particular requirement for metrics, it simply doesn't give enough
> granularity (there's only one value per entire connector), but it doesn't
> require any changes to connectors. The framework allocates all of these and
> there are already framework-defined config values that all connectors share
> (some for only sinks or sources), so the framework can handle all of this
> without changes to connectors. Further, with connector-specific overrides,
> you could get task-specific values if interpolation were supported in the
> config value (as we now do with managed secrets). For example, it could
> support something like client.id=connector-${taskId} and the task ID would
> be substituted automatically into the string.
>
> I don't necessarily like that solution (seems complicated and not a great
> user experience), but it could work.
>
> -Ewen
>
>
>
>
> On Thu, Dec 20, 2018 at 5:05 PM Paul Davidson 
> wrote:
>
> > Hi everyone,
> >
> > I would like to start a discussion around the following KIP:
> > *
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-411%3A+Add+option+to+make+Kafka+Connect+task+client+ID+values+unique
> > <
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-411%3A+Add+option+to+make+Kafka+Connect+task+client+ID+values+unique
> > >*
> >
> > This proposes a small change to allow Kafka Connect the option to
> > auto-generate unique client IDs for each task. This enables granular
> > monitoring of the producer / consumer client in each task.
> >
> > Feedback is appreciated, thanks in advance!
> >
> > Paul
> >
>


Re: [DISCUSS] KIP-411: Add option to make Kafka Connect task client ID values unique

2019-01-04 Thread Ewen Cheslack-Postava
Hi Paul,

Thanks for the KIP. A few comments.

To me, biggest question here is if we can fix this behavior without adding
a config. In particular, today, we don't even set the client.id for the
producer and consumer at all, right? The *only* way it is set is if you
include an override in the worker config, but in that case you need to be
explicitly opting in with a `producer.` or `consumer.` prefix, i.e. the
settings are `producer.client.id` and `consumer.client.id`. Otherwise, I
think we're getting the default behavior where we generate unique,
per-process IDs, i.e. via this logic
https://github.com/apache/kafka/blob/trunk/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java#L662-L664

If that's the case, would it maybe be possible to compatibly change the
default to use task IDs in the client ID, but only if we don't see an
existing override from the worker config? This would only change the
behavior when someone is using the default, but since the default would
just use what is effectively a random ID that is useless for monitoring
metrics, presumably this wouldn't affect any existing users. I think that
would avoid having to introduce the config, give better out of the box
behavior, and still be a safe, compatible change to make.


Other than that, just two minor comments. On the config naming, not sure
about a better name, but I think the config name could be a bit clearer if
we need to have it. Maybe something including "task" like
"task.based.client.ids" or something like that (or change the type to be an
enum and make it something like task.client.ids=[default|task] and leave it
open for extension in the future if needed).

Finally, you have this:

*"Allow overriding client.id  on a per-connector basis"*
>
> This is a much more complex change, and would require individual
> connectors to be updated to support the change. In contrast, the proposed
> approach would immediately allow detailed consumer/producer monitoring for
> all existing connectors.
>

I don't think this is quite accurate. I think the reason to reject is that
for your particular requirement for metrics, it simply doesn't give enough
granularity (there's only one value per entire connector), but it doesn't
require any changes to connectors. The framework allocates all of these and
there are already framework-defined config values that all connectors share
(some for only sinks or sources), so the framework can handle all of this
without changes to connectors. Further, with connector-specific overrides,
you could get task-specific values if interpolation were supported in the
config value (as we now do with managed secrets). For example, it could
support something like client.id=connector-${taskId} and the task ID would
be substituted automatically into the string.

I don't necessarily like that solution (seems complicated and not a great
user experience), but it could work.

-Ewen




On Thu, Dec 20, 2018 at 5:05 PM Paul Davidson 
wrote:

> Hi everyone,
>
> I would like to start a discussion around the following KIP:
> *
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-411%3A+Add+option+to+make+Kafka+Connect+task+client+ID+values+unique
> <
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-411%3A+Add+option+to+make+Kafka+Connect+task+client+ID+values+unique
> >*
>
> This proposes a small change to allow Kafka Connect the option to
> auto-generate unique client IDs for each task. This enables granular
> monitoring of the producer / consumer client in each task.
>
> Feedback is appreciated, thanks in advance!
>
> Paul
>