I actually prefer to see those at INFO level. The reason is that the config
system in an application can be complex. Some configs can be overridden in
different layers and it may not be easy to determine what the final binding
value is. The logging in Kafka will serve as the source of truth.

For reference, ZK client logs all overridden values during initialization.
It's a one time thing during starting up, so shouldn't add much noise. It's
very useful for debugging subtle config issues.

Exposing final configs programmatically is potentially useful. If we don't
want to log overridden values out of box, an app can achieve the same thing
using the programming api. The only missing thing is that we won't know
those unused property keys, which is probably less important than seeing
the overridden values.

Thanks,

Jun


On Mon, Feb 10, 2014 at 10:15 AM, Jay Kreps <jay.kr...@gmail.com> wrote:

> Hey Jun,
>
> I think that is reasonable but would object to having it be debug logging?
> I think logging out a bunch of noise during normal operation in a client
> library is pretty ugly. Also, is there value in exposing the final configs
> programmatically?
>
> -Jay
>
>
>
> On Sun, Feb 9, 2014 at 9:23 PM, Jun Rao <jun...@gmail.com> wrote:
>
> > +1 on the new config. Just one comment. Currently, when initiating a
> config
> > (e.g. ProducerConfig), we log those overridden property values and unused
> > property keys (likely due to mis-spelling). This has been very useful for
> > config verification. It would be good to add similar support in the new
> > config.
> >
> > Thanks,
> >
> > Jun
> >
> >
> > On Tue, Feb 4, 2014 at 9:34 AM, Jay Kreps <jay.kr...@gmail.com> wrote:
> >
> > > We touched on this a bit in previous discussions, but I wanted to draw
> > out
> > > the approach to config specifically as an item of discussion.
> > >
> > > The new producer and consumer use a similar key-value config approach
> as
> > > the existing scala clients but have different implementation code to
> help
> > > define these configs. The plan is to use the same approach on the
> server,
> > > once the new clients are complete; so if we agree on this approach it
> > will
> > > be the new default across the board.
> > >
> > > Let me split this into two parts. First I will try to motivate the use
> of
> > > key-value pairs as a configuration api. Then let me discuss the
> mechanics
> > > of specifying and parsing these. If we agree on the public api then the
> > > public api then the implementation details are interesting as this will
> > be
> > > shared across producer, consumer, and broker and potentially some
> tools;
> > > but if we disagree about the api then there is no point in discussing
> the
> > > implementation.
> > >
> > > Let me explain the rationale for this. In a sense a key-value map of
> > > configs is the worst possible API to the programmer using the clients.
> > Let
> > > me contrast the pros and cons versus a POJO and motivate why I think it
> > is
> > > still superior overall.
> > >
> > > Pro: An application can externalize the configuration of its kafka
> > clients
> > > into its own configuration. Whatever config management system the
> client
> > > application is using will likely support key-value pairs, so the client
> > > should be able to directly pull whatever configurations are present and
> > use
> > > them in its client. This means that any configuration the client
> supports
> > > can be added to any application at runtime. With the pojo approach the
> > > client application has to expose each pojo getter as some config
> > parameter.
> > > The result of many applications doing this is that the config is
> > different
> > > for each and it is very hard to have a standard client config shared
> > > across. Moving config into config files allows the usual tooling
> (version
> > > control, review, audit, config deployments separate from code pushes,
> > > etc.).
> > >
> > > Pro: Backwards and forwards compatibility. Provided we stick to our
> java
> > > api many internals can evolve and expose new configs. The application
> can
> > > support both the new and old client by just specifying a config that
> will
> > > be unused in the older version (and of course the reverse--we can
> remove
> > > obsolete configs).
> > >
> > > Pro: We can use a similar mechanism for both the client and the server.
> > > Since most people run the server as a stand-alone process it needs a
> > config
> > > file.
> > >
> > > Pro: Systems like Samza that need to ship configs across the network
> can
> > > easily do so as configs have a natural serialized form. This can be
> done
> > > with pojos using java serialization but it is ugly and has bizare
> failure
> > > cases.
> > >
> > > Con: The IDE gives nice auto-completion for pojos.
> > >
> > > Con: There are some advantages to javadoc as a documentation mechanism
> > for
> > > java people.
> > >
> > > Basically to me this is about operability versus niceness of api and I
> > > think operability is more important.
> > >
> > > Let me now give some details of the config support classes in
> > > kafka.common.config and how they are intended to be used.
> > >
> > > The goal of this code is the following:
> > > 1. Make specifying configs, their expected type (string, numbers,
> lists,
> > > etc) simple and declarative
> > > 2. Allow for validating simple checks (numeric range checks, etc)
> > > 3. Make the config "self-documenting". I.e. we should be able to write
> > code
> > > that generates the configuration documentation off the config def.
> > > 4. Specify default values.
> > > 5. Track which configs actually get used.
> > > 6. Make it easy to get config values.
> > >
> > > There are two classes there: ConfigDef and AbstractConfig. ConfigDef
> > > defines the specification of the accepted configurations and
> > AbstractConfig
> > > is a helper class for implementing the configuration class. The
> > difference
> > > is kind of like the difference between a "class" and an "object":
> > ConfigDef
> > > is for specifying the configurations that are accepted, AbstractConfig
> is
> > > the base class for an instance of these configs.
> > >
> > > You can see this in action here:
> > >
> > >
> >
> https://git-wip-us.apache.org/repos/asf?p=kafka.git;a=blob_plain;f=clients/src/main/java/kafka/clients/producer/ProducerConfig.java;hb=HEAD
> > >
> > > (Ignore the static config names in there for now...I'm not actually
> sure
> > > that is the best approach).
> > >
> > > So the way this works is that the config specification is defined as:
> > >
> > >         config = new ConfigDef().define("bootstrap.brokers", Type.LIST,
> > > "documentation")
> > >
> > >                                 .define("metadata.timeout.ms",
> > Type.LONG,
> > > 60 * 1000, atLeast(0), "documentation")
> > >                                 .define("max.partition.size", Type.INT,
> > > 16384, atLeast(0), "documentation")
> > >
> > >
> > > This is used in a ProducerConfig class which extends AbstractConfig to
> > get
> > > access to some helper methods as well as the logic for tracking which
> > > configs get accessed.
> > >
> > > Currently I have included static String variables for each of the
> config
> > > names in that class. However I actually think that is not very helpful
> as
> > > the javadoc for them doesn't give the constant value and requires
> > > duplicating the documentation. To understand this point look at the
> > javadoc
> > > and note that the doc on the string is not the same as what we define
> in
> > > the ConfigDef. We could just have the javadoc for the config string be
> > the
> > > source of truth but it is actually pretty inconvient for that as it
> > doesn't
> > > show you the value of the constant, just the variable name (unless you
> > > discover how to unhide it). That is fine for the clients, but for the
> > > server would be very weird especially for non-java people. We could
> > attempt
> > > to duplicate documentation between the javadoc and the ConfigDef but
> > given
> > > our struggle to get well-documented config in a single place this seems
> > > unwise.
> > >
> > > So I recommend we have a single source for documentation of these and
> > that
> > > that source be the website documentation on configuration that covers
> > > clients and server and that that be generated off the config defs. The
> > > javadoc on KafkaProducer will link to this table so it should be quite
> > > convenient to discover. This makes things a little more typo prone, but
> > > that should be easily caught by the key detection. This will also make
> it
> > > possible for us to retire configs in the future without causing compile
> > > failures and add configs without having use of them break backwards
> > > compatibility. This is useful during upgrades where you want to be
> > > compatible with the old and new version so you can roll forwards and
> > > backwards.
> > >
> > > -Jay
> > >
> >
>

Reply via email to