From dev-return-103570-archive-asf-public=cust-asf.ponee.io@kafka.apache.org Thu Apr 25 21:18:15 2019 Return-Path: X-Original-To: archive-asf-public@cust-asf.ponee.io Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [207.244.88.153]) by mx-eu-01.ponee.io (Postfix) with SMTP id 60FD9180638 for ; Thu, 25 Apr 2019 23:18:15 +0200 (CEST) Received: (qmail 35644 invoked by uid 500); 25 Apr 2019 21:18:12 -0000 Mailing-List: contact dev-help@kafka.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@kafka.apache.org Delivered-To: mailing list dev@kafka.apache.org Received: (qmail 35622 invoked by uid 99); 25 Apr 2019 21:18:11 -0000 Received: from pnap-us-west-generic-nat.apache.org (HELO spamd1-us-west.apache.org) (209.188.14.142) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 25 Apr 2019 21:18:11 +0000 Received: from localhost (localhost [127.0.0.1]) by spamd1-us-west.apache.org (ASF Mail Server at spamd1-us-west.apache.org) with ESMTP id 28528C1CC8 for ; Thu, 25 Apr 2019 21:18:10 +0000 (UTC) X-Virus-Scanned: Debian amavisd-new at spamd1-us-west.apache.org X-Spam-Flag: NO X-Spam-Score: 1.8 X-Spam-Level: * X-Spam-Status: No, score=1.8 tagged_above=-999 required=6.31 tests=[DKIM_SIGNED=0.1, DKIM_VALID=-0.1, DKIM_VALID_AU=-0.1, DKIM_VALID_EF=-0.1, HTML_MESSAGE=2, RCVD_IN_DNSWL_NONE=-0.0001, SPF_PASS=-0.001, URIBL_BLOCKED=0.001] autolearn=disabled Authentication-Results: spamd1-us-west.apache.org (amavisd-new); dkim=pass (2048-bit key) header.d=confluent.io Received: from mx1-lw-eu.apache.org ([10.40.0.8]) by localhost (spamd1-us-west.apache.org [10.40.0.7]) (amavisd-new, port 10024) with ESMTP id 7bZh8-au9QbV for ; Thu, 25 Apr 2019 21:18:07 +0000 (UTC) Received: from mail-vs1-f53.google.com (mail-vs1-f53.google.com [209.85.217.53]) by mx1-lw-eu.apache.org (ASF Mail Server at mx1-lw-eu.apache.org) with ESMTPS id 03CD25FB8D for ; Thu, 25 Apr 2019 21:18:06 +0000 (UTC) Received: by mail-vs1-f53.google.com with SMTP id s2so737661vsi.5 for ; Thu, 25 Apr 2019 14:18:06 -0700 (PDT) DKIM-Signature: v=1; a=rsa-sha256; c=relaxed/relaxed; d=confluent.io; s=google; h=mime-version:references:in-reply-to:from:date:message-id:subject:to; bh=Ih+r9afKNc52GTqbbnrzhCYV/cBLdYTzFaGTTpnmw0o=; b=XAD+Gtr0j/v6rWhpDggh6KsHz/pSGGZtA1t6+DS5B03jQAf/QfUlFRnF4ze5/xWCCl xICzFB6GFRj4SiisU+7FYWAZAf3CBxpSDRPqmUmuUTYZxlDFInyYeAf6jeuPpJnCjl8m gHlDOKydyEznW/cthIah74yM8Z5l1rFMwLgO9Pm1Sogzuw3HNluPRtrptVbs5/ECZg8K 3bWokOVNFizEzBMDxSdOHQRD6+/TQULXnWhlNGjoo1V+ds9RL/KJkLxI4mmPt/xcJQSv PYiRqSVCks4OEwSwYZVhuGNyd7XH1S7BUJfdOZw322jkZ+KBMvbz/gk/aO3yL7Ob/OFQ WiNQ== X-Google-DKIM-Signature: v=1; a=rsa-sha256; c=relaxed/relaxed; d=1e100.net; s=20161025; h=x-gm-message-state:mime-version:references:in-reply-to:from:date :message-id:subject:to; bh=Ih+r9afKNc52GTqbbnrzhCYV/cBLdYTzFaGTTpnmw0o=; b=Un4IgMkdoKPCsQuaQos+03LrseHz+HIqB07j/SjCQfF0BPlQq12EWFFF84xDkyHfK7 /XHX29YdG2FWs7QLVgFVyR62yxVD8Ds4a4WuK2dqv/NRtZBGI6aFoIcbPdtmhRPntYSf P6HaFr3Ha8N7WsqWcqxYUNMGsTxH1KopwQhlPEV0f5866TWgeFuPRfhnX3L9rTQPJtKy 6JGt9XUJ3bG7qVYB+/R96+kVxS1X6Ig0sKpfK6X7gC+adiWCyrmHI0hvszMyMvkG45Ai cmtunQeyyqpftByqtllye7hiTu9tqUbKJLBhZovZHc+DunUe23qBcRpBr7eRKeBaOfhL Lvtg== X-Gm-Message-State: APjAAAWfSXa3UgdXt9fyv2MLwkNCsC59awpsTZhIIzMffrscae/0mh2F 5DjDFlMUOua1IoHaj2gvY4OclXAoqzLrLrjhJKsuJMz9 X-Google-Smtp-Source: APXvYqy99HSg4O/ghHfvhSsNpmsEhj74eiouXx5USzmrPbghqG5RcS75wfzTHNBfZqGo5EohVbbvgYQ3bGSpQSE6OD4= X-Received: by 2002:a67:da8b:: with SMTP id w11mr7765066vsj.139.1556227085381; Thu, 25 Apr 2019 14:18:05 -0700 (PDT) MIME-Version: 1.0 References: In-Reply-To: From: Chris Egerton Date: Thu, 25 Apr 2019 14:17:54 -0700 Message-ID: Subject: Re: [DISCUSS] KIP-458: Connector Client Config Override Policy To: dev@kafka.apache.org Content-Type: multipart/alternative; boundary="0000000000009d537b0587615a52" --0000000000009d537b0587615a52 Content-Type: text/plain; charset="UTF-8" Hi Magesh, Agreed that we should avoid `dlq.admin`. I also don't have a strong opinion between `connector.` and `.override`, but I have a slight inclination toward `.override` since `connector.` feels a little redundant given that the whole configuration is for the connector and the use of "override" may shed a little light on how the properties for these clients are computed and help make the learning curve a little gentler on new devs and users. Regardless, I think the larger issue of conflicts with existing properties (both in MM2 and potentially other connectors) has been satisfactorily addressed, so I'm happy. Cheers, Chris On Wed, Apr 24, 2019 at 11:14 AM Magesh Nandakumar wrote: > HI Chrise, > > You are right about the "admin." prefix creating conflicts. Here are few > options that I can think of > > 1. Use `dlq.admin` since admin client is used only for DLQ. But this might > not really be the case in the future. So, we should possibly drop this idea > :) > 2. Use `connector.producer`, `connector.consumer` and `connector.admin` - > provides better context that its connector specific property > 3. Use `producer.override`, '`consumer.override` and `admin.override` - > provides better clarity that these are overrides. > > I don't have a strong opinion in choosing between #2 and #3. Let me > know what you think. > > Thanks > Magesh > > On Wed, Apr 24, 2019 at 10:25 AM Chris Egerton > wrote: > > > Hi Magesh, > > > > Next round :) > > > > 1. It looks like MM2 will also support "admin." properties that affect > > AdminClients it creates and uses, which IIUC is the same prefix name to > be > > used for managing the DLQ for sink connectors in this KIP. Doesn't that > > still leave room for conflict? I'm imagining a scenario like this: a > > Connect worker is configured to use the > > PrincipalConnectorClientConfigPolicy, someone tries to start an instance > of > > an MM2 sink with "admin." properties beyond just > "admin.sasl.jaas.config", > > and gets rejected because those properties are then interpreted by the > > worker as overrides for the AdminClient it uses to manage the DLQ. > > 2. (LGTM) > > 3. I'm convinced by this, as long as nobody else identifies a common use > > case that would involve a similar client config policy implementation > that > > would be limited to a small set of whitelisted configs. For now keeping > the > > PrincipalConnectorClientConfigPolicy sounds fine to me. > > > > Cheers, > > > > Chris > > > > On Tue, Apr 23, 2019 at 10:30 PM Magesh Nandakumar > > > wrote: > > > > > Hi all, > > > > > > I also have a draft implementation of the KIP > > > https://github.com/apache/kafka/pull/6624. I would still need to > include > > > more tests and docs but I thought it would be useful to have this for > the > > > KIP discussion. Looking forward to all of your valuable feedback. > > > > > > Thanks > > > Magesh > > > > > > On Tue, Apr 23, 2019 at 10:27 PM Magesh Nandakumar < > mageshn@confluent.io > > > > > > wrote: > > > > > > > Chrise, > > > > > > > > Thanks a lot for your feedback. I will address them in order of your > > > > questions/comments. > > > > > > > > 1. Thanks for bringing this to my attention about KIP-382. I had a > > closer > > > > look at the KIP and IIUC, the KIP allows `consumer.` prefix for > > > SourceConnector > > > > and producer. prefix for SinkConnector since those are additional > > > > connector properties to help resolve the Kafka cluster other than the > > one > > > > Connect framework knows about. Whereas, the proposal in KIP-458 > applies > > > > producer policies for SinkConnectors and consumer policies > > > > SourceConnectors. So, from what I understand this new policy should > > work > > > > without any issues even for Mirror Maker 2.0. > > > > 2. I have updated the KIP to use a default value of null and use that > > to > > > > determine if we need to ignore overrides. > > > > 3. I would still prefer to keep the special > > > PrincipalConnectorClientConfigPolicy > > > > since that is one of the most common use cases one would choose to > use > > > this > > > > feature. If we make it a general case, that would involve users > > requiring > > > > to add additional configuration and they might require well more than > > > just > > > > the list of configs but might also want some restriction on values. > If > > > the > > > > concern is about users wanting principal and also other configs, it > > would > > > > still be possible by means of a custom implementation. As is, I would > > > > prefer to keep the proposal to be the same for this. Let me know your > > > > thoughts. > > > > > > > > Thanks, > > > > Magesh > > > > > > > > > > > > On Mon, Apr 22, 2019 at 3:44 PM Chris Egerton > > > wrote: > > > > > > > >> Hi Magesh, > > > >> > > > >> This is an exciting KIP! I have a few questions/comments but > overall I > > > >> like > > > >> the direction it's headed in and hope to see it included in the > > Connect > > > >> framework soon. > > > >> > > > >> 1. With the proposed "consumer.", "producer.", and "admin." > prefixes, > > > how > > > >> will this interact with connectors such as the upcoming Mirror Maker > > 2.0 > > > >> (KIP-382) that already support properties with those prefixes? Would > > it > > > be > > > >> possible for a user to configure MM2 with those properties without > > them > > > >> being interpreted as Connect client overrides, without isolating MM2 > > > onto > > > >> its own cluster and using the IgnoreConnectorClientConfigPolicy > > policy? > > > >> 2. Is the IgnoreConnectorClientConfigPolicy class necessary? The > > default > > > >> for the connector.client.config.policy property could simply be null > > > >> instead of a new policy that, as far as I can tell, isn't an actual > > > policy > > > >> in that its validate(...) method is never invoked and instead > > > represents a > > > >> special case to the Connect framework that says "Drop all overrides > > and > > > >> never use me". > > > >> 3. The PrincipalConnectorClientConfigPolicy seems like a specific > > > instance > > > >> of a more general use case: allow exactly a small set of overrides > and > > > no > > > >> others. Why not generalize here and create a policy that accepts a > > list > > > of > > > >> allowed overrides during configuration? > > > >> > > > >> Thanks again for the KIP. > > > >> > > > >> Cheers, > > > >> > > > >> Chris > > > >> > > > >> On Fri, Apr 19, 2019 at 2:53 PM Magesh Nandakumar < > > mageshn@confluent.io > > > > > > > >> wrote: > > > >> > > > >> > Hi all, > > > >> > > > > >> > I've posted "KIP-458: Connector Client Config Override Policy", > > which > > > >> > allows users to override the connector client configurations based > > on > > > a > > > >> > policy defined by the administrator. > > > >> > > > > >> > The KIP can be found at > > > >> > > > > >> > > > > >> > > > > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-458%3A+Connector+Client+Config+Override+Policy > > > >> > . > > > >> > > > > >> > Looking forward for the discussion on the KIP and all of your > > > thoughts & > > > >> > feedback on this enhancement to Connect. > > > >> > > > > >> > Thanks, > > > >> > Magesh > > > >> > > > > >> > > > > > > > > > > --0000000000009d537b0587615a52--