You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@kafka.apache.org by Randall Hauch <rh...@gmail.com> on 2017/09/07 23:50:58 UTC

[DISCUSS] KIP-196: Add metrics to Kafka Connect framework

Hi everyone.

I've created a new KIP to add metrics to the Kafka Connect framework:
https://cwiki.apache.org/confluence/display/KAFKA/KIP-196%3A+Add+metrics+to+Kafka+Connect+framework

The KIP approval deadline is looming, so if you're interested in Kafka
Connect metrics please review and provide feedback as soon as possible. I'm
interested not only in whether the metrics are sufficient and appropriate,
but also in whether the MBean naming conventions are okay.

Best regards,

Randall

Re: [DISCUSS] KIP-196: Add metrics to Kafka Connect framework

Posted by Randall Hauch <rh...@gmail.com>.
Hi, James. I was mistaken about how the Kafka metrics are converted to
MBeans and attributes. The MBean is constructed from the group and tags,
and the metrics show up as attributes on the MBean. I'll update the KIP to
reflect this.

On Tue, Sep 12, 2017 at 1:43 AM, James Cheng <wu...@gmail.com> wrote:

> Thanks for the KIP, Randall.
>
> The KIP has one MBean per metric name. Can I suggest an alternate grouping?
>
> kafka.connect:type=connector-metrics,connector=([-.\w]+)
>         connector-type
>         connector-class
>         connector-version
>         status
>
> kafka.connect:type=task-metrics,connector=([-.\w]+),task=([\d]+)
>         status
>         pause-ratio
>         offset-commit-success-percentage
>         offset-commit-failure-percentage
>         offset-commit-max-time
>         offset-commit-99p-time
>         offset-commit-95p-time
>         offset-commit-90p-time
>         offset-commit-75p-time
>         offset-commit-50p-time
>         batch-size-max
>         batch-size-avg
>
> kafka.connect:type=source-task-metrics,connector=([-.\w]+),task=([\d]+)
>         source-record-poll-rate
>         source-record-write-rate
>
> kafka.connect:type=sink-task-metrics,connector=([-.\w]+),task=([\d]+)
>         sink-record-read-rate
>         sink-record-send-rate
>         sink-record-lag-max
>         partition-count
>         offset-commit-95p-time
>         offset-commit-90p-time
>         offset-commit-75p-time
>         offset-commit-50p-time
>         batch-size-max
>         batch-size-avg
>
> kafka.connect:type=sink-task-metrics,connector=([-.\w]+),
> task=([\d]+),topic=([-.\w]+),partition=([\d]+)
>         sink-record-lag
>         sink-record-lag-avg
>         sink-record-lag-max
>
> kafka.connect:type=connect-coordinator-metrics
>         task-count
>         connector-count
>         leader-name
>         state
>         rest-request-rate
>
> kafka.connect:type=connect-coordinator-metrics,name=assigned-tasks
>         assigned-tasks (existing metric, so can't merge in above without
> breaking compatibility)
> kafka.connect:type=connect-coordinator-metrics,name=assigned-connectors
> (existing metric, so can't merge in above without breaking compatibility)
>         assigned-connectors (existing metric, so can't merge in above
> without breaking compatibility)
>
> kafka.connect:type=connect-worker-rebalance-metrics
>         rebalance-success-total
>         rebalance-success-percentage
>         rebalance-failure-total
>         rebalance-failure-percentage
>         rebalance-max-time
>         rebalance-99p-time
>         rebalance-95p-time
>         rebalance-90p-time
>         rebalance-75p-time
>         rebalance-50p-time
>         time-since-last-rebalance
>         task-failure-rate
>
> This lets you use a single MBean selector to select multiple related
> attributes all at once. You can use JMX's wildcards to target which
> connectors or tasks or topics or partitions you care about.
>
> Also notice that for the topic and partition level metrics, the attributes
> are named identically ("sink-record-lag-avg" instead of
> "sink-record-{topic}-{partition}.records-lag-avg"), so monitoring systems
> have a consistent string they can use, instead of needing to
> prefix-and-suffix matching against the attribute name. And TBH, it
> integrates better with the work I'm doing in https://issues.apache.org/
> jira/browse/KAFKA-3480
>
> -James
>
> > On Sep 7, 2017, at 4:50 PM, Randall Hauch <rh...@gmail.com> wrote:
> >
> > Hi everyone.
> >
> > I've created a new KIP to add metrics to the Kafka Connect framework:
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> 196%3A+Add+metrics+to+Kafka+Connect+framework
> >
> > The KIP approval deadline is looming, so if you're interested in Kafka
> > Connect metrics please review and provide feedback as soon as possible.
> I'm
> > interested not only in whether the metrics are sufficient and
> appropriate,
> > but also in whether the MBean naming conventions are okay.
> >
> > Best regards,
> >
> > Randall
>
>

Re: [DISCUSS] KIP-196: Add metrics to Kafka Connect framework

Posted by James Cheng <wu...@gmail.com>.
Thanks for the KIP, Randall.

The KIP has one MBean per metric name. Can I suggest an alternate grouping?

kafka.connect:type=connector-metrics,connector=([-.\w]+)
	connector-type
	connector-class
	connector-version
	status

kafka.connect:type=task-metrics,connector=([-.\w]+),task=([\d]+)
	status
	pause-ratio
	offset-commit-success-percentage
	offset-commit-failure-percentage
	offset-commit-max-time
	offset-commit-99p-time
	offset-commit-95p-time
	offset-commit-90p-time
	offset-commit-75p-time
	offset-commit-50p-time
	batch-size-max
	batch-size-avg

kafka.connect:type=source-task-metrics,connector=([-.\w]+),task=([\d]+)
	source-record-poll-rate
	source-record-write-rate

kafka.connect:type=sink-task-metrics,connector=([-.\w]+),task=([\d]+)
	sink-record-read-rate
	sink-record-send-rate
	sink-record-lag-max
	partition-count
	offset-commit-95p-time
	offset-commit-90p-time
	offset-commit-75p-time
	offset-commit-50p-time
	batch-size-max
	batch-size-avg

kafka.connect:type=sink-task-metrics,connector=([-.\w]+),task=([\d]+),topic=([-.\w]+),partition=([\d]+)
	sink-record-lag
	sink-record-lag-avg
	sink-record-lag-max

kafka.connect:type=connect-coordinator-metrics
	task-count
	connector-count
	leader-name
	state
	rest-request-rate

kafka.connect:type=connect-coordinator-metrics,name=assigned-tasks 
	assigned-tasks (existing metric, so can't merge in above without breaking compatibility)
kafka.connect:type=connect-coordinator-metrics,name=assigned-connectors (existing metric, so can't merge in above without breaking compatibility)
	assigned-connectors (existing metric, so can't merge in above without breaking compatibility)

kafka.connect:type=connect-worker-rebalance-metrics
	rebalance-success-total
	rebalance-success-percentage
	rebalance-failure-total
	rebalance-failure-percentage
	rebalance-max-time
	rebalance-99p-time
	rebalance-95p-time
	rebalance-90p-time
	rebalance-75p-time
	rebalance-50p-time
	time-since-last-rebalance
	task-failure-rate

This lets you use a single MBean selector to select multiple related attributes all at once. You can use JMX's wildcards to target which connectors or tasks or topics or partitions you care about.

Also notice that for the topic and partition level metrics, the attributes are named identically ("sink-record-lag-avg" instead of "sink-record-{topic}-{partition}.records-lag-avg"), so monitoring systems have a consistent string they can use, instead of needing to prefix-and-suffix matching against the attribute name. And TBH, it integrates better with the work I'm doing in https://issues.apache.org/jira/browse/KAFKA-3480

-James

> On Sep 7, 2017, at 4:50 PM, Randall Hauch <rh...@gmail.com> wrote:
> 
> Hi everyone.
> 
> I've created a new KIP to add metrics to the Kafka Connect framework:
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-196%3A+Add+metrics+to+Kafka+Connect+framework
> 
> The KIP approval deadline is looming, so if you're interested in Kafka
> Connect metrics please review and provide feedback as soon as possible. I'm
> interested not only in whether the metrics are sufficient and appropriate,
> but also in whether the MBean naming conventions are okay.
> 
> Best regards,
> 
> Randall


Re: [DISCUSS] KIP-196: Add metrics to Kafka Connect framework

Posted by Ewen Cheslack-Postava <ew...@confluent.io>.
On Mon, Sep 11, 2017 at 4:50 PM, Randall Hauch <rh...@gmail.com> wrote:

> Thanks, Ewen. Comments inline below.
>
> On Mon, Sep 11, 2017 at 5:46 PM, Ewen Cheslack-Postava <ew...@confluent.io>
> wrote:
>
> > Randall,
> >
> > A couple of questions:
> >
> > * Some metrics don't seem to have unique names? e.g.
> > source-record-produce-rate and source-record-produce-total seem like they
> > are duplicated. Looks like maybe just an oversight that the second ones
> > should be changed from "produce" to "write".
> >
>
> Nice catch. You are correct - should be "write" instead of "produce". I
> will correct.
>
>
> > * I think there's a stray extra character in a couple of
> > places: kafka.connect:type=source-task-metrics,name=source-
> > record-produce-total,worker=([-.\w]+)l,connector=([-.\w]+),task=([\d]+)
> > has an extra char after the worker name.
> >
>
> Thanks. Removed in 2 places.
>
>
> > * Are the produce totals actually useful given rebalancing would cancel
> > them out anyway? Doesn't seem like you could do much with them.
> >
>
> Yes, the totals would be since the last rebalance. Maybe that isn't that
> useful. Might be better to capture the offsets and lag as Roger was
> suggestion. Thoughts?
>
>
Sounds like you've since removed it. re: offsets/lag, is this something
that makes sense for source connectors? I don't think I quite understand
the suggestion.


>
> > * Why do transformations get their own metric but not converters? And are
> > we concerned at all about the performance impact of getting such fine
> > grained info? Getting current time isn't free and we've seen before that
> we
> > ended up w/ accidental performance regressions as we tried to check it
> too
> > frequently to enforce timeouts fine grained in the producer (iirc).
> > Batching helps w/ this, but on the consumer side, a max.poll.records=1
> > setting could put you in a bad place, especially since transforms might
> be
> > very lightweight (or nothing) and converters are expected to be
> relatively
> > cheap as well.
> >
>
> We could remove the read, transform, and put time-based metrics for sink
> tasks, and poll, transform, and write time-based metrics. Can/should they
> be replaced with anything else?
>

We can start with any breakdown we want. I think the only tricky issue that
arises is if we decide we *do* want to include more detail. Then the
metrics become a bit confusing (or the naming is really important) because
there is some overlap in the metrics (e.g. if we converted
transformation/conversion/send to "processing" time, but then later broke
them down into those components). But I think that's easily resolved with a
nice little image showing the finest granularity components and the
aggregate components as labels over their constituent parts.


>
>
> > * If we include the worker id everywhere and don't have metrics without
> > that included, isn't that a pain for users that dump this data into some
> > other system? They have to know which worker the connector/task is
> > currently on *or* need to do extra work to merge the metrics from across
> > machines. Including versions with the worker ID can make sense for
> > completeness and accuracy (e.g. technically there are still very slim
> risks
> > of having a task running twice due to zombies), but it seems like bad
> > usability for the common case.
> >
>
> Part of the reason was also to help identify where each of the metrics came
> from, but per the next comment this may not be as useful, either.
> So remove the worker ID in all the task and connector metric names? What
> about the worker metrics?
>

Does worker id work as a tag to keep it isolated w/o including it in the
name? tbh, JMX style metric naming always seemed overly complicated and
messy and I've never fully grokked the details of Kafka metrics as a
result. (To me, this is also a huge usability issue w/ Kafka as it stands
today -- Kafka metrics don't seem to integrate naturally with almost
anything that doesn't assume JMX/Java style metrics.)


>
>
> > * Is aggregating things like source record rate at the (worker,
> connector)
> > level really useful since you're just going to need to do additional
> > aggregation anyway once you've collected metrics across all workers? I'd
> > rather add a smaller number of metrics w/ clear use cases than just try
> to
> > be exhaustive and then have to maintain stuff that nobody actually uses.
> >
>
> Yes, the connector aggregate metrics are maybe not as useful if you also
> have to aggregate them from different workers. Removing them probably also
> reduces the risk of them being misinterpretted.
>

I swear I was not at all motivated by reducing the size of the patch that
this KIP will need reviewed :)


>
>
> > * You have status for connectors but not for tasks. Any reason why? Seems
> > like it'd make sense to expose both, especially since users generally
> care
> > about task status more than connector status (not many connectors
> actually
> > run a monitoring thread.)
> >
>
> Ack.
>
>
> > * Is number of tasks for each connector a useful metric? Not sure whether
> > someone would find this useful or not. Probably not for alerts, but might
> > be useful to be able to check it via your metrics dashboard.
> >
>
> Seems like it might be useful, at least in terms of tracking the number of
> tasks over time. Might not be as useful for connectors that have relatively
> static tasks, but it would be more interesting/useful for connectors that
> create tasks dynamically and periodically request task reconfigurations.
>

Yeah, most will be static. It mostly seems useful a) just for historical
reference, b) possibly to alert on for crazy configs (e.g.
max.tasks=1000000 from a newbie user), and c) possibly to alert on changes
in general just so you know that connectors are changing.


>
> > * Same questions re: granularity of sink tasks/connectors timing and
> > whether the connectors need all the roll-ups of individual (worker, task)
> > values to (worker, connector) level.
> >
>
> I'm fine with taking out the aggregates to keep things simple and prevent
> misunderstanding.
>
>
> > * If we expose the who the worker currently thinks is leader, it might
> also
> > make sense to expose the underlying epoch. Not actually sure if we expose
> > that for the consumer today, but it's an indicator of who is properly up
> to
> > date.
> >
>
> Ack.
>
>
> > * Why worker-level offset commit stats? It's not clear to me that these
> are
> > useful without considering the specific connector.
> >
>
> So would they make more sense on the tasks? Again, on the worker they're
> aggregates.
>

Yeah, task level could make sense. I was just trying to think through how I
would put each of the metrics to use and I wasn't sure what I could infer
from the worker-level commit stats since sink connectors vary a lot in how
they process commits.

-Ewen


>
>
> >
> > -Ewen
> >
> >
> > On Mon, Sep 11, 2017 at 9:43 AM, Randall Hauch <rh...@gmail.com> wrote:
> >
> > > Thanks for reviewing. Responses inline below.
> > >
> > > On Mon, Sep 11, 2017 at 11:22 AM, Roger Hoover <roger.hoover@gmail.com
> >
> > > wrote:
> > >
> > > > Randall,
> > > >
> > > > Thank you for the KIP.  This should improve visibility greatly.  I
> had
> > a
> > > > few questions/ideas for more metrics.
> > > >
> > > >
> > > >    1. What's the relationship between the worker state and the
> > connector
> > > >    status?  Does the 'paused' status at the Connector level include
> the
> > > > time
> > > >    that worker is 'rebalancing'?
> > > >
> > >
> > > The worker state metric simply reports whether the worker is running or
> > > rebalancing. This state is independent of how many connectors are
> > > deployed/running/paused. During a rebalance, the connectors are being
> > > stopped and restarted but are effectively not running.
> > >
> > >
> > > >    2. Are the "Source Connector" metrics like record rate an
> > aggregation
> > > of
> > > >    the "Source Task" metrics?
> > > >
> > >
> > > Yes.
> > >
> > >
> > > >       - How much value is there is monitoring at the "Source
> Connector"
> > > >       level (other than status) if the number of constituent tasks
> may
> > > > change
> > > >       over time?
> > > >
> > >
> > > The task metrics allow you to know whether the tasks are evenly loaded
> > and
> > > each making progress. The aggregate connector metrics tell you how much
> > > work has been performed by all the tasks in that worker. Both are
> useful
> > > IMO.
> > >
> > >
> > > >       - I'm imagining that it's most useful to collect metrics at the
> > > task
> > > >       level as the task-level metrics should be stable regardless of
> > > tasks
> > > >       shifting to different workers
> > > >
> > >
> > > Correct, this is where the most value is because it is the most fine
> > > grained.
> > >
> > >
> > > >       - If so, can we duplicate the Connector Status down at the task
> > > level
> > > >          so that all important metrics can be tracked by task?
> > > >
> > >
> > > Possibly. The challenge is that the threads running the tasks are
> blocked
> > > when a connector is paused.
> > >
> > >
> > > >          3. For the Sink Task metrics
> > > >       - Can we add offset lag and timestamp lag on commit?
> > > >          - After records are flushed/committed
> > > >             - what is the diff between the record timestamps and
> commit
> > > >             time (histogram)?  this is a measure of end-to-end
> pipeline
> > > > latency
> > > >             - what is the diff between record offsets and latest
> offset
> > > of
> > > >             their partition at commit time (histogram)? this is a
> > > > measure of whether
> > > >             this particular task is keeping up
> > > >
> > >
> > > Yeah, possibly. Will have to compare with the consumer metrics to see
> > what
> > > we can get.
> > >
> > >
> > > >          - How about flush error rate?  Assuming the sink connectors
> > are
> > > >       using retries, it would be helpful to know how many errors
> > they're
> > > > seeing
> > > >
> > >
> > > We could add a metric to track how many times the framework receives a
> > > retry exception and then retries, but the connectors may also do this
> on
> > > their own.
> > >
> > >
> > > >       - Can we tell at the framework level how many records were
> > inserted
> > > >       vs updated vs deleted?
> > > >
> > >
> > > No, there's no distinction in the Connect framework.
> > >
> > >
> > > >       - Batching stats
> > > >          - Histogram of flush batch size
> > > >          - Counts of flush trigger method (time vs max batch size)
> > > >
> > >
> > > Should be able to add these.
> > >
> > >
> > > >
> > > > Cheers,
> > > >
> > > > Roger
> > > >
> > > > On Sun, Sep 10, 2017 at 8:45 AM, Randall Hauch <rh...@gmail.com>
> > wrote:
> > > >
> > > > > Thanks, Gwen.
> > > > >
> > > > > That's a great idea, so I've changed the KIP to add those metrics.
> > I've
> > > > > also made a few other changes:
> > > > >
> > > > >
> > > > >    1. The context of all metrics is limited to the activity within
> > the
> > > > >    worker. This wasn't clear before, so I changed the motivation
> and
> > > > metric
> > > > >    descriptions to explicitly state this.
> > > > >    2. Added the worker ID to all MBean attributes. In addition to
> > > > hopefully
> > > > >    making this same scope obvious from within JMX or other metric
> > > > reporting
> > > > >    system. This is also similar to how the Kafka producer and
> > consumer
> > > > > metrics
> > > > >    include the client ID in their MBean attributes. Hopefully this
> > does
> > > > not
> > > > >    negatively impact or complicate how external reporting systems'
> > > > > aggregate
> > > > >    metrics from multiple workers.
> > > > >    3. Stated explicitly that aggregating metrics across workers was
> > out
> > > > of
> > > > >    scope of this KIP.
> > > > >    4. Added metrics to report the connector class and version for
> > both
> > > > sink
> > > > >    and source connectors.
> > > > >
> > > > > Check this KIP's history for details of these changes.
> > > > >
> > > > > Please let me know if you have any other suggestions. I hope to
> start
> > > the
> > > > > voting soon!
> > > > >
> > > > > Best regards,
> > > > >
> > > > > Randall
> > > > >
> > > > > On Thu, Sep 7, 2017 at 9:35 PM, Gwen Shapira <gw...@confluent.io>
> > > wrote:
> > > > >
> > > > > > Thanks for the KIP, Randall. Those are badly needed!
> > > > > >
> > > > > > Can we have two metrics with record rate per task? One before SMT
> > and
> > > > one
> > > > > > after?
> > > > > > We can have cases where we read 5000 rows from JDBC but write 5
> to
> > > > Kafka,
> > > > > > or read 5000 records from Kafka and write 5 due to filtering. I
> > think
> > > > its
> > > > > > important to know both numbers.
> > > > > >
> > > > > >
> > > > > > Gwen
> > > > > >
> > > > > > On Thu, Sep 7, 2017 at 7:50 PM, Randall Hauch <rh...@gmail.com>
> > > > wrote:
> > > > > >
> > > > > > > Hi everyone.
> > > > > > >
> > > > > > > I've created a new KIP to add metrics to the Kafka Connect
> > > framework:
> > > > > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > > > > > > 196%3A+Add+metrics+to+Kafka+Connect+framework
> > > > > > >
> > > > > > > The KIP approval deadline is looming, so if you're interested
> in
> > > > Kafka
> > > > > > > Connect metrics please review and provide feedback as soon as
> > > > possible.
> > > > > > I'm
> > > > > > > interested not only in whether the metrics are sufficient and
> > > > > > appropriate,
> > > > > > > but also in whether the MBean naming conventions are okay.
> > > > > > >
> > > > > > > Best regards,
> > > > > > >
> > > > > > > Randall
> > > > > > >
> > > > > >
> > > > > >
> > > > > >
> > > > > > --
> > > > > > *Gwen Shapira*
> > > > > > Product Manager | Confluent
> > > > > > 650.450.2760 | @gwenshap
> > > > > > Follow us: Twitter <https://twitter.com/ConfluentInc> | blog
> > > > > > <http://www.confluent.io/blog>
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-196: Add metrics to Kafka Connect framework

Posted by Randall Hauch <rh...@gmail.com>.
Regarding the existing rebalance metrics under
"kafka.connect:type=connect-coordinator-metrics", I think we should just
plan on reusing them rather than duplicating them.

On Tue, Sep 12, 2017 at 5:06 PM, Ewen Cheslack-Postava <ew...@confluent.io>
wrote:

> Requests are generally substantial batches of data, you are not guaranteed
> that for the processing batches both because source connectors can hand you
> batches of whatever size they want and consumer's max.poll.records can be
> overridden.
>
> Both SMTs and converters are a concern because they can both be relatively
> cheap such that just checking the time in between them could possibly dwarf
> the cost of applying them.
>
> Also, another thought re: rebalance metrics: we are already getting some
> info via AbstractCoordinator and those actually provide a bit more detail
> in some ways (e.g. join & sync vs the entire rebalance). Not sure if we
> want to effectively duplicate some info so it can all be located under
> Connect names or rely on the existing metrics for some of these.
>
> -Ewen
>
> On Tue, Sep 12, 2017 at 2:05 PM, Roger Hoover <ro...@gmail.com>
> wrote:
>
> > Ewen,
> >
> > I don't know the details of the perf concern.  How is it that the Kafka
> > broker can keep latency stats per request without suffering too much
> > performance?  Maybe SMTs are the only concern b/c they are per-message.
> If
> > so, let's remove those and keep timing info for everything else like
> > flushes, which are batch-based.
> >
> >
> > On Tue, Sep 12, 2017 at 1:32 PM, Ewen Cheslack-Postava <
> ewen@confluent.io>
> > wrote:
> >
> > > On Tue, Sep 12, 2017 at 10:55 AM, Gwen Shapira <gw...@confluent.io>
> > wrote:
> > >
> > > > Ewen, you gave a nice talk at Kafka Summit where you warned about the
> > > > danger of SMTs that slow down the data pipe. If we don't provide the
> > time
> > > > metrics, how will users know when their SMTs are causing performance
> > > > issues?
> > > >
> > >
> > > Metrics aren't the only way to gain insight about performance and
> always
> > > measuring this even when it's not necessarily being used may not make
> > > sense. SMT authors are much better off starting out with a JMH or
> similar
> > > benchmark. What I was referring to in the talk is more about
> > understanding
> > > that the processing for SMTs is entirely synchronous and that means
> > certain
> > > classes of operations will just generally be a bad idea, e.g. anything
> > that
> > > goes out over the network to another service. You don't even really
> need
> > > performance info to determine that that type of transformation will
> cause
> > > problems.
> > >
> > > But my point wasn't that timing info isn't useful. It's that we know
> that
> > > getting timestamps is pretty expensive and we'll already be doing so
> > > elsewhere (e.g. if a source record doesn't include a timestamp). For
> some
> > > use cases such as ByteArrayConverter + no SMTs + lightweight processing
> > > (e.g. just gets handed to a background thread that deals with sending
> the
> > > data), it wouldn't be out of the question that adding 4 or so more
> calls
> > to
> > > get timestamps could become a bottleneck. Since I don't know if it
> would
> > > but we have definitely seen the issue come up before, I would be
> > > conservative in adding the metrics unless we had some numbers showing
> it
> > > doesn't matter or doesn't matter much.
> > >
> > > In general, I don't think metrics that require always-on measurement
> are
> > a
> > > good way to get fine grained performance information. Instrumenting
> > > different phases that imply different types of performance problems can
> > be
> > > helpful (e.g. "processing time" that should be CPU/memory throughput
> > bound
> > > vs. "send time" that, at least for many connectors, is more likely to
> be
> > IO
> > > bound), but if you want finer-grained details, you probably either want
> > > something that can be toggled on/off temporarily or just use a tool
> > that's
> > > really designed for the job, i.e. a profiler like perf.
> > >
> > > -Ewen
> > >
> > >
> > > >
> > > > Gwen
> > > >
> > > > On Mon, Sep 11, 2017 at 7:50 PM Ewen Cheslack-Postava <
> > ewen@confluent.io
> > > >
> > > > wrote:
> > > >
> > > > > re: questions about additional metrics, I think we'll undoubtedly
> > find
> > > > more
> > > > > that people want in practice, but as I mentioned earlier I think
> it's
> > > > > better to add the ones we know we need and then fill out the rest
> as
> > we
> > > > > figure it out. So, e.g., batch size metrics sound like they could
> be
> > > > > useful, but I'd probably wait until we have a clear use case. It
> > seems
> > > > > likely that it could be useful in diagnosing slow connectors (e.g.
> > the
> > > > > implementation just does something inefficient), but I'm not really
> > > sure
> > > > > about that yet.
> > > > >
> > > > > -Ewen
> > > > >
> > > > > On Mon, Sep 11, 2017 at 7:11 PM, Randall Hauch <rh...@gmail.com>
> > > wrote:
> > > > >
> > > > > > Based on Roger and Ewen's feedback, I removed the aggregate
> metrics
> > > as
> > > > > they
> > > > > > would be difficult to make use of without extra work. This
> > simplified
> > > > > > things a great deal, and I took the opportunity to reorganize the
> > > > groups
> > > > > of
> > > > > > metrics. Also, based upon Ewen's concerns regarding measuring
> > > > > > times/durations, I removed all time-related metrics except for
> the
> > > > offset
> > > > > > commits and rebalances, which are infrequent enough to warrant
> the
> > > > > capture
> > > > > > of percentiles. Roger asked about capturing batch size metrics
> for
> > > > source
> > > > > > and sink tasks, and offset lag metrics for sink tasks. Finally,
> > Ewen
> > > > > > pointed out that all count/total metrics are only valid since the
> > > most
> > > > > > recent rebalance and are therefore less meaningful, and were
> > removed.
> > > > > >
> > > > > > On Mon, Sep 11, 2017 at 6:50 PM, Randall Hauch <rhauch@gmail.com
> >
> > > > wrote:
> > > > > >
> > > > > > > Thanks, Ewen. Comments inline below.
> > > > > > >
> > > > > > > On Mon, Sep 11, 2017 at 5:46 PM, Ewen Cheslack-Postava <
> > > > > > ewen@confluent.io>
> > > > > > > wrote:
> > > > > > >
> > > > > > >> Randall,
> > > > > > >>
> > > > > > >> A couple of questions:
> > > > > > >>
> > > > > > >> * Some metrics don't seem to have unique names? e.g.
> > > > > > >> source-record-produce-rate and source-record-produce-total
> seem
> > > like
> > > > > > they
> > > > > > >> are duplicated. Looks like maybe just an oversight that the
> > second
> > > > > ones
> > > > > > >> should be changed from "produce" to "write".
> > > > > > >>
> > > > > > >
> > > > > > > Nice catch. You are correct - should be "write" instead of
> > > > "produce". I
> > > > > > > will correct.
> > > > > > >
> > > > > > >
> > > > > > >> * I think there's a stray extra character in a couple of
> > > > > > >> places: kafka.connect:type=source-task
> > > -metrics,name=source-record-
> > > > > > >> produce-total,worker=([-.\w]+)l,connector=([-.\w]+),task=([\
> > d]+)
> > > > > > >> has an extra char after the worker name.
> > > > > > >>
> > > > > > >
> > > > > > > Thanks. Removed in 2 places.
> > > > > > >
> > > > > > >
> > > > > > >> * Are the produce totals actually useful given rebalancing
> would
> > > > > cancel
> > > > > > >> them out anyway? Doesn't seem like you could do much with
> them.
> > > > > > >>
> > > > > > >
> > > > > > > Yes, the totals would be since the last rebalance. Maybe that
> > isn't
> > > > > that
> > > > > > > useful. Might be better to capture the offsets and lag as Roger
> > was
> > > > > > > suggestion. Thoughts?
> > > > > > >
> > > > > > >
> > > > > > >> * Why do transformations get their own metric but not
> > converters?
> > > > And
> > > > > > are
> > > > > > >> we concerned at all about the performance impact of getting
> such
> > > > fine
> > > > > > >> grained info? Getting current time isn't free and we've seen
> > > before
> > > > > that
> > > > > > >> we
> > > > > > >> ended up w/ accidental performance regressions as we tried to
> > > check
> > > > it
> > > > > > too
> > > > > > >> frequently to enforce timeouts fine grained in the producer
> > > (iirc).
> > > > > > >> Batching helps w/ this, but on the consumer side, a
> > > > max.poll.records=1
> > > > > > >> setting could put you in a bad place, especially since
> > transforms
> > > > > might
> > > > > > be
> > > > > > >> very lightweight (or nothing) and converters are expected to
> be
> > > > > > relatively
> > > > > > >> cheap as well.
> > > > > > >>
> > > > > > >
> > > > > > > We could remove the read, transform, and put time-based metrics
> > for
> > > > > sink
> > > > > > > tasks, and poll, transform, and write time-based metrics.
> > > Can/should
> > > > > they
> > > > > > > be replaced with anything else?
> > > > > > >
> > > > > > >
> > > > > > >> * If we include the worker id everywhere and don't have
> metrics
> > > > > without
> > > > > > >> that included, isn't that a pain for users that dump this data
> > > into
> > > > > some
> > > > > > >> other system? They have to know which worker the
> connector/task
> > is
> > > > > > >> currently on *or* need to do extra work to merge the metrics
> > from
> > > > > across
> > > > > > >> machines. Including versions with the worker ID can make sense
> > for
> > > > > > >> completeness and accuracy (e.g. technically there are still
> very
> > > > slim
> > > > > > >> risks
> > > > > > >> of having a task running twice due to zombies), but it seems
> > like
> > > > bad
> > > > > > >> usability for the common case.
> > > > > > >>
> > > > > > >
> > > > > > > Part of the reason was also to help identify where each of the
> > > > metrics
> > > > > > > came from, but per the next comment this may not be as useful,
> > > > either.
> > > > > > > So remove the worker ID in all the task and connector metric
> > names?
> > > > > What
> > > > > > > about the worker metrics?
> > > > > > >
> > > > > > >
> > > > > > >> * Is aggregating things like source record rate at the
> (worker,
> > > > > > connector)
> > > > > > >> level really useful since you're just going to need to do
> > > additional
> > > > > > >> aggregation anyway once you've collected metrics across all
> > > workers?
> > > > > I'd
> > > > > > >> rather add a smaller number of metrics w/ clear use cases than
> > > just
> > > > > try
> > > > > > to
> > > > > > >> be exhaustive and then have to maintain stuff that nobody
> > actually
> > > > > uses.
> > > > > > >>
> > > > > > >
> > > > > > > Yes, the connector aggregate metrics are maybe not as useful if
> > you
> > > > > also
> > > > > > > have to aggregate them from different workers. Removing them
> > > probably
> > > > > > also
> > > > > > > reduces the risk of them being misinterpretted.
> > > > > > >
> > > > > > >
> > > > > > >> * You have status for connectors but not for tasks. Any reason
> > > why?
> > > > > > Seems
> > > > > > >> like it'd make sense to expose both, especially since users
> > > > generally
> > > > > > care
> > > > > > >> about task status more than connector status (not many
> > connectors
> > > > > > actually
> > > > > > >> run a monitoring thread.)
> > > > > > >>
> > > > > > >
> > > > > > > Ack.
> > > > > > >
> > > > > > >
> > > > > > >> * Is number of tasks for each connector a useful metric? Not
> > sure
> > > > > > whether
> > > > > > >> someone would find this useful or not. Probably not for
> alerts,
> > > but
> > > > > > might
> > > > > > >> be useful to be able to check it via your metrics dashboard.
> > > > > > >>
> > > > > > >
> > > > > > > Seems like it might be useful, at least in terms of tracking
> the
> > > > number
> > > > > > of
> > > > > > > tasks over time. Might not be as useful for connectors that
> have
> > > > > > relatively
> > > > > > > static tasks, but it would be more interesting/useful for
> > > connectors
> > > > > that
> > > > > > > create tasks dynamically and periodically request task
> > > > > reconfigurations.
> > > > > > >
> > > > > > >
> > > > > > >> * Same questions re: granularity of sink tasks/connectors
> timing
> > > and
> > > > > > >> whether the connectors need all the roll-ups of individual
> > > (worker,
> > > > > > task)
> > > > > > >> values to (worker, connector) level.
> > > > > > >>
> > > > > > >
> > > > > > > I'm fine with taking out the aggregates to keep things simple
> and
> > > > > prevent
> > > > > > > misunderstanding.
> > > > > > >
> > > > > > >
> > > > > > >> * If we expose the who the worker currently thinks is leader,
> it
> > > > might
> > > > > > >> also
> > > > > > >> make sense to expose the underlying epoch. Not actually sure
> if
> > we
> > > > > > expose
> > > > > > >> that for the consumer today, but it's an indicator of who is
> > > > properly
> > > > > up
> > > > > > >> to
> > > > > > >> date.
> > > > > > >>
> > > > > > >
> > > > > > > Ack.
> > > > > > >
> > > > > > >
> > > > > > >> * Why worker-level offset commit stats? It's not clear to me
> > that
> > > > > these
> > > > > > >> are
> > > > > > >> useful without considering the specific connector.
> > > > > > >>
> > > > > > >
> > > > > > > So would they make more sense on the tasks? Again, on the
> worker
> > > > > they're
> > > > > > > aggregates.
> > > > > > >
> > > > > > >
> > > > > > >>
> > > > > > >> -Ewen
> > > > > > >>
> > > > > > >>
> > > > > > >> On Mon, Sep 11, 2017 at 9:43 AM, Randall Hauch <
> > rhauch@gmail.com>
> > > > > > wrote:
> > > > > > >>
> > > > > > >> > Thanks for reviewing. Responses inline below.
> > > > > > >> >
> > > > > > >> > On Mon, Sep 11, 2017 at 11:22 AM, Roger Hoover <
> > > > > > roger.hoover@gmail.com>
> > > > > > >> > wrote:
> > > > > > >> >
> > > > > > >> > > Randall,
> > > > > > >> > >
> > > > > > >> > > Thank you for the KIP.  This should improve visibility
> > > > greatly.  I
> > > > > > >> had a
> > > > > > >> > > few questions/ideas for more metrics.
> > > > > > >> > >
> > > > > > >> > >
> > > > > > >> > >    1. What's the relationship between the worker state and
> > the
> > > > > > >> connector
> > > > > > >> > >    status?  Does the 'paused' status at the Connector
> level
> > > > > include
> > > > > > >> the
> > > > > > >> > > time
> > > > > > >> > >    that worker is 'rebalancing'?
> > > > > > >> > >
> > > > > > >> >
> > > > > > >> > The worker state metric simply reports whether the worker is
> > > > running
> > > > > > or
> > > > > > >> > rebalancing. This state is independent of how many
> connectors
> > > are
> > > > > > >> > deployed/running/paused. During a rebalance, the connectors
> > are
> > > > > being
> > > > > > >> > stopped and restarted but are effectively not running.
> > > > > > >> >
> > > > > > >> >
> > > > > > >> > >    2. Are the "Source Connector" metrics like record rate
> an
> > > > > > >> aggregation
> > > > > > >> > of
> > > > > > >> > >    the "Source Task" metrics?
> > > > > > >> > >
> > > > > > >> >
> > > > > > >> > Yes.
> > > > > > >> >
> > > > > > >> >
> > > > > > >> > >       - How much value is there is monitoring at the
> "Source
> > > > > > >> Connector"
> > > > > > >> > >       level (other than status) if the number of
> constituent
> > > > tasks
> > > > > > may
> > > > > > >> > > change
> > > > > > >> > >       over time?
> > > > > > >> > >
> > > > > > >> >
> > > > > > >> > The task metrics allow you to know whether the tasks are
> > evenly
> > > > > loaded
> > > > > > >> and
> > > > > > >> > each making progress. The aggregate connector metrics tell
> you
> > > how
> > > > > > much
> > > > > > >> > work has been performed by all the tasks in that worker.
> Both
> > > are
> > > > > > useful
> > > > > > >> > IMO.
> > > > > > >> >
> > > > > > >> >
> > > > > > >> > >       - I'm imagining that it's most useful to collect
> > metrics
> > > > at
> > > > > > the
> > > > > > >> > task
> > > > > > >> > >       level as the task-level metrics should be stable
> > > > regardless
> > > > > of
> > > > > > >> > tasks
> > > > > > >> > >       shifting to different workers
> > > > > > >> > >
> > > > > > >> >
> > > > > > >> > Correct, this is where the most value is because it is the
> > most
> > > > fine
> > > > > > >> > grained.
> > > > > > >> >
> > > > > > >> >
> > > > > > >> > >       - If so, can we duplicate the Connector Status down
> at
> > > the
> > > > > > task
> > > > > > >> > level
> > > > > > >> > >          so that all important metrics can be tracked by
> > task?
> > > > > > >> > >
> > > > > > >> >
> > > > > > >> > Possibly. The challenge is that the threads running the
> tasks
> > > are
> > > > > > >> blocked
> > > > > > >> > when a connector is paused.
> > > > > > >> >
> > > > > > >> >
> > > > > > >> > >          3. For the Sink Task metrics
> > > > > > >> > >       - Can we add offset lag and timestamp lag on commit?
> > > > > > >> > >          - After records are flushed/committed
> > > > > > >> > >             - what is the diff between the record
> timestamps
> > > and
> > > > > > >> commit
> > > > > > >> > >             time (histogram)?  this is a measure of
> > end-to-end
> > > > > > >> pipeline
> > > > > > >> > > latency
> > > > > > >> > >             - what is the diff between record offsets and
> > > latest
> > > > > > >> offset
> > > > > > >> > of
> > > > > > >> > >             their partition at commit time (histogram)?
> this
> > > is
> > > > a
> > > > > > >> > > measure of whether
> > > > > > >> > >             this particular task is keeping up
> > > > > > >> > >
> > > > > > >> >
> > > > > > >> > Yeah, possibly. Will have to compare with the consumer
> metrics
> > > to
> > > > > see
> > > > > > >> what
> > > > > > >> > we can get.
> > > > > > >> >
> > > > > > >> >
> > > > > > >> > >          - How about flush error rate?  Assuming the sink
> > > > > connectors
> > > > > > >> are
> > > > > > >> > >       using retries, it would be helpful to know how many
> > > errors
> > > > > > >> they're
> > > > > > >> > > seeing
> > > > > > >> > >
> > > > > > >> >
> > > > > > >> > We could add a metric to track how many times the framework
> > > > > receives a
> > > > > > >> > retry exception and then retries, but the connectors may
> also
> > do
> > > > > this
> > > > > > on
> > > > > > >> > their own.
> > > > > > >> >
> > > > > > >> >
> > > > > > >> > >       - Can we tell at the framework level how many
> records
> > > were
> > > > > > >> inserted
> > > > > > >> > >       vs updated vs deleted?
> > > > > > >> > >
> > > > > > >> >
> > > > > > >> > No, there's no distinction in the Connect framework.
> > > > > > >> >
> > > > > > >> >
> > > > > > >> > >       - Batching stats
> > > > > > >> > >          - Histogram of flush batch size
> > > > > > >> > >          - Counts of flush trigger method (time vs max
> batch
> > > > size)
> > > > > > >> > >
> > > > > > >> >
> > > > > > >> > Should be able to add these.
> > > > > > >> >
> > > > > > >> >
> > > > > > >> > >
> > > > > > >> > > Cheers,
> > > > > > >> > >
> > > > > > >> > > Roger
> > > > > > >> > >
> > > > > > >> > > On Sun, Sep 10, 2017 at 8:45 AM, Randall Hauch <
> > > > rhauch@gmail.com>
> > > > > > >> wrote:
> > > > > > >> > >
> > > > > > >> > > > Thanks, Gwen.
> > > > > > >> > > >
> > > > > > >> > > > That's a great idea, so I've changed the KIP to add
> those
> > > > > metrics.
> > > > > > >> I've
> > > > > > >> > > > also made a few other changes:
> > > > > > >> > > >
> > > > > > >> > > >
> > > > > > >> > > >    1. The context of all metrics is limited to the
> > activity
> > > > > within
> > > > > > >> the
> > > > > > >> > > >    worker. This wasn't clear before, so I changed the
> > > > motivation
> > > > > > and
> > > > > > >> > > metric
> > > > > > >> > > >    descriptions to explicitly state this.
> > > > > > >> > > >    2. Added the worker ID to all MBean attributes. In
> > > addition
> > > > > to
> > > > > > >> > > hopefully
> > > > > > >> > > >    making this same scope obvious from within JMX or
> other
> > > > > metric
> > > > > > >> > > reporting
> > > > > > >> > > >    system. This is also similar to how the Kafka
> producer
> > > and
> > > > > > >> consumer
> > > > > > >> > > > metrics
> > > > > > >> > > >    include the client ID in their MBean attributes.
> > > Hopefully
> > > > > this
> > > > > > >> does
> > > > > > >> > > not
> > > > > > >> > > >    negatively impact or complicate how external
> reporting
> > > > > systems'
> > > > > > >> > > > aggregate
> > > > > > >> > > >    metrics from multiple workers.
> > > > > > >> > > >    3. Stated explicitly that aggregating metrics across
> > > > workers
> > > > > > was
> > > > > > >> out
> > > > > > >> > > of
> > > > > > >> > > >    scope of this KIP.
> > > > > > >> > > >    4. Added metrics to report the connector class and
> > > version
> > > > > for
> > > > > > >> both
> > > > > > >> > > sink
> > > > > > >> > > >    and source connectors.
> > > > > > >> > > >
> > > > > > >> > > > Check this KIP's history for details of these changes.
> > > > > > >> > > >
> > > > > > >> > > > Please let me know if you have any other suggestions. I
> > hope
> > > > to
> > > > > > >> start
> > > > > > >> > the
> > > > > > >> > > > voting soon!
> > > > > > >> > > >
> > > > > > >> > > > Best regards,
> > > > > > >> > > >
> > > > > > >> > > > Randall
> > > > > > >> > > >
> > > > > > >> > > > On Thu, Sep 7, 2017 at 9:35 PM, Gwen Shapira <
> > > > gwen@confluent.io
> > > > > >
> > > > > > >> > wrote:
> > > > > > >> > > >
> > > > > > >> > > > > Thanks for the KIP, Randall. Those are badly needed!
> > > > > > >> > > > >
> > > > > > >> > > > > Can we have two metrics with record rate per task? One
> > > > before
> > > > > > SMT
> > > > > > >> and
> > > > > > >> > > one
> > > > > > >> > > > > after?
> > > > > > >> > > > > We can have cases where we read 5000 rows from JDBC
> but
> > > > write
> > > > > 5
> > > > > > to
> > > > > > >> > > Kafka,
> > > > > > >> > > > > or read 5000 records from Kafka and write 5 due to
> > > > filtering.
> > > > > I
> > > > > > >> think
> > > > > > >> > > its
> > > > > > >> > > > > important to know both numbers.
> > > > > > >> > > > >
> > > > > > >> > > > >
> > > > > > >> > > > > Gwen
> > > > > > >> > > > >
> > > > > > >> > > > > On Thu, Sep 7, 2017 at 7:50 PM, Randall Hauch <
> > > > > rhauch@gmail.com
> > > > > > >
> > > > > > >> > > wrote:
> > > > > > >> > > > >
> > > > > > >> > > > > > Hi everyone.
> > > > > > >> > > > > >
> > > > > > >> > > > > > I've created a new KIP to add metrics to the Kafka
> > > Connect
> > > > > > >> > framework:
> > > > > > >> > > > > > https://cwiki.apache.org/
> > confluence/display/KAFKA/KIP-
> > > > > > >> > > > > > 196%3A+Add+metrics+to+Kafka+Connect+framework
> > > > > > >> > > > > >
> > > > > > >> > > > > > The KIP approval deadline is looming, so if you're
> > > > > interested
> > > > > > in
> > > > > > >> > > Kafka
> > > > > > >> > > > > > Connect metrics please review and provide feedback
> as
> > > soon
> > > > > as
> > > > > > >> > > possible.
> > > > > > >> > > > > I'm
> > > > > > >> > > > > > interested not only in whether the metrics are
> > > sufficient
> > > > > and
> > > > > > >> > > > > appropriate,
> > > > > > >> > > > > > but also in whether the MBean naming conventions are
> > > okay.
> > > > > > >> > > > > >
> > > > > > >> > > > > > Best regards,
> > > > > > >> > > > > >
> > > > > > >> > > > > > Randall
> > > > > > >> > > > > >
> > > > > > >> > > > >
> > > > > > >> > > > >
> > > > > > >> > > > >
> > > > > > >> > > > > --
> > > > > > >> > > > > *Gwen Shapira*
> > > > > > >> > > > > Product Manager | Confluent
> > > > > > >> > > > > 650.450.2760 <(650)%20450-2760> | @gwenshap
> > > > > > >> > > > > Follow us: Twitter <https://twitter.com/ConfluentInc>
> |
> > > > blog
> > > > > > >> > > > > <http://www.confluent.io/blog>
> > > > > > >> > > > >
> > > > > > >> > > >
> > > > > > >> > >
> > > > > > >> >
> > > > > > >>
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-196: Add metrics to Kafka Connect framework

Posted by Roger Hoover <ro...@gmail.com>.
Sorry, one more thing occurred to me.  Can the names of the time-based
metrics include their units?  That makes it much easier for people
consuming the metrics to interpret them correctly.

For example, offset-commit-max-time would become offset-commit-max-time-ms
or offset-commit-max-time-microsecs (-us?) or whatever you plan to make the
unit be.

On Tue, Sep 12, 2017 at 6:19 PM, Sriram Subramanian <ra...@confluent.io>
wrote:

> FWIW, I agree that time metrics have been very useful in the past. The
> reasoning around perf overhead seems reasonable as well. Can we agree on a
> subset of time metrics that we feel would be super useful for debugging?
>
> On Tue, Sep 12, 2017 at 6:08 PM, Roger Hoover <ro...@gmail.com>
> wrote:
>
> > Thanks, Ewen.
> >
> > I agree with you on the overhead of measuring time for SMTs and
> > converters.  I'd still argue for keeping other metrics like flush time
> b/c
> > even small batches should still be small overhead compared to writing to
> a
> > sink.
> >
> > On Tue, Sep 12, 2017 at 3:06 PM, Ewen Cheslack-Postava <
> ewen@confluent.io>
> > wrote:
> >
> > > Requests are generally substantial batches of data, you are not
> > guaranteed
> > > that for the processing batches both because source connectors can hand
> > you
> > > batches of whatever size they want and consumer's max.poll.records can
> be
> > > overridden.
> > >
> > > Both SMTs and converters are a concern because they can both be
> > relatively
> > > cheap such that just checking the time in between them could possibly
> > dwarf
> > > the cost of applying them.
> > >
> > > Also, another thought re: rebalance metrics: we are already getting
> some
> > > info via AbstractCoordinator and those actually provide a bit more
> detail
> > > in some ways (e.g. join & sync vs the entire rebalance). Not sure if we
> > > want to effectively duplicate some info so it can all be located under
> > > Connect names or rely on the existing metrics for some of these.
> > >
> > > -Ewen
> > >
> > > On Tue, Sep 12, 2017 at 2:05 PM, Roger Hoover <ro...@gmail.com>
> > > wrote:
> > >
> > > > Ewen,
> > > >
> > > > I don't know the details of the perf concern.  How is it that the
> Kafka
> > > > broker can keep latency stats per request without suffering too much
> > > > performance?  Maybe SMTs are the only concern b/c they are
> per-message.
> > > If
> > > > so, let's remove those and keep timing info for everything else like
> > > > flushes, which are batch-based.
> > > >
> > > >
> > > > On Tue, Sep 12, 2017 at 1:32 PM, Ewen Cheslack-Postava <
> > > ewen@confluent.io>
> > > > wrote:
> > > >
> > > > > On Tue, Sep 12, 2017 at 10:55 AM, Gwen Shapira <gw...@confluent.io>
> > > > wrote:
> > > > >
> > > > > > Ewen, you gave a nice talk at Kafka Summit where you warned about
> > the
> > > > > > danger of SMTs that slow down the data pipe. If we don't provide
> > the
> > > > time
> > > > > > metrics, how will users know when their SMTs are causing
> > performance
> > > > > > issues?
> > > > > >
> > > > >
> > > > > Metrics aren't the only way to gain insight about performance and
> > > always
> > > > > measuring this even when it's not necessarily being used may not
> make
> > > > > sense. SMT authors are much better off starting out with a JMH or
> > > similar
> > > > > benchmark. What I was referring to in the talk is more about
> > > > understanding
> > > > > that the processing for SMTs is entirely synchronous and that means
> > > > certain
> > > > > classes of operations will just generally be a bad idea, e.g.
> > anything
> > > > that
> > > > > goes out over the network to another service. You don't even really
> > > need
> > > > > performance info to determine that that type of transformation will
> > > cause
> > > > > problems.
> > > > >
> > > > > But my point wasn't that timing info isn't useful. It's that we
> know
> > > that
> > > > > getting timestamps is pretty expensive and we'll already be doing
> so
> > > > > elsewhere (e.g. if a source record doesn't include a timestamp).
> For
> > > some
> > > > > use cases such as ByteArrayConverter + no SMTs + lightweight
> > processing
> > > > > (e.g. just gets handed to a background thread that deals with
> sending
> > > the
> > > > > data), it wouldn't be out of the question that adding 4 or so more
> > > calls
> > > > to
> > > > > get timestamps could become a bottleneck. Since I don't know if it
> > > would
> > > > > but we have definitely seen the issue come up before, I would be
> > > > > conservative in adding the metrics unless we had some numbers
> showing
> > > it
> > > > > doesn't matter or doesn't matter much.
> > > > >
> > > > > In general, I don't think metrics that require always-on
> measurement
> > > are
> > > > a
> > > > > good way to get fine grained performance information. Instrumenting
> > > > > different phases that imply different types of performance problems
> > can
> > > > be
> > > > > helpful (e.g. "processing time" that should be CPU/memory
> throughput
> > > > bound
> > > > > vs. "send time" that, at least for many connectors, is more likely
> to
> > > be
> > > > IO
> > > > > bound), but if you want finer-grained details, you probably either
> > want
> > > > > something that can be toggled on/off temporarily or just use a tool
> > > > that's
> > > > > really designed for the job, i.e. a profiler like perf.
> > > > >
> > > > > -Ewen
> > > > >
> > > > >
> > > > > >
> > > > > > Gwen
> > > > > >
> > > > > > On Mon, Sep 11, 2017 at 7:50 PM Ewen Cheslack-Postava <
> > > > ewen@confluent.io
> > > > > >
> > > > > > wrote:
> > > > > >
> > > > > > > re: questions about additional metrics, I think we'll
> undoubtedly
> > > > find
> > > > > > more
> > > > > > > that people want in practice, but as I mentioned earlier I
> think
> > > it's
> > > > > > > better to add the ones we know we need and then fill out the
> rest
> > > as
> > > > we
> > > > > > > figure it out. So, e.g., batch size metrics sound like they
> could
> > > be
> > > > > > > useful, but I'd probably wait until we have a clear use case.
> It
> > > > seems
> > > > > > > likely that it could be useful in diagnosing slow connectors
> > (e.g.
> > > > the
> > > > > > > implementation just does something inefficient), but I'm not
> > really
> > > > > sure
> > > > > > > about that yet.
> > > > > > >
> > > > > > > -Ewen
> > > > > > >
> > > > > > > On Mon, Sep 11, 2017 at 7:11 PM, Randall Hauch <
> rhauch@gmail.com
> > >
> > > > > wrote:
> > > > > > >
> > > > > > > > Based on Roger and Ewen's feedback, I removed the aggregate
> > > metrics
> > > > > as
> > > > > > > they
> > > > > > > > would be difficult to make use of without extra work. This
> > > > simplified
> > > > > > > > things a great deal, and I took the opportunity to reorganize
> > the
> > > > > > groups
> > > > > > > of
> > > > > > > > metrics. Also, based upon Ewen's concerns regarding measuring
> > > > > > > > times/durations, I removed all time-related metrics except
> for
> > > the
> > > > > > offset
> > > > > > > > commits and rebalances, which are infrequent enough to
> warrant
> > > the
> > > > > > > capture
> > > > > > > > of percentiles. Roger asked about capturing batch size
> metrics
> > > for
> > > > > > source
> > > > > > > > and sink tasks, and offset lag metrics for sink tasks.
> Finally,
> > > > Ewen
> > > > > > > > pointed out that all count/total metrics are only valid since
> > the
> > > > > most
> > > > > > > > recent rebalance and are therefore less meaningful, and were
> > > > removed.
> > > > > > > >
> > > > > > > > On Mon, Sep 11, 2017 at 6:50 PM, Randall Hauch <
> > rhauch@gmail.com
> > > >
> > > > > > wrote:
> > > > > > > >
> > > > > > > > > Thanks, Ewen. Comments inline below.
> > > > > > > > >
> > > > > > > > > On Mon, Sep 11, 2017 at 5:46 PM, Ewen Cheslack-Postava <
> > > > > > > > ewen@confluent.io>
> > > > > > > > > wrote:
> > > > > > > > >
> > > > > > > > >> Randall,
> > > > > > > > >>
> > > > > > > > >> A couple of questions:
> > > > > > > > >>
> > > > > > > > >> * Some metrics don't seem to have unique names? e.g.
> > > > > > > > >> source-record-produce-rate and source-record-produce-total
> > > seem
> > > > > like
> > > > > > > > they
> > > > > > > > >> are duplicated. Looks like maybe just an oversight that
> the
> > > > second
> > > > > > > ones
> > > > > > > > >> should be changed from "produce" to "write".
> > > > > > > > >>
> > > > > > > > >
> > > > > > > > > Nice catch. You are correct - should be "write" instead of
> > > > > > "produce". I
> > > > > > > > > will correct.
> > > > > > > > >
> > > > > > > > >
> > > > > > > > >> * I think there's a stray extra character in a couple of
> > > > > > > > >> places: kafka.connect:type=source-task
> > > > > -metrics,name=source-record-
> > > > > > > > >> produce-total,worker=([-.\w]+)
> > l,connector=([-.\w]+),task=([\
> > > > d]+)
> > > > > > > > >> has an extra char after the worker name.
> > > > > > > > >>
> > > > > > > > >
> > > > > > > > > Thanks. Removed in 2 places.
> > > > > > > > >
> > > > > > > > >
> > > > > > > > >> * Are the produce totals actually useful given rebalancing
> > > would
> > > > > > > cancel
> > > > > > > > >> them out anyway? Doesn't seem like you could do much with
> > > them.
> > > > > > > > >>
> > > > > > > > >
> > > > > > > > > Yes, the totals would be since the last rebalance. Maybe
> that
> > > > isn't
> > > > > > > that
> > > > > > > > > useful. Might be better to capture the offsets and lag as
> > Roger
> > > > was
> > > > > > > > > suggestion. Thoughts?
> > > > > > > > >
> > > > > > > > >
> > > > > > > > >> * Why do transformations get their own metric but not
> > > > converters?
> > > > > > And
> > > > > > > > are
> > > > > > > > >> we concerned at all about the performance impact of
> getting
> > > such
> > > > > > fine
> > > > > > > > >> grained info? Getting current time isn't free and we've
> seen
> > > > > before
> > > > > > > that
> > > > > > > > >> we
> > > > > > > > >> ended up w/ accidental performance regressions as we tried
> > to
> > > > > check
> > > > > > it
> > > > > > > > too
> > > > > > > > >> frequently to enforce timeouts fine grained in the
> producer
> > > > > (iirc).
> > > > > > > > >> Batching helps w/ this, but on the consumer side, a
> > > > > > max.poll.records=1
> > > > > > > > >> setting could put you in a bad place, especially since
> > > > transforms
> > > > > > > might
> > > > > > > > be
> > > > > > > > >> very lightweight (or nothing) and converters are expected
> to
> > > be
> > > > > > > > relatively
> > > > > > > > >> cheap as well.
> > > > > > > > >>
> > > > > > > > >
> > > > > > > > > We could remove the read, transform, and put time-based
> > metrics
> > > > for
> > > > > > > sink
> > > > > > > > > tasks, and poll, transform, and write time-based metrics.
> > > > > Can/should
> > > > > > > they
> > > > > > > > > be replaced with anything else?
> > > > > > > > >
> > > > > > > > >
> > > > > > > > >> * If we include the worker id everywhere and don't have
> > > metrics
> > > > > > > without
> > > > > > > > >> that included, isn't that a pain for users that dump this
> > data
> > > > > into
> > > > > > > some
> > > > > > > > >> other system? They have to know which worker the
> > > connector/task
> > > > is
> > > > > > > > >> currently on *or* need to do extra work to merge the
> metrics
> > > > from
> > > > > > > across
> > > > > > > > >> machines. Including versions with the worker ID can make
> > sense
> > > > for
> > > > > > > > >> completeness and accuracy (e.g. technically there are
> still
> > > very
> > > > > > slim
> > > > > > > > >> risks
> > > > > > > > >> of having a task running twice due to zombies), but it
> seems
> > > > like
> > > > > > bad
> > > > > > > > >> usability for the common case.
> > > > > > > > >>
> > > > > > > > >
> > > > > > > > > Part of the reason was also to help identify where each of
> > the
> > > > > > metrics
> > > > > > > > > came from, but per the next comment this may not be as
> > useful,
> > > > > > either.
> > > > > > > > > So remove the worker ID in all the task and connector
> metric
> > > > names?
> > > > > > > What
> > > > > > > > > about the worker metrics?
> > > > > > > > >
> > > > > > > > >
> > > > > > > > >> * Is aggregating things like source record rate at the
> > > (worker,
> > > > > > > > connector)
> > > > > > > > >> level really useful since you're just going to need to do
> > > > > additional
> > > > > > > > >> aggregation anyway once you've collected metrics across
> all
> > > > > workers?
> > > > > > > I'd
> > > > > > > > >> rather add a smaller number of metrics w/ clear use cases
> > than
> > > > > just
> > > > > > > try
> > > > > > > > to
> > > > > > > > >> be exhaustive and then have to maintain stuff that nobody
> > > > actually
> > > > > > > uses.
> > > > > > > > >>
> > > > > > > > >
> > > > > > > > > Yes, the connector aggregate metrics are maybe not as
> useful
> > if
> > > > you
> > > > > > > also
> > > > > > > > > have to aggregate them from different workers. Removing
> them
> > > > > probably
> > > > > > > > also
> > > > > > > > > reduces the risk of them being misinterpretted.
> > > > > > > > >
> > > > > > > > >
> > > > > > > > >> * You have status for connectors but not for tasks. Any
> > reason
> > > > > why?
> > > > > > > > Seems
> > > > > > > > >> like it'd make sense to expose both, especially since
> users
> > > > > > generally
> > > > > > > > care
> > > > > > > > >> about task status more than connector status (not many
> > > > connectors
> > > > > > > > actually
> > > > > > > > >> run a monitoring thread.)
> > > > > > > > >>
> > > > > > > > >
> > > > > > > > > Ack.
> > > > > > > > >
> > > > > > > > >
> > > > > > > > >> * Is number of tasks for each connector a useful metric?
> Not
> > > > sure
> > > > > > > > whether
> > > > > > > > >> someone would find this useful or not. Probably not for
> > > alerts,
> > > > > but
> > > > > > > > might
> > > > > > > > >> be useful to be able to check it via your metrics
> dashboard.
> > > > > > > > >>
> > > > > > > > >
> > > > > > > > > Seems like it might be useful, at least in terms of
> tracking
> > > the
> > > > > > number
> > > > > > > > of
> > > > > > > > > tasks over time. Might not be as useful for connectors that
> > > have
> > > > > > > > relatively
> > > > > > > > > static tasks, but it would be more interesting/useful for
> > > > > connectors
> > > > > > > that
> > > > > > > > > create tasks dynamically and periodically request task
> > > > > > > reconfigurations.
> > > > > > > > >
> > > > > > > > >
> > > > > > > > >> * Same questions re: granularity of sink tasks/connectors
> > > timing
> > > > > and
> > > > > > > > >> whether the connectors need all the roll-ups of individual
> > > > > (worker,
> > > > > > > > task)
> > > > > > > > >> values to (worker, connector) level.
> > > > > > > > >>
> > > > > > > > >
> > > > > > > > > I'm fine with taking out the aggregates to keep things
> simple
> > > and
> > > > > > > prevent
> > > > > > > > > misunderstanding.
> > > > > > > > >
> > > > > > > > >
> > > > > > > > >> * If we expose the who the worker currently thinks is
> > leader,
> > > it
> > > > > > might
> > > > > > > > >> also
> > > > > > > > >> make sense to expose the underlying epoch. Not actually
> sure
> > > if
> > > > we
> > > > > > > > expose
> > > > > > > > >> that for the consumer today, but it's an indicator of who
> is
> > > > > > properly
> > > > > > > up
> > > > > > > > >> to
> > > > > > > > >> date.
> > > > > > > > >>
> > > > > > > > >
> > > > > > > > > Ack.
> > > > > > > > >
> > > > > > > > >
> > > > > > > > >> * Why worker-level offset commit stats? It's not clear to
> me
> > > > that
> > > > > > > these
> > > > > > > > >> are
> > > > > > > > >> useful without considering the specific connector.
> > > > > > > > >>
> > > > > > > > >
> > > > > > > > > So would they make more sense on the tasks? Again, on the
> > > worker
> > > > > > > they're
> > > > > > > > > aggregates.
> > > > > > > > >
> > > > > > > > >
> > > > > > > > >>
> > > > > > > > >> -Ewen
> > > > > > > > >>
> > > > > > > > >>
> > > > > > > > >> On Mon, Sep 11, 2017 at 9:43 AM, Randall Hauch <
> > > > rhauch@gmail.com>
> > > > > > > > wrote:
> > > > > > > > >>
> > > > > > > > >> > Thanks for reviewing. Responses inline below.
> > > > > > > > >> >
> > > > > > > > >> > On Mon, Sep 11, 2017 at 11:22 AM, Roger Hoover <
> > > > > > > > roger.hoover@gmail.com>
> > > > > > > > >> > wrote:
> > > > > > > > >> >
> > > > > > > > >> > > Randall,
> > > > > > > > >> > >
> > > > > > > > >> > > Thank you for the KIP.  This should improve visibility
> > > > > > greatly.  I
> > > > > > > > >> had a
> > > > > > > > >> > > few questions/ideas for more metrics.
> > > > > > > > >> > >
> > > > > > > > >> > >
> > > > > > > > >> > >    1. What's the relationship between the worker state
> > and
> > > > the
> > > > > > > > >> connector
> > > > > > > > >> > >    status?  Does the 'paused' status at the Connector
> > > level
> > > > > > > include
> > > > > > > > >> the
> > > > > > > > >> > > time
> > > > > > > > >> > >    that worker is 'rebalancing'?
> > > > > > > > >> > >
> > > > > > > > >> >
> > > > > > > > >> > The worker state metric simply reports whether the
> worker
> > is
> > > > > > running
> > > > > > > > or
> > > > > > > > >> > rebalancing. This state is independent of how many
> > > connectors
> > > > > are
> > > > > > > > >> > deployed/running/paused. During a rebalance, the
> > connectors
> > > > are
> > > > > > > being
> > > > > > > > >> > stopped and restarted but are effectively not running.
> > > > > > > > >> >
> > > > > > > > >> >
> > > > > > > > >> > >    2. Are the "Source Connector" metrics like record
> > rate
> > > an
> > > > > > > > >> aggregation
> > > > > > > > >> > of
> > > > > > > > >> > >    the "Source Task" metrics?
> > > > > > > > >> > >
> > > > > > > > >> >
> > > > > > > > >> > Yes.
> > > > > > > > >> >
> > > > > > > > >> >
> > > > > > > > >> > >       - How much value is there is monitoring at the
> > > "Source
> > > > > > > > >> Connector"
> > > > > > > > >> > >       level (other than status) if the number of
> > > constituent
> > > > > > tasks
> > > > > > > > may
> > > > > > > > >> > > change
> > > > > > > > >> > >       over time?
> > > > > > > > >> > >
> > > > > > > > >> >
> > > > > > > > >> > The task metrics allow you to know whether the tasks are
> > > > evenly
> > > > > > > loaded
> > > > > > > > >> and
> > > > > > > > >> > each making progress. The aggregate connector metrics
> tell
> > > you
> > > > > how
> > > > > > > > much
> > > > > > > > >> > work has been performed by all the tasks in that worker.
> > > Both
> > > > > are
> > > > > > > > useful
> > > > > > > > >> > IMO.
> > > > > > > > >> >
> > > > > > > > >> >
> > > > > > > > >> > >       - I'm imagining that it's most useful to collect
> > > > metrics
> > > > > > at
> > > > > > > > the
> > > > > > > > >> > task
> > > > > > > > >> > >       level as the task-level metrics should be stable
> > > > > > regardless
> > > > > > > of
> > > > > > > > >> > tasks
> > > > > > > > >> > >       shifting to different workers
> > > > > > > > >> > >
> > > > > > > > >> >
> > > > > > > > >> > Correct, this is where the most value is because it is
> the
> > > > most
> > > > > > fine
> > > > > > > > >> > grained.
> > > > > > > > >> >
> > > > > > > > >> >
> > > > > > > > >> > >       - If so, can we duplicate the Connector Status
> > down
> > > at
> > > > > the
> > > > > > > > task
> > > > > > > > >> > level
> > > > > > > > >> > >          so that all important metrics can be tracked
> by
> > > > task?
> > > > > > > > >> > >
> > > > > > > > >> >
> > > > > > > > >> > Possibly. The challenge is that the threads running the
> > > tasks
> > > > > are
> > > > > > > > >> blocked
> > > > > > > > >> > when a connector is paused.
> > > > > > > > >> >
> > > > > > > > >> >
> > > > > > > > >> > >          3. For the Sink Task metrics
> > > > > > > > >> > >       - Can we add offset lag and timestamp lag on
> > commit?
> > > > > > > > >> > >          - After records are flushed/committed
> > > > > > > > >> > >             - what is the diff between the record
> > > timestamps
> > > > > and
> > > > > > > > >> commit
> > > > > > > > >> > >             time (histogram)?  this is a measure of
> > > > end-to-end
> > > > > > > > >> pipeline
> > > > > > > > >> > > latency
> > > > > > > > >> > >             - what is the diff between record offsets
> > and
> > > > > latest
> > > > > > > > >> offset
> > > > > > > > >> > of
> > > > > > > > >> > >             their partition at commit time
> (histogram)?
> > > this
> > > > > is
> > > > > > a
> > > > > > > > >> > > measure of whether
> > > > > > > > >> > >             this particular task is keeping up
> > > > > > > > >> > >
> > > > > > > > >> >
> > > > > > > > >> > Yeah, possibly. Will have to compare with the consumer
> > > metrics
> > > > > to
> > > > > > > see
> > > > > > > > >> what
> > > > > > > > >> > we can get.
> > > > > > > > >> >
> > > > > > > > >> >
> > > > > > > > >> > >          - How about flush error rate?  Assuming the
> > sink
> > > > > > > connectors
> > > > > > > > >> are
> > > > > > > > >> > >       using retries, it would be helpful to know how
> > many
> > > > > errors
> > > > > > > > >> they're
> > > > > > > > >> > > seeing
> > > > > > > > >> > >
> > > > > > > > >> >
> > > > > > > > >> > We could add a metric to track how many times the
> > framework
> > > > > > > receives a
> > > > > > > > >> > retry exception and then retries, but the connectors may
> > > also
> > > > do
> > > > > > > this
> > > > > > > > on
> > > > > > > > >> > their own.
> > > > > > > > >> >
> > > > > > > > >> >
> > > > > > > > >> > >       - Can we tell at the framework level how many
> > > records
> > > > > were
> > > > > > > > >> inserted
> > > > > > > > >> > >       vs updated vs deleted?
> > > > > > > > >> > >
> > > > > > > > >> >
> > > > > > > > >> > No, there's no distinction in the Connect framework.
> > > > > > > > >> >
> > > > > > > > >> >
> > > > > > > > >> > >       - Batching stats
> > > > > > > > >> > >          - Histogram of flush batch size
> > > > > > > > >> > >          - Counts of flush trigger method (time vs max
> > > batch
> > > > > > size)
> > > > > > > > >> > >
> > > > > > > > >> >
> > > > > > > > >> > Should be able to add these.
> > > > > > > > >> >
> > > > > > > > >> >
> > > > > > > > >> > >
> > > > > > > > >> > > Cheers,
> > > > > > > > >> > >
> > > > > > > > >> > > Roger
> > > > > > > > >> > >
> > > > > > > > >> > > On Sun, Sep 10, 2017 at 8:45 AM, Randall Hauch <
> > > > > > rhauch@gmail.com>
> > > > > > > > >> wrote:
> > > > > > > > >> > >
> > > > > > > > >> > > > Thanks, Gwen.
> > > > > > > > >> > > >
> > > > > > > > >> > > > That's a great idea, so I've changed the KIP to add
> > > those
> > > > > > > metrics.
> > > > > > > > >> I've
> > > > > > > > >> > > > also made a few other changes:
> > > > > > > > >> > > >
> > > > > > > > >> > > >
> > > > > > > > >> > > >    1. The context of all metrics is limited to the
> > > > activity
> > > > > > > within
> > > > > > > > >> the
> > > > > > > > >> > > >    worker. This wasn't clear before, so I changed
> the
> > > > > > motivation
> > > > > > > > and
> > > > > > > > >> > > metric
> > > > > > > > >> > > >    descriptions to explicitly state this.
> > > > > > > > >> > > >    2. Added the worker ID to all MBean attributes.
> In
> > > > > addition
> > > > > > > to
> > > > > > > > >> > > hopefully
> > > > > > > > >> > > >    making this same scope obvious from within JMX or
> > > other
> > > > > > > metric
> > > > > > > > >> > > reporting
> > > > > > > > >> > > >    system. This is also similar to how the Kafka
> > > producer
> > > > > and
> > > > > > > > >> consumer
> > > > > > > > >> > > > metrics
> > > > > > > > >> > > >    include the client ID in their MBean attributes.
> > > > > Hopefully
> > > > > > > this
> > > > > > > > >> does
> > > > > > > > >> > > not
> > > > > > > > >> > > >    negatively impact or complicate how external
> > > reporting
> > > > > > > systems'
> > > > > > > > >> > > > aggregate
> > > > > > > > >> > > >    metrics from multiple workers.
> > > > > > > > >> > > >    3. Stated explicitly that aggregating metrics
> > across
> > > > > > workers
> > > > > > > > was
> > > > > > > > >> out
> > > > > > > > >> > > of
> > > > > > > > >> > > >    scope of this KIP.
> > > > > > > > >> > > >    4. Added metrics to report the connector class
> and
> > > > > version
> > > > > > > for
> > > > > > > > >> both
> > > > > > > > >> > > sink
> > > > > > > > >> > > >    and source connectors.
> > > > > > > > >> > > >
> > > > > > > > >> > > > Check this KIP's history for details of these
> changes.
> > > > > > > > >> > > >
> > > > > > > > >> > > > Please let me know if you have any other
> suggestions.
> > I
> > > > hope
> > > > > > to
> > > > > > > > >> start
> > > > > > > > >> > the
> > > > > > > > >> > > > voting soon!
> > > > > > > > >> > > >
> > > > > > > > >> > > > Best regards,
> > > > > > > > >> > > >
> > > > > > > > >> > > > Randall
> > > > > > > > >> > > >
> > > > > > > > >> > > > On Thu, Sep 7, 2017 at 9:35 PM, Gwen Shapira <
> > > > > > gwen@confluent.io
> > > > > > > >
> > > > > > > > >> > wrote:
> > > > > > > > >> > > >
> > > > > > > > >> > > > > Thanks for the KIP, Randall. Those are badly
> needed!
> > > > > > > > >> > > > >
> > > > > > > > >> > > > > Can we have two metrics with record rate per task?
> > One
> > > > > > before
> > > > > > > > SMT
> > > > > > > > >> and
> > > > > > > > >> > > one
> > > > > > > > >> > > > > after?
> > > > > > > > >> > > > > We can have cases where we read 5000 rows from
> JDBC
> > > but
> > > > > > write
> > > > > > > 5
> > > > > > > > to
> > > > > > > > >> > > Kafka,
> > > > > > > > >> > > > > or read 5000 records from Kafka and write 5 due to
> > > > > > filtering.
> > > > > > > I
> > > > > > > > >> think
> > > > > > > > >> > > its
> > > > > > > > >> > > > > important to know both numbers.
> > > > > > > > >> > > > >
> > > > > > > > >> > > > >
> > > > > > > > >> > > > > Gwen
> > > > > > > > >> > > > >
> > > > > > > > >> > > > > On Thu, Sep 7, 2017 at 7:50 PM, Randall Hauch <
> > > > > > > rhauch@gmail.com
> > > > > > > > >
> > > > > > > > >> > > wrote:
> > > > > > > > >> > > > >
> > > > > > > > >> > > > > > Hi everyone.
> > > > > > > > >> > > > > >
> > > > > > > > >> > > > > > I've created a new KIP to add metrics to the
> Kafka
> > > > > Connect
> > > > > > > > >> > framework:
> > > > > > > > >> > > > > > https://cwiki.apache.org/
> > > > confluence/display/KAFKA/KIP-
> > > > > > > > >> > > > > > 196%3A+Add+metrics+to+Kafka+Connect+framework
> > > > > > > > >> > > > > >
> > > > > > > > >> > > > > > The KIP approval deadline is looming, so if
> you're
> > > > > > > interested
> > > > > > > > in
> > > > > > > > >> > > Kafka
> > > > > > > > >> > > > > > Connect metrics please review and provide
> feedback
> > > as
> > > > > soon
> > > > > > > as
> > > > > > > > >> > > possible.
> > > > > > > > >> > > > > I'm
> > > > > > > > >> > > > > > interested not only in whether the metrics are
> > > > > sufficient
> > > > > > > and
> > > > > > > > >> > > > > appropriate,
> > > > > > > > >> > > > > > but also in whether the MBean naming conventions
> > are
> > > > > okay.
> > > > > > > > >> > > > > >
> > > > > > > > >> > > > > > Best regards,
> > > > > > > > >> > > > > >
> > > > > > > > >> > > > > > Randall
> > > > > > > > >> > > > > >
> > > > > > > > >> > > > >
> > > > > > > > >> > > > >
> > > > > > > > >> > > > >
> > > > > > > > >> > > > > --
> > > > > > > > >> > > > > *Gwen Shapira*
> > > > > > > > >> > > > > Product Manager | Confluent
> > > > > > > > >> > > > > 650.450.2760 <(650)%20450-2760> | @gwenshap
> > > > > > > > >> > > > > Follow us: Twitter <https://twitter.com/
> > ConfluentInc>
> > > |
> > > > > > blog
> > > > > > > > >> > > > > <http://www.confluent.io/blog>
> > > > > > > > >> > > > >
> > > > > > > > >> > > >
> > > > > > > > >> > >
> > > > > > > > >> >
> > > > > > > > >>
> > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-196: Add metrics to Kafka Connect framework

Posted by Sriram Subramanian <ra...@confluent.io>.
FWIW, I agree that time metrics have been very useful in the past. The
reasoning around perf overhead seems reasonable as well. Can we agree on a
subset of time metrics that we feel would be super useful for debugging?

On Tue, Sep 12, 2017 at 6:08 PM, Roger Hoover <ro...@gmail.com>
wrote:

> Thanks, Ewen.
>
> I agree with you on the overhead of measuring time for SMTs and
> converters.  I'd still argue for keeping other metrics like flush time b/c
> even small batches should still be small overhead compared to writing to a
> sink.
>
> On Tue, Sep 12, 2017 at 3:06 PM, Ewen Cheslack-Postava <ew...@confluent.io>
> wrote:
>
> > Requests are generally substantial batches of data, you are not
> guaranteed
> > that for the processing batches both because source connectors can hand
> you
> > batches of whatever size they want and consumer's max.poll.records can be
> > overridden.
> >
> > Both SMTs and converters are a concern because they can both be
> relatively
> > cheap such that just checking the time in between them could possibly
> dwarf
> > the cost of applying them.
> >
> > Also, another thought re: rebalance metrics: we are already getting some
> > info via AbstractCoordinator and those actually provide a bit more detail
> > in some ways (e.g. join & sync vs the entire rebalance). Not sure if we
> > want to effectively duplicate some info so it can all be located under
> > Connect names or rely on the existing metrics for some of these.
> >
> > -Ewen
> >
> > On Tue, Sep 12, 2017 at 2:05 PM, Roger Hoover <ro...@gmail.com>
> > wrote:
> >
> > > Ewen,
> > >
> > > I don't know the details of the perf concern.  How is it that the Kafka
> > > broker can keep latency stats per request without suffering too much
> > > performance?  Maybe SMTs are the only concern b/c they are per-message.
> > If
> > > so, let's remove those and keep timing info for everything else like
> > > flushes, which are batch-based.
> > >
> > >
> > > On Tue, Sep 12, 2017 at 1:32 PM, Ewen Cheslack-Postava <
> > ewen@confluent.io>
> > > wrote:
> > >
> > > > On Tue, Sep 12, 2017 at 10:55 AM, Gwen Shapira <gw...@confluent.io>
> > > wrote:
> > > >
> > > > > Ewen, you gave a nice talk at Kafka Summit where you warned about
> the
> > > > > danger of SMTs that slow down the data pipe. If we don't provide
> the
> > > time
> > > > > metrics, how will users know when their SMTs are causing
> performance
> > > > > issues?
> > > > >
> > > >
> > > > Metrics aren't the only way to gain insight about performance and
> > always
> > > > measuring this even when it's not necessarily being used may not make
> > > > sense. SMT authors are much better off starting out with a JMH or
> > similar
> > > > benchmark. What I was referring to in the talk is more about
> > > understanding
> > > > that the processing for SMTs is entirely synchronous and that means
> > > certain
> > > > classes of operations will just generally be a bad idea, e.g.
> anything
> > > that
> > > > goes out over the network to another service. You don't even really
> > need
> > > > performance info to determine that that type of transformation will
> > cause
> > > > problems.
> > > >
> > > > But my point wasn't that timing info isn't useful. It's that we know
> > that
> > > > getting timestamps is pretty expensive and we'll already be doing so
> > > > elsewhere (e.g. if a source record doesn't include a timestamp). For
> > some
> > > > use cases such as ByteArrayConverter + no SMTs + lightweight
> processing
> > > > (e.g. just gets handed to a background thread that deals with sending
> > the
> > > > data), it wouldn't be out of the question that adding 4 or so more
> > calls
> > > to
> > > > get timestamps could become a bottleneck. Since I don't know if it
> > would
> > > > but we have definitely seen the issue come up before, I would be
> > > > conservative in adding the metrics unless we had some numbers showing
> > it
> > > > doesn't matter or doesn't matter much.
> > > >
> > > > In general, I don't think metrics that require always-on measurement
> > are
> > > a
> > > > good way to get fine grained performance information. Instrumenting
> > > > different phases that imply different types of performance problems
> can
> > > be
> > > > helpful (e.g. "processing time" that should be CPU/memory throughput
> > > bound
> > > > vs. "send time" that, at least for many connectors, is more likely to
> > be
> > > IO
> > > > bound), but if you want finer-grained details, you probably either
> want
> > > > something that can be toggled on/off temporarily or just use a tool
> > > that's
> > > > really designed for the job, i.e. a profiler like perf.
> > > >
> > > > -Ewen
> > > >
> > > >
> > > > >
> > > > > Gwen
> > > > >
> > > > > On Mon, Sep 11, 2017 at 7:50 PM Ewen Cheslack-Postava <
> > > ewen@confluent.io
> > > > >
> > > > > wrote:
> > > > >
> > > > > > re: questions about additional metrics, I think we'll undoubtedly
> > > find
> > > > > more
> > > > > > that people want in practice, but as I mentioned earlier I think
> > it's
> > > > > > better to add the ones we know we need and then fill out the rest
> > as
> > > we
> > > > > > figure it out. So, e.g., batch size metrics sound like they could
> > be
> > > > > > useful, but I'd probably wait until we have a clear use case. It
> > > seems
> > > > > > likely that it could be useful in diagnosing slow connectors
> (e.g.
> > > the
> > > > > > implementation just does something inefficient), but I'm not
> really
> > > > sure
> > > > > > about that yet.
> > > > > >
> > > > > > -Ewen
> > > > > >
> > > > > > On Mon, Sep 11, 2017 at 7:11 PM, Randall Hauch <rhauch@gmail.com
> >
> > > > wrote:
> > > > > >
> > > > > > > Based on Roger and Ewen's feedback, I removed the aggregate
> > metrics
> > > > as
> > > > > > they
> > > > > > > would be difficult to make use of without extra work. This
> > > simplified
> > > > > > > things a great deal, and I took the opportunity to reorganize
> the
> > > > > groups
> > > > > > of
> > > > > > > metrics. Also, based upon Ewen's concerns regarding measuring
> > > > > > > times/durations, I removed all time-related metrics except for
> > the
> > > > > offset
> > > > > > > commits and rebalances, which are infrequent enough to warrant
> > the
> > > > > > capture
> > > > > > > of percentiles. Roger asked about capturing batch size metrics
> > for
> > > > > source
> > > > > > > and sink tasks, and offset lag metrics for sink tasks. Finally,
> > > Ewen
> > > > > > > pointed out that all count/total metrics are only valid since
> the
> > > > most
> > > > > > > recent rebalance and are therefore less meaningful, and were
> > > removed.
> > > > > > >
> > > > > > > On Mon, Sep 11, 2017 at 6:50 PM, Randall Hauch <
> rhauch@gmail.com
> > >
> > > > > wrote:
> > > > > > >
> > > > > > > > Thanks, Ewen. Comments inline below.
> > > > > > > >
> > > > > > > > On Mon, Sep 11, 2017 at 5:46 PM, Ewen Cheslack-Postava <
> > > > > > > ewen@confluent.io>
> > > > > > > > wrote:
> > > > > > > >
> > > > > > > >> Randall,
> > > > > > > >>
> > > > > > > >> A couple of questions:
> > > > > > > >>
> > > > > > > >> * Some metrics don't seem to have unique names? e.g.
> > > > > > > >> source-record-produce-rate and source-record-produce-total
> > seem
> > > > like
> > > > > > > they
> > > > > > > >> are duplicated. Looks like maybe just an oversight that the
> > > second
> > > > > > ones
> > > > > > > >> should be changed from "produce" to "write".
> > > > > > > >>
> > > > > > > >
> > > > > > > > Nice catch. You are correct - should be "write" instead of
> > > > > "produce". I
> > > > > > > > will correct.
> > > > > > > >
> > > > > > > >
> > > > > > > >> * I think there's a stray extra character in a couple of
> > > > > > > >> places: kafka.connect:type=source-task
> > > > -metrics,name=source-record-
> > > > > > > >> produce-total,worker=([-.\w]+)
> l,connector=([-.\w]+),task=([\
> > > d]+)
> > > > > > > >> has an extra char after the worker name.
> > > > > > > >>
> > > > > > > >
> > > > > > > > Thanks. Removed in 2 places.
> > > > > > > >
> > > > > > > >
> > > > > > > >> * Are the produce totals actually useful given rebalancing
> > would
> > > > > > cancel
> > > > > > > >> them out anyway? Doesn't seem like you could do much with
> > them.
> > > > > > > >>
> > > > > > > >
> > > > > > > > Yes, the totals would be since the last rebalance. Maybe that
> > > isn't
> > > > > > that
> > > > > > > > useful. Might be better to capture the offsets and lag as
> Roger
> > > was
> > > > > > > > suggestion. Thoughts?
> > > > > > > >
> > > > > > > >
> > > > > > > >> * Why do transformations get their own metric but not
> > > converters?
> > > > > And
> > > > > > > are
> > > > > > > >> we concerned at all about the performance impact of getting
> > such
> > > > > fine
> > > > > > > >> grained info? Getting current time isn't free and we've seen
> > > > before
> > > > > > that
> > > > > > > >> we
> > > > > > > >> ended up w/ accidental performance regressions as we tried
> to
> > > > check
> > > > > it
> > > > > > > too
> > > > > > > >> frequently to enforce timeouts fine grained in the producer
> > > > (iirc).
> > > > > > > >> Batching helps w/ this, but on the consumer side, a
> > > > > max.poll.records=1
> > > > > > > >> setting could put you in a bad place, especially since
> > > transforms
> > > > > > might
> > > > > > > be
> > > > > > > >> very lightweight (or nothing) and converters are expected to
> > be
> > > > > > > relatively
> > > > > > > >> cheap as well.
> > > > > > > >>
> > > > > > > >
> > > > > > > > We could remove the read, transform, and put time-based
> metrics
> > > for
> > > > > > sink
> > > > > > > > tasks, and poll, transform, and write time-based metrics.
> > > > Can/should
> > > > > > they
> > > > > > > > be replaced with anything else?
> > > > > > > >
> > > > > > > >
> > > > > > > >> * If we include the worker id everywhere and don't have
> > metrics
> > > > > > without
> > > > > > > >> that included, isn't that a pain for users that dump this
> data
> > > > into
> > > > > > some
> > > > > > > >> other system? They have to know which worker the
> > connector/task
> > > is
> > > > > > > >> currently on *or* need to do extra work to merge the metrics
> > > from
> > > > > > across
> > > > > > > >> machines. Including versions with the worker ID can make
> sense
> > > for
> > > > > > > >> completeness and accuracy (e.g. technically there are still
> > very
> > > > > slim
> > > > > > > >> risks
> > > > > > > >> of having a task running twice due to zombies), but it seems
> > > like
> > > > > bad
> > > > > > > >> usability for the common case.
> > > > > > > >>
> > > > > > > >
> > > > > > > > Part of the reason was also to help identify where each of
> the
> > > > > metrics
> > > > > > > > came from, but per the next comment this may not be as
> useful,
> > > > > either.
> > > > > > > > So remove the worker ID in all the task and connector metric
> > > names?
> > > > > > What
> > > > > > > > about the worker metrics?
> > > > > > > >
> > > > > > > >
> > > > > > > >> * Is aggregating things like source record rate at the
> > (worker,
> > > > > > > connector)
> > > > > > > >> level really useful since you're just going to need to do
> > > > additional
> > > > > > > >> aggregation anyway once you've collected metrics across all
> > > > workers?
> > > > > > I'd
> > > > > > > >> rather add a smaller number of metrics w/ clear use cases
> than
> > > > just
> > > > > > try
> > > > > > > to
> > > > > > > >> be exhaustive and then have to maintain stuff that nobody
> > > actually
> > > > > > uses.
> > > > > > > >>
> > > > > > > >
> > > > > > > > Yes, the connector aggregate metrics are maybe not as useful
> if
> > > you
> > > > > > also
> > > > > > > > have to aggregate them from different workers. Removing them
> > > > probably
> > > > > > > also
> > > > > > > > reduces the risk of them being misinterpretted.
> > > > > > > >
> > > > > > > >
> > > > > > > >> * You have status for connectors but not for tasks. Any
> reason
> > > > why?
> > > > > > > Seems
> > > > > > > >> like it'd make sense to expose both, especially since users
> > > > > generally
> > > > > > > care
> > > > > > > >> about task status more than connector status (not many
> > > connectors
> > > > > > > actually
> > > > > > > >> run a monitoring thread.)
> > > > > > > >>
> > > > > > > >
> > > > > > > > Ack.
> > > > > > > >
> > > > > > > >
> > > > > > > >> * Is number of tasks for each connector a useful metric? Not
> > > sure
> > > > > > > whether
> > > > > > > >> someone would find this useful or not. Probably not for
> > alerts,
> > > > but
> > > > > > > might
> > > > > > > >> be useful to be able to check it via your metrics dashboard.
> > > > > > > >>
> > > > > > > >
> > > > > > > > Seems like it might be useful, at least in terms of tracking
> > the
> > > > > number
> > > > > > > of
> > > > > > > > tasks over time. Might not be as useful for connectors that
> > have
> > > > > > > relatively
> > > > > > > > static tasks, but it would be more interesting/useful for
> > > > connectors
> > > > > > that
> > > > > > > > create tasks dynamically and periodically request task
> > > > > > reconfigurations.
> > > > > > > >
> > > > > > > >
> > > > > > > >> * Same questions re: granularity of sink tasks/connectors
> > timing
> > > > and
> > > > > > > >> whether the connectors need all the roll-ups of individual
> > > > (worker,
> > > > > > > task)
> > > > > > > >> values to (worker, connector) level.
> > > > > > > >>
> > > > > > > >
> > > > > > > > I'm fine with taking out the aggregates to keep things simple
> > and
> > > > > > prevent
> > > > > > > > misunderstanding.
> > > > > > > >
> > > > > > > >
> > > > > > > >> * If we expose the who the worker currently thinks is
> leader,
> > it
> > > > > might
> > > > > > > >> also
> > > > > > > >> make sense to expose the underlying epoch. Not actually sure
> > if
> > > we
> > > > > > > expose
> > > > > > > >> that for the consumer today, but it's an indicator of who is
> > > > > properly
> > > > > > up
> > > > > > > >> to
> > > > > > > >> date.
> > > > > > > >>
> > > > > > > >
> > > > > > > > Ack.
> > > > > > > >
> > > > > > > >
> > > > > > > >> * Why worker-level offset commit stats? It's not clear to me
> > > that
> > > > > > these
> > > > > > > >> are
> > > > > > > >> useful without considering the specific connector.
> > > > > > > >>
> > > > > > > >
> > > > > > > > So would they make more sense on the tasks? Again, on the
> > worker
> > > > > > they're
> > > > > > > > aggregates.
> > > > > > > >
> > > > > > > >
> > > > > > > >>
> > > > > > > >> -Ewen
> > > > > > > >>
> > > > > > > >>
> > > > > > > >> On Mon, Sep 11, 2017 at 9:43 AM, Randall Hauch <
> > > rhauch@gmail.com>
> > > > > > > wrote:
> > > > > > > >>
> > > > > > > >> > Thanks for reviewing. Responses inline below.
> > > > > > > >> >
> > > > > > > >> > On Mon, Sep 11, 2017 at 11:22 AM, Roger Hoover <
> > > > > > > roger.hoover@gmail.com>
> > > > > > > >> > wrote:
> > > > > > > >> >
> > > > > > > >> > > Randall,
> > > > > > > >> > >
> > > > > > > >> > > Thank you for the KIP.  This should improve visibility
> > > > > greatly.  I
> > > > > > > >> had a
> > > > > > > >> > > few questions/ideas for more metrics.
> > > > > > > >> > >
> > > > > > > >> > >
> > > > > > > >> > >    1. What's the relationship between the worker state
> and
> > > the
> > > > > > > >> connector
> > > > > > > >> > >    status?  Does the 'paused' status at the Connector
> > level
> > > > > > include
> > > > > > > >> the
> > > > > > > >> > > time
> > > > > > > >> > >    that worker is 'rebalancing'?
> > > > > > > >> > >
> > > > > > > >> >
> > > > > > > >> > The worker state metric simply reports whether the worker
> is
> > > > > running
> > > > > > > or
> > > > > > > >> > rebalancing. This state is independent of how many
> > connectors
> > > > are
> > > > > > > >> > deployed/running/paused. During a rebalance, the
> connectors
> > > are
> > > > > > being
> > > > > > > >> > stopped and restarted but are effectively not running.
> > > > > > > >> >
> > > > > > > >> >
> > > > > > > >> > >    2. Are the "Source Connector" metrics like record
> rate
> > an
> > > > > > > >> aggregation
> > > > > > > >> > of
> > > > > > > >> > >    the "Source Task" metrics?
> > > > > > > >> > >
> > > > > > > >> >
> > > > > > > >> > Yes.
> > > > > > > >> >
> > > > > > > >> >
> > > > > > > >> > >       - How much value is there is monitoring at the
> > "Source
> > > > > > > >> Connector"
> > > > > > > >> > >       level (other than status) if the number of
> > constituent
> > > > > tasks
> > > > > > > may
> > > > > > > >> > > change
> > > > > > > >> > >       over time?
> > > > > > > >> > >
> > > > > > > >> >
> > > > > > > >> > The task metrics allow you to know whether the tasks are
> > > evenly
> > > > > > loaded
> > > > > > > >> and
> > > > > > > >> > each making progress. The aggregate connector metrics tell
> > you
> > > > how
> > > > > > > much
> > > > > > > >> > work has been performed by all the tasks in that worker.
> > Both
> > > > are
> > > > > > > useful
> > > > > > > >> > IMO.
> > > > > > > >> >
> > > > > > > >> >
> > > > > > > >> > >       - I'm imagining that it's most useful to collect
> > > metrics
> > > > > at
> > > > > > > the
> > > > > > > >> > task
> > > > > > > >> > >       level as the task-level metrics should be stable
> > > > > regardless
> > > > > > of
> > > > > > > >> > tasks
> > > > > > > >> > >       shifting to different workers
> > > > > > > >> > >
> > > > > > > >> >
> > > > > > > >> > Correct, this is where the most value is because it is the
> > > most
> > > > > fine
> > > > > > > >> > grained.
> > > > > > > >> >
> > > > > > > >> >
> > > > > > > >> > >       - If so, can we duplicate the Connector Status
> down
> > at
> > > > the
> > > > > > > task
> > > > > > > >> > level
> > > > > > > >> > >          so that all important metrics can be tracked by
> > > task?
> > > > > > > >> > >
> > > > > > > >> >
> > > > > > > >> > Possibly. The challenge is that the threads running the
> > tasks
> > > > are
> > > > > > > >> blocked
> > > > > > > >> > when a connector is paused.
> > > > > > > >> >
> > > > > > > >> >
> > > > > > > >> > >          3. For the Sink Task metrics
> > > > > > > >> > >       - Can we add offset lag and timestamp lag on
> commit?
> > > > > > > >> > >          - After records are flushed/committed
> > > > > > > >> > >             - what is the diff between the record
> > timestamps
> > > > and
> > > > > > > >> commit
> > > > > > > >> > >             time (histogram)?  this is a measure of
> > > end-to-end
> > > > > > > >> pipeline
> > > > > > > >> > > latency
> > > > > > > >> > >             - what is the diff between record offsets
> and
> > > > latest
> > > > > > > >> offset
> > > > > > > >> > of
> > > > > > > >> > >             their partition at commit time (histogram)?
> > this
> > > > is
> > > > > a
> > > > > > > >> > > measure of whether
> > > > > > > >> > >             this particular task is keeping up
> > > > > > > >> > >
> > > > > > > >> >
> > > > > > > >> > Yeah, possibly. Will have to compare with the consumer
> > metrics
> > > > to
> > > > > > see
> > > > > > > >> what
> > > > > > > >> > we can get.
> > > > > > > >> >
> > > > > > > >> >
> > > > > > > >> > >          - How about flush error rate?  Assuming the
> sink
> > > > > > connectors
> > > > > > > >> are
> > > > > > > >> > >       using retries, it would be helpful to know how
> many
> > > > errors
> > > > > > > >> they're
> > > > > > > >> > > seeing
> > > > > > > >> > >
> > > > > > > >> >
> > > > > > > >> > We could add a metric to track how many times the
> framework
> > > > > > receives a
> > > > > > > >> > retry exception and then retries, but the connectors may
> > also
> > > do
> > > > > > this
> > > > > > > on
> > > > > > > >> > their own.
> > > > > > > >> >
> > > > > > > >> >
> > > > > > > >> > >       - Can we tell at the framework level how many
> > records
> > > > were
> > > > > > > >> inserted
> > > > > > > >> > >       vs updated vs deleted?
> > > > > > > >> > >
> > > > > > > >> >
> > > > > > > >> > No, there's no distinction in the Connect framework.
> > > > > > > >> >
> > > > > > > >> >
> > > > > > > >> > >       - Batching stats
> > > > > > > >> > >          - Histogram of flush batch size
> > > > > > > >> > >          - Counts of flush trigger method (time vs max
> > batch
> > > > > size)
> > > > > > > >> > >
> > > > > > > >> >
> > > > > > > >> > Should be able to add these.
> > > > > > > >> >
> > > > > > > >> >
> > > > > > > >> > >
> > > > > > > >> > > Cheers,
> > > > > > > >> > >
> > > > > > > >> > > Roger
> > > > > > > >> > >
> > > > > > > >> > > On Sun, Sep 10, 2017 at 8:45 AM, Randall Hauch <
> > > > > rhauch@gmail.com>
> > > > > > > >> wrote:
> > > > > > > >> > >
> > > > > > > >> > > > Thanks, Gwen.
> > > > > > > >> > > >
> > > > > > > >> > > > That's a great idea, so I've changed the KIP to add
> > those
> > > > > > metrics.
> > > > > > > >> I've
> > > > > > > >> > > > also made a few other changes:
> > > > > > > >> > > >
> > > > > > > >> > > >
> > > > > > > >> > > >    1. The context of all metrics is limited to the
> > > activity
> > > > > > within
> > > > > > > >> the
> > > > > > > >> > > >    worker. This wasn't clear before, so I changed the
> > > > > motivation
> > > > > > > and
> > > > > > > >> > > metric
> > > > > > > >> > > >    descriptions to explicitly state this.
> > > > > > > >> > > >    2. Added the worker ID to all MBean attributes. In
> > > > addition
> > > > > > to
> > > > > > > >> > > hopefully
> > > > > > > >> > > >    making this same scope obvious from within JMX or
> > other
> > > > > > metric
> > > > > > > >> > > reporting
> > > > > > > >> > > >    system. This is also similar to how the Kafka
> > producer
> > > > and
> > > > > > > >> consumer
> > > > > > > >> > > > metrics
> > > > > > > >> > > >    include the client ID in their MBean attributes.
> > > > Hopefully
> > > > > > this
> > > > > > > >> does
> > > > > > > >> > > not
> > > > > > > >> > > >    negatively impact or complicate how external
> > reporting
> > > > > > systems'
> > > > > > > >> > > > aggregate
> > > > > > > >> > > >    metrics from multiple workers.
> > > > > > > >> > > >    3. Stated explicitly that aggregating metrics
> across
> > > > > workers
> > > > > > > was
> > > > > > > >> out
> > > > > > > >> > > of
> > > > > > > >> > > >    scope of this KIP.
> > > > > > > >> > > >    4. Added metrics to report the connector class and
> > > > version
> > > > > > for
> > > > > > > >> both
> > > > > > > >> > > sink
> > > > > > > >> > > >    and source connectors.
> > > > > > > >> > > >
> > > > > > > >> > > > Check this KIP's history for details of these changes.
> > > > > > > >> > > >
> > > > > > > >> > > > Please let me know if you have any other suggestions.
> I
> > > hope
> > > > > to
> > > > > > > >> start
> > > > > > > >> > the
> > > > > > > >> > > > voting soon!
> > > > > > > >> > > >
> > > > > > > >> > > > Best regards,
> > > > > > > >> > > >
> > > > > > > >> > > > Randall
> > > > > > > >> > > >
> > > > > > > >> > > > On Thu, Sep 7, 2017 at 9:35 PM, Gwen Shapira <
> > > > > gwen@confluent.io
> > > > > > >
> > > > > > > >> > wrote:
> > > > > > > >> > > >
> > > > > > > >> > > > > Thanks for the KIP, Randall. Those are badly needed!
> > > > > > > >> > > > >
> > > > > > > >> > > > > Can we have two metrics with record rate per task?
> One
> > > > > before
> > > > > > > SMT
> > > > > > > >> and
> > > > > > > >> > > one
> > > > > > > >> > > > > after?
> > > > > > > >> > > > > We can have cases where we read 5000 rows from JDBC
> > but
> > > > > write
> > > > > > 5
> > > > > > > to
> > > > > > > >> > > Kafka,
> > > > > > > >> > > > > or read 5000 records from Kafka and write 5 due to
> > > > > filtering.
> > > > > > I
> > > > > > > >> think
> > > > > > > >> > > its
> > > > > > > >> > > > > important to know both numbers.
> > > > > > > >> > > > >
> > > > > > > >> > > > >
> > > > > > > >> > > > > Gwen
> > > > > > > >> > > > >
> > > > > > > >> > > > > On Thu, Sep 7, 2017 at 7:50 PM, Randall Hauch <
> > > > > > rhauch@gmail.com
> > > > > > > >
> > > > > > > >> > > wrote:
> > > > > > > >> > > > >
> > > > > > > >> > > > > > Hi everyone.
> > > > > > > >> > > > > >
> > > > > > > >> > > > > > I've created a new KIP to add metrics to the Kafka
> > > > Connect
> > > > > > > >> > framework:
> > > > > > > >> > > > > > https://cwiki.apache.org/
> > > confluence/display/KAFKA/KIP-
> > > > > > > >> > > > > > 196%3A+Add+metrics+to+Kafka+Connect+framework
> > > > > > > >> > > > > >
> > > > > > > >> > > > > > The KIP approval deadline is looming, so if you're
> > > > > > interested
> > > > > > > in
> > > > > > > >> > > Kafka
> > > > > > > >> > > > > > Connect metrics please review and provide feedback
> > as
> > > > soon
> > > > > > as
> > > > > > > >> > > possible.
> > > > > > > >> > > > > I'm
> > > > > > > >> > > > > > interested not only in whether the metrics are
> > > > sufficient
> > > > > > and
> > > > > > > >> > > > > appropriate,
> > > > > > > >> > > > > > but also in whether the MBean naming conventions
> are
> > > > okay.
> > > > > > > >> > > > > >
> > > > > > > >> > > > > > Best regards,
> > > > > > > >> > > > > >
> > > > > > > >> > > > > > Randall
> > > > > > > >> > > > > >
> > > > > > > >> > > > >
> > > > > > > >> > > > >
> > > > > > > >> > > > >
> > > > > > > >> > > > > --
> > > > > > > >> > > > > *Gwen Shapira*
> > > > > > > >> > > > > Product Manager | Confluent
> > > > > > > >> > > > > 650.450.2760 <(650)%20450-2760> | @gwenshap
> > > > > > > >> > > > > Follow us: Twitter <https://twitter.com/
> ConfluentInc>
> > |
> > > > > blog
> > > > > > > >> > > > > <http://www.confluent.io/blog>
> > > > > > > >> > > > >
> > > > > > > >> > > >
> > > > > > > >> > >
> > > > > > > >> >
> > > > > > > >>
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-196: Add metrics to Kafka Connect framework

Posted by Roger Hoover <ro...@gmail.com>.
Thanks, Ewen.

I agree with you on the overhead of measuring time for SMTs and
converters.  I'd still argue for keeping other metrics like flush time b/c
even small batches should still be small overhead compared to writing to a
sink.

On Tue, Sep 12, 2017 at 3:06 PM, Ewen Cheslack-Postava <ew...@confluent.io>
wrote:

> Requests are generally substantial batches of data, you are not guaranteed
> that for the processing batches both because source connectors can hand you
> batches of whatever size they want and consumer's max.poll.records can be
> overridden.
>
> Both SMTs and converters are a concern because they can both be relatively
> cheap such that just checking the time in between them could possibly dwarf
> the cost of applying them.
>
> Also, another thought re: rebalance metrics: we are already getting some
> info via AbstractCoordinator and those actually provide a bit more detail
> in some ways (e.g. join & sync vs the entire rebalance). Not sure if we
> want to effectively duplicate some info so it can all be located under
> Connect names or rely on the existing metrics for some of these.
>
> -Ewen
>
> On Tue, Sep 12, 2017 at 2:05 PM, Roger Hoover <ro...@gmail.com>
> wrote:
>
> > Ewen,
> >
> > I don't know the details of the perf concern.  How is it that the Kafka
> > broker can keep latency stats per request without suffering too much
> > performance?  Maybe SMTs are the only concern b/c they are per-message.
> If
> > so, let's remove those and keep timing info for everything else like
> > flushes, which are batch-based.
> >
> >
> > On Tue, Sep 12, 2017 at 1:32 PM, Ewen Cheslack-Postava <
> ewen@confluent.io>
> > wrote:
> >
> > > On Tue, Sep 12, 2017 at 10:55 AM, Gwen Shapira <gw...@confluent.io>
> > wrote:
> > >
> > > > Ewen, you gave a nice talk at Kafka Summit where you warned about the
> > > > danger of SMTs that slow down the data pipe. If we don't provide the
> > time
> > > > metrics, how will users know when their SMTs are causing performance
> > > > issues?
> > > >
> > >
> > > Metrics aren't the only way to gain insight about performance and
> always
> > > measuring this even when it's not necessarily being used may not make
> > > sense. SMT authors are much better off starting out with a JMH or
> similar
> > > benchmark. What I was referring to in the talk is more about
> > understanding
> > > that the processing for SMTs is entirely synchronous and that means
> > certain
> > > classes of operations will just generally be a bad idea, e.g. anything
> > that
> > > goes out over the network to another service. You don't even really
> need
> > > performance info to determine that that type of transformation will
> cause
> > > problems.
> > >
> > > But my point wasn't that timing info isn't useful. It's that we know
> that
> > > getting timestamps is pretty expensive and we'll already be doing so
> > > elsewhere (e.g. if a source record doesn't include a timestamp). For
> some
> > > use cases such as ByteArrayConverter + no SMTs + lightweight processing
> > > (e.g. just gets handed to a background thread that deals with sending
> the
> > > data), it wouldn't be out of the question that adding 4 or so more
> calls
> > to
> > > get timestamps could become a bottleneck. Since I don't know if it
> would
> > > but we have definitely seen the issue come up before, I would be
> > > conservative in adding the metrics unless we had some numbers showing
> it
> > > doesn't matter or doesn't matter much.
> > >
> > > In general, I don't think metrics that require always-on measurement
> are
> > a
> > > good way to get fine grained performance information. Instrumenting
> > > different phases that imply different types of performance problems can
> > be
> > > helpful (e.g. "processing time" that should be CPU/memory throughput
> > bound
> > > vs. "send time" that, at least for many connectors, is more likely to
> be
> > IO
> > > bound), but if you want finer-grained details, you probably either want
> > > something that can be toggled on/off temporarily or just use a tool
> > that's
> > > really designed for the job, i.e. a profiler like perf.
> > >
> > > -Ewen
> > >
> > >
> > > >
> > > > Gwen
> > > >
> > > > On Mon, Sep 11, 2017 at 7:50 PM Ewen Cheslack-Postava <
> > ewen@confluent.io
> > > >
> > > > wrote:
> > > >
> > > > > re: questions about additional metrics, I think we'll undoubtedly
> > find
> > > > more
> > > > > that people want in practice, but as I mentioned earlier I think
> it's
> > > > > better to add the ones we know we need and then fill out the rest
> as
> > we
> > > > > figure it out. So, e.g., batch size metrics sound like they could
> be
> > > > > useful, but I'd probably wait until we have a clear use case. It
> > seems
> > > > > likely that it could be useful in diagnosing slow connectors (e.g.
> > the
> > > > > implementation just does something inefficient), but I'm not really
> > > sure
> > > > > about that yet.
> > > > >
> > > > > -Ewen
> > > > >
> > > > > On Mon, Sep 11, 2017 at 7:11 PM, Randall Hauch <rh...@gmail.com>
> > > wrote:
> > > > >
> > > > > > Based on Roger and Ewen's feedback, I removed the aggregate
> metrics
> > > as
> > > > > they
> > > > > > would be difficult to make use of without extra work. This
> > simplified
> > > > > > things a great deal, and I took the opportunity to reorganize the
> > > > groups
> > > > > of
> > > > > > metrics. Also, based upon Ewen's concerns regarding measuring
> > > > > > times/durations, I removed all time-related metrics except for
> the
> > > > offset
> > > > > > commits and rebalances, which are infrequent enough to warrant
> the
> > > > > capture
> > > > > > of percentiles. Roger asked about capturing batch size metrics
> for
> > > > source
> > > > > > and sink tasks, and offset lag metrics for sink tasks. Finally,
> > Ewen
> > > > > > pointed out that all count/total metrics are only valid since the
> > > most
> > > > > > recent rebalance and are therefore less meaningful, and were
> > removed.
> > > > > >
> > > > > > On Mon, Sep 11, 2017 at 6:50 PM, Randall Hauch <rhauch@gmail.com
> >
> > > > wrote:
> > > > > >
> > > > > > > Thanks, Ewen. Comments inline below.
> > > > > > >
> > > > > > > On Mon, Sep 11, 2017 at 5:46 PM, Ewen Cheslack-Postava <
> > > > > > ewen@confluent.io>
> > > > > > > wrote:
> > > > > > >
> > > > > > >> Randall,
> > > > > > >>
> > > > > > >> A couple of questions:
> > > > > > >>
> > > > > > >> * Some metrics don't seem to have unique names? e.g.
> > > > > > >> source-record-produce-rate and source-record-produce-total
> seem
> > > like
> > > > > > they
> > > > > > >> are duplicated. Looks like maybe just an oversight that the
> > second
> > > > > ones
> > > > > > >> should be changed from "produce" to "write".
> > > > > > >>
> > > > > > >
> > > > > > > Nice catch. You are correct - should be "write" instead of
> > > > "produce". I
> > > > > > > will correct.
> > > > > > >
> > > > > > >
> > > > > > >> * I think there's a stray extra character in a couple of
> > > > > > >> places: kafka.connect:type=source-task
> > > -metrics,name=source-record-
> > > > > > >> produce-total,worker=([-.\w]+)l,connector=([-.\w]+),task=([\
> > d]+)
> > > > > > >> has an extra char after the worker name.
> > > > > > >>
> > > > > > >
> > > > > > > Thanks. Removed in 2 places.
> > > > > > >
> > > > > > >
> > > > > > >> * Are the produce totals actually useful given rebalancing
> would
> > > > > cancel
> > > > > > >> them out anyway? Doesn't seem like you could do much with
> them.
> > > > > > >>
> > > > > > >
> > > > > > > Yes, the totals would be since the last rebalance. Maybe that
> > isn't
> > > > > that
> > > > > > > useful. Might be better to capture the offsets and lag as Roger
> > was
> > > > > > > suggestion. Thoughts?
> > > > > > >
> > > > > > >
> > > > > > >> * Why do transformations get their own metric but not
> > converters?
> > > > And
> > > > > > are
> > > > > > >> we concerned at all about the performance impact of getting
> such
> > > > fine
> > > > > > >> grained info? Getting current time isn't free and we've seen
> > > before
> > > > > that
> > > > > > >> we
> > > > > > >> ended up w/ accidental performance regressions as we tried to
> > > check
> > > > it
> > > > > > too
> > > > > > >> frequently to enforce timeouts fine grained in the producer
> > > (iirc).
> > > > > > >> Batching helps w/ this, but on the consumer side, a
> > > > max.poll.records=1
> > > > > > >> setting could put you in a bad place, especially since
> > transforms
> > > > > might
> > > > > > be
> > > > > > >> very lightweight (or nothing) and converters are expected to
> be
> > > > > > relatively
> > > > > > >> cheap as well.
> > > > > > >>
> > > > > > >
> > > > > > > We could remove the read, transform, and put time-based metrics
> > for
> > > > > sink
> > > > > > > tasks, and poll, transform, and write time-based metrics.
> > > Can/should
> > > > > they
> > > > > > > be replaced with anything else?
> > > > > > >
> > > > > > >
> > > > > > >> * If we include the worker id everywhere and don't have
> metrics
> > > > > without
> > > > > > >> that included, isn't that a pain for users that dump this data
> > > into
> > > > > some
> > > > > > >> other system? They have to know which worker the
> connector/task
> > is
> > > > > > >> currently on *or* need to do extra work to merge the metrics
> > from
> > > > > across
> > > > > > >> machines. Including versions with the worker ID can make sense
> > for
> > > > > > >> completeness and accuracy (e.g. technically there are still
> very
> > > > slim
> > > > > > >> risks
> > > > > > >> of having a task running twice due to zombies), but it seems
> > like
> > > > bad
> > > > > > >> usability for the common case.
> > > > > > >>
> > > > > > >
> > > > > > > Part of the reason was also to help identify where each of the
> > > > metrics
> > > > > > > came from, but per the next comment this may not be as useful,
> > > > either.
> > > > > > > So remove the worker ID in all the task and connector metric
> > names?
> > > > > What
> > > > > > > about the worker metrics?
> > > > > > >
> > > > > > >
> > > > > > >> * Is aggregating things like source record rate at the
> (worker,
> > > > > > connector)
> > > > > > >> level really useful since you're just going to need to do
> > > additional
> > > > > > >> aggregation anyway once you've collected metrics across all
> > > workers?
> > > > > I'd
> > > > > > >> rather add a smaller number of metrics w/ clear use cases than
> > > just
> > > > > try
> > > > > > to
> > > > > > >> be exhaustive and then have to maintain stuff that nobody
> > actually
> > > > > uses.
> > > > > > >>
> > > > > > >
> > > > > > > Yes, the connector aggregate metrics are maybe not as useful if
> > you
> > > > > also
> > > > > > > have to aggregate them from different workers. Removing them
> > > probably
> > > > > > also
> > > > > > > reduces the risk of them being misinterpretted.
> > > > > > >
> > > > > > >
> > > > > > >> * You have status for connectors but not for tasks. Any reason
> > > why?
> > > > > > Seems
> > > > > > >> like it'd make sense to expose both, especially since users
> > > > generally
> > > > > > care
> > > > > > >> about task status more than connector status (not many
> > connectors
> > > > > > actually
> > > > > > >> run a monitoring thread.)
> > > > > > >>
> > > > > > >
> > > > > > > Ack.
> > > > > > >
> > > > > > >
> > > > > > >> * Is number of tasks for each connector a useful metric? Not
> > sure
> > > > > > whether
> > > > > > >> someone would find this useful or not. Probably not for
> alerts,
> > > but
> > > > > > might
> > > > > > >> be useful to be able to check it via your metrics dashboard.
> > > > > > >>
> > > > > > >
> > > > > > > Seems like it might be useful, at least in terms of tracking
> the
> > > > number
> > > > > > of
> > > > > > > tasks over time. Might not be as useful for connectors that
> have
> > > > > > relatively
> > > > > > > static tasks, but it would be more interesting/useful for
> > > connectors
> > > > > that
> > > > > > > create tasks dynamically and periodically request task
> > > > > reconfigurations.
> > > > > > >
> > > > > > >
> > > > > > >> * Same questions re: granularity of sink tasks/connectors
> timing
> > > and
> > > > > > >> whether the connectors need all the roll-ups of individual
> > > (worker,
> > > > > > task)
> > > > > > >> values to (worker, connector) level.
> > > > > > >>
> > > > > > >
> > > > > > > I'm fine with taking out the aggregates to keep things simple
> and
> > > > > prevent
> > > > > > > misunderstanding.
> > > > > > >
> > > > > > >
> > > > > > >> * If we expose the who the worker currently thinks is leader,
> it
> > > > might
> > > > > > >> also
> > > > > > >> make sense to expose the underlying epoch. Not actually sure
> if
> > we
> > > > > > expose
> > > > > > >> that for the consumer today, but it's an indicator of who is
> > > > properly
> > > > > up
> > > > > > >> to
> > > > > > >> date.
> > > > > > >>
> > > > > > >
> > > > > > > Ack.
> > > > > > >
> > > > > > >
> > > > > > >> * Why worker-level offset commit stats? It's not clear to me
> > that
> > > > > these
> > > > > > >> are
> > > > > > >> useful without considering the specific connector.
> > > > > > >>
> > > > > > >
> > > > > > > So would they make more sense on the tasks? Again, on the
> worker
> > > > > they're
> > > > > > > aggregates.
> > > > > > >
> > > > > > >
> > > > > > >>
> > > > > > >> -Ewen
> > > > > > >>
> > > > > > >>
> > > > > > >> On Mon, Sep 11, 2017 at 9:43 AM, Randall Hauch <
> > rhauch@gmail.com>
> > > > > > wrote:
> > > > > > >>
> > > > > > >> > Thanks for reviewing. Responses inline below.
> > > > > > >> >
> > > > > > >> > On Mon, Sep 11, 2017 at 11:22 AM, Roger Hoover <
> > > > > > roger.hoover@gmail.com>
> > > > > > >> > wrote:
> > > > > > >> >
> > > > > > >> > > Randall,
> > > > > > >> > >
> > > > > > >> > > Thank you for the KIP.  This should improve visibility
> > > > greatly.  I
> > > > > > >> had a
> > > > > > >> > > few questions/ideas for more metrics.
> > > > > > >> > >
> > > > > > >> > >
> > > > > > >> > >    1. What's the relationship between the worker state and
> > the
> > > > > > >> connector
> > > > > > >> > >    status?  Does the 'paused' status at the Connector
> level
> > > > > include
> > > > > > >> the
> > > > > > >> > > time
> > > > > > >> > >    that worker is 'rebalancing'?
> > > > > > >> > >
> > > > > > >> >
> > > > > > >> > The worker state metric simply reports whether the worker is
> > > > running
> > > > > > or
> > > > > > >> > rebalancing. This state is independent of how many
> connectors
> > > are
> > > > > > >> > deployed/running/paused. During a rebalance, the connectors
> > are
> > > > > being
> > > > > > >> > stopped and restarted but are effectively not running.
> > > > > > >> >
> > > > > > >> >
> > > > > > >> > >    2. Are the "Source Connector" metrics like record rate
> an
> > > > > > >> aggregation
> > > > > > >> > of
> > > > > > >> > >    the "Source Task" metrics?
> > > > > > >> > >
> > > > > > >> >
> > > > > > >> > Yes.
> > > > > > >> >
> > > > > > >> >
> > > > > > >> > >       - How much value is there is monitoring at the
> "Source
> > > > > > >> Connector"
> > > > > > >> > >       level (other than status) if the number of
> constituent
> > > > tasks
> > > > > > may
> > > > > > >> > > change
> > > > > > >> > >       over time?
> > > > > > >> > >
> > > > > > >> >
> > > > > > >> > The task metrics allow you to know whether the tasks are
> > evenly
> > > > > loaded
> > > > > > >> and
> > > > > > >> > each making progress. The aggregate connector metrics tell
> you
> > > how
> > > > > > much
> > > > > > >> > work has been performed by all the tasks in that worker.
> Both
> > > are
> > > > > > useful
> > > > > > >> > IMO.
> > > > > > >> >
> > > > > > >> >
> > > > > > >> > >       - I'm imagining that it's most useful to collect
> > metrics
> > > > at
> > > > > > the
> > > > > > >> > task
> > > > > > >> > >       level as the task-level metrics should be stable
> > > > regardless
> > > > > of
> > > > > > >> > tasks
> > > > > > >> > >       shifting to different workers
> > > > > > >> > >
> > > > > > >> >
> > > > > > >> > Correct, this is where the most value is because it is the
> > most
> > > > fine
> > > > > > >> > grained.
> > > > > > >> >
> > > > > > >> >
> > > > > > >> > >       - If so, can we duplicate the Connector Status down
> at
> > > the
> > > > > > task
> > > > > > >> > level
> > > > > > >> > >          so that all important metrics can be tracked by
> > task?
> > > > > > >> > >
> > > > > > >> >
> > > > > > >> > Possibly. The challenge is that the threads running the
> tasks
> > > are
> > > > > > >> blocked
> > > > > > >> > when a connector is paused.
> > > > > > >> >
> > > > > > >> >
> > > > > > >> > >          3. For the Sink Task metrics
> > > > > > >> > >       - Can we add offset lag and timestamp lag on commit?
> > > > > > >> > >          - After records are flushed/committed
> > > > > > >> > >             - what is the diff between the record
> timestamps
> > > and
> > > > > > >> commit
> > > > > > >> > >             time (histogram)?  this is a measure of
> > end-to-end
> > > > > > >> pipeline
> > > > > > >> > > latency
> > > > > > >> > >             - what is the diff between record offsets and
> > > latest
> > > > > > >> offset
> > > > > > >> > of
> > > > > > >> > >             their partition at commit time (histogram)?
> this
> > > is
> > > > a
> > > > > > >> > > measure of whether
> > > > > > >> > >             this particular task is keeping up
> > > > > > >> > >
> > > > > > >> >
> > > > > > >> > Yeah, possibly. Will have to compare with the consumer
> metrics
> > > to
> > > > > see
> > > > > > >> what
> > > > > > >> > we can get.
> > > > > > >> >
> > > > > > >> >
> > > > > > >> > >          - How about flush error rate?  Assuming the sink
> > > > > connectors
> > > > > > >> are
> > > > > > >> > >       using retries, it would be helpful to know how many
> > > errors
> > > > > > >> they're
> > > > > > >> > > seeing
> > > > > > >> > >
> > > > > > >> >
> > > > > > >> > We could add a metric to track how many times the framework
> > > > > receives a
> > > > > > >> > retry exception and then retries, but the connectors may
> also
> > do
> > > > > this
> > > > > > on
> > > > > > >> > their own.
> > > > > > >> >
> > > > > > >> >
> > > > > > >> > >       - Can we tell at the framework level how many
> records
> > > were
> > > > > > >> inserted
> > > > > > >> > >       vs updated vs deleted?
> > > > > > >> > >
> > > > > > >> >
> > > > > > >> > No, there's no distinction in the Connect framework.
> > > > > > >> >
> > > > > > >> >
> > > > > > >> > >       - Batching stats
> > > > > > >> > >          - Histogram of flush batch size
> > > > > > >> > >          - Counts of flush trigger method (time vs max
> batch
> > > > size)
> > > > > > >> > >
> > > > > > >> >
> > > > > > >> > Should be able to add these.
> > > > > > >> >
> > > > > > >> >
> > > > > > >> > >
> > > > > > >> > > Cheers,
> > > > > > >> > >
> > > > > > >> > > Roger
> > > > > > >> > >
> > > > > > >> > > On Sun, Sep 10, 2017 at 8:45 AM, Randall Hauch <
> > > > rhauch@gmail.com>
> > > > > > >> wrote:
> > > > > > >> > >
> > > > > > >> > > > Thanks, Gwen.
> > > > > > >> > > >
> > > > > > >> > > > That's a great idea, so I've changed the KIP to add
> those
> > > > > metrics.
> > > > > > >> I've
> > > > > > >> > > > also made a few other changes:
> > > > > > >> > > >
> > > > > > >> > > >
> > > > > > >> > > >    1. The context of all metrics is limited to the
> > activity
> > > > > within
> > > > > > >> the
> > > > > > >> > > >    worker. This wasn't clear before, so I changed the
> > > > motivation
> > > > > > and
> > > > > > >> > > metric
> > > > > > >> > > >    descriptions to explicitly state this.
> > > > > > >> > > >    2. Added the worker ID to all MBean attributes. In
> > > addition
> > > > > to
> > > > > > >> > > hopefully
> > > > > > >> > > >    making this same scope obvious from within JMX or
> other
> > > > > metric
> > > > > > >> > > reporting
> > > > > > >> > > >    system. This is also similar to how the Kafka
> producer
> > > and
> > > > > > >> consumer
> > > > > > >> > > > metrics
> > > > > > >> > > >    include the client ID in their MBean attributes.
> > > Hopefully
> > > > > this
> > > > > > >> does
> > > > > > >> > > not
> > > > > > >> > > >    negatively impact or complicate how external
> reporting
> > > > > systems'
> > > > > > >> > > > aggregate
> > > > > > >> > > >    metrics from multiple workers.
> > > > > > >> > > >    3. Stated explicitly that aggregating metrics across
> > > > workers
> > > > > > was
> > > > > > >> out
> > > > > > >> > > of
> > > > > > >> > > >    scope of this KIP.
> > > > > > >> > > >    4. Added metrics to report the connector class and
> > > version
> > > > > for
> > > > > > >> both
> > > > > > >> > > sink
> > > > > > >> > > >    and source connectors.
> > > > > > >> > > >
> > > > > > >> > > > Check this KIP's history for details of these changes.
> > > > > > >> > > >
> > > > > > >> > > > Please let me know if you have any other suggestions. I
> > hope
> > > > to
> > > > > > >> start
> > > > > > >> > the
> > > > > > >> > > > voting soon!
> > > > > > >> > > >
> > > > > > >> > > > Best regards,
> > > > > > >> > > >
> > > > > > >> > > > Randall
> > > > > > >> > > >
> > > > > > >> > > > On Thu, Sep 7, 2017 at 9:35 PM, Gwen Shapira <
> > > > gwen@confluent.io
> > > > > >
> > > > > > >> > wrote:
> > > > > > >> > > >
> > > > > > >> > > > > Thanks for the KIP, Randall. Those are badly needed!
> > > > > > >> > > > >
> > > > > > >> > > > > Can we have two metrics with record rate per task? One
> > > > before
> > > > > > SMT
> > > > > > >> and
> > > > > > >> > > one
> > > > > > >> > > > > after?
> > > > > > >> > > > > We can have cases where we read 5000 rows from JDBC
> but
> > > > write
> > > > > 5
> > > > > > to
> > > > > > >> > > Kafka,
> > > > > > >> > > > > or read 5000 records from Kafka and write 5 due to
> > > > filtering.
> > > > > I
> > > > > > >> think
> > > > > > >> > > its
> > > > > > >> > > > > important to know both numbers.
> > > > > > >> > > > >
> > > > > > >> > > > >
> > > > > > >> > > > > Gwen
> > > > > > >> > > > >
> > > > > > >> > > > > On Thu, Sep 7, 2017 at 7:50 PM, Randall Hauch <
> > > > > rhauch@gmail.com
> > > > > > >
> > > > > > >> > > wrote:
> > > > > > >> > > > >
> > > > > > >> > > > > > Hi everyone.
> > > > > > >> > > > > >
> > > > > > >> > > > > > I've created a new KIP to add metrics to the Kafka
> > > Connect
> > > > > > >> > framework:
> > > > > > >> > > > > > https://cwiki.apache.org/
> > confluence/display/KAFKA/KIP-
> > > > > > >> > > > > > 196%3A+Add+metrics+to+Kafka+Connect+framework
> > > > > > >> > > > > >
> > > > > > >> > > > > > The KIP approval deadline is looming, so if you're
> > > > > interested
> > > > > > in
> > > > > > >> > > Kafka
> > > > > > >> > > > > > Connect metrics please review and provide feedback
> as
> > > soon
> > > > > as
> > > > > > >> > > possible.
> > > > > > >> > > > > I'm
> > > > > > >> > > > > > interested not only in whether the metrics are
> > > sufficient
> > > > > and
> > > > > > >> > > > > appropriate,
> > > > > > >> > > > > > but also in whether the MBean naming conventions are
> > > okay.
> > > > > > >> > > > > >
> > > > > > >> > > > > > Best regards,
> > > > > > >> > > > > >
> > > > > > >> > > > > > Randall
> > > > > > >> > > > > >
> > > > > > >> > > > >
> > > > > > >> > > > >
> > > > > > >> > > > >
> > > > > > >> > > > > --
> > > > > > >> > > > > *Gwen Shapira*
> > > > > > >> > > > > Product Manager | Confluent
> > > > > > >> > > > > 650.450.2760 <(650)%20450-2760> | @gwenshap
> > > > > > >> > > > > Follow us: Twitter <https://twitter.com/ConfluentInc>
> |
> > > > blog
> > > > > > >> > > > > <http://www.confluent.io/blog>
> > > > > > >> > > > >
> > > > > > >> > > >
> > > > > > >> > >
> > > > > > >> >
> > > > > > >>
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-196: Add metrics to Kafka Connect framework

Posted by Ewen Cheslack-Postava <ew...@confluent.io>.
Requests are generally substantial batches of data, you are not guaranteed
that for the processing batches both because source connectors can hand you
batches of whatever size they want and consumer's max.poll.records can be
overridden.

Both SMTs and converters are a concern because they can both be relatively
cheap such that just checking the time in between them could possibly dwarf
the cost of applying them.

Also, another thought re: rebalance metrics: we are already getting some
info via AbstractCoordinator and those actually provide a bit more detail
in some ways (e.g. join & sync vs the entire rebalance). Not sure if we
want to effectively duplicate some info so it can all be located under
Connect names or rely on the existing metrics for some of these.

-Ewen

On Tue, Sep 12, 2017 at 2:05 PM, Roger Hoover <ro...@gmail.com>
wrote:

> Ewen,
>
> I don't know the details of the perf concern.  How is it that the Kafka
> broker can keep latency stats per request without suffering too much
> performance?  Maybe SMTs are the only concern b/c they are per-message.  If
> so, let's remove those and keep timing info for everything else like
> flushes, which are batch-based.
>
>
> On Tue, Sep 12, 2017 at 1:32 PM, Ewen Cheslack-Postava <ew...@confluent.io>
> wrote:
>
> > On Tue, Sep 12, 2017 at 10:55 AM, Gwen Shapira <gw...@confluent.io>
> wrote:
> >
> > > Ewen, you gave a nice talk at Kafka Summit where you warned about the
> > > danger of SMTs that slow down the data pipe. If we don't provide the
> time
> > > metrics, how will users know when their SMTs are causing performance
> > > issues?
> > >
> >
> > Metrics aren't the only way to gain insight about performance and always
> > measuring this even when it's not necessarily being used may not make
> > sense. SMT authors are much better off starting out with a JMH or similar
> > benchmark. What I was referring to in the talk is more about
> understanding
> > that the processing for SMTs is entirely synchronous and that means
> certain
> > classes of operations will just generally be a bad idea, e.g. anything
> that
> > goes out over the network to another service. You don't even really need
> > performance info to determine that that type of transformation will cause
> > problems.
> >
> > But my point wasn't that timing info isn't useful. It's that we know that
> > getting timestamps is pretty expensive and we'll already be doing so
> > elsewhere (e.g. if a source record doesn't include a timestamp). For some
> > use cases such as ByteArrayConverter + no SMTs + lightweight processing
> > (e.g. just gets handed to a background thread that deals with sending the
> > data), it wouldn't be out of the question that adding 4 or so more calls
> to
> > get timestamps could become a bottleneck. Since I don't know if it would
> > but we have definitely seen the issue come up before, I would be
> > conservative in adding the metrics unless we had some numbers showing it
> > doesn't matter or doesn't matter much.
> >
> > In general, I don't think metrics that require always-on measurement are
> a
> > good way to get fine grained performance information. Instrumenting
> > different phases that imply different types of performance problems can
> be
> > helpful (e.g. "processing time" that should be CPU/memory throughput
> bound
> > vs. "send time" that, at least for many connectors, is more likely to be
> IO
> > bound), but if you want finer-grained details, you probably either want
> > something that can be toggled on/off temporarily or just use a tool
> that's
> > really designed for the job, i.e. a profiler like perf.
> >
> > -Ewen
> >
> >
> > >
> > > Gwen
> > >
> > > On Mon, Sep 11, 2017 at 7:50 PM Ewen Cheslack-Postava <
> ewen@confluent.io
> > >
> > > wrote:
> > >
> > > > re: questions about additional metrics, I think we'll undoubtedly
> find
> > > more
> > > > that people want in practice, but as I mentioned earlier I think it's
> > > > better to add the ones we know we need and then fill out the rest as
> we
> > > > figure it out. So, e.g., batch size metrics sound like they could be
> > > > useful, but I'd probably wait until we have a clear use case. It
> seems
> > > > likely that it could be useful in diagnosing slow connectors (e.g.
> the
> > > > implementation just does something inefficient), but I'm not really
> > sure
> > > > about that yet.
> > > >
> > > > -Ewen
> > > >
> > > > On Mon, Sep 11, 2017 at 7:11 PM, Randall Hauch <rh...@gmail.com>
> > wrote:
> > > >
> > > > > Based on Roger and Ewen's feedback, I removed the aggregate metrics
> > as
> > > > they
> > > > > would be difficult to make use of without extra work. This
> simplified
> > > > > things a great deal, and I took the opportunity to reorganize the
> > > groups
> > > > of
> > > > > metrics. Also, based upon Ewen's concerns regarding measuring
> > > > > times/durations, I removed all time-related metrics except for the
> > > offset
> > > > > commits and rebalances, which are infrequent enough to warrant the
> > > > capture
> > > > > of percentiles. Roger asked about capturing batch size metrics for
> > > source
> > > > > and sink tasks, and offset lag metrics for sink tasks. Finally,
> Ewen
> > > > > pointed out that all count/total metrics are only valid since the
> > most
> > > > > recent rebalance and are therefore less meaningful, and were
> removed.
> > > > >
> > > > > On Mon, Sep 11, 2017 at 6:50 PM, Randall Hauch <rh...@gmail.com>
> > > wrote:
> > > > >
> > > > > > Thanks, Ewen. Comments inline below.
> > > > > >
> > > > > > On Mon, Sep 11, 2017 at 5:46 PM, Ewen Cheslack-Postava <
> > > > > ewen@confluent.io>
> > > > > > wrote:
> > > > > >
> > > > > >> Randall,
> > > > > >>
> > > > > >> A couple of questions:
> > > > > >>
> > > > > >> * Some metrics don't seem to have unique names? e.g.
> > > > > >> source-record-produce-rate and source-record-produce-total seem
> > like
> > > > > they
> > > > > >> are duplicated. Looks like maybe just an oversight that the
> second
> > > > ones
> > > > > >> should be changed from "produce" to "write".
> > > > > >>
> > > > > >
> > > > > > Nice catch. You are correct - should be "write" instead of
> > > "produce". I
> > > > > > will correct.
> > > > > >
> > > > > >
> > > > > >> * I think there's a stray extra character in a couple of
> > > > > >> places: kafka.connect:type=source-task
> > -metrics,name=source-record-
> > > > > >> produce-total,worker=([-.\w]+)l,connector=([-.\w]+),task=([\
> d]+)
> > > > > >> has an extra char after the worker name.
> > > > > >>
> > > > > >
> > > > > > Thanks. Removed in 2 places.
> > > > > >
> > > > > >
> > > > > >> * Are the produce totals actually useful given rebalancing would
> > > > cancel
> > > > > >> them out anyway? Doesn't seem like you could do much with them.
> > > > > >>
> > > > > >
> > > > > > Yes, the totals would be since the last rebalance. Maybe that
> isn't
> > > > that
> > > > > > useful. Might be better to capture the offsets and lag as Roger
> was
> > > > > > suggestion. Thoughts?
> > > > > >
> > > > > >
> > > > > >> * Why do transformations get their own metric but not
> converters?
> > > And
> > > > > are
> > > > > >> we concerned at all about the performance impact of getting such
> > > fine
> > > > > >> grained info? Getting current time isn't free and we've seen
> > before
> > > > that
> > > > > >> we
> > > > > >> ended up w/ accidental performance regressions as we tried to
> > check
> > > it
> > > > > too
> > > > > >> frequently to enforce timeouts fine grained in the producer
> > (iirc).
> > > > > >> Batching helps w/ this, but on the consumer side, a
> > > max.poll.records=1
> > > > > >> setting could put you in a bad place, especially since
> transforms
> > > > might
> > > > > be
> > > > > >> very lightweight (or nothing) and converters are expected to be
> > > > > relatively
> > > > > >> cheap as well.
> > > > > >>
> > > > > >
> > > > > > We could remove the read, transform, and put time-based metrics
> for
> > > > sink
> > > > > > tasks, and poll, transform, and write time-based metrics.
> > Can/should
> > > > they
> > > > > > be replaced with anything else?
> > > > > >
> > > > > >
> > > > > >> * If we include the worker id everywhere and don't have metrics
> > > > without
> > > > > >> that included, isn't that a pain for users that dump this data
> > into
> > > > some
> > > > > >> other system? They have to know which worker the connector/task
> is
> > > > > >> currently on *or* need to do extra work to merge the metrics
> from
> > > > across
> > > > > >> machines. Including versions with the worker ID can make sense
> for
> > > > > >> completeness and accuracy (e.g. technically there are still very
> > > slim
> > > > > >> risks
> > > > > >> of having a task running twice due to zombies), but it seems
> like
> > > bad
> > > > > >> usability for the common case.
> > > > > >>
> > > > > >
> > > > > > Part of the reason was also to help identify where each of the
> > > metrics
> > > > > > came from, but per the next comment this may not be as useful,
> > > either.
> > > > > > So remove the worker ID in all the task and connector metric
> names?
> > > > What
> > > > > > about the worker metrics?
> > > > > >
> > > > > >
> > > > > >> * Is aggregating things like source record rate at the (worker,
> > > > > connector)
> > > > > >> level really useful since you're just going to need to do
> > additional
> > > > > >> aggregation anyway once you've collected metrics across all
> > workers?
> > > > I'd
> > > > > >> rather add a smaller number of metrics w/ clear use cases than
> > just
> > > > try
> > > > > to
> > > > > >> be exhaustive and then have to maintain stuff that nobody
> actually
> > > > uses.
> > > > > >>
> > > > > >
> > > > > > Yes, the connector aggregate metrics are maybe not as useful if
> you
> > > > also
> > > > > > have to aggregate them from different workers. Removing them
> > probably
> > > > > also
> > > > > > reduces the risk of them being misinterpretted.
> > > > > >
> > > > > >
> > > > > >> * You have status for connectors but not for tasks. Any reason
> > why?
> > > > > Seems
> > > > > >> like it'd make sense to expose both, especially since users
> > > generally
> > > > > care
> > > > > >> about task status more than connector status (not many
> connectors
> > > > > actually
> > > > > >> run a monitoring thread.)
> > > > > >>
> > > > > >
> > > > > > Ack.
> > > > > >
> > > > > >
> > > > > >> * Is number of tasks for each connector a useful metric? Not
> sure
> > > > > whether
> > > > > >> someone would find this useful or not. Probably not for alerts,
> > but
> > > > > might
> > > > > >> be useful to be able to check it via your metrics dashboard.
> > > > > >>
> > > > > >
> > > > > > Seems like it might be useful, at least in terms of tracking the
> > > number
> > > > > of
> > > > > > tasks over time. Might not be as useful for connectors that have
> > > > > relatively
> > > > > > static tasks, but it would be more interesting/useful for
> > connectors
> > > > that
> > > > > > create tasks dynamically and periodically request task
> > > > reconfigurations.
> > > > > >
> > > > > >
> > > > > >> * Same questions re: granularity of sink tasks/connectors timing
> > and
> > > > > >> whether the connectors need all the roll-ups of individual
> > (worker,
> > > > > task)
> > > > > >> values to (worker, connector) level.
> > > > > >>
> > > > > >
> > > > > > I'm fine with taking out the aggregates to keep things simple and
> > > > prevent
> > > > > > misunderstanding.
> > > > > >
> > > > > >
> > > > > >> * If we expose the who the worker currently thinks is leader, it
> > > might
> > > > > >> also
> > > > > >> make sense to expose the underlying epoch. Not actually sure if
> we
> > > > > expose
> > > > > >> that for the consumer today, but it's an indicator of who is
> > > properly
> > > > up
> > > > > >> to
> > > > > >> date.
> > > > > >>
> > > > > >
> > > > > > Ack.
> > > > > >
> > > > > >
> > > > > >> * Why worker-level offset commit stats? It's not clear to me
> that
> > > > these
> > > > > >> are
> > > > > >> useful without considering the specific connector.
> > > > > >>
> > > > > >
> > > > > > So would they make more sense on the tasks? Again, on the worker
> > > > they're
> > > > > > aggregates.
> > > > > >
> > > > > >
> > > > > >>
> > > > > >> -Ewen
> > > > > >>
> > > > > >>
> > > > > >> On Mon, Sep 11, 2017 at 9:43 AM, Randall Hauch <
> rhauch@gmail.com>
> > > > > wrote:
> > > > > >>
> > > > > >> > Thanks for reviewing. Responses inline below.
> > > > > >> >
> > > > > >> > On Mon, Sep 11, 2017 at 11:22 AM, Roger Hoover <
> > > > > roger.hoover@gmail.com>
> > > > > >> > wrote:
> > > > > >> >
> > > > > >> > > Randall,
> > > > > >> > >
> > > > > >> > > Thank you for the KIP.  This should improve visibility
> > > greatly.  I
> > > > > >> had a
> > > > > >> > > few questions/ideas for more metrics.
> > > > > >> > >
> > > > > >> > >
> > > > > >> > >    1. What's the relationship between the worker state and
> the
> > > > > >> connector
> > > > > >> > >    status?  Does the 'paused' status at the Connector level
> > > > include
> > > > > >> the
> > > > > >> > > time
> > > > > >> > >    that worker is 'rebalancing'?
> > > > > >> > >
> > > > > >> >
> > > > > >> > The worker state metric simply reports whether the worker is
> > > running
> > > > > or
> > > > > >> > rebalancing. This state is independent of how many connectors
> > are
> > > > > >> > deployed/running/paused. During a rebalance, the connectors
> are
> > > > being
> > > > > >> > stopped and restarted but are effectively not running.
> > > > > >> >
> > > > > >> >
> > > > > >> > >    2. Are the "Source Connector" metrics like record rate an
> > > > > >> aggregation
> > > > > >> > of
> > > > > >> > >    the "Source Task" metrics?
> > > > > >> > >
> > > > > >> >
> > > > > >> > Yes.
> > > > > >> >
> > > > > >> >
> > > > > >> > >       - How much value is there is monitoring at the "Source
> > > > > >> Connector"
> > > > > >> > >       level (other than status) if the number of constituent
> > > tasks
> > > > > may
> > > > > >> > > change
> > > > > >> > >       over time?
> > > > > >> > >
> > > > > >> >
> > > > > >> > The task metrics allow you to know whether the tasks are
> evenly
> > > > loaded
> > > > > >> and
> > > > > >> > each making progress. The aggregate connector metrics tell you
> > how
> > > > > much
> > > > > >> > work has been performed by all the tasks in that worker. Both
> > are
> > > > > useful
> > > > > >> > IMO.
> > > > > >> >
> > > > > >> >
> > > > > >> > >       - I'm imagining that it's most useful to collect
> metrics
> > > at
> > > > > the
> > > > > >> > task
> > > > > >> > >       level as the task-level metrics should be stable
> > > regardless
> > > > of
> > > > > >> > tasks
> > > > > >> > >       shifting to different workers
> > > > > >> > >
> > > > > >> >
> > > > > >> > Correct, this is where the most value is because it is the
> most
> > > fine
> > > > > >> > grained.
> > > > > >> >
> > > > > >> >
> > > > > >> > >       - If so, can we duplicate the Connector Status down at
> > the
> > > > > task
> > > > > >> > level
> > > > > >> > >          so that all important metrics can be tracked by
> task?
> > > > > >> > >
> > > > > >> >
> > > > > >> > Possibly. The challenge is that the threads running the tasks
> > are
> > > > > >> blocked
> > > > > >> > when a connector is paused.
> > > > > >> >
> > > > > >> >
> > > > > >> > >          3. For the Sink Task metrics
> > > > > >> > >       - Can we add offset lag and timestamp lag on commit?
> > > > > >> > >          - After records are flushed/committed
> > > > > >> > >             - what is the diff between the record timestamps
> > and
> > > > > >> commit
> > > > > >> > >             time (histogram)?  this is a measure of
> end-to-end
> > > > > >> pipeline
> > > > > >> > > latency
> > > > > >> > >             - what is the diff between record offsets and
> > latest
> > > > > >> offset
> > > > > >> > of
> > > > > >> > >             their partition at commit time (histogram)? this
> > is
> > > a
> > > > > >> > > measure of whether
> > > > > >> > >             this particular task is keeping up
> > > > > >> > >
> > > > > >> >
> > > > > >> > Yeah, possibly. Will have to compare with the consumer metrics
> > to
> > > > see
> > > > > >> what
> > > > > >> > we can get.
> > > > > >> >
> > > > > >> >
> > > > > >> > >          - How about flush error rate?  Assuming the sink
> > > > connectors
> > > > > >> are
> > > > > >> > >       using retries, it would be helpful to know how many
> > errors
> > > > > >> they're
> > > > > >> > > seeing
> > > > > >> > >
> > > > > >> >
> > > > > >> > We could add a metric to track how many times the framework
> > > > receives a
> > > > > >> > retry exception and then retries, but the connectors may also
> do
> > > > this
> > > > > on
> > > > > >> > their own.
> > > > > >> >
> > > > > >> >
> > > > > >> > >       - Can we tell at the framework level how many records
> > were
> > > > > >> inserted
> > > > > >> > >       vs updated vs deleted?
> > > > > >> > >
> > > > > >> >
> > > > > >> > No, there's no distinction in the Connect framework.
> > > > > >> >
> > > > > >> >
> > > > > >> > >       - Batching stats
> > > > > >> > >          - Histogram of flush batch size
> > > > > >> > >          - Counts of flush trigger method (time vs max batch
> > > size)
> > > > > >> > >
> > > > > >> >
> > > > > >> > Should be able to add these.
> > > > > >> >
> > > > > >> >
> > > > > >> > >
> > > > > >> > > Cheers,
> > > > > >> > >
> > > > > >> > > Roger
> > > > > >> > >
> > > > > >> > > On Sun, Sep 10, 2017 at 8:45 AM, Randall Hauch <
> > > rhauch@gmail.com>
> > > > > >> wrote:
> > > > > >> > >
> > > > > >> > > > Thanks, Gwen.
> > > > > >> > > >
> > > > > >> > > > That's a great idea, so I've changed the KIP to add those
> > > > metrics.
> > > > > >> I've
> > > > > >> > > > also made a few other changes:
> > > > > >> > > >
> > > > > >> > > >
> > > > > >> > > >    1. The context of all metrics is limited to the
> activity
> > > > within
> > > > > >> the
> > > > > >> > > >    worker. This wasn't clear before, so I changed the
> > > motivation
> > > > > and
> > > > > >> > > metric
> > > > > >> > > >    descriptions to explicitly state this.
> > > > > >> > > >    2. Added the worker ID to all MBean attributes. In
> > addition
> > > > to
> > > > > >> > > hopefully
> > > > > >> > > >    making this same scope obvious from within JMX or other
> > > > metric
> > > > > >> > > reporting
> > > > > >> > > >    system. This is also similar to how the Kafka producer
> > and
> > > > > >> consumer
> > > > > >> > > > metrics
> > > > > >> > > >    include the client ID in their MBean attributes.
> > Hopefully
> > > > this
> > > > > >> does
> > > > > >> > > not
> > > > > >> > > >    negatively impact or complicate how external reporting
> > > > systems'
> > > > > >> > > > aggregate
> > > > > >> > > >    metrics from multiple workers.
> > > > > >> > > >    3. Stated explicitly that aggregating metrics across
> > > workers
> > > > > was
> > > > > >> out
> > > > > >> > > of
> > > > > >> > > >    scope of this KIP.
> > > > > >> > > >    4. Added metrics to report the connector class and
> > version
> > > > for
> > > > > >> both
> > > > > >> > > sink
> > > > > >> > > >    and source connectors.
> > > > > >> > > >
> > > > > >> > > > Check this KIP's history for details of these changes.
> > > > > >> > > >
> > > > > >> > > > Please let me know if you have any other suggestions. I
> hope
> > > to
> > > > > >> start
> > > > > >> > the
> > > > > >> > > > voting soon!
> > > > > >> > > >
> > > > > >> > > > Best regards,
> > > > > >> > > >
> > > > > >> > > > Randall
> > > > > >> > > >
> > > > > >> > > > On Thu, Sep 7, 2017 at 9:35 PM, Gwen Shapira <
> > > gwen@confluent.io
> > > > >
> > > > > >> > wrote:
> > > > > >> > > >
> > > > > >> > > > > Thanks for the KIP, Randall. Those are badly needed!
> > > > > >> > > > >
> > > > > >> > > > > Can we have two metrics with record rate per task? One
> > > before
> > > > > SMT
> > > > > >> and
> > > > > >> > > one
> > > > > >> > > > > after?
> > > > > >> > > > > We can have cases where we read 5000 rows from JDBC but
> > > write
> > > > 5
> > > > > to
> > > > > >> > > Kafka,
> > > > > >> > > > > or read 5000 records from Kafka and write 5 due to
> > > filtering.
> > > > I
> > > > > >> think
> > > > > >> > > its
> > > > > >> > > > > important to know both numbers.
> > > > > >> > > > >
> > > > > >> > > > >
> > > > > >> > > > > Gwen
> > > > > >> > > > >
> > > > > >> > > > > On Thu, Sep 7, 2017 at 7:50 PM, Randall Hauch <
> > > > rhauch@gmail.com
> > > > > >
> > > > > >> > > wrote:
> > > > > >> > > > >
> > > > > >> > > > > > Hi everyone.
> > > > > >> > > > > >
> > > > > >> > > > > > I've created a new KIP to add metrics to the Kafka
> > Connect
> > > > > >> > framework:
> > > > > >> > > > > > https://cwiki.apache.org/
> confluence/display/KAFKA/KIP-
> > > > > >> > > > > > 196%3A+Add+metrics+to+Kafka+Connect+framework
> > > > > >> > > > > >
> > > > > >> > > > > > The KIP approval deadline is looming, so if you're
> > > > interested
> > > > > in
> > > > > >> > > Kafka
> > > > > >> > > > > > Connect metrics please review and provide feedback as
> > soon
> > > > as
> > > > > >> > > possible.
> > > > > >> > > > > I'm
> > > > > >> > > > > > interested not only in whether the metrics are
> > sufficient
> > > > and
> > > > > >> > > > > appropriate,
> > > > > >> > > > > > but also in whether the MBean naming conventions are
> > okay.
> > > > > >> > > > > >
> > > > > >> > > > > > Best regards,
> > > > > >> > > > > >
> > > > > >> > > > > > Randall
> > > > > >> > > > > >
> > > > > >> > > > >
> > > > > >> > > > >
> > > > > >> > > > >
> > > > > >> > > > > --
> > > > > >> > > > > *Gwen Shapira*
> > > > > >> > > > > Product Manager | Confluent
> > > > > >> > > > > 650.450.2760 <(650)%20450-2760> | @gwenshap
> > > > > >> > > > > Follow us: Twitter <https://twitter.com/ConfluentInc> |
> > > blog
> > > > > >> > > > > <http://www.confluent.io/blog>
> > > > > >> > > > >
> > > > > >> > > >
> > > > > >> > >
> > > > > >> >
> > > > > >>
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-196: Add metrics to Kafka Connect framework

Posted by Roger Hoover <ro...@gmail.com>.
Ewen,

I don't know the details of the perf concern.  How is it that the Kafka
broker can keep latency stats per request without suffering too much
performance?  Maybe SMTs are the only concern b/c they are per-message.  If
so, let's remove those and keep timing info for everything else like
flushes, which are batch-based.


On Tue, Sep 12, 2017 at 1:32 PM, Ewen Cheslack-Postava <ew...@confluent.io>
wrote:

> On Tue, Sep 12, 2017 at 10:55 AM, Gwen Shapira <gw...@confluent.io> wrote:
>
> > Ewen, you gave a nice talk at Kafka Summit where you warned about the
> > danger of SMTs that slow down the data pipe. If we don't provide the time
> > metrics, how will users know when their SMTs are causing performance
> > issues?
> >
>
> Metrics aren't the only way to gain insight about performance and always
> measuring this even when it's not necessarily being used may not make
> sense. SMT authors are much better off starting out with a JMH or similar
> benchmark. What I was referring to in the talk is more about understanding
> that the processing for SMTs is entirely synchronous and that means certain
> classes of operations will just generally be a bad idea, e.g. anything that
> goes out over the network to another service. You don't even really need
> performance info to determine that that type of transformation will cause
> problems.
>
> But my point wasn't that timing info isn't useful. It's that we know that
> getting timestamps is pretty expensive and we'll already be doing so
> elsewhere (e.g. if a source record doesn't include a timestamp). For some
> use cases such as ByteArrayConverter + no SMTs + lightweight processing
> (e.g. just gets handed to a background thread that deals with sending the
> data), it wouldn't be out of the question that adding 4 or so more calls to
> get timestamps could become a bottleneck. Since I don't know if it would
> but we have definitely seen the issue come up before, I would be
> conservative in adding the metrics unless we had some numbers showing it
> doesn't matter or doesn't matter much.
>
> In general, I don't think metrics that require always-on measurement are a
> good way to get fine grained performance information. Instrumenting
> different phases that imply different types of performance problems can be
> helpful (e.g. "processing time" that should be CPU/memory throughput bound
> vs. "send time" that, at least for many connectors, is more likely to be IO
> bound), but if you want finer-grained details, you probably either want
> something that can be toggled on/off temporarily or just use a tool that's
> really designed for the job, i.e. a profiler like perf.
>
> -Ewen
>
>
> >
> > Gwen
> >
> > On Mon, Sep 11, 2017 at 7:50 PM Ewen Cheslack-Postava <ewen@confluent.io
> >
> > wrote:
> >
> > > re: questions about additional metrics, I think we'll undoubtedly find
> > more
> > > that people want in practice, but as I mentioned earlier I think it's
> > > better to add the ones we know we need and then fill out the rest as we
> > > figure it out. So, e.g., batch size metrics sound like they could be
> > > useful, but I'd probably wait until we have a clear use case. It seems
> > > likely that it could be useful in diagnosing slow connectors (e.g. the
> > > implementation just does something inefficient), but I'm not really
> sure
> > > about that yet.
> > >
> > > -Ewen
> > >
> > > On Mon, Sep 11, 2017 at 7:11 PM, Randall Hauch <rh...@gmail.com>
> wrote:
> > >
> > > > Based on Roger and Ewen's feedback, I removed the aggregate metrics
> as
> > > they
> > > > would be difficult to make use of without extra work. This simplified
> > > > things a great deal, and I took the opportunity to reorganize the
> > groups
> > > of
> > > > metrics. Also, based upon Ewen's concerns regarding measuring
> > > > times/durations, I removed all time-related metrics except for the
> > offset
> > > > commits and rebalances, which are infrequent enough to warrant the
> > > capture
> > > > of percentiles. Roger asked about capturing batch size metrics for
> > source
> > > > and sink tasks, and offset lag metrics for sink tasks. Finally, Ewen
> > > > pointed out that all count/total metrics are only valid since the
> most
> > > > recent rebalance and are therefore less meaningful, and were removed.
> > > >
> > > > On Mon, Sep 11, 2017 at 6:50 PM, Randall Hauch <rh...@gmail.com>
> > wrote:
> > > >
> > > > > Thanks, Ewen. Comments inline below.
> > > > >
> > > > > On Mon, Sep 11, 2017 at 5:46 PM, Ewen Cheslack-Postava <
> > > > ewen@confluent.io>
> > > > > wrote:
> > > > >
> > > > >> Randall,
> > > > >>
> > > > >> A couple of questions:
> > > > >>
> > > > >> * Some metrics don't seem to have unique names? e.g.
> > > > >> source-record-produce-rate and source-record-produce-total seem
> like
> > > > they
> > > > >> are duplicated. Looks like maybe just an oversight that the second
> > > ones
> > > > >> should be changed from "produce" to "write".
> > > > >>
> > > > >
> > > > > Nice catch. You are correct - should be "write" instead of
> > "produce". I
> > > > > will correct.
> > > > >
> > > > >
> > > > >> * I think there's a stray extra character in a couple of
> > > > >> places: kafka.connect:type=source-task
> -metrics,name=source-record-
> > > > >> produce-total,worker=([-.\w]+)l,connector=([-.\w]+),task=([\d]+)
> > > > >> has an extra char after the worker name.
> > > > >>
> > > > >
> > > > > Thanks. Removed in 2 places.
> > > > >
> > > > >
> > > > >> * Are the produce totals actually useful given rebalancing would
> > > cancel
> > > > >> them out anyway? Doesn't seem like you could do much with them.
> > > > >>
> > > > >
> > > > > Yes, the totals would be since the last rebalance. Maybe that isn't
> > > that
> > > > > useful. Might be better to capture the offsets and lag as Roger was
> > > > > suggestion. Thoughts?
> > > > >
> > > > >
> > > > >> * Why do transformations get their own metric but not converters?
> > And
> > > > are
> > > > >> we concerned at all about the performance impact of getting such
> > fine
> > > > >> grained info? Getting current time isn't free and we've seen
> before
> > > that
> > > > >> we
> > > > >> ended up w/ accidental performance regressions as we tried to
> check
> > it
> > > > too
> > > > >> frequently to enforce timeouts fine grained in the producer
> (iirc).
> > > > >> Batching helps w/ this, but on the consumer side, a
> > max.poll.records=1
> > > > >> setting could put you in a bad place, especially since transforms
> > > might
> > > > be
> > > > >> very lightweight (or nothing) and converters are expected to be
> > > > relatively
> > > > >> cheap as well.
> > > > >>
> > > > >
> > > > > We could remove the read, transform, and put time-based metrics for
> > > sink
> > > > > tasks, and poll, transform, and write time-based metrics.
> Can/should
> > > they
> > > > > be replaced with anything else?
> > > > >
> > > > >
> > > > >> * If we include the worker id everywhere and don't have metrics
> > > without
> > > > >> that included, isn't that a pain for users that dump this data
> into
> > > some
> > > > >> other system? They have to know which worker the connector/task is
> > > > >> currently on *or* need to do extra work to merge the metrics from
> > > across
> > > > >> machines. Including versions with the worker ID can make sense for
> > > > >> completeness and accuracy (e.g. technically there are still very
> > slim
> > > > >> risks
> > > > >> of having a task running twice due to zombies), but it seems like
> > bad
> > > > >> usability for the common case.
> > > > >>
> > > > >
> > > > > Part of the reason was also to help identify where each of the
> > metrics
> > > > > came from, but per the next comment this may not be as useful,
> > either.
> > > > > So remove the worker ID in all the task and connector metric names?
> > > What
> > > > > about the worker metrics?
> > > > >
> > > > >
> > > > >> * Is aggregating things like source record rate at the (worker,
> > > > connector)
> > > > >> level really useful since you're just going to need to do
> additional
> > > > >> aggregation anyway once you've collected metrics across all
> workers?
> > > I'd
> > > > >> rather add a smaller number of metrics w/ clear use cases than
> just
> > > try
> > > > to
> > > > >> be exhaustive and then have to maintain stuff that nobody actually
> > > uses.
> > > > >>
> > > > >
> > > > > Yes, the connector aggregate metrics are maybe not as useful if you
> > > also
> > > > > have to aggregate them from different workers. Removing them
> probably
> > > > also
> > > > > reduces the risk of them being misinterpretted.
> > > > >
> > > > >
> > > > >> * You have status for connectors but not for tasks. Any reason
> why?
> > > > Seems
> > > > >> like it'd make sense to expose both, especially since users
> > generally
> > > > care
> > > > >> about task status more than connector status (not many connectors
> > > > actually
> > > > >> run a monitoring thread.)
> > > > >>
> > > > >
> > > > > Ack.
> > > > >
> > > > >
> > > > >> * Is number of tasks for each connector a useful metric? Not sure
> > > > whether
> > > > >> someone would find this useful or not. Probably not for alerts,
> but
> > > > might
> > > > >> be useful to be able to check it via your metrics dashboard.
> > > > >>
> > > > >
> > > > > Seems like it might be useful, at least in terms of tracking the
> > number
> > > > of
> > > > > tasks over time. Might not be as useful for connectors that have
> > > > relatively
> > > > > static tasks, but it would be more interesting/useful for
> connectors
> > > that
> > > > > create tasks dynamically and periodically request task
> > > reconfigurations.
> > > > >
> > > > >
> > > > >> * Same questions re: granularity of sink tasks/connectors timing
> and
> > > > >> whether the connectors need all the roll-ups of individual
> (worker,
> > > > task)
> > > > >> values to (worker, connector) level.
> > > > >>
> > > > >
> > > > > I'm fine with taking out the aggregates to keep things simple and
> > > prevent
> > > > > misunderstanding.
> > > > >
> > > > >
> > > > >> * If we expose the who the worker currently thinks is leader, it
> > might
> > > > >> also
> > > > >> make sense to expose the underlying epoch. Not actually sure if we
> > > > expose
> > > > >> that for the consumer today, but it's an indicator of who is
> > properly
> > > up
> > > > >> to
> > > > >> date.
> > > > >>
> > > > >
> > > > > Ack.
> > > > >
> > > > >
> > > > >> * Why worker-level offset commit stats? It's not clear to me that
> > > these
> > > > >> are
> > > > >> useful without considering the specific connector.
> > > > >>
> > > > >
> > > > > So would they make more sense on the tasks? Again, on the worker
> > > they're
> > > > > aggregates.
> > > > >
> > > > >
> > > > >>
> > > > >> -Ewen
> > > > >>
> > > > >>
> > > > >> On Mon, Sep 11, 2017 at 9:43 AM, Randall Hauch <rh...@gmail.com>
> > > > wrote:
> > > > >>
> > > > >> > Thanks for reviewing. Responses inline below.
> > > > >> >
> > > > >> > On Mon, Sep 11, 2017 at 11:22 AM, Roger Hoover <
> > > > roger.hoover@gmail.com>
> > > > >> > wrote:
> > > > >> >
> > > > >> > > Randall,
> > > > >> > >
> > > > >> > > Thank you for the KIP.  This should improve visibility
> > greatly.  I
> > > > >> had a
> > > > >> > > few questions/ideas for more metrics.
> > > > >> > >
> > > > >> > >
> > > > >> > >    1. What's the relationship between the worker state and the
> > > > >> connector
> > > > >> > >    status?  Does the 'paused' status at the Connector level
> > > include
> > > > >> the
> > > > >> > > time
> > > > >> > >    that worker is 'rebalancing'?
> > > > >> > >
> > > > >> >
> > > > >> > The worker state metric simply reports whether the worker is
> > running
> > > > or
> > > > >> > rebalancing. This state is independent of how many connectors
> are
> > > > >> > deployed/running/paused. During a rebalance, the connectors are
> > > being
> > > > >> > stopped and restarted but are effectively not running.
> > > > >> >
> > > > >> >
> > > > >> > >    2. Are the "Source Connector" metrics like record rate an
> > > > >> aggregation
> > > > >> > of
> > > > >> > >    the "Source Task" metrics?
> > > > >> > >
> > > > >> >
> > > > >> > Yes.
> > > > >> >
> > > > >> >
> > > > >> > >       - How much value is there is monitoring at the "Source
> > > > >> Connector"
> > > > >> > >       level (other than status) if the number of constituent
> > tasks
> > > > may
> > > > >> > > change
> > > > >> > >       over time?
> > > > >> > >
> > > > >> >
> > > > >> > The task metrics allow you to know whether the tasks are evenly
> > > loaded
> > > > >> and
> > > > >> > each making progress. The aggregate connector metrics tell you
> how
> > > > much
> > > > >> > work has been performed by all the tasks in that worker. Both
> are
> > > > useful
> > > > >> > IMO.
> > > > >> >
> > > > >> >
> > > > >> > >       - I'm imagining that it's most useful to collect metrics
> > at
> > > > the
> > > > >> > task
> > > > >> > >       level as the task-level metrics should be stable
> > regardless
> > > of
> > > > >> > tasks
> > > > >> > >       shifting to different workers
> > > > >> > >
> > > > >> >
> > > > >> > Correct, this is where the most value is because it is the most
> > fine
> > > > >> > grained.
> > > > >> >
> > > > >> >
> > > > >> > >       - If so, can we duplicate the Connector Status down at
> the
> > > > task
> > > > >> > level
> > > > >> > >          so that all important metrics can be tracked by task?
> > > > >> > >
> > > > >> >
> > > > >> > Possibly. The challenge is that the threads running the tasks
> are
> > > > >> blocked
> > > > >> > when a connector is paused.
> > > > >> >
> > > > >> >
> > > > >> > >          3. For the Sink Task metrics
> > > > >> > >       - Can we add offset lag and timestamp lag on commit?
> > > > >> > >          - After records are flushed/committed
> > > > >> > >             - what is the diff between the record timestamps
> and
> > > > >> commit
> > > > >> > >             time (histogram)?  this is a measure of end-to-end
> > > > >> pipeline
> > > > >> > > latency
> > > > >> > >             - what is the diff between record offsets and
> latest
> > > > >> offset
> > > > >> > of
> > > > >> > >             their partition at commit time (histogram)? this
> is
> > a
> > > > >> > > measure of whether
> > > > >> > >             this particular task is keeping up
> > > > >> > >
> > > > >> >
> > > > >> > Yeah, possibly. Will have to compare with the consumer metrics
> to
> > > see
> > > > >> what
> > > > >> > we can get.
> > > > >> >
> > > > >> >
> > > > >> > >          - How about flush error rate?  Assuming the sink
> > > connectors
> > > > >> are
> > > > >> > >       using retries, it would be helpful to know how many
> errors
> > > > >> they're
> > > > >> > > seeing
> > > > >> > >
> > > > >> >
> > > > >> > We could add a metric to track how many times the framework
> > > receives a
> > > > >> > retry exception and then retries, but the connectors may also do
> > > this
> > > > on
> > > > >> > their own.
> > > > >> >
> > > > >> >
> > > > >> > >       - Can we tell at the framework level how many records
> were
> > > > >> inserted
> > > > >> > >       vs updated vs deleted?
> > > > >> > >
> > > > >> >
> > > > >> > No, there's no distinction in the Connect framework.
> > > > >> >
> > > > >> >
> > > > >> > >       - Batching stats
> > > > >> > >          - Histogram of flush batch size
> > > > >> > >          - Counts of flush trigger method (time vs max batch
> > size)
> > > > >> > >
> > > > >> >
> > > > >> > Should be able to add these.
> > > > >> >
> > > > >> >
> > > > >> > >
> > > > >> > > Cheers,
> > > > >> > >
> > > > >> > > Roger
> > > > >> > >
> > > > >> > > On Sun, Sep 10, 2017 at 8:45 AM, Randall Hauch <
> > rhauch@gmail.com>
> > > > >> wrote:
> > > > >> > >
> > > > >> > > > Thanks, Gwen.
> > > > >> > > >
> > > > >> > > > That's a great idea, so I've changed the KIP to add those
> > > metrics.
> > > > >> I've
> > > > >> > > > also made a few other changes:
> > > > >> > > >
> > > > >> > > >
> > > > >> > > >    1. The context of all metrics is limited to the activity
> > > within
> > > > >> the
> > > > >> > > >    worker. This wasn't clear before, so I changed the
> > motivation
> > > > and
> > > > >> > > metric
> > > > >> > > >    descriptions to explicitly state this.
> > > > >> > > >    2. Added the worker ID to all MBean attributes. In
> addition
> > > to
> > > > >> > > hopefully
> > > > >> > > >    making this same scope obvious from within JMX or other
> > > metric
> > > > >> > > reporting
> > > > >> > > >    system. This is also similar to how the Kafka producer
> and
> > > > >> consumer
> > > > >> > > > metrics
> > > > >> > > >    include the client ID in their MBean attributes.
> Hopefully
> > > this
> > > > >> does
> > > > >> > > not
> > > > >> > > >    negatively impact or complicate how external reporting
> > > systems'
> > > > >> > > > aggregate
> > > > >> > > >    metrics from multiple workers.
> > > > >> > > >    3. Stated explicitly that aggregating metrics across
> > workers
> > > > was
> > > > >> out
> > > > >> > > of
> > > > >> > > >    scope of this KIP.
> > > > >> > > >    4. Added metrics to report the connector class and
> version
> > > for
> > > > >> both
> > > > >> > > sink
> > > > >> > > >    and source connectors.
> > > > >> > > >
> > > > >> > > > Check this KIP's history for details of these changes.
> > > > >> > > >
> > > > >> > > > Please let me know if you have any other suggestions. I hope
> > to
> > > > >> start
> > > > >> > the
> > > > >> > > > voting soon!
> > > > >> > > >
> > > > >> > > > Best regards,
> > > > >> > > >
> > > > >> > > > Randall
> > > > >> > > >
> > > > >> > > > On Thu, Sep 7, 2017 at 9:35 PM, Gwen Shapira <
> > gwen@confluent.io
> > > >
> > > > >> > wrote:
> > > > >> > > >
> > > > >> > > > > Thanks for the KIP, Randall. Those are badly needed!
> > > > >> > > > >
> > > > >> > > > > Can we have two metrics with record rate per task? One
> > before
> > > > SMT
> > > > >> and
> > > > >> > > one
> > > > >> > > > > after?
> > > > >> > > > > We can have cases where we read 5000 rows from JDBC but
> > write
> > > 5
> > > > to
> > > > >> > > Kafka,
> > > > >> > > > > or read 5000 records from Kafka and write 5 due to
> > filtering.
> > > I
> > > > >> think
> > > > >> > > its
> > > > >> > > > > important to know both numbers.
> > > > >> > > > >
> > > > >> > > > >
> > > > >> > > > > Gwen
> > > > >> > > > >
> > > > >> > > > > On Thu, Sep 7, 2017 at 7:50 PM, Randall Hauch <
> > > rhauch@gmail.com
> > > > >
> > > > >> > > wrote:
> > > > >> > > > >
> > > > >> > > > > > Hi everyone.
> > > > >> > > > > >
> > > > >> > > > > > I've created a new KIP to add metrics to the Kafka
> Connect
> > > > >> > framework:
> > > > >> > > > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > > > >> > > > > > 196%3A+Add+metrics+to+Kafka+Connect+framework
> > > > >> > > > > >
> > > > >> > > > > > The KIP approval deadline is looming, so if you're
> > > interested
> > > > in
> > > > >> > > Kafka
> > > > >> > > > > > Connect metrics please review and provide feedback as
> soon
> > > as
> > > > >> > > possible.
> > > > >> > > > > I'm
> > > > >> > > > > > interested not only in whether the metrics are
> sufficient
> > > and
> > > > >> > > > > appropriate,
> > > > >> > > > > > but also in whether the MBean naming conventions are
> okay.
> > > > >> > > > > >
> > > > >> > > > > > Best regards,
> > > > >> > > > > >
> > > > >> > > > > > Randall
> > > > >> > > > > >
> > > > >> > > > >
> > > > >> > > > >
> > > > >> > > > >
> > > > >> > > > > --
> > > > >> > > > > *Gwen Shapira*
> > > > >> > > > > Product Manager | Confluent
> > > > >> > > > > 650.450.2760 <(650)%20450-2760> | @gwenshap
> > > > >> > > > > Follow us: Twitter <https://twitter.com/ConfluentInc> |
> > blog
> > > > >> > > > > <http://www.confluent.io/blog>
> > > > >> > > > >
> > > > >> > > >
> > > > >> > >
> > > > >> >
> > > > >>
> > > > >
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-196: Add metrics to Kafka Connect framework

Posted by Ewen Cheslack-Postava <ew...@confluent.io>.
On Tue, Sep 12, 2017 at 10:55 AM, Gwen Shapira <gw...@confluent.io> wrote:

> Ewen, you gave a nice talk at Kafka Summit where you warned about the
> danger of SMTs that slow down the data pipe. If we don't provide the time
> metrics, how will users know when their SMTs are causing performance
> issues?
>

Metrics aren't the only way to gain insight about performance and always
measuring this even when it's not necessarily being used may not make
sense. SMT authors are much better off starting out with a JMH or similar
benchmark. What I was referring to in the talk is more about understanding
that the processing for SMTs is entirely synchronous and that means certain
classes of operations will just generally be a bad idea, e.g. anything that
goes out over the network to another service. You don't even really need
performance info to determine that that type of transformation will cause
problems.

But my point wasn't that timing info isn't useful. It's that we know that
getting timestamps is pretty expensive and we'll already be doing so
elsewhere (e.g. if a source record doesn't include a timestamp). For some
use cases such as ByteArrayConverter + no SMTs + lightweight processing
(e.g. just gets handed to a background thread that deals with sending the
data), it wouldn't be out of the question that adding 4 or so more calls to
get timestamps could become a bottleneck. Since I don't know if it would
but we have definitely seen the issue come up before, I would be
conservative in adding the metrics unless we had some numbers showing it
doesn't matter or doesn't matter much.

In general, I don't think metrics that require always-on measurement are a
good way to get fine grained performance information. Instrumenting
different phases that imply different types of performance problems can be
helpful (e.g. "processing time" that should be CPU/memory throughput bound
vs. "send time" that, at least for many connectors, is more likely to be IO
bound), but if you want finer-grained details, you probably either want
something that can be toggled on/off temporarily or just use a tool that's
really designed for the job, i.e. a profiler like perf.

-Ewen


>
> Gwen
>
> On Mon, Sep 11, 2017 at 7:50 PM Ewen Cheslack-Postava <ew...@confluent.io>
> wrote:
>
> > re: questions about additional metrics, I think we'll undoubtedly find
> more
> > that people want in practice, but as I mentioned earlier I think it's
> > better to add the ones we know we need and then fill out the rest as we
> > figure it out. So, e.g., batch size metrics sound like they could be
> > useful, but I'd probably wait until we have a clear use case. It seems
> > likely that it could be useful in diagnosing slow connectors (e.g. the
> > implementation just does something inefficient), but I'm not really sure
> > about that yet.
> >
> > -Ewen
> >
> > On Mon, Sep 11, 2017 at 7:11 PM, Randall Hauch <rh...@gmail.com> wrote:
> >
> > > Based on Roger and Ewen's feedback, I removed the aggregate metrics as
> > they
> > > would be difficult to make use of without extra work. This simplified
> > > things a great deal, and I took the opportunity to reorganize the
> groups
> > of
> > > metrics. Also, based upon Ewen's concerns regarding measuring
> > > times/durations, I removed all time-related metrics except for the
> offset
> > > commits and rebalances, which are infrequent enough to warrant the
> > capture
> > > of percentiles. Roger asked about capturing batch size metrics for
> source
> > > and sink tasks, and offset lag metrics for sink tasks. Finally, Ewen
> > > pointed out that all count/total metrics are only valid since the most
> > > recent rebalance and are therefore less meaningful, and were removed.
> > >
> > > On Mon, Sep 11, 2017 at 6:50 PM, Randall Hauch <rh...@gmail.com>
> wrote:
> > >
> > > > Thanks, Ewen. Comments inline below.
> > > >
> > > > On Mon, Sep 11, 2017 at 5:46 PM, Ewen Cheslack-Postava <
> > > ewen@confluent.io>
> > > > wrote:
> > > >
> > > >> Randall,
> > > >>
> > > >> A couple of questions:
> > > >>
> > > >> * Some metrics don't seem to have unique names? e.g.
> > > >> source-record-produce-rate and source-record-produce-total seem like
> > > they
> > > >> are duplicated. Looks like maybe just an oversight that the second
> > ones
> > > >> should be changed from "produce" to "write".
> > > >>
> > > >
> > > > Nice catch. You are correct - should be "write" instead of
> "produce". I
> > > > will correct.
> > > >
> > > >
> > > >> * I think there's a stray extra character in a couple of
> > > >> places: kafka.connect:type=source-task-metrics,name=source-record-
> > > >> produce-total,worker=([-.\w]+)l,connector=([-.\w]+),task=([\d]+)
> > > >> has an extra char after the worker name.
> > > >>
> > > >
> > > > Thanks. Removed in 2 places.
> > > >
> > > >
> > > >> * Are the produce totals actually useful given rebalancing would
> > cancel
> > > >> them out anyway? Doesn't seem like you could do much with them.
> > > >>
> > > >
> > > > Yes, the totals would be since the last rebalance. Maybe that isn't
> > that
> > > > useful. Might be better to capture the offsets and lag as Roger was
> > > > suggestion. Thoughts?
> > > >
> > > >
> > > >> * Why do transformations get their own metric but not converters?
> And
> > > are
> > > >> we concerned at all about the performance impact of getting such
> fine
> > > >> grained info? Getting current time isn't free and we've seen before
> > that
> > > >> we
> > > >> ended up w/ accidental performance regressions as we tried to check
> it
> > > too
> > > >> frequently to enforce timeouts fine grained in the producer (iirc).
> > > >> Batching helps w/ this, but on the consumer side, a
> max.poll.records=1
> > > >> setting could put you in a bad place, especially since transforms
> > might
> > > be
> > > >> very lightweight (or nothing) and converters are expected to be
> > > relatively
> > > >> cheap as well.
> > > >>
> > > >
> > > > We could remove the read, transform, and put time-based metrics for
> > sink
> > > > tasks, and poll, transform, and write time-based metrics. Can/should
> > they
> > > > be replaced with anything else?
> > > >
> > > >
> > > >> * If we include the worker id everywhere and don't have metrics
> > without
> > > >> that included, isn't that a pain for users that dump this data into
> > some
> > > >> other system? They have to know which worker the connector/task is
> > > >> currently on *or* need to do extra work to merge the metrics from
> > across
> > > >> machines. Including versions with the worker ID can make sense for
> > > >> completeness and accuracy (e.g. technically there are still very
> slim
> > > >> risks
> > > >> of having a task running twice due to zombies), but it seems like
> bad
> > > >> usability for the common case.
> > > >>
> > > >
> > > > Part of the reason was also to help identify where each of the
> metrics
> > > > came from, but per the next comment this may not be as useful,
> either.
> > > > So remove the worker ID in all the task and connector metric names?
> > What
> > > > about the worker metrics?
> > > >
> > > >
> > > >> * Is aggregating things like source record rate at the (worker,
> > > connector)
> > > >> level really useful since you're just going to need to do additional
> > > >> aggregation anyway once you've collected metrics across all workers?
> > I'd
> > > >> rather add a smaller number of metrics w/ clear use cases than just
> > try
> > > to
> > > >> be exhaustive and then have to maintain stuff that nobody actually
> > uses.
> > > >>
> > > >
> > > > Yes, the connector aggregate metrics are maybe not as useful if you
> > also
> > > > have to aggregate them from different workers. Removing them probably
> > > also
> > > > reduces the risk of them being misinterpretted.
> > > >
> > > >
> > > >> * You have status for connectors but not for tasks. Any reason why?
> > > Seems
> > > >> like it'd make sense to expose both, especially since users
> generally
> > > care
> > > >> about task status more than connector status (not many connectors
> > > actually
> > > >> run a monitoring thread.)
> > > >>
> > > >
> > > > Ack.
> > > >
> > > >
> > > >> * Is number of tasks for each connector a useful metric? Not sure
> > > whether
> > > >> someone would find this useful or not. Probably not for alerts, but
> > > might
> > > >> be useful to be able to check it via your metrics dashboard.
> > > >>
> > > >
> > > > Seems like it might be useful, at least in terms of tracking the
> number
> > > of
> > > > tasks over time. Might not be as useful for connectors that have
> > > relatively
> > > > static tasks, but it would be more interesting/useful for connectors
> > that
> > > > create tasks dynamically and periodically request task
> > reconfigurations.
> > > >
> > > >
> > > >> * Same questions re: granularity of sink tasks/connectors timing and
> > > >> whether the connectors need all the roll-ups of individual (worker,
> > > task)
> > > >> values to (worker, connector) level.
> > > >>
> > > >
> > > > I'm fine with taking out the aggregates to keep things simple and
> > prevent
> > > > misunderstanding.
> > > >
> > > >
> > > >> * If we expose the who the worker currently thinks is leader, it
> might
> > > >> also
> > > >> make sense to expose the underlying epoch. Not actually sure if we
> > > expose
> > > >> that for the consumer today, but it's an indicator of who is
> properly
> > up
> > > >> to
> > > >> date.
> > > >>
> > > >
> > > > Ack.
> > > >
> > > >
> > > >> * Why worker-level offset commit stats? It's not clear to me that
> > these
> > > >> are
> > > >> useful without considering the specific connector.
> > > >>
> > > >
> > > > So would they make more sense on the tasks? Again, on the worker
> > they're
> > > > aggregates.
> > > >
> > > >
> > > >>
> > > >> -Ewen
> > > >>
> > > >>
> > > >> On Mon, Sep 11, 2017 at 9:43 AM, Randall Hauch <rh...@gmail.com>
> > > wrote:
> > > >>
> > > >> > Thanks for reviewing. Responses inline below.
> > > >> >
> > > >> > On Mon, Sep 11, 2017 at 11:22 AM, Roger Hoover <
> > > roger.hoover@gmail.com>
> > > >> > wrote:
> > > >> >
> > > >> > > Randall,
> > > >> > >
> > > >> > > Thank you for the KIP.  This should improve visibility
> greatly.  I
> > > >> had a
> > > >> > > few questions/ideas for more metrics.
> > > >> > >
> > > >> > >
> > > >> > >    1. What's the relationship between the worker state and the
> > > >> connector
> > > >> > >    status?  Does the 'paused' status at the Connector level
> > include
> > > >> the
> > > >> > > time
> > > >> > >    that worker is 'rebalancing'?
> > > >> > >
> > > >> >
> > > >> > The worker state metric simply reports whether the worker is
> running
> > > or
> > > >> > rebalancing. This state is independent of how many connectors are
> > > >> > deployed/running/paused. During a rebalance, the connectors are
> > being
> > > >> > stopped and restarted but are effectively not running.
> > > >> >
> > > >> >
> > > >> > >    2. Are the "Source Connector" metrics like record rate an
> > > >> aggregation
> > > >> > of
> > > >> > >    the "Source Task" metrics?
> > > >> > >
> > > >> >
> > > >> > Yes.
> > > >> >
> > > >> >
> > > >> > >       - How much value is there is monitoring at the "Source
> > > >> Connector"
> > > >> > >       level (other than status) if the number of constituent
> tasks
> > > may
> > > >> > > change
> > > >> > >       over time?
> > > >> > >
> > > >> >
> > > >> > The task metrics allow you to know whether the tasks are evenly
> > loaded
> > > >> and
> > > >> > each making progress. The aggregate connector metrics tell you how
> > > much
> > > >> > work has been performed by all the tasks in that worker. Both are
> > > useful
> > > >> > IMO.
> > > >> >
> > > >> >
> > > >> > >       - I'm imagining that it's most useful to collect metrics
> at
> > > the
> > > >> > task
> > > >> > >       level as the task-level metrics should be stable
> regardless
> > of
> > > >> > tasks
> > > >> > >       shifting to different workers
> > > >> > >
> > > >> >
> > > >> > Correct, this is where the most value is because it is the most
> fine
> > > >> > grained.
> > > >> >
> > > >> >
> > > >> > >       - If so, can we duplicate the Connector Status down at the
> > > task
> > > >> > level
> > > >> > >          so that all important metrics can be tracked by task?
> > > >> > >
> > > >> >
> > > >> > Possibly. The challenge is that the threads running the tasks are
> > > >> blocked
> > > >> > when a connector is paused.
> > > >> >
> > > >> >
> > > >> > >          3. For the Sink Task metrics
> > > >> > >       - Can we add offset lag and timestamp lag on commit?
> > > >> > >          - After records are flushed/committed
> > > >> > >             - what is the diff between the record timestamps and
> > > >> commit
> > > >> > >             time (histogram)?  this is a measure of end-to-end
> > > >> pipeline
> > > >> > > latency
> > > >> > >             - what is the diff between record offsets and latest
> > > >> offset
> > > >> > of
> > > >> > >             their partition at commit time (histogram)? this is
> a
> > > >> > > measure of whether
> > > >> > >             this particular task is keeping up
> > > >> > >
> > > >> >
> > > >> > Yeah, possibly. Will have to compare with the consumer metrics to
> > see
> > > >> what
> > > >> > we can get.
> > > >> >
> > > >> >
> > > >> > >          - How about flush error rate?  Assuming the sink
> > connectors
> > > >> are
> > > >> > >       using retries, it would be helpful to know how many errors
> > > >> they're
> > > >> > > seeing
> > > >> > >
> > > >> >
> > > >> > We could add a metric to track how many times the framework
> > receives a
> > > >> > retry exception and then retries, but the connectors may also do
> > this
> > > on
> > > >> > their own.
> > > >> >
> > > >> >
> > > >> > >       - Can we tell at the framework level how many records were
> > > >> inserted
> > > >> > >       vs updated vs deleted?
> > > >> > >
> > > >> >
> > > >> > No, there's no distinction in the Connect framework.
> > > >> >
> > > >> >
> > > >> > >       - Batching stats
> > > >> > >          - Histogram of flush batch size
> > > >> > >          - Counts of flush trigger method (time vs max batch
> size)
> > > >> > >
> > > >> >
> > > >> > Should be able to add these.
> > > >> >
> > > >> >
> > > >> > >
> > > >> > > Cheers,
> > > >> > >
> > > >> > > Roger
> > > >> > >
> > > >> > > On Sun, Sep 10, 2017 at 8:45 AM, Randall Hauch <
> rhauch@gmail.com>
> > > >> wrote:
> > > >> > >
> > > >> > > > Thanks, Gwen.
> > > >> > > >
> > > >> > > > That's a great idea, so I've changed the KIP to add those
> > metrics.
> > > >> I've
> > > >> > > > also made a few other changes:
> > > >> > > >
> > > >> > > >
> > > >> > > >    1. The context of all metrics is limited to the activity
> > within
> > > >> the
> > > >> > > >    worker. This wasn't clear before, so I changed the
> motivation
> > > and
> > > >> > > metric
> > > >> > > >    descriptions to explicitly state this.
> > > >> > > >    2. Added the worker ID to all MBean attributes. In addition
> > to
> > > >> > > hopefully
> > > >> > > >    making this same scope obvious from within JMX or other
> > metric
> > > >> > > reporting
> > > >> > > >    system. This is also similar to how the Kafka producer and
> > > >> consumer
> > > >> > > > metrics
> > > >> > > >    include the client ID in their MBean attributes. Hopefully
> > this
> > > >> does
> > > >> > > not
> > > >> > > >    negatively impact or complicate how external reporting
> > systems'
> > > >> > > > aggregate
> > > >> > > >    metrics from multiple workers.
> > > >> > > >    3. Stated explicitly that aggregating metrics across
> workers
> > > was
> > > >> out
> > > >> > > of
> > > >> > > >    scope of this KIP.
> > > >> > > >    4. Added metrics to report the connector class and version
> > for
> > > >> both
> > > >> > > sink
> > > >> > > >    and source connectors.
> > > >> > > >
> > > >> > > > Check this KIP's history for details of these changes.
> > > >> > > >
> > > >> > > > Please let me know if you have any other suggestions. I hope
> to
> > > >> start
> > > >> > the
> > > >> > > > voting soon!
> > > >> > > >
> > > >> > > > Best regards,
> > > >> > > >
> > > >> > > > Randall
> > > >> > > >
> > > >> > > > On Thu, Sep 7, 2017 at 9:35 PM, Gwen Shapira <
> gwen@confluent.io
> > >
> > > >> > wrote:
> > > >> > > >
> > > >> > > > > Thanks for the KIP, Randall. Those are badly needed!
> > > >> > > > >
> > > >> > > > > Can we have two metrics with record rate per task? One
> before
> > > SMT
> > > >> and
> > > >> > > one
> > > >> > > > > after?
> > > >> > > > > We can have cases where we read 5000 rows from JDBC but
> write
> > 5
> > > to
> > > >> > > Kafka,
> > > >> > > > > or read 5000 records from Kafka and write 5 due to
> filtering.
> > I
> > > >> think
> > > >> > > its
> > > >> > > > > important to know both numbers.
> > > >> > > > >
> > > >> > > > >
> > > >> > > > > Gwen
> > > >> > > > >
> > > >> > > > > On Thu, Sep 7, 2017 at 7:50 PM, Randall Hauch <
> > rhauch@gmail.com
> > > >
> > > >> > > wrote:
> > > >> > > > >
> > > >> > > > > > Hi everyone.
> > > >> > > > > >
> > > >> > > > > > I've created a new KIP to add metrics to the Kafka Connect
> > > >> > framework:
> > > >> > > > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > > >> > > > > > 196%3A+Add+metrics+to+Kafka+Connect+framework
> > > >> > > > > >
> > > >> > > > > > The KIP approval deadline is looming, so if you're
> > interested
> > > in
> > > >> > > Kafka
> > > >> > > > > > Connect metrics please review and provide feedback as soon
> > as
> > > >> > > possible.
> > > >> > > > > I'm
> > > >> > > > > > interested not only in whether the metrics are sufficient
> > and
> > > >> > > > > appropriate,
> > > >> > > > > > but also in whether the MBean naming conventions are okay.
> > > >> > > > > >
> > > >> > > > > > Best regards,
> > > >> > > > > >
> > > >> > > > > > Randall
> > > >> > > > > >
> > > >> > > > >
> > > >> > > > >
> > > >> > > > >
> > > >> > > > > --
> > > >> > > > > *Gwen Shapira*
> > > >> > > > > Product Manager | Confluent
> > > >> > > > > 650.450.2760 <(650)%20450-2760> | @gwenshap
> > > >> > > > > Follow us: Twitter <https://twitter.com/ConfluentInc> |
> blog
> > > >> > > > > <http://www.confluent.io/blog>
> > > >> > > > >
> > > >> > > >
> > > >> > >
> > > >> >
> > > >>
> > > >
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-196: Add metrics to Kafka Connect framework

Posted by Gwen Shapira <gw...@confluent.io>.
Ewen, you gave a nice talk at Kafka Summit where you warned about the
danger of SMTs that slow down the data pipe. If we don't provide the time
metrics, how will users know when their SMTs are causing performance issues?

Gwen

On Mon, Sep 11, 2017 at 7:50 PM Ewen Cheslack-Postava <ew...@confluent.io>
wrote:

> re: questions about additional metrics, I think we'll undoubtedly find more
> that people want in practice, but as I mentioned earlier I think it's
> better to add the ones we know we need and then fill out the rest as we
> figure it out. So, e.g., batch size metrics sound like they could be
> useful, but I'd probably wait until we have a clear use case. It seems
> likely that it could be useful in diagnosing slow connectors (e.g. the
> implementation just does something inefficient), but I'm not really sure
> about that yet.
>
> -Ewen
>
> On Mon, Sep 11, 2017 at 7:11 PM, Randall Hauch <rh...@gmail.com> wrote:
>
> > Based on Roger and Ewen's feedback, I removed the aggregate metrics as
> they
> > would be difficult to make use of without extra work. This simplified
> > things a great deal, and I took the opportunity to reorganize the groups
> of
> > metrics. Also, based upon Ewen's concerns regarding measuring
> > times/durations, I removed all time-related metrics except for the offset
> > commits and rebalances, which are infrequent enough to warrant the
> capture
> > of percentiles. Roger asked about capturing batch size metrics for source
> > and sink tasks, and offset lag metrics for sink tasks. Finally, Ewen
> > pointed out that all count/total metrics are only valid since the most
> > recent rebalance and are therefore less meaningful, and were removed.
> >
> > On Mon, Sep 11, 2017 at 6:50 PM, Randall Hauch <rh...@gmail.com> wrote:
> >
> > > Thanks, Ewen. Comments inline below.
> > >
> > > On Mon, Sep 11, 2017 at 5:46 PM, Ewen Cheslack-Postava <
> > ewen@confluent.io>
> > > wrote:
> > >
> > >> Randall,
> > >>
> > >> A couple of questions:
> > >>
> > >> * Some metrics don't seem to have unique names? e.g.
> > >> source-record-produce-rate and source-record-produce-total seem like
> > they
> > >> are duplicated. Looks like maybe just an oversight that the second
> ones
> > >> should be changed from "produce" to "write".
> > >>
> > >
> > > Nice catch. You are correct - should be "write" instead of "produce". I
> > > will correct.
> > >
> > >
> > >> * I think there's a stray extra character in a couple of
> > >> places: kafka.connect:type=source-task-metrics,name=source-record-
> > >> produce-total,worker=([-.\w]+)l,connector=([-.\w]+),task=([\d]+)
> > >> has an extra char after the worker name.
> > >>
> > >
> > > Thanks. Removed in 2 places.
> > >
> > >
> > >> * Are the produce totals actually useful given rebalancing would
> cancel
> > >> them out anyway? Doesn't seem like you could do much with them.
> > >>
> > >
> > > Yes, the totals would be since the last rebalance. Maybe that isn't
> that
> > > useful. Might be better to capture the offsets and lag as Roger was
> > > suggestion. Thoughts?
> > >
> > >
> > >> * Why do transformations get their own metric but not converters? And
> > are
> > >> we concerned at all about the performance impact of getting such fine
> > >> grained info? Getting current time isn't free and we've seen before
> that
> > >> we
> > >> ended up w/ accidental performance regressions as we tried to check it
> > too
> > >> frequently to enforce timeouts fine grained in the producer (iirc).
> > >> Batching helps w/ this, but on the consumer side, a max.poll.records=1
> > >> setting could put you in a bad place, especially since transforms
> might
> > be
> > >> very lightweight (or nothing) and converters are expected to be
> > relatively
> > >> cheap as well.
> > >>
> > >
> > > We could remove the read, transform, and put time-based metrics for
> sink
> > > tasks, and poll, transform, and write time-based metrics. Can/should
> they
> > > be replaced with anything else?
> > >
> > >
> > >> * If we include the worker id everywhere and don't have metrics
> without
> > >> that included, isn't that a pain for users that dump this data into
> some
> > >> other system? They have to know which worker the connector/task is
> > >> currently on *or* need to do extra work to merge the metrics from
> across
> > >> machines. Including versions with the worker ID can make sense for
> > >> completeness and accuracy (e.g. technically there are still very slim
> > >> risks
> > >> of having a task running twice due to zombies), but it seems like bad
> > >> usability for the common case.
> > >>
> > >
> > > Part of the reason was also to help identify where each of the metrics
> > > came from, but per the next comment this may not be as useful, either.
> > > So remove the worker ID in all the task and connector metric names?
> What
> > > about the worker metrics?
> > >
> > >
> > >> * Is aggregating things like source record rate at the (worker,
> > connector)
> > >> level really useful since you're just going to need to do additional
> > >> aggregation anyway once you've collected metrics across all workers?
> I'd
> > >> rather add a smaller number of metrics w/ clear use cases than just
> try
> > to
> > >> be exhaustive and then have to maintain stuff that nobody actually
> uses.
> > >>
> > >
> > > Yes, the connector aggregate metrics are maybe not as useful if you
> also
> > > have to aggregate them from different workers. Removing them probably
> > also
> > > reduces the risk of them being misinterpretted.
> > >
> > >
> > >> * You have status for connectors but not for tasks. Any reason why?
> > Seems
> > >> like it'd make sense to expose both, especially since users generally
> > care
> > >> about task status more than connector status (not many connectors
> > actually
> > >> run a monitoring thread.)
> > >>
> > >
> > > Ack.
> > >
> > >
> > >> * Is number of tasks for each connector a useful metric? Not sure
> > whether
> > >> someone would find this useful or not. Probably not for alerts, but
> > might
> > >> be useful to be able to check it via your metrics dashboard.
> > >>
> > >
> > > Seems like it might be useful, at least in terms of tracking the number
> > of
> > > tasks over time. Might not be as useful for connectors that have
> > relatively
> > > static tasks, but it would be more interesting/useful for connectors
> that
> > > create tasks dynamically and periodically request task
> reconfigurations.
> > >
> > >
> > >> * Same questions re: granularity of sink tasks/connectors timing and
> > >> whether the connectors need all the roll-ups of individual (worker,
> > task)
> > >> values to (worker, connector) level.
> > >>
> > >
> > > I'm fine with taking out the aggregates to keep things simple and
> prevent
> > > misunderstanding.
> > >
> > >
> > >> * If we expose the who the worker currently thinks is leader, it might
> > >> also
> > >> make sense to expose the underlying epoch. Not actually sure if we
> > expose
> > >> that for the consumer today, but it's an indicator of who is properly
> up
> > >> to
> > >> date.
> > >>
> > >
> > > Ack.
> > >
> > >
> > >> * Why worker-level offset commit stats? It's not clear to me that
> these
> > >> are
> > >> useful without considering the specific connector.
> > >>
> > >
> > > So would they make more sense on the tasks? Again, on the worker
> they're
> > > aggregates.
> > >
> > >
> > >>
> > >> -Ewen
> > >>
> > >>
> > >> On Mon, Sep 11, 2017 at 9:43 AM, Randall Hauch <rh...@gmail.com>
> > wrote:
> > >>
> > >> > Thanks for reviewing. Responses inline below.
> > >> >
> > >> > On Mon, Sep 11, 2017 at 11:22 AM, Roger Hoover <
> > roger.hoover@gmail.com>
> > >> > wrote:
> > >> >
> > >> > > Randall,
> > >> > >
> > >> > > Thank you for the KIP.  This should improve visibility greatly.  I
> > >> had a
> > >> > > few questions/ideas for more metrics.
> > >> > >
> > >> > >
> > >> > >    1. What's the relationship between the worker state and the
> > >> connector
> > >> > >    status?  Does the 'paused' status at the Connector level
> include
> > >> the
> > >> > > time
> > >> > >    that worker is 'rebalancing'?
> > >> > >
> > >> >
> > >> > The worker state metric simply reports whether the worker is running
> > or
> > >> > rebalancing. This state is independent of how many connectors are
> > >> > deployed/running/paused. During a rebalance, the connectors are
> being
> > >> > stopped and restarted but are effectively not running.
> > >> >
> > >> >
> > >> > >    2. Are the "Source Connector" metrics like record rate an
> > >> aggregation
> > >> > of
> > >> > >    the "Source Task" metrics?
> > >> > >
> > >> >
> > >> > Yes.
> > >> >
> > >> >
> > >> > >       - How much value is there is monitoring at the "Source
> > >> Connector"
> > >> > >       level (other than status) if the number of constituent tasks
> > may
> > >> > > change
> > >> > >       over time?
> > >> > >
> > >> >
> > >> > The task metrics allow you to know whether the tasks are evenly
> loaded
> > >> and
> > >> > each making progress. The aggregate connector metrics tell you how
> > much
> > >> > work has been performed by all the tasks in that worker. Both are
> > useful
> > >> > IMO.
> > >> >
> > >> >
> > >> > >       - I'm imagining that it's most useful to collect metrics at
> > the
> > >> > task
> > >> > >       level as the task-level metrics should be stable regardless
> of
> > >> > tasks
> > >> > >       shifting to different workers
> > >> > >
> > >> >
> > >> > Correct, this is where the most value is because it is the most fine
> > >> > grained.
> > >> >
> > >> >
> > >> > >       - If so, can we duplicate the Connector Status down at the
> > task
> > >> > level
> > >> > >          so that all important metrics can be tracked by task?
> > >> > >
> > >> >
> > >> > Possibly. The challenge is that the threads running the tasks are
> > >> blocked
> > >> > when a connector is paused.
> > >> >
> > >> >
> > >> > >          3. For the Sink Task metrics
> > >> > >       - Can we add offset lag and timestamp lag on commit?
> > >> > >          - After records are flushed/committed
> > >> > >             - what is the diff between the record timestamps and
> > >> commit
> > >> > >             time (histogram)?  this is a measure of end-to-end
> > >> pipeline
> > >> > > latency
> > >> > >             - what is the diff between record offsets and latest
> > >> offset
> > >> > of
> > >> > >             their partition at commit time (histogram)? this is a
> > >> > > measure of whether
> > >> > >             this particular task is keeping up
> > >> > >
> > >> >
> > >> > Yeah, possibly. Will have to compare with the consumer metrics to
> see
> > >> what
> > >> > we can get.
> > >> >
> > >> >
> > >> > >          - How about flush error rate?  Assuming the sink
> connectors
> > >> are
> > >> > >       using retries, it would be helpful to know how many errors
> > >> they're
> > >> > > seeing
> > >> > >
> > >> >
> > >> > We could add a metric to track how many times the framework
> receives a
> > >> > retry exception and then retries, but the connectors may also do
> this
> > on
> > >> > their own.
> > >> >
> > >> >
> > >> > >       - Can we tell at the framework level how many records were
> > >> inserted
> > >> > >       vs updated vs deleted?
> > >> > >
> > >> >
> > >> > No, there's no distinction in the Connect framework.
> > >> >
> > >> >
> > >> > >       - Batching stats
> > >> > >          - Histogram of flush batch size
> > >> > >          - Counts of flush trigger method (time vs max batch size)
> > >> > >
> > >> >
> > >> > Should be able to add these.
> > >> >
> > >> >
> > >> > >
> > >> > > Cheers,
> > >> > >
> > >> > > Roger
> > >> > >
> > >> > > On Sun, Sep 10, 2017 at 8:45 AM, Randall Hauch <rh...@gmail.com>
> > >> wrote:
> > >> > >
> > >> > > > Thanks, Gwen.
> > >> > > >
> > >> > > > That's a great idea, so I've changed the KIP to add those
> metrics.
> > >> I've
> > >> > > > also made a few other changes:
> > >> > > >
> > >> > > >
> > >> > > >    1. The context of all metrics is limited to the activity
> within
> > >> the
> > >> > > >    worker. This wasn't clear before, so I changed the motivation
> > and
> > >> > > metric
> > >> > > >    descriptions to explicitly state this.
> > >> > > >    2. Added the worker ID to all MBean attributes. In addition
> to
> > >> > > hopefully
> > >> > > >    making this same scope obvious from within JMX or other
> metric
> > >> > > reporting
> > >> > > >    system. This is also similar to how the Kafka producer and
> > >> consumer
> > >> > > > metrics
> > >> > > >    include the client ID in their MBean attributes. Hopefully
> this
> > >> does
> > >> > > not
> > >> > > >    negatively impact or complicate how external reporting
> systems'
> > >> > > > aggregate
> > >> > > >    metrics from multiple workers.
> > >> > > >    3. Stated explicitly that aggregating metrics across workers
> > was
> > >> out
> > >> > > of
> > >> > > >    scope of this KIP.
> > >> > > >    4. Added metrics to report the connector class and version
> for
> > >> both
> > >> > > sink
> > >> > > >    and source connectors.
> > >> > > >
> > >> > > > Check this KIP's history for details of these changes.
> > >> > > >
> > >> > > > Please let me know if you have any other suggestions. I hope to
> > >> start
> > >> > the
> > >> > > > voting soon!
> > >> > > >
> > >> > > > Best regards,
> > >> > > >
> > >> > > > Randall
> > >> > > >
> > >> > > > On Thu, Sep 7, 2017 at 9:35 PM, Gwen Shapira <gwen@confluent.io
> >
> > >> > wrote:
> > >> > > >
> > >> > > > > Thanks for the KIP, Randall. Those are badly needed!
> > >> > > > >
> > >> > > > > Can we have two metrics with record rate per task? One before
> > SMT
> > >> and
> > >> > > one
> > >> > > > > after?
> > >> > > > > We can have cases where we read 5000 rows from JDBC but write
> 5
> > to
> > >> > > Kafka,
> > >> > > > > or read 5000 records from Kafka and write 5 due to filtering.
> I
> > >> think
> > >> > > its
> > >> > > > > important to know both numbers.
> > >> > > > >
> > >> > > > >
> > >> > > > > Gwen
> > >> > > > >
> > >> > > > > On Thu, Sep 7, 2017 at 7:50 PM, Randall Hauch <
> rhauch@gmail.com
> > >
> > >> > > wrote:
> > >> > > > >
> > >> > > > > > Hi everyone.
> > >> > > > > >
> > >> > > > > > I've created a new KIP to add metrics to the Kafka Connect
> > >> > framework:
> > >> > > > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > >> > > > > > 196%3A+Add+metrics+to+Kafka+Connect+framework
> > >> > > > > >
> > >> > > > > > The KIP approval deadline is looming, so if you're
> interested
> > in
> > >> > > Kafka
> > >> > > > > > Connect metrics please review and provide feedback as soon
> as
> > >> > > possible.
> > >> > > > > I'm
> > >> > > > > > interested not only in whether the metrics are sufficient
> and
> > >> > > > > appropriate,
> > >> > > > > > but also in whether the MBean naming conventions are okay.
> > >> > > > > >
> > >> > > > > > Best regards,
> > >> > > > > >
> > >> > > > > > Randall
> > >> > > > > >
> > >> > > > >
> > >> > > > >
> > >> > > > >
> > >> > > > > --
> > >> > > > > *Gwen Shapira*
> > >> > > > > Product Manager | Confluent
> > >> > > > > 650.450.2760 <(650)%20450-2760> | @gwenshap
> > >> > > > > Follow us: Twitter <https://twitter.com/ConfluentInc> | blog
> > >> > > > > <http://www.confluent.io/blog>
> > >> > > > >
> > >> > > >
> > >> > >
> > >> >
> > >>
> > >
> > >
> >
>

Re: [DISCUSS] KIP-196: Add metrics to Kafka Connect framework

Posted by Randall Hauch <rh...@gmail.com>.
Okay, I think I've incorporated all feedback except for Gwen and Roger than
would like to have timing metrics. Given the deadline and Ewen's concern
about degraded performance, I think it's prudent to leave those out of this
KIP and proceed as is.



On Tue, Sep 12, 2017 at 12:48 PM, Randall Hauch <rh...@gmail.com> wrote:

>
>
> On Tue, Sep 12, 2017 at 10:36 AM, Roger Hoover <ro...@gmail.com>
> wrote:
>
>> Randall/Ewen,
>>
>> I think the timing info is still useful even if it's measured since the
>> last rebalance.  How else do you know where time is being spent?
>>
>
> I think Ewen's concern (correct me if I'm wrong) is that measuring
> time-based metrics might result in excessive performance degradation,
> especially when batch sizes are small.
>
>
>>
>> The use case for seeing the batch size is that you generally have two
>> knobs
>> to configure - max batch size and max wait time.  The batch size metrics
>> would tell you how full your batches are based on your current linger time
>> so you can adjust the config.
>>
>
> It does seem that batch sizes are useful, and the KIP includes these
> ("batch-size-max" and "batch-size-avg").
>
>
>>
>> Cheers,
>>
>> Roger
>>
>> On Mon, Sep 11, 2017 at 7:50 PM, Ewen Cheslack-Postava <ewen@confluent.io
>> >
>> wrote:
>>
>> > re: questions about additional metrics, I think we'll undoubtedly find
>> more
>> > that people want in practice, but as I mentioned earlier I think it's
>> > better to add the ones we know we need and then fill out the rest as we
>> > figure it out. So, e.g., batch size metrics sound like they could be
>> > useful, but I'd probably wait until we have a clear use case. It seems
>> > likely that it could be useful in diagnosing slow connectors (e.g. the
>> > implementation just does something inefficient), but I'm not really sure
>> > about that yet.
>> >
>> > -Ewen
>> >
>> > On Mon, Sep 11, 2017 at 7:11 PM, Randall Hauch <rh...@gmail.com>
>> wrote:
>> >
>> > > Based on Roger and Ewen's feedback, I removed the aggregate metrics as
>> > they
>> > > would be difficult to make use of without extra work. This simplified
>> > > things a great deal, and I took the opportunity to reorganize the
>> groups
>> > of
>> > > metrics. Also, based upon Ewen's concerns regarding measuring
>> > > times/durations, I removed all time-related metrics except for the
>> offset
>> > > commits and rebalances, which are infrequent enough to warrant the
>> > capture
>> > > of percentiles. Roger asked about capturing batch size metrics for
>> source
>> > > and sink tasks, and offset lag metrics for sink tasks. Finally, Ewen
>> > > pointed out that all count/total metrics are only valid since the most
>> > > recent rebalance and are therefore less meaningful, and were removed.
>> > >
>> > > On Mon, Sep 11, 2017 at 6:50 PM, Randall Hauch <rh...@gmail.com>
>> wrote:
>> > >
>> > > > Thanks, Ewen. Comments inline below.
>> > > >
>> > > > On Mon, Sep 11, 2017 at 5:46 PM, Ewen Cheslack-Postava <
>> > > ewen@confluent.io>
>> > > > wrote:
>> > > >
>> > > >> Randall,
>> > > >>
>> > > >> A couple of questions:
>> > > >>
>> > > >> * Some metrics don't seem to have unique names? e.g.
>> > > >> source-record-produce-rate and source-record-produce-total seem
>> like
>> > > they
>> > > >> are duplicated. Looks like maybe just an oversight that the second
>> > ones
>> > > >> should be changed from "produce" to "write".
>> > > >>
>> > > >
>> > > > Nice catch. You are correct - should be "write" instead of
>> "produce". I
>> > > > will correct.
>> > > >
>> > > >
>> > > >> * I think there's a stray extra character in a couple of
>> > > >> places: kafka.connect:type=source-task-metrics,name=source-record-
>> > > >> produce-total,worker=([-.\w]+)l,connector=([-.\w]+),task=([\d]+)
>> > > >> has an extra char after the worker name.
>> > > >>
>> > > >
>> > > > Thanks. Removed in 2 places.
>> > > >
>> > > >
>> > > >> * Are the produce totals actually useful given rebalancing would
>> > cancel
>> > > >> them out anyway? Doesn't seem like you could do much with them.
>> > > >>
>> > > >
>> > > > Yes, the totals would be since the last rebalance. Maybe that isn't
>> > that
>> > > > useful. Might be better to capture the offsets and lag as Roger was
>> > > > suggestion. Thoughts?
>> > > >
>> > > >
>> > > >> * Why do transformations get their own metric but not converters?
>> And
>> > > are
>> > > >> we concerned at all about the performance impact of getting such
>> fine
>> > > >> grained info? Getting current time isn't free and we've seen before
>> > that
>> > > >> we
>> > > >> ended up w/ accidental performance regressions as we tried to
>> check it
>> > > too
>> > > >> frequently to enforce timeouts fine grained in the producer (iirc).
>> > > >> Batching helps w/ this, but on the consumer side, a
>> max.poll.records=1
>> > > >> setting could put you in a bad place, especially since transforms
>> > might
>> > > be
>> > > >> very lightweight (or nothing) and converters are expected to be
>> > > relatively
>> > > >> cheap as well.
>> > > >>
>> > > >
>> > > > We could remove the read, transform, and put time-based metrics for
>> > sink
>> > > > tasks, and poll, transform, and write time-based metrics. Can/should
>> > they
>> > > > be replaced with anything else?
>> > > >
>> > > >
>> > > >> * If we include the worker id everywhere and don't have metrics
>> > without
>> > > >> that included, isn't that a pain for users that dump this data into
>> > some
>> > > >> other system? They have to know which worker the connector/task is
>> > > >> currently on *or* need to do extra work to merge the metrics from
>> > across
>> > > >> machines. Including versions with the worker ID can make sense for
>> > > >> completeness and accuracy (e.g. technically there are still very
>> slim
>> > > >> risks
>> > > >> of having a task running twice due to zombies), but it seems like
>> bad
>> > > >> usability for the common case.
>> > > >>
>> > > >
>> > > > Part of the reason was also to help identify where each of the
>> metrics
>> > > > came from, but per the next comment this may not be as useful,
>> either.
>> > > > So remove the worker ID in all the task and connector metric names?
>> > What
>> > > > about the worker metrics?
>> > > >
>> > > >
>> > > >> * Is aggregating things like source record rate at the (worker,
>> > > connector)
>> > > >> level really useful since you're just going to need to do
>> additional
>> > > >> aggregation anyway once you've collected metrics across all
>> workers?
>> > I'd
>> > > >> rather add a smaller number of metrics w/ clear use cases than just
>> > try
>> > > to
>> > > >> be exhaustive and then have to maintain stuff that nobody actually
>> > uses.
>> > > >>
>> > > >
>> > > > Yes, the connector aggregate metrics are maybe not as useful if you
>> > also
>> > > > have to aggregate them from different workers. Removing them
>> probably
>> > > also
>> > > > reduces the risk of them being misinterpretted.
>> > > >
>> > > >
>> > > >> * You have status for connectors but not for tasks. Any reason why?
>> > > Seems
>> > > >> like it'd make sense to expose both, especially since users
>> generally
>> > > care
>> > > >> about task status more than connector status (not many connectors
>> > > actually
>> > > >> run a monitoring thread.)
>> > > >>
>> > > >
>> > > > Ack.
>> > > >
>> > > >
>> > > >> * Is number of tasks for each connector a useful metric? Not sure
>> > > whether
>> > > >> someone would find this useful or not. Probably not for alerts, but
>> > > might
>> > > >> be useful to be able to check it via your metrics dashboard.
>> > > >>
>> > > >
>> > > > Seems like it might be useful, at least in terms of tracking the
>> number
>> > > of
>> > > > tasks over time. Might not be as useful for connectors that have
>> > > relatively
>> > > > static tasks, but it would be more interesting/useful for connectors
>> > that
>> > > > create tasks dynamically and periodically request task
>> > reconfigurations.
>> > > >
>> > > >
>> > > >> * Same questions re: granularity of sink tasks/connectors timing
>> and
>> > > >> whether the connectors need all the roll-ups of individual (worker,
>> > > task)
>> > > >> values to (worker, connector) level.
>> > > >>
>> > > >
>> > > > I'm fine with taking out the aggregates to keep things simple and
>> > prevent
>> > > > misunderstanding.
>> > > >
>> > > >
>> > > >> * If we expose the who the worker currently thinks is leader, it
>> might
>> > > >> also
>> > > >> make sense to expose the underlying epoch. Not actually sure if we
>> > > expose
>> > > >> that for the consumer today, but it's an indicator of who is
>> properly
>> > up
>> > > >> to
>> > > >> date.
>> > > >>
>> > > >
>> > > > Ack.
>> > > >
>> > > >
>> > > >> * Why worker-level offset commit stats? It's not clear to me that
>> > these
>> > > >> are
>> > > >> useful without considering the specific connector.
>> > > >>
>> > > >
>> > > > So would they make more sense on the tasks? Again, on the worker
>> > they're
>> > > > aggregates.
>> > > >
>> > > >
>> > > >>
>> > > >> -Ewen
>> > > >>
>> > > >>
>> > > >> On Mon, Sep 11, 2017 at 9:43 AM, Randall Hauch <rh...@gmail.com>
>> > > wrote:
>> > > >>
>> > > >> > Thanks for reviewing. Responses inline below.
>> > > >> >
>> > > >> > On Mon, Sep 11, 2017 at 11:22 AM, Roger Hoover <
>> > > roger.hoover@gmail.com>
>> > > >> > wrote:
>> > > >> >
>> > > >> > > Randall,
>> > > >> > >
>> > > >> > > Thank you for the KIP.  This should improve visibility
>> greatly.  I
>> > > >> had a
>> > > >> > > few questions/ideas for more metrics.
>> > > >> > >
>> > > >> > >
>> > > >> > >    1. What's the relationship between the worker state and the
>> > > >> connector
>> > > >> > >    status?  Does the 'paused' status at the Connector level
>> > include
>> > > >> the
>> > > >> > > time
>> > > >> > >    that worker is 'rebalancing'?
>> > > >> > >
>> > > >> >
>> > > >> > The worker state metric simply reports whether the worker is
>> running
>> > > or
>> > > >> > rebalancing. This state is independent of how many connectors are
>> > > >> > deployed/running/paused. During a rebalance, the connectors are
>> > being
>> > > >> > stopped and restarted but are effectively not running.
>> > > >> >
>> > > >> >
>> > > >> > >    2. Are the "Source Connector" metrics like record rate an
>> > > >> aggregation
>> > > >> > of
>> > > >> > >    the "Source Task" metrics?
>> > > >> > >
>> > > >> >
>> > > >> > Yes.
>> > > >> >
>> > > >> >
>> > > >> > >       - How much value is there is monitoring at the "Source
>> > > >> Connector"
>> > > >> > >       level (other than status) if the number of constituent
>> tasks
>> > > may
>> > > >> > > change
>> > > >> > >       over time?
>> > > >> > >
>> > > >> >
>> > > >> > The task metrics allow you to know whether the tasks are evenly
>> > loaded
>> > > >> and
>> > > >> > each making progress. The aggregate connector metrics tell you
>> how
>> > > much
>> > > >> > work has been performed by all the tasks in that worker. Both are
>> > > useful
>> > > >> > IMO.
>> > > >> >
>> > > >> >
>> > > >> > >       - I'm imagining that it's most useful to collect metrics
>> at
>> > > the
>> > > >> > task
>> > > >> > >       level as the task-level metrics should be stable
>> regardless
>> > of
>> > > >> > tasks
>> > > >> > >       shifting to different workers
>> > > >> > >
>> > > >> >
>> > > >> > Correct, this is where the most value is because it is the most
>> fine
>> > > >> > grained.
>> > > >> >
>> > > >> >
>> > > >> > >       - If so, can we duplicate the Connector Status down at
>> the
>> > > task
>> > > >> > level
>> > > >> > >          so that all important metrics can be tracked by task?
>> > > >> > >
>> > > >> >
>> > > >> > Possibly. The challenge is that the threads running the tasks are
>> > > >> blocked
>> > > >> > when a connector is paused.
>> > > >> >
>> > > >> >
>> > > >> > >          3. For the Sink Task metrics
>> > > >> > >       - Can we add offset lag and timestamp lag on commit?
>> > > >> > >          - After records are flushed/committed
>> > > >> > >             - what is the diff between the record timestamps
>> and
>> > > >> commit
>> > > >> > >             time (histogram)?  this is a measure of end-to-end
>> > > >> pipeline
>> > > >> > > latency
>> > > >> > >             - what is the diff between record offsets and
>> latest
>> > > >> offset
>> > > >> > of
>> > > >> > >             their partition at commit time (histogram)? this
>> is a
>> > > >> > > measure of whether
>> > > >> > >             this particular task is keeping up
>> > > >> > >
>> > > >> >
>> > > >> > Yeah, possibly. Will have to compare with the consumer metrics to
>> > see
>> > > >> what
>> > > >> > we can get.
>> > > >> >
>> > > >> >
>> > > >> > >          - How about flush error rate?  Assuming the sink
>> > connectors
>> > > >> are
>> > > >> > >       using retries, it would be helpful to know how many
>> errors
>> > > >> they're
>> > > >> > > seeing
>> > > >> > >
>> > > >> >
>> > > >> > We could add a metric to track how many times the framework
>> > receives a
>> > > >> > retry exception and then retries, but the connectors may also do
>> > this
>> > > on
>> > > >> > their own.
>> > > >> >
>> > > >> >
>> > > >> > >       - Can we tell at the framework level how many records
>> were
>> > > >> inserted
>> > > >> > >       vs updated vs deleted?
>> > > >> > >
>> > > >> >
>> > > >> > No, there's no distinction in the Connect framework.
>> > > >> >
>> > > >> >
>> > > >> > >       - Batching stats
>> > > >> > >          - Histogram of flush batch size
>> > > >> > >          - Counts of flush trigger method (time vs max batch
>> size)
>> > > >> > >
>> > > >> >
>> > > >> > Should be able to add these.
>> > > >> >
>> > > >> >
>> > > >> > >
>> > > >> > > Cheers,
>> > > >> > >
>> > > >> > > Roger
>> > > >> > >
>> > > >> > > On Sun, Sep 10, 2017 at 8:45 AM, Randall Hauch <
>> rhauch@gmail.com>
>> > > >> wrote:
>> > > >> > >
>> > > >> > > > Thanks, Gwen.
>> > > >> > > >
>> > > >> > > > That's a great idea, so I've changed the KIP to add those
>> > metrics.
>> > > >> I've
>> > > >> > > > also made a few other changes:
>> > > >> > > >
>> > > >> > > >
>> > > >> > > >    1. The context of all metrics is limited to the activity
>> > within
>> > > >> the
>> > > >> > > >    worker. This wasn't clear before, so I changed the
>> motivation
>> > > and
>> > > >> > > metric
>> > > >> > > >    descriptions to explicitly state this.
>> > > >> > > >    2. Added the worker ID to all MBean attributes. In
>> addition
>> > to
>> > > >> > > hopefully
>> > > >> > > >    making this same scope obvious from within JMX or other
>> > metric
>> > > >> > > reporting
>> > > >> > > >    system. This is also similar to how the Kafka producer and
>> > > >> consumer
>> > > >> > > > metrics
>> > > >> > > >    include the client ID in their MBean attributes. Hopefully
>> > this
>> > > >> does
>> > > >> > > not
>> > > >> > > >    negatively impact or complicate how external reporting
>> > systems'
>> > > >> > > > aggregate
>> > > >> > > >    metrics from multiple workers.
>> > > >> > > >    3. Stated explicitly that aggregating metrics across
>> workers
>> > > was
>> > > >> out
>> > > >> > > of
>> > > >> > > >    scope of this KIP.
>> > > >> > > >    4. Added metrics to report the connector class and version
>> > for
>> > > >> both
>> > > >> > > sink
>> > > >> > > >    and source connectors.
>> > > >> > > >
>> > > >> > > > Check this KIP's history for details of these changes.
>> > > >> > > >
>> > > >> > > > Please let me know if you have any other suggestions. I hope
>> to
>> > > >> start
>> > > >> > the
>> > > >> > > > voting soon!
>> > > >> > > >
>> > > >> > > > Best regards,
>> > > >> > > >
>> > > >> > > > Randall
>> > > >> > > >
>> > > >> > > > On Thu, Sep 7, 2017 at 9:35 PM, Gwen Shapira <
>> gwen@confluent.io
>> > >
>> > > >> > wrote:
>> > > >> > > >
>> > > >> > > > > Thanks for the KIP, Randall. Those are badly needed!
>> > > >> > > > >
>> > > >> > > > > Can we have two metrics with record rate per task? One
>> before
>> > > SMT
>> > > >> and
>> > > >> > > one
>> > > >> > > > > after?
>> > > >> > > > > We can have cases where we read 5000 rows from JDBC but
>> write
>> > 5
>> > > to
>> > > >> > > Kafka,
>> > > >> > > > > or read 5000 records from Kafka and write 5 due to
>> filtering.
>> > I
>> > > >> think
>> > > >> > > its
>> > > >> > > > > important to know both numbers.
>> > > >> > > > >
>> > > >> > > > >
>> > > >> > > > > Gwen
>> > > >> > > > >
>> > > >> > > > > On Thu, Sep 7, 2017 at 7:50 PM, Randall Hauch <
>> > rhauch@gmail.com
>> > > >
>> > > >> > > wrote:
>> > > >> > > > >
>> > > >> > > > > > Hi everyone.
>> > > >> > > > > >
>> > > >> > > > > > I've created a new KIP to add metrics to the Kafka
>> Connect
>> > > >> > framework:
>> > > >> > > > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-
>> > > >> > > > > > 196%3A+Add+metrics+to+Kafka+Connect+framework
>> > > >> > > > > >
>> > > >> > > > > > The KIP approval deadline is looming, so if you're
>> > interested
>> > > in
>> > > >> > > Kafka
>> > > >> > > > > > Connect metrics please review and provide feedback as
>> soon
>> > as
>> > > >> > > possible.
>> > > >> > > > > I'm
>> > > >> > > > > > interested not only in whether the metrics are sufficient
>> > and
>> > > >> > > > > appropriate,
>> > > >> > > > > > but also in whether the MBean naming conventions are
>> okay.
>> > > >> > > > > >
>> > > >> > > > > > Best regards,
>> > > >> > > > > >
>> > > >> > > > > > Randall
>> > > >> > > > > >
>> > > >> > > > >
>> > > >> > > > >
>> > > >> > > > >
>> > > >> > > > > --
>> > > >> > > > > *Gwen Shapira*
>> > > >> > > > > Product Manager | Confluent
>> > > >> > > > > 650.450.2760 | @gwenshap
>> > > >> > > > > Follow us: Twitter <https://twitter.com/ConfluentInc> |
>> blog
>> > > >> > > > > <http://www.confluent.io/blog>
>> > > >> > > > >
>> > > >> > > >
>> > > >> > >
>> > > >> >
>> > > >>
>> > > >
>> > > >
>> > >
>> >
>>
>
>

Re: [DISCUSS] KIP-196: Add metrics to Kafka Connect framework

Posted by Randall Hauch <rh...@gmail.com>.
On Tue, Sep 12, 2017 at 10:36 AM, Roger Hoover <ro...@gmail.com>
wrote:

> Randall/Ewen,
>
> I think the timing info is still useful even if it's measured since the
> last rebalance.  How else do you know where time is being spent?
>

I think Ewen's concern (correct me if I'm wrong) is that measuring
time-based metrics might result in excessive performance degradation,
especially when batch sizes are small.


>
> The use case for seeing the batch size is that you generally have two knobs
> to configure - max batch size and max wait time.  The batch size metrics
> would tell you how full your batches are based on your current linger time
> so you can adjust the config.
>

It does seem that batch sizes are useful, and the KIP includes these
("batch-size-max" and "batch-size-avg").


>
> Cheers,
>
> Roger
>
> On Mon, Sep 11, 2017 at 7:50 PM, Ewen Cheslack-Postava <ew...@confluent.io>
> wrote:
>
> > re: questions about additional metrics, I think we'll undoubtedly find
> more
> > that people want in practice, but as I mentioned earlier I think it's
> > better to add the ones we know we need and then fill out the rest as we
> > figure it out. So, e.g., batch size metrics sound like they could be
> > useful, but I'd probably wait until we have a clear use case. It seems
> > likely that it could be useful in diagnosing slow connectors (e.g. the
> > implementation just does something inefficient), but I'm not really sure
> > about that yet.
> >
> > -Ewen
> >
> > On Mon, Sep 11, 2017 at 7:11 PM, Randall Hauch <rh...@gmail.com> wrote:
> >
> > > Based on Roger and Ewen's feedback, I removed the aggregate metrics as
> > they
> > > would be difficult to make use of without extra work. This simplified
> > > things a great deal, and I took the opportunity to reorganize the
> groups
> > of
> > > metrics. Also, based upon Ewen's concerns regarding measuring
> > > times/durations, I removed all time-related metrics except for the
> offset
> > > commits and rebalances, which are infrequent enough to warrant the
> > capture
> > > of percentiles. Roger asked about capturing batch size metrics for
> source
> > > and sink tasks, and offset lag metrics for sink tasks. Finally, Ewen
> > > pointed out that all count/total metrics are only valid since the most
> > > recent rebalance and are therefore less meaningful, and were removed.
> > >
> > > On Mon, Sep 11, 2017 at 6:50 PM, Randall Hauch <rh...@gmail.com>
> wrote:
> > >
> > > > Thanks, Ewen. Comments inline below.
> > > >
> > > > On Mon, Sep 11, 2017 at 5:46 PM, Ewen Cheslack-Postava <
> > > ewen@confluent.io>
> > > > wrote:
> > > >
> > > >> Randall,
> > > >>
> > > >> A couple of questions:
> > > >>
> > > >> * Some metrics don't seem to have unique names? e.g.
> > > >> source-record-produce-rate and source-record-produce-total seem like
> > > they
> > > >> are duplicated. Looks like maybe just an oversight that the second
> > ones
> > > >> should be changed from "produce" to "write".
> > > >>
> > > >
> > > > Nice catch. You are correct - should be "write" instead of
> "produce". I
> > > > will correct.
> > > >
> > > >
> > > >> * I think there's a stray extra character in a couple of
> > > >> places: kafka.connect:type=source-task-metrics,name=source-record-
> > > >> produce-total,worker=([-.\w]+)l,connector=([-.\w]+),task=([\d]+)
> > > >> has an extra char after the worker name.
> > > >>
> > > >
> > > > Thanks. Removed in 2 places.
> > > >
> > > >
> > > >> * Are the produce totals actually useful given rebalancing would
> > cancel
> > > >> them out anyway? Doesn't seem like you could do much with them.
> > > >>
> > > >
> > > > Yes, the totals would be since the last rebalance. Maybe that isn't
> > that
> > > > useful. Might be better to capture the offsets and lag as Roger was
> > > > suggestion. Thoughts?
> > > >
> > > >
> > > >> * Why do transformations get their own metric but not converters?
> And
> > > are
> > > >> we concerned at all about the performance impact of getting such
> fine
> > > >> grained info? Getting current time isn't free and we've seen before
> > that
> > > >> we
> > > >> ended up w/ accidental performance regressions as we tried to check
> it
> > > too
> > > >> frequently to enforce timeouts fine grained in the producer (iirc).
> > > >> Batching helps w/ this, but on the consumer side, a
> max.poll.records=1
> > > >> setting could put you in a bad place, especially since transforms
> > might
> > > be
> > > >> very lightweight (or nothing) and converters are expected to be
> > > relatively
> > > >> cheap as well.
> > > >>
> > > >
> > > > We could remove the read, transform, and put time-based metrics for
> > sink
> > > > tasks, and poll, transform, and write time-based metrics. Can/should
> > they
> > > > be replaced with anything else?
> > > >
> > > >
> > > >> * If we include the worker id everywhere and don't have metrics
> > without
> > > >> that included, isn't that a pain for users that dump this data into
> > some
> > > >> other system? They have to know which worker the connector/task is
> > > >> currently on *or* need to do extra work to merge the metrics from
> > across
> > > >> machines. Including versions with the worker ID can make sense for
> > > >> completeness and accuracy (e.g. technically there are still very
> slim
> > > >> risks
> > > >> of having a task running twice due to zombies), but it seems like
> bad
> > > >> usability for the common case.
> > > >>
> > > >
> > > > Part of the reason was also to help identify where each of the
> metrics
> > > > came from, but per the next comment this may not be as useful,
> either.
> > > > So remove the worker ID in all the task and connector metric names?
> > What
> > > > about the worker metrics?
> > > >
> > > >
> > > >> * Is aggregating things like source record rate at the (worker,
> > > connector)
> > > >> level really useful since you're just going to need to do additional
> > > >> aggregation anyway once you've collected metrics across all workers?
> > I'd
> > > >> rather add a smaller number of metrics w/ clear use cases than just
> > try
> > > to
> > > >> be exhaustive and then have to maintain stuff that nobody actually
> > uses.
> > > >>
> > > >
> > > > Yes, the connector aggregate metrics are maybe not as useful if you
> > also
> > > > have to aggregate them from different workers. Removing them probably
> > > also
> > > > reduces the risk of them being misinterpretted.
> > > >
> > > >
> > > >> * You have status for connectors but not for tasks. Any reason why?
> > > Seems
> > > >> like it'd make sense to expose both, especially since users
> generally
> > > care
> > > >> about task status more than connector status (not many connectors
> > > actually
> > > >> run a monitoring thread.)
> > > >>
> > > >
> > > > Ack.
> > > >
> > > >
> > > >> * Is number of tasks for each connector a useful metric? Not sure
> > > whether
> > > >> someone would find this useful or not. Probably not for alerts, but
> > > might
> > > >> be useful to be able to check it via your metrics dashboard.
> > > >>
> > > >
> > > > Seems like it might be useful, at least in terms of tracking the
> number
> > > of
> > > > tasks over time. Might not be as useful for connectors that have
> > > relatively
> > > > static tasks, but it would be more interesting/useful for connectors
> > that
> > > > create tasks dynamically and periodically request task
> > reconfigurations.
> > > >
> > > >
> > > >> * Same questions re: granularity of sink tasks/connectors timing and
> > > >> whether the connectors need all the roll-ups of individual (worker,
> > > task)
> > > >> values to (worker, connector) level.
> > > >>
> > > >
> > > > I'm fine with taking out the aggregates to keep things simple and
> > prevent
> > > > misunderstanding.
> > > >
> > > >
> > > >> * If we expose the who the worker currently thinks is leader, it
> might
> > > >> also
> > > >> make sense to expose the underlying epoch. Not actually sure if we
> > > expose
> > > >> that for the consumer today, but it's an indicator of who is
> properly
> > up
> > > >> to
> > > >> date.
> > > >>
> > > >
> > > > Ack.
> > > >
> > > >
> > > >> * Why worker-level offset commit stats? It's not clear to me that
> > these
> > > >> are
> > > >> useful without considering the specific connector.
> > > >>
> > > >
> > > > So would they make more sense on the tasks? Again, on the worker
> > they're
> > > > aggregates.
> > > >
> > > >
> > > >>
> > > >> -Ewen
> > > >>
> > > >>
> > > >> On Mon, Sep 11, 2017 at 9:43 AM, Randall Hauch <rh...@gmail.com>
> > > wrote:
> > > >>
> > > >> > Thanks for reviewing. Responses inline below.
> > > >> >
> > > >> > On Mon, Sep 11, 2017 at 11:22 AM, Roger Hoover <
> > > roger.hoover@gmail.com>
> > > >> > wrote:
> > > >> >
> > > >> > > Randall,
> > > >> > >
> > > >> > > Thank you for the KIP.  This should improve visibility
> greatly.  I
> > > >> had a
> > > >> > > few questions/ideas for more metrics.
> > > >> > >
> > > >> > >
> > > >> > >    1. What's the relationship between the worker state and the
> > > >> connector
> > > >> > >    status?  Does the 'paused' status at the Connector level
> > include
> > > >> the
> > > >> > > time
> > > >> > >    that worker is 'rebalancing'?
> > > >> > >
> > > >> >
> > > >> > The worker state metric simply reports whether the worker is
> running
> > > or
> > > >> > rebalancing. This state is independent of how many connectors are
> > > >> > deployed/running/paused. During a rebalance, the connectors are
> > being
> > > >> > stopped and restarted but are effectively not running.
> > > >> >
> > > >> >
> > > >> > >    2. Are the "Source Connector" metrics like record rate an
> > > >> aggregation
> > > >> > of
> > > >> > >    the "Source Task" metrics?
> > > >> > >
> > > >> >
> > > >> > Yes.
> > > >> >
> > > >> >
> > > >> > >       - How much value is there is monitoring at the "Source
> > > >> Connector"
> > > >> > >       level (other than status) if the number of constituent
> tasks
> > > may
> > > >> > > change
> > > >> > >       over time?
> > > >> > >
> > > >> >
> > > >> > The task metrics allow you to know whether the tasks are evenly
> > loaded
> > > >> and
> > > >> > each making progress. The aggregate connector metrics tell you how
> > > much
> > > >> > work has been performed by all the tasks in that worker. Both are
> > > useful
> > > >> > IMO.
> > > >> >
> > > >> >
> > > >> > >       - I'm imagining that it's most useful to collect metrics
> at
> > > the
> > > >> > task
> > > >> > >       level as the task-level metrics should be stable
> regardless
> > of
> > > >> > tasks
> > > >> > >       shifting to different workers
> > > >> > >
> > > >> >
> > > >> > Correct, this is where the most value is because it is the most
> fine
> > > >> > grained.
> > > >> >
> > > >> >
> > > >> > >       - If so, can we duplicate the Connector Status down at the
> > > task
> > > >> > level
> > > >> > >          so that all important metrics can be tracked by task?
> > > >> > >
> > > >> >
> > > >> > Possibly. The challenge is that the threads running the tasks are
> > > >> blocked
> > > >> > when a connector is paused.
> > > >> >
> > > >> >
> > > >> > >          3. For the Sink Task metrics
> > > >> > >       - Can we add offset lag and timestamp lag on commit?
> > > >> > >          - After records are flushed/committed
> > > >> > >             - what is the diff between the record timestamps and
> > > >> commit
> > > >> > >             time (histogram)?  this is a measure of end-to-end
> > > >> pipeline
> > > >> > > latency
> > > >> > >             - what is the diff between record offsets and latest
> > > >> offset
> > > >> > of
> > > >> > >             their partition at commit time (histogram)? this is
> a
> > > >> > > measure of whether
> > > >> > >             this particular task is keeping up
> > > >> > >
> > > >> >
> > > >> > Yeah, possibly. Will have to compare with the consumer metrics to
> > see
> > > >> what
> > > >> > we can get.
> > > >> >
> > > >> >
> > > >> > >          - How about flush error rate?  Assuming the sink
> > connectors
> > > >> are
> > > >> > >       using retries, it would be helpful to know how many errors
> > > >> they're
> > > >> > > seeing
> > > >> > >
> > > >> >
> > > >> > We could add a metric to track how many times the framework
> > receives a
> > > >> > retry exception and then retries, but the connectors may also do
> > this
> > > on
> > > >> > their own.
> > > >> >
> > > >> >
> > > >> > >       - Can we tell at the framework level how many records were
> > > >> inserted
> > > >> > >       vs updated vs deleted?
> > > >> > >
> > > >> >
> > > >> > No, there's no distinction in the Connect framework.
> > > >> >
> > > >> >
> > > >> > >       - Batching stats
> > > >> > >          - Histogram of flush batch size
> > > >> > >          - Counts of flush trigger method (time vs max batch
> size)
> > > >> > >
> > > >> >
> > > >> > Should be able to add these.
> > > >> >
> > > >> >
> > > >> > >
> > > >> > > Cheers,
> > > >> > >
> > > >> > > Roger
> > > >> > >
> > > >> > > On Sun, Sep 10, 2017 at 8:45 AM, Randall Hauch <
> rhauch@gmail.com>
> > > >> wrote:
> > > >> > >
> > > >> > > > Thanks, Gwen.
> > > >> > > >
> > > >> > > > That's a great idea, so I've changed the KIP to add those
> > metrics.
> > > >> I've
> > > >> > > > also made a few other changes:
> > > >> > > >
> > > >> > > >
> > > >> > > >    1. The context of all metrics is limited to the activity
> > within
> > > >> the
> > > >> > > >    worker. This wasn't clear before, so I changed the
> motivation
> > > and
> > > >> > > metric
> > > >> > > >    descriptions to explicitly state this.
> > > >> > > >    2. Added the worker ID to all MBean attributes. In addition
> > to
> > > >> > > hopefully
> > > >> > > >    making this same scope obvious from within JMX or other
> > metric
> > > >> > > reporting
> > > >> > > >    system. This is also similar to how the Kafka producer and
> > > >> consumer
> > > >> > > > metrics
> > > >> > > >    include the client ID in their MBean attributes. Hopefully
> > this
> > > >> does
> > > >> > > not
> > > >> > > >    negatively impact or complicate how external reporting
> > systems'
> > > >> > > > aggregate
> > > >> > > >    metrics from multiple workers.
> > > >> > > >    3. Stated explicitly that aggregating metrics across
> workers
> > > was
> > > >> out
> > > >> > > of
> > > >> > > >    scope of this KIP.
> > > >> > > >    4. Added metrics to report the connector class and version
> > for
> > > >> both
> > > >> > > sink
> > > >> > > >    and source connectors.
> > > >> > > >
> > > >> > > > Check this KIP's history for details of these changes.
> > > >> > > >
> > > >> > > > Please let me know if you have any other suggestions. I hope
> to
> > > >> start
> > > >> > the
> > > >> > > > voting soon!
> > > >> > > >
> > > >> > > > Best regards,
> > > >> > > >
> > > >> > > > Randall
> > > >> > > >
> > > >> > > > On Thu, Sep 7, 2017 at 9:35 PM, Gwen Shapira <
> gwen@confluent.io
> > >
> > > >> > wrote:
> > > >> > > >
> > > >> > > > > Thanks for the KIP, Randall. Those are badly needed!
> > > >> > > > >
> > > >> > > > > Can we have two metrics with record rate per task? One
> before
> > > SMT
> > > >> and
> > > >> > > one
> > > >> > > > > after?
> > > >> > > > > We can have cases where we read 5000 rows from JDBC but
> write
> > 5
> > > to
> > > >> > > Kafka,
> > > >> > > > > or read 5000 records from Kafka and write 5 due to
> filtering.
> > I
> > > >> think
> > > >> > > its
> > > >> > > > > important to know both numbers.
> > > >> > > > >
> > > >> > > > >
> > > >> > > > > Gwen
> > > >> > > > >
> > > >> > > > > On Thu, Sep 7, 2017 at 7:50 PM, Randall Hauch <
> > rhauch@gmail.com
> > > >
> > > >> > > wrote:
> > > >> > > > >
> > > >> > > > > > Hi everyone.
> > > >> > > > > >
> > > >> > > > > > I've created a new KIP to add metrics to the Kafka Connect
> > > >> > framework:
> > > >> > > > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > > >> > > > > > 196%3A+Add+metrics+to+Kafka+Connect+framework
> > > >> > > > > >
> > > >> > > > > > The KIP approval deadline is looming, so if you're
> > interested
> > > in
> > > >> > > Kafka
> > > >> > > > > > Connect metrics please review and provide feedback as soon
> > as
> > > >> > > possible.
> > > >> > > > > I'm
> > > >> > > > > > interested not only in whether the metrics are sufficient
> > and
> > > >> > > > > appropriate,
> > > >> > > > > > but also in whether the MBean naming conventions are okay.
> > > >> > > > > >
> > > >> > > > > > Best regards,
> > > >> > > > > >
> > > >> > > > > > Randall
> > > >> > > > > >
> > > >> > > > >
> > > >> > > > >
> > > >> > > > >
> > > >> > > > > --
> > > >> > > > > *Gwen Shapira*
> > > >> > > > > Product Manager | Confluent
> > > >> > > > > 650.450.2760 | @gwenshap
> > > >> > > > > Follow us: Twitter <https://twitter.com/ConfluentInc> |
> blog
> > > >> > > > > <http://www.confluent.io/blog>
> > > >> > > > >
> > > >> > > >
> > > >> > >
> > > >> >
> > > >>
> > > >
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-196: Add metrics to Kafka Connect framework

Posted by Roger Hoover <ro...@gmail.com>.
Randall/Ewen,

I think the timing info is still useful even if it's measured since the
last rebalance.  How else do you know where time is being spent?

The use case for seeing the batch size is that you generally have two knobs
to configure - max batch size and max wait time.  The batch size metrics
would tell you how full your batches are based on your current linger time
so you can adjust the config.

Cheers,

Roger

On Mon, Sep 11, 2017 at 7:50 PM, Ewen Cheslack-Postava <ew...@confluent.io>
wrote:

> re: questions about additional metrics, I think we'll undoubtedly find more
> that people want in practice, but as I mentioned earlier I think it's
> better to add the ones we know we need and then fill out the rest as we
> figure it out. So, e.g., batch size metrics sound like they could be
> useful, but I'd probably wait until we have a clear use case. It seems
> likely that it could be useful in diagnosing slow connectors (e.g. the
> implementation just does something inefficient), but I'm not really sure
> about that yet.
>
> -Ewen
>
> On Mon, Sep 11, 2017 at 7:11 PM, Randall Hauch <rh...@gmail.com> wrote:
>
> > Based on Roger and Ewen's feedback, I removed the aggregate metrics as
> they
> > would be difficult to make use of without extra work. This simplified
> > things a great deal, and I took the opportunity to reorganize the groups
> of
> > metrics. Also, based upon Ewen's concerns regarding measuring
> > times/durations, I removed all time-related metrics except for the offset
> > commits and rebalances, which are infrequent enough to warrant the
> capture
> > of percentiles. Roger asked about capturing batch size metrics for source
> > and sink tasks, and offset lag metrics for sink tasks. Finally, Ewen
> > pointed out that all count/total metrics are only valid since the most
> > recent rebalance and are therefore less meaningful, and were removed.
> >
> > On Mon, Sep 11, 2017 at 6:50 PM, Randall Hauch <rh...@gmail.com> wrote:
> >
> > > Thanks, Ewen. Comments inline below.
> > >
> > > On Mon, Sep 11, 2017 at 5:46 PM, Ewen Cheslack-Postava <
> > ewen@confluent.io>
> > > wrote:
> > >
> > >> Randall,
> > >>
> > >> A couple of questions:
> > >>
> > >> * Some metrics don't seem to have unique names? e.g.
> > >> source-record-produce-rate and source-record-produce-total seem like
> > they
> > >> are duplicated. Looks like maybe just an oversight that the second
> ones
> > >> should be changed from "produce" to "write".
> > >>
> > >
> > > Nice catch. You are correct - should be "write" instead of "produce". I
> > > will correct.
> > >
> > >
> > >> * I think there's a stray extra character in a couple of
> > >> places: kafka.connect:type=source-task-metrics,name=source-record-
> > >> produce-total,worker=([-.\w]+)l,connector=([-.\w]+),task=([\d]+)
> > >> has an extra char after the worker name.
> > >>
> > >
> > > Thanks. Removed in 2 places.
> > >
> > >
> > >> * Are the produce totals actually useful given rebalancing would
> cancel
> > >> them out anyway? Doesn't seem like you could do much with them.
> > >>
> > >
> > > Yes, the totals would be since the last rebalance. Maybe that isn't
> that
> > > useful. Might be better to capture the offsets and lag as Roger was
> > > suggestion. Thoughts?
> > >
> > >
> > >> * Why do transformations get their own metric but not converters? And
> > are
> > >> we concerned at all about the performance impact of getting such fine
> > >> grained info? Getting current time isn't free and we've seen before
> that
> > >> we
> > >> ended up w/ accidental performance regressions as we tried to check it
> > too
> > >> frequently to enforce timeouts fine grained in the producer (iirc).
> > >> Batching helps w/ this, but on the consumer side, a max.poll.records=1
> > >> setting could put you in a bad place, especially since transforms
> might
> > be
> > >> very lightweight (or nothing) and converters are expected to be
> > relatively
> > >> cheap as well.
> > >>
> > >
> > > We could remove the read, transform, and put time-based metrics for
> sink
> > > tasks, and poll, transform, and write time-based metrics. Can/should
> they
> > > be replaced with anything else?
> > >
> > >
> > >> * If we include the worker id everywhere and don't have metrics
> without
> > >> that included, isn't that a pain for users that dump this data into
> some
> > >> other system? They have to know which worker the connector/task is
> > >> currently on *or* need to do extra work to merge the metrics from
> across
> > >> machines. Including versions with the worker ID can make sense for
> > >> completeness and accuracy (e.g. technically there are still very slim
> > >> risks
> > >> of having a task running twice due to zombies), but it seems like bad
> > >> usability for the common case.
> > >>
> > >
> > > Part of the reason was also to help identify where each of the metrics
> > > came from, but per the next comment this may not be as useful, either.
> > > So remove the worker ID in all the task and connector metric names?
> What
> > > about the worker metrics?
> > >
> > >
> > >> * Is aggregating things like source record rate at the (worker,
> > connector)
> > >> level really useful since you're just going to need to do additional
> > >> aggregation anyway once you've collected metrics across all workers?
> I'd
> > >> rather add a smaller number of metrics w/ clear use cases than just
> try
> > to
> > >> be exhaustive and then have to maintain stuff that nobody actually
> uses.
> > >>
> > >
> > > Yes, the connector aggregate metrics are maybe not as useful if you
> also
> > > have to aggregate them from different workers. Removing them probably
> > also
> > > reduces the risk of them being misinterpretted.
> > >
> > >
> > >> * You have status for connectors but not for tasks. Any reason why?
> > Seems
> > >> like it'd make sense to expose both, especially since users generally
> > care
> > >> about task status more than connector status (not many connectors
> > actually
> > >> run a monitoring thread.)
> > >>
> > >
> > > Ack.
> > >
> > >
> > >> * Is number of tasks for each connector a useful metric? Not sure
> > whether
> > >> someone would find this useful or not. Probably not for alerts, but
> > might
> > >> be useful to be able to check it via your metrics dashboard.
> > >>
> > >
> > > Seems like it might be useful, at least in terms of tracking the number
> > of
> > > tasks over time. Might not be as useful for connectors that have
> > relatively
> > > static tasks, but it would be more interesting/useful for connectors
> that
> > > create tasks dynamically and periodically request task
> reconfigurations.
> > >
> > >
> > >> * Same questions re: granularity of sink tasks/connectors timing and
> > >> whether the connectors need all the roll-ups of individual (worker,
> > task)
> > >> values to (worker, connector) level.
> > >>
> > >
> > > I'm fine with taking out the aggregates to keep things simple and
> prevent
> > > misunderstanding.
> > >
> > >
> > >> * If we expose the who the worker currently thinks is leader, it might
> > >> also
> > >> make sense to expose the underlying epoch. Not actually sure if we
> > expose
> > >> that for the consumer today, but it's an indicator of who is properly
> up
> > >> to
> > >> date.
> > >>
> > >
> > > Ack.
> > >
> > >
> > >> * Why worker-level offset commit stats? It's not clear to me that
> these
> > >> are
> > >> useful without considering the specific connector.
> > >>
> > >
> > > So would they make more sense on the tasks? Again, on the worker
> they're
> > > aggregates.
> > >
> > >
> > >>
> > >> -Ewen
> > >>
> > >>
> > >> On Mon, Sep 11, 2017 at 9:43 AM, Randall Hauch <rh...@gmail.com>
> > wrote:
> > >>
> > >> > Thanks for reviewing. Responses inline below.
> > >> >
> > >> > On Mon, Sep 11, 2017 at 11:22 AM, Roger Hoover <
> > roger.hoover@gmail.com>
> > >> > wrote:
> > >> >
> > >> > > Randall,
> > >> > >
> > >> > > Thank you for the KIP.  This should improve visibility greatly.  I
> > >> had a
> > >> > > few questions/ideas for more metrics.
> > >> > >
> > >> > >
> > >> > >    1. What's the relationship between the worker state and the
> > >> connector
> > >> > >    status?  Does the 'paused' status at the Connector level
> include
> > >> the
> > >> > > time
> > >> > >    that worker is 'rebalancing'?
> > >> > >
> > >> >
> > >> > The worker state metric simply reports whether the worker is running
> > or
> > >> > rebalancing. This state is independent of how many connectors are
> > >> > deployed/running/paused. During a rebalance, the connectors are
> being
> > >> > stopped and restarted but are effectively not running.
> > >> >
> > >> >
> > >> > >    2. Are the "Source Connector" metrics like record rate an
> > >> aggregation
> > >> > of
> > >> > >    the "Source Task" metrics?
> > >> > >
> > >> >
> > >> > Yes.
> > >> >
> > >> >
> > >> > >       - How much value is there is monitoring at the "Source
> > >> Connector"
> > >> > >       level (other than status) if the number of constituent tasks
> > may
> > >> > > change
> > >> > >       over time?
> > >> > >
> > >> >
> > >> > The task metrics allow you to know whether the tasks are evenly
> loaded
> > >> and
> > >> > each making progress. The aggregate connector metrics tell you how
> > much
> > >> > work has been performed by all the tasks in that worker. Both are
> > useful
> > >> > IMO.
> > >> >
> > >> >
> > >> > >       - I'm imagining that it's most useful to collect metrics at
> > the
> > >> > task
> > >> > >       level as the task-level metrics should be stable regardless
> of
> > >> > tasks
> > >> > >       shifting to different workers
> > >> > >
> > >> >
> > >> > Correct, this is where the most value is because it is the most fine
> > >> > grained.
> > >> >
> > >> >
> > >> > >       - If so, can we duplicate the Connector Status down at the
> > task
> > >> > level
> > >> > >          so that all important metrics can be tracked by task?
> > >> > >
> > >> >
> > >> > Possibly. The challenge is that the threads running the tasks are
> > >> blocked
> > >> > when a connector is paused.
> > >> >
> > >> >
> > >> > >          3. For the Sink Task metrics
> > >> > >       - Can we add offset lag and timestamp lag on commit?
> > >> > >          - After records are flushed/committed
> > >> > >             - what is the diff between the record timestamps and
> > >> commit
> > >> > >             time (histogram)?  this is a measure of end-to-end
> > >> pipeline
> > >> > > latency
> > >> > >             - what is the diff between record offsets and latest
> > >> offset
> > >> > of
> > >> > >             their partition at commit time (histogram)? this is a
> > >> > > measure of whether
> > >> > >             this particular task is keeping up
> > >> > >
> > >> >
> > >> > Yeah, possibly. Will have to compare with the consumer metrics to
> see
> > >> what
> > >> > we can get.
> > >> >
> > >> >
> > >> > >          - How about flush error rate?  Assuming the sink
> connectors
> > >> are
> > >> > >       using retries, it would be helpful to know how many errors
> > >> they're
> > >> > > seeing
> > >> > >
> > >> >
> > >> > We could add a metric to track how many times the framework
> receives a
> > >> > retry exception and then retries, but the connectors may also do
> this
> > on
> > >> > their own.
> > >> >
> > >> >
> > >> > >       - Can we tell at the framework level how many records were
> > >> inserted
> > >> > >       vs updated vs deleted?
> > >> > >
> > >> >
> > >> > No, there's no distinction in the Connect framework.
> > >> >
> > >> >
> > >> > >       - Batching stats
> > >> > >          - Histogram of flush batch size
> > >> > >          - Counts of flush trigger method (time vs max batch size)
> > >> > >
> > >> >
> > >> > Should be able to add these.
> > >> >
> > >> >
> > >> > >
> > >> > > Cheers,
> > >> > >
> > >> > > Roger
> > >> > >
> > >> > > On Sun, Sep 10, 2017 at 8:45 AM, Randall Hauch <rh...@gmail.com>
> > >> wrote:
> > >> > >
> > >> > > > Thanks, Gwen.
> > >> > > >
> > >> > > > That's a great idea, so I've changed the KIP to add those
> metrics.
> > >> I've
> > >> > > > also made a few other changes:
> > >> > > >
> > >> > > >
> > >> > > >    1. The context of all metrics is limited to the activity
> within
> > >> the
> > >> > > >    worker. This wasn't clear before, so I changed the motivation
> > and
> > >> > > metric
> > >> > > >    descriptions to explicitly state this.
> > >> > > >    2. Added the worker ID to all MBean attributes. In addition
> to
> > >> > > hopefully
> > >> > > >    making this same scope obvious from within JMX or other
> metric
> > >> > > reporting
> > >> > > >    system. This is also similar to how the Kafka producer and
> > >> consumer
> > >> > > > metrics
> > >> > > >    include the client ID in their MBean attributes. Hopefully
> this
> > >> does
> > >> > > not
> > >> > > >    negatively impact or complicate how external reporting
> systems'
> > >> > > > aggregate
> > >> > > >    metrics from multiple workers.
> > >> > > >    3. Stated explicitly that aggregating metrics across workers
> > was
> > >> out
> > >> > > of
> > >> > > >    scope of this KIP.
> > >> > > >    4. Added metrics to report the connector class and version
> for
> > >> both
> > >> > > sink
> > >> > > >    and source connectors.
> > >> > > >
> > >> > > > Check this KIP's history for details of these changes.
> > >> > > >
> > >> > > > Please let me know if you have any other suggestions. I hope to
> > >> start
> > >> > the
> > >> > > > voting soon!
> > >> > > >
> > >> > > > Best regards,
> > >> > > >
> > >> > > > Randall
> > >> > > >
> > >> > > > On Thu, Sep 7, 2017 at 9:35 PM, Gwen Shapira <gwen@confluent.io
> >
> > >> > wrote:
> > >> > > >
> > >> > > > > Thanks for the KIP, Randall. Those are badly needed!
> > >> > > > >
> > >> > > > > Can we have two metrics with record rate per task? One before
> > SMT
> > >> and
> > >> > > one
> > >> > > > > after?
> > >> > > > > We can have cases where we read 5000 rows from JDBC but write
> 5
> > to
> > >> > > Kafka,
> > >> > > > > or read 5000 records from Kafka and write 5 due to filtering.
> I
> > >> think
> > >> > > its
> > >> > > > > important to know both numbers.
> > >> > > > >
> > >> > > > >
> > >> > > > > Gwen
> > >> > > > >
> > >> > > > > On Thu, Sep 7, 2017 at 7:50 PM, Randall Hauch <
> rhauch@gmail.com
> > >
> > >> > > wrote:
> > >> > > > >
> > >> > > > > > Hi everyone.
> > >> > > > > >
> > >> > > > > > I've created a new KIP to add metrics to the Kafka Connect
> > >> > framework:
> > >> > > > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > >> > > > > > 196%3A+Add+metrics+to+Kafka+Connect+framework
> > >> > > > > >
> > >> > > > > > The KIP approval deadline is looming, so if you're
> interested
> > in
> > >> > > Kafka
> > >> > > > > > Connect metrics please review and provide feedback as soon
> as
> > >> > > possible.
> > >> > > > > I'm
> > >> > > > > > interested not only in whether the metrics are sufficient
> and
> > >> > > > > appropriate,
> > >> > > > > > but also in whether the MBean naming conventions are okay.
> > >> > > > > >
> > >> > > > > > Best regards,
> > >> > > > > >
> > >> > > > > > Randall
> > >> > > > > >
> > >> > > > >
> > >> > > > >
> > >> > > > >
> > >> > > > > --
> > >> > > > > *Gwen Shapira*
> > >> > > > > Product Manager | Confluent
> > >> > > > > 650.450.2760 | @gwenshap
> > >> > > > > Follow us: Twitter <https://twitter.com/ConfluentInc> | blog
> > >> > > > > <http://www.confluent.io/blog>
> > >> > > > >
> > >> > > >
> > >> > >
> > >> >
> > >>
> > >
> > >
> >
>

Re: [DISCUSS] KIP-196: Add metrics to Kafka Connect framework

Posted by Ewen Cheslack-Postava <ew...@confluent.io>.
re: questions about additional metrics, I think we'll undoubtedly find more
that people want in practice, but as I mentioned earlier I think it's
better to add the ones we know we need and then fill out the rest as we
figure it out. So, e.g., batch size metrics sound like they could be
useful, but I'd probably wait until we have a clear use case. It seems
likely that it could be useful in diagnosing slow connectors (e.g. the
implementation just does something inefficient), but I'm not really sure
about that yet.

-Ewen

On Mon, Sep 11, 2017 at 7:11 PM, Randall Hauch <rh...@gmail.com> wrote:

> Based on Roger and Ewen's feedback, I removed the aggregate metrics as they
> would be difficult to make use of without extra work. This simplified
> things a great deal, and I took the opportunity to reorganize the groups of
> metrics. Also, based upon Ewen's concerns regarding measuring
> times/durations, I removed all time-related metrics except for the offset
> commits and rebalances, which are infrequent enough to warrant the capture
> of percentiles. Roger asked about capturing batch size metrics for source
> and sink tasks, and offset lag metrics for sink tasks. Finally, Ewen
> pointed out that all count/total metrics are only valid since the most
> recent rebalance and are therefore less meaningful, and were removed.
>
> On Mon, Sep 11, 2017 at 6:50 PM, Randall Hauch <rh...@gmail.com> wrote:
>
> > Thanks, Ewen. Comments inline below.
> >
> > On Mon, Sep 11, 2017 at 5:46 PM, Ewen Cheslack-Postava <
> ewen@confluent.io>
> > wrote:
> >
> >> Randall,
> >>
> >> A couple of questions:
> >>
> >> * Some metrics don't seem to have unique names? e.g.
> >> source-record-produce-rate and source-record-produce-total seem like
> they
> >> are duplicated. Looks like maybe just an oversight that the second ones
> >> should be changed from "produce" to "write".
> >>
> >
> > Nice catch. You are correct - should be "write" instead of "produce". I
> > will correct.
> >
> >
> >> * I think there's a stray extra character in a couple of
> >> places: kafka.connect:type=source-task-metrics,name=source-record-
> >> produce-total,worker=([-.\w]+)l,connector=([-.\w]+),task=([\d]+)
> >> has an extra char after the worker name.
> >>
> >
> > Thanks. Removed in 2 places.
> >
> >
> >> * Are the produce totals actually useful given rebalancing would cancel
> >> them out anyway? Doesn't seem like you could do much with them.
> >>
> >
> > Yes, the totals would be since the last rebalance. Maybe that isn't that
> > useful. Might be better to capture the offsets and lag as Roger was
> > suggestion. Thoughts?
> >
> >
> >> * Why do transformations get their own metric but not converters? And
> are
> >> we concerned at all about the performance impact of getting such fine
> >> grained info? Getting current time isn't free and we've seen before that
> >> we
> >> ended up w/ accidental performance regressions as we tried to check it
> too
> >> frequently to enforce timeouts fine grained in the producer (iirc).
> >> Batching helps w/ this, but on the consumer side, a max.poll.records=1
> >> setting could put you in a bad place, especially since transforms might
> be
> >> very lightweight (or nothing) and converters are expected to be
> relatively
> >> cheap as well.
> >>
> >
> > We could remove the read, transform, and put time-based metrics for sink
> > tasks, and poll, transform, and write time-based metrics. Can/should they
> > be replaced with anything else?
> >
> >
> >> * If we include the worker id everywhere and don't have metrics without
> >> that included, isn't that a pain for users that dump this data into some
> >> other system? They have to know which worker the connector/task is
> >> currently on *or* need to do extra work to merge the metrics from across
> >> machines. Including versions with the worker ID can make sense for
> >> completeness and accuracy (e.g. technically there are still very slim
> >> risks
> >> of having a task running twice due to zombies), but it seems like bad
> >> usability for the common case.
> >>
> >
> > Part of the reason was also to help identify where each of the metrics
> > came from, but per the next comment this may not be as useful, either.
> > So remove the worker ID in all the task and connector metric names? What
> > about the worker metrics?
> >
> >
> >> * Is aggregating things like source record rate at the (worker,
> connector)
> >> level really useful since you're just going to need to do additional
> >> aggregation anyway once you've collected metrics across all workers? I'd
> >> rather add a smaller number of metrics w/ clear use cases than just try
> to
> >> be exhaustive and then have to maintain stuff that nobody actually uses.
> >>
> >
> > Yes, the connector aggregate metrics are maybe not as useful if you also
> > have to aggregate them from different workers. Removing them probably
> also
> > reduces the risk of them being misinterpretted.
> >
> >
> >> * You have status for connectors but not for tasks. Any reason why?
> Seems
> >> like it'd make sense to expose both, especially since users generally
> care
> >> about task status more than connector status (not many connectors
> actually
> >> run a monitoring thread.)
> >>
> >
> > Ack.
> >
> >
> >> * Is number of tasks for each connector a useful metric? Not sure
> whether
> >> someone would find this useful or not. Probably not for alerts, but
> might
> >> be useful to be able to check it via your metrics dashboard.
> >>
> >
> > Seems like it might be useful, at least in terms of tracking the number
> of
> > tasks over time. Might not be as useful for connectors that have
> relatively
> > static tasks, but it would be more interesting/useful for connectors that
> > create tasks dynamically and periodically request task reconfigurations.
> >
> >
> >> * Same questions re: granularity of sink tasks/connectors timing and
> >> whether the connectors need all the roll-ups of individual (worker,
> task)
> >> values to (worker, connector) level.
> >>
> >
> > I'm fine with taking out the aggregates to keep things simple and prevent
> > misunderstanding.
> >
> >
> >> * If we expose the who the worker currently thinks is leader, it might
> >> also
> >> make sense to expose the underlying epoch. Not actually sure if we
> expose
> >> that for the consumer today, but it's an indicator of who is properly up
> >> to
> >> date.
> >>
> >
> > Ack.
> >
> >
> >> * Why worker-level offset commit stats? It's not clear to me that these
> >> are
> >> useful without considering the specific connector.
> >>
> >
> > So would they make more sense on the tasks? Again, on the worker they're
> > aggregates.
> >
> >
> >>
> >> -Ewen
> >>
> >>
> >> On Mon, Sep 11, 2017 at 9:43 AM, Randall Hauch <rh...@gmail.com>
> wrote:
> >>
> >> > Thanks for reviewing. Responses inline below.
> >> >
> >> > On Mon, Sep 11, 2017 at 11:22 AM, Roger Hoover <
> roger.hoover@gmail.com>
> >> > wrote:
> >> >
> >> > > Randall,
> >> > >
> >> > > Thank you for the KIP.  This should improve visibility greatly.  I
> >> had a
> >> > > few questions/ideas for more metrics.
> >> > >
> >> > >
> >> > >    1. What's the relationship between the worker state and the
> >> connector
> >> > >    status?  Does the 'paused' status at the Connector level include
> >> the
> >> > > time
> >> > >    that worker is 'rebalancing'?
> >> > >
> >> >
> >> > The worker state metric simply reports whether the worker is running
> or
> >> > rebalancing. This state is independent of how many connectors are
> >> > deployed/running/paused. During a rebalance, the connectors are being
> >> > stopped and restarted but are effectively not running.
> >> >
> >> >
> >> > >    2. Are the "Source Connector" metrics like record rate an
> >> aggregation
> >> > of
> >> > >    the "Source Task" metrics?
> >> > >
> >> >
> >> > Yes.
> >> >
> >> >
> >> > >       - How much value is there is monitoring at the "Source
> >> Connector"
> >> > >       level (other than status) if the number of constituent tasks
> may
> >> > > change
> >> > >       over time?
> >> > >
> >> >
> >> > The task metrics allow you to know whether the tasks are evenly loaded
> >> and
> >> > each making progress. The aggregate connector metrics tell you how
> much
> >> > work has been performed by all the tasks in that worker. Both are
> useful
> >> > IMO.
> >> >
> >> >
> >> > >       - I'm imagining that it's most useful to collect metrics at
> the
> >> > task
> >> > >       level as the task-level metrics should be stable regardless of
> >> > tasks
> >> > >       shifting to different workers
> >> > >
> >> >
> >> > Correct, this is where the most value is because it is the most fine
> >> > grained.
> >> >
> >> >
> >> > >       - If so, can we duplicate the Connector Status down at the
> task
> >> > level
> >> > >          so that all important metrics can be tracked by task?
> >> > >
> >> >
> >> > Possibly. The challenge is that the threads running the tasks are
> >> blocked
> >> > when a connector is paused.
> >> >
> >> >
> >> > >          3. For the Sink Task metrics
> >> > >       - Can we add offset lag and timestamp lag on commit?
> >> > >          - After records are flushed/committed
> >> > >             - what is the diff between the record timestamps and
> >> commit
> >> > >             time (histogram)?  this is a measure of end-to-end
> >> pipeline
> >> > > latency
> >> > >             - what is the diff between record offsets and latest
> >> offset
> >> > of
> >> > >             their partition at commit time (histogram)? this is a
> >> > > measure of whether
> >> > >             this particular task is keeping up
> >> > >
> >> >
> >> > Yeah, possibly. Will have to compare with the consumer metrics to see
> >> what
> >> > we can get.
> >> >
> >> >
> >> > >          - How about flush error rate?  Assuming the sink connectors
> >> are
> >> > >       using retries, it would be helpful to know how many errors
> >> they're
> >> > > seeing
> >> > >
> >> >
> >> > We could add a metric to track how many times the framework receives a
> >> > retry exception and then retries, but the connectors may also do this
> on
> >> > their own.
> >> >
> >> >
> >> > >       - Can we tell at the framework level how many records were
> >> inserted
> >> > >       vs updated vs deleted?
> >> > >
> >> >
> >> > No, there's no distinction in the Connect framework.
> >> >
> >> >
> >> > >       - Batching stats
> >> > >          - Histogram of flush batch size
> >> > >          - Counts of flush trigger method (time vs max batch size)
> >> > >
> >> >
> >> > Should be able to add these.
> >> >
> >> >
> >> > >
> >> > > Cheers,
> >> > >
> >> > > Roger
> >> > >
> >> > > On Sun, Sep 10, 2017 at 8:45 AM, Randall Hauch <rh...@gmail.com>
> >> wrote:
> >> > >
> >> > > > Thanks, Gwen.
> >> > > >
> >> > > > That's a great idea, so I've changed the KIP to add those metrics.
> >> I've
> >> > > > also made a few other changes:
> >> > > >
> >> > > >
> >> > > >    1. The context of all metrics is limited to the activity within
> >> the
> >> > > >    worker. This wasn't clear before, so I changed the motivation
> and
> >> > > metric
> >> > > >    descriptions to explicitly state this.
> >> > > >    2. Added the worker ID to all MBean attributes. In addition to
> >> > > hopefully
> >> > > >    making this same scope obvious from within JMX or other metric
> >> > > reporting
> >> > > >    system. This is also similar to how the Kafka producer and
> >> consumer
> >> > > > metrics
> >> > > >    include the client ID in their MBean attributes. Hopefully this
> >> does
> >> > > not
> >> > > >    negatively impact or complicate how external reporting systems'
> >> > > > aggregate
> >> > > >    metrics from multiple workers.
> >> > > >    3. Stated explicitly that aggregating metrics across workers
> was
> >> out
> >> > > of
> >> > > >    scope of this KIP.
> >> > > >    4. Added metrics to report the connector class and version for
> >> both
> >> > > sink
> >> > > >    and source connectors.
> >> > > >
> >> > > > Check this KIP's history for details of these changes.
> >> > > >
> >> > > > Please let me know if you have any other suggestions. I hope to
> >> start
> >> > the
> >> > > > voting soon!
> >> > > >
> >> > > > Best regards,
> >> > > >
> >> > > > Randall
> >> > > >
> >> > > > On Thu, Sep 7, 2017 at 9:35 PM, Gwen Shapira <gw...@confluent.io>
> >> > wrote:
> >> > > >
> >> > > > > Thanks for the KIP, Randall. Those are badly needed!
> >> > > > >
> >> > > > > Can we have two metrics with record rate per task? One before
> SMT
> >> and
> >> > > one
> >> > > > > after?
> >> > > > > We can have cases where we read 5000 rows from JDBC but write 5
> to
> >> > > Kafka,
> >> > > > > or read 5000 records from Kafka and write 5 due to filtering. I
> >> think
> >> > > its
> >> > > > > important to know both numbers.
> >> > > > >
> >> > > > >
> >> > > > > Gwen
> >> > > > >
> >> > > > > On Thu, Sep 7, 2017 at 7:50 PM, Randall Hauch <rhauch@gmail.com
> >
> >> > > wrote:
> >> > > > >
> >> > > > > > Hi everyone.
> >> > > > > >
> >> > > > > > I've created a new KIP to add metrics to the Kafka Connect
> >> > framework:
> >> > > > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> >> > > > > > 196%3A+Add+metrics+to+Kafka+Connect+framework
> >> > > > > >
> >> > > > > > The KIP approval deadline is looming, so if you're interested
> in
> >> > > Kafka
> >> > > > > > Connect metrics please review and provide feedback as soon as
> >> > > possible.
> >> > > > > I'm
> >> > > > > > interested not only in whether the metrics are sufficient and
> >> > > > > appropriate,
> >> > > > > > but also in whether the MBean naming conventions are okay.
> >> > > > > >
> >> > > > > > Best regards,
> >> > > > > >
> >> > > > > > Randall
> >> > > > > >
> >> > > > >
> >> > > > >
> >> > > > >
> >> > > > > --
> >> > > > > *Gwen Shapira*
> >> > > > > Product Manager | Confluent
> >> > > > > 650.450.2760 | @gwenshap
> >> > > > > Follow us: Twitter <https://twitter.com/ConfluentInc> | blog
> >> > > > > <http://www.confluent.io/blog>
> >> > > > >
> >> > > >
> >> > >
> >> >
> >>
> >
> >
>

Re: [DISCUSS] KIP-196: Add metrics to Kafka Connect framework

Posted by Randall Hauch <rh...@gmail.com>.
Based on Roger and Ewen's feedback, I removed the aggregate metrics as they
would be difficult to make use of without extra work. This simplified
things a great deal, and I took the opportunity to reorganize the groups of
metrics. Also, based upon Ewen's concerns regarding measuring
times/durations, I removed all time-related metrics except for the offset
commits and rebalances, which are infrequent enough to warrant the capture
of percentiles. Roger asked about capturing batch size metrics for source
and sink tasks, and offset lag metrics for sink tasks. Finally, Ewen
pointed out that all count/total metrics are only valid since the most
recent rebalance and are therefore less meaningful, and were removed.

On Mon, Sep 11, 2017 at 6:50 PM, Randall Hauch <rh...@gmail.com> wrote:

> Thanks, Ewen. Comments inline below.
>
> On Mon, Sep 11, 2017 at 5:46 PM, Ewen Cheslack-Postava <ew...@confluent.io>
> wrote:
>
>> Randall,
>>
>> A couple of questions:
>>
>> * Some metrics don't seem to have unique names? e.g.
>> source-record-produce-rate and source-record-produce-total seem like they
>> are duplicated. Looks like maybe just an oversight that the second ones
>> should be changed from "produce" to "write".
>>
>
> Nice catch. You are correct - should be "write" instead of "produce". I
> will correct.
>
>
>> * I think there's a stray extra character in a couple of
>> places: kafka.connect:type=source-task-metrics,name=source-record-
>> produce-total,worker=([-.\w]+)l,connector=([-.\w]+),task=([\d]+)
>> has an extra char after the worker name.
>>
>
> Thanks. Removed in 2 places.
>
>
>> * Are the produce totals actually useful given rebalancing would cancel
>> them out anyway? Doesn't seem like you could do much with them.
>>
>
> Yes, the totals would be since the last rebalance. Maybe that isn't that
> useful. Might be better to capture the offsets and lag as Roger was
> suggestion. Thoughts?
>
>
>> * Why do transformations get their own metric but not converters? And are
>> we concerned at all about the performance impact of getting such fine
>> grained info? Getting current time isn't free and we've seen before that
>> we
>> ended up w/ accidental performance regressions as we tried to check it too
>> frequently to enforce timeouts fine grained in the producer (iirc).
>> Batching helps w/ this, but on the consumer side, a max.poll.records=1
>> setting could put you in a bad place, especially since transforms might be
>> very lightweight (or nothing) and converters are expected to be relatively
>> cheap as well.
>>
>
> We could remove the read, transform, and put time-based metrics for sink
> tasks, and poll, transform, and write time-based metrics. Can/should they
> be replaced with anything else?
>
>
>> * If we include the worker id everywhere and don't have metrics without
>> that included, isn't that a pain for users that dump this data into some
>> other system? They have to know which worker the connector/task is
>> currently on *or* need to do extra work to merge the metrics from across
>> machines. Including versions with the worker ID can make sense for
>> completeness and accuracy (e.g. technically there are still very slim
>> risks
>> of having a task running twice due to zombies), but it seems like bad
>> usability for the common case.
>>
>
> Part of the reason was also to help identify where each of the metrics
> came from, but per the next comment this may not be as useful, either.
> So remove the worker ID in all the task and connector metric names? What
> about the worker metrics?
>
>
>> * Is aggregating things like source record rate at the (worker, connector)
>> level really useful since you're just going to need to do additional
>> aggregation anyway once you've collected metrics across all workers? I'd
>> rather add a smaller number of metrics w/ clear use cases than just try to
>> be exhaustive and then have to maintain stuff that nobody actually uses.
>>
>
> Yes, the connector aggregate metrics are maybe not as useful if you also
> have to aggregate them from different workers. Removing them probably also
> reduces the risk of them being misinterpretted.
>
>
>> * You have status for connectors but not for tasks. Any reason why? Seems
>> like it'd make sense to expose both, especially since users generally care
>> about task status more than connector status (not many connectors actually
>> run a monitoring thread.)
>>
>
> Ack.
>
>
>> * Is number of tasks for each connector a useful metric? Not sure whether
>> someone would find this useful or not. Probably not for alerts, but might
>> be useful to be able to check it via your metrics dashboard.
>>
>
> Seems like it might be useful, at least in terms of tracking the number of
> tasks over time. Might not be as useful for connectors that have relatively
> static tasks, but it would be more interesting/useful for connectors that
> create tasks dynamically and periodically request task reconfigurations.
>
>
>> * Same questions re: granularity of sink tasks/connectors timing and
>> whether the connectors need all the roll-ups of individual (worker, task)
>> values to (worker, connector) level.
>>
>
> I'm fine with taking out the aggregates to keep things simple and prevent
> misunderstanding.
>
>
>> * If we expose the who the worker currently thinks is leader, it might
>> also
>> make sense to expose the underlying epoch. Not actually sure if we expose
>> that for the consumer today, but it's an indicator of who is properly up
>> to
>> date.
>>
>
> Ack.
>
>
>> * Why worker-level offset commit stats? It's not clear to me that these
>> are
>> useful without considering the specific connector.
>>
>
> So would they make more sense on the tasks? Again, on the worker they're
> aggregates.
>
>
>>
>> -Ewen
>>
>>
>> On Mon, Sep 11, 2017 at 9:43 AM, Randall Hauch <rh...@gmail.com> wrote:
>>
>> > Thanks for reviewing. Responses inline below.
>> >
>> > On Mon, Sep 11, 2017 at 11:22 AM, Roger Hoover <ro...@gmail.com>
>> > wrote:
>> >
>> > > Randall,
>> > >
>> > > Thank you for the KIP.  This should improve visibility greatly.  I
>> had a
>> > > few questions/ideas for more metrics.
>> > >
>> > >
>> > >    1. What's the relationship between the worker state and the
>> connector
>> > >    status?  Does the 'paused' status at the Connector level include
>> the
>> > > time
>> > >    that worker is 'rebalancing'?
>> > >
>> >
>> > The worker state metric simply reports whether the worker is running or
>> > rebalancing. This state is independent of how many connectors are
>> > deployed/running/paused. During a rebalance, the connectors are being
>> > stopped and restarted but are effectively not running.
>> >
>> >
>> > >    2. Are the "Source Connector" metrics like record rate an
>> aggregation
>> > of
>> > >    the "Source Task" metrics?
>> > >
>> >
>> > Yes.
>> >
>> >
>> > >       - How much value is there is monitoring at the "Source
>> Connector"
>> > >       level (other than status) if the number of constituent tasks may
>> > > change
>> > >       over time?
>> > >
>> >
>> > The task metrics allow you to know whether the tasks are evenly loaded
>> and
>> > each making progress. The aggregate connector metrics tell you how much
>> > work has been performed by all the tasks in that worker. Both are useful
>> > IMO.
>> >
>> >
>> > >       - I'm imagining that it's most useful to collect metrics at the
>> > task
>> > >       level as the task-level metrics should be stable regardless of
>> > tasks
>> > >       shifting to different workers
>> > >
>> >
>> > Correct, this is where the most value is because it is the most fine
>> > grained.
>> >
>> >
>> > >       - If so, can we duplicate the Connector Status down at the task
>> > level
>> > >          so that all important metrics can be tracked by task?
>> > >
>> >
>> > Possibly. The challenge is that the threads running the tasks are
>> blocked
>> > when a connector is paused.
>> >
>> >
>> > >          3. For the Sink Task metrics
>> > >       - Can we add offset lag and timestamp lag on commit?
>> > >          - After records are flushed/committed
>> > >             - what is the diff between the record timestamps and
>> commit
>> > >             time (histogram)?  this is a measure of end-to-end
>> pipeline
>> > > latency
>> > >             - what is the diff between record offsets and latest
>> offset
>> > of
>> > >             their partition at commit time (histogram)? this is a
>> > > measure of whether
>> > >             this particular task is keeping up
>> > >
>> >
>> > Yeah, possibly. Will have to compare with the consumer metrics to see
>> what
>> > we can get.
>> >
>> >
>> > >          - How about flush error rate?  Assuming the sink connectors
>> are
>> > >       using retries, it would be helpful to know how many errors
>> they're
>> > > seeing
>> > >
>> >
>> > We could add a metric to track how many times the framework receives a
>> > retry exception and then retries, but the connectors may also do this on
>> > their own.
>> >
>> >
>> > >       - Can we tell at the framework level how many records were
>> inserted
>> > >       vs updated vs deleted?
>> > >
>> >
>> > No, there's no distinction in the Connect framework.
>> >
>> >
>> > >       - Batching stats
>> > >          - Histogram of flush batch size
>> > >          - Counts of flush trigger method (time vs max batch size)
>> > >
>> >
>> > Should be able to add these.
>> >
>> >
>> > >
>> > > Cheers,
>> > >
>> > > Roger
>> > >
>> > > On Sun, Sep 10, 2017 at 8:45 AM, Randall Hauch <rh...@gmail.com>
>> wrote:
>> > >
>> > > > Thanks, Gwen.
>> > > >
>> > > > That's a great idea, so I've changed the KIP to add those metrics.
>> I've
>> > > > also made a few other changes:
>> > > >
>> > > >
>> > > >    1. The context of all metrics is limited to the activity within
>> the
>> > > >    worker. This wasn't clear before, so I changed the motivation and
>> > > metric
>> > > >    descriptions to explicitly state this.
>> > > >    2. Added the worker ID to all MBean attributes. In addition to
>> > > hopefully
>> > > >    making this same scope obvious from within JMX or other metric
>> > > reporting
>> > > >    system. This is also similar to how the Kafka producer and
>> consumer
>> > > > metrics
>> > > >    include the client ID in their MBean attributes. Hopefully this
>> does
>> > > not
>> > > >    negatively impact or complicate how external reporting systems'
>> > > > aggregate
>> > > >    metrics from multiple workers.
>> > > >    3. Stated explicitly that aggregating metrics across workers was
>> out
>> > > of
>> > > >    scope of this KIP.
>> > > >    4. Added metrics to report the connector class and version for
>> both
>> > > sink
>> > > >    and source connectors.
>> > > >
>> > > > Check this KIP's history for details of these changes.
>> > > >
>> > > > Please let me know if you have any other suggestions. I hope to
>> start
>> > the
>> > > > voting soon!
>> > > >
>> > > > Best regards,
>> > > >
>> > > > Randall
>> > > >
>> > > > On Thu, Sep 7, 2017 at 9:35 PM, Gwen Shapira <gw...@confluent.io>
>> > wrote:
>> > > >
>> > > > > Thanks for the KIP, Randall. Those are badly needed!
>> > > > >
>> > > > > Can we have two metrics with record rate per task? One before SMT
>> and
>> > > one
>> > > > > after?
>> > > > > We can have cases where we read 5000 rows from JDBC but write 5 to
>> > > Kafka,
>> > > > > or read 5000 records from Kafka and write 5 due to filtering. I
>> think
>> > > its
>> > > > > important to know both numbers.
>> > > > >
>> > > > >
>> > > > > Gwen
>> > > > >
>> > > > > On Thu, Sep 7, 2017 at 7:50 PM, Randall Hauch <rh...@gmail.com>
>> > > wrote:
>> > > > >
>> > > > > > Hi everyone.
>> > > > > >
>> > > > > > I've created a new KIP to add metrics to the Kafka Connect
>> > framework:
>> > > > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-
>> > > > > > 196%3A+Add+metrics+to+Kafka+Connect+framework
>> > > > > >
>> > > > > > The KIP approval deadline is looming, so if you're interested in
>> > > Kafka
>> > > > > > Connect metrics please review and provide feedback as soon as
>> > > possible.
>> > > > > I'm
>> > > > > > interested not only in whether the metrics are sufficient and
>> > > > > appropriate,
>> > > > > > but also in whether the MBean naming conventions are okay.
>> > > > > >
>> > > > > > Best regards,
>> > > > > >
>> > > > > > Randall
>> > > > > >
>> > > > >
>> > > > >
>> > > > >
>> > > > > --
>> > > > > *Gwen Shapira*
>> > > > > Product Manager | Confluent
>> > > > > 650.450.2760 | @gwenshap
>> > > > > Follow us: Twitter <https://twitter.com/ConfluentInc> | blog
>> > > > > <http://www.confluent.io/blog>
>> > > > >
>> > > >
>> > >
>> >
>>
>
>

Re: [DISCUSS] KIP-196: Add metrics to Kafka Connect framework

Posted by Randall Hauch <rh...@gmail.com>.
Thanks, Ewen. Comments inline below.

On Mon, Sep 11, 2017 at 5:46 PM, Ewen Cheslack-Postava <ew...@confluent.io>
wrote:

> Randall,
>
> A couple of questions:
>
> * Some metrics don't seem to have unique names? e.g.
> source-record-produce-rate and source-record-produce-total seem like they
> are duplicated. Looks like maybe just an oversight that the second ones
> should be changed from "produce" to "write".
>

Nice catch. You are correct - should be "write" instead of "produce". I
will correct.


> * I think there's a stray extra character in a couple of
> places: kafka.connect:type=source-task-metrics,name=source-
> record-produce-total,worker=([-.\w]+)l,connector=([-.\w]+),task=([\d]+)
> has an extra char after the worker name.
>

Thanks. Removed in 2 places.


> * Are the produce totals actually useful given rebalancing would cancel
> them out anyway? Doesn't seem like you could do much with them.
>

Yes, the totals would be since the last rebalance. Maybe that isn't that
useful. Might be better to capture the offsets and lag as Roger was
suggestion. Thoughts?


> * Why do transformations get their own metric but not converters? And are
> we concerned at all about the performance impact of getting such fine
> grained info? Getting current time isn't free and we've seen before that we
> ended up w/ accidental performance regressions as we tried to check it too
> frequently to enforce timeouts fine grained in the producer (iirc).
> Batching helps w/ this, but on the consumer side, a max.poll.records=1
> setting could put you in a bad place, especially since transforms might be
> very lightweight (or nothing) and converters are expected to be relatively
> cheap as well.
>

We could remove the read, transform, and put time-based metrics for sink
tasks, and poll, transform, and write time-based metrics. Can/should they
be replaced with anything else?


> * If we include the worker id everywhere and don't have metrics without
> that included, isn't that a pain for users that dump this data into some
> other system? They have to know which worker the connector/task is
> currently on *or* need to do extra work to merge the metrics from across
> machines. Including versions with the worker ID can make sense for
> completeness and accuracy (e.g. technically there are still very slim risks
> of having a task running twice due to zombies), but it seems like bad
> usability for the common case.
>

Part of the reason was also to help identify where each of the metrics came
from, but per the next comment this may not be as useful, either.
So remove the worker ID in all the task and connector metric names? What
about the worker metrics?


> * Is aggregating things like source record rate at the (worker, connector)
> level really useful since you're just going to need to do additional
> aggregation anyway once you've collected metrics across all workers? I'd
> rather add a smaller number of metrics w/ clear use cases than just try to
> be exhaustive and then have to maintain stuff that nobody actually uses.
>

Yes, the connector aggregate metrics are maybe not as useful if you also
have to aggregate them from different workers. Removing them probably also
reduces the risk of them being misinterpretted.


> * You have status for connectors but not for tasks. Any reason why? Seems
> like it'd make sense to expose both, especially since users generally care
> about task status more than connector status (not many connectors actually
> run a monitoring thread.)
>

Ack.


> * Is number of tasks for each connector a useful metric? Not sure whether
> someone would find this useful or not. Probably not for alerts, but might
> be useful to be able to check it via your metrics dashboard.
>

Seems like it might be useful, at least in terms of tracking the number of
tasks over time. Might not be as useful for connectors that have relatively
static tasks, but it would be more interesting/useful for connectors that
create tasks dynamically and periodically request task reconfigurations.


> * Same questions re: granularity of sink tasks/connectors timing and
> whether the connectors need all the roll-ups of individual (worker, task)
> values to (worker, connector) level.
>

I'm fine with taking out the aggregates to keep things simple and prevent
misunderstanding.


> * If we expose the who the worker currently thinks is leader, it might also
> make sense to expose the underlying epoch. Not actually sure if we expose
> that for the consumer today, but it's an indicator of who is properly up to
> date.
>

Ack.


> * Why worker-level offset commit stats? It's not clear to me that these are
> useful without considering the specific connector.
>

So would they make more sense on the tasks? Again, on the worker they're
aggregates.


>
> -Ewen
>
>
> On Mon, Sep 11, 2017 at 9:43 AM, Randall Hauch <rh...@gmail.com> wrote:
>
> > Thanks for reviewing. Responses inline below.
> >
> > On Mon, Sep 11, 2017 at 11:22 AM, Roger Hoover <ro...@gmail.com>
> > wrote:
> >
> > > Randall,
> > >
> > > Thank you for the KIP.  This should improve visibility greatly.  I had
> a
> > > few questions/ideas for more metrics.
> > >
> > >
> > >    1. What's the relationship between the worker state and the
> connector
> > >    status?  Does the 'paused' status at the Connector level include the
> > > time
> > >    that worker is 'rebalancing'?
> > >
> >
> > The worker state metric simply reports whether the worker is running or
> > rebalancing. This state is independent of how many connectors are
> > deployed/running/paused. During a rebalance, the connectors are being
> > stopped and restarted but are effectively not running.
> >
> >
> > >    2. Are the "Source Connector" metrics like record rate an
> aggregation
> > of
> > >    the "Source Task" metrics?
> > >
> >
> > Yes.
> >
> >
> > >       - How much value is there is monitoring at the "Source Connector"
> > >       level (other than status) if the number of constituent tasks may
> > > change
> > >       over time?
> > >
> >
> > The task metrics allow you to know whether the tasks are evenly loaded
> and
> > each making progress. The aggregate connector metrics tell you how much
> > work has been performed by all the tasks in that worker. Both are useful
> > IMO.
> >
> >
> > >       - I'm imagining that it's most useful to collect metrics at the
> > task
> > >       level as the task-level metrics should be stable regardless of
> > tasks
> > >       shifting to different workers
> > >
> >
> > Correct, this is where the most value is because it is the most fine
> > grained.
> >
> >
> > >       - If so, can we duplicate the Connector Status down at the task
> > level
> > >          so that all important metrics can be tracked by task?
> > >
> >
> > Possibly. The challenge is that the threads running the tasks are blocked
> > when a connector is paused.
> >
> >
> > >          3. For the Sink Task metrics
> > >       - Can we add offset lag and timestamp lag on commit?
> > >          - After records are flushed/committed
> > >             - what is the diff between the record timestamps and commit
> > >             time (histogram)?  this is a measure of end-to-end pipeline
> > > latency
> > >             - what is the diff between record offsets and latest offset
> > of
> > >             their partition at commit time (histogram)? this is a
> > > measure of whether
> > >             this particular task is keeping up
> > >
> >
> > Yeah, possibly. Will have to compare with the consumer metrics to see
> what
> > we can get.
> >
> >
> > >          - How about flush error rate?  Assuming the sink connectors
> are
> > >       using retries, it would be helpful to know how many errors
> they're
> > > seeing
> > >
> >
> > We could add a metric to track how many times the framework receives a
> > retry exception and then retries, but the connectors may also do this on
> > their own.
> >
> >
> > >       - Can we tell at the framework level how many records were
> inserted
> > >       vs updated vs deleted?
> > >
> >
> > No, there's no distinction in the Connect framework.
> >
> >
> > >       - Batching stats
> > >          - Histogram of flush batch size
> > >          - Counts of flush trigger method (time vs max batch size)
> > >
> >
> > Should be able to add these.
> >
> >
> > >
> > > Cheers,
> > >
> > > Roger
> > >
> > > On Sun, Sep 10, 2017 at 8:45 AM, Randall Hauch <rh...@gmail.com>
> wrote:
> > >
> > > > Thanks, Gwen.
> > > >
> > > > That's a great idea, so I've changed the KIP to add those metrics.
> I've
> > > > also made a few other changes:
> > > >
> > > >
> > > >    1. The context of all metrics is limited to the activity within
> the
> > > >    worker. This wasn't clear before, so I changed the motivation and
> > > metric
> > > >    descriptions to explicitly state this.
> > > >    2. Added the worker ID to all MBean attributes. In addition to
> > > hopefully
> > > >    making this same scope obvious from within JMX or other metric
> > > reporting
> > > >    system. This is also similar to how the Kafka producer and
> consumer
> > > > metrics
> > > >    include the client ID in their MBean attributes. Hopefully this
> does
> > > not
> > > >    negatively impact or complicate how external reporting systems'
> > > > aggregate
> > > >    metrics from multiple workers.
> > > >    3. Stated explicitly that aggregating metrics across workers was
> out
> > > of
> > > >    scope of this KIP.
> > > >    4. Added metrics to report the connector class and version for
> both
> > > sink
> > > >    and source connectors.
> > > >
> > > > Check this KIP's history for details of these changes.
> > > >
> > > > Please let me know if you have any other suggestions. I hope to start
> > the
> > > > voting soon!
> > > >
> > > > Best regards,
> > > >
> > > > Randall
> > > >
> > > > On Thu, Sep 7, 2017 at 9:35 PM, Gwen Shapira <gw...@confluent.io>
> > wrote:
> > > >
> > > > > Thanks for the KIP, Randall. Those are badly needed!
> > > > >
> > > > > Can we have two metrics with record rate per task? One before SMT
> and
> > > one
> > > > > after?
> > > > > We can have cases where we read 5000 rows from JDBC but write 5 to
> > > Kafka,
> > > > > or read 5000 records from Kafka and write 5 due to filtering. I
> think
> > > its
> > > > > important to know both numbers.
> > > > >
> > > > >
> > > > > Gwen
> > > > >
> > > > > On Thu, Sep 7, 2017 at 7:50 PM, Randall Hauch <rh...@gmail.com>
> > > wrote:
> > > > >
> > > > > > Hi everyone.
> > > > > >
> > > > > > I've created a new KIP to add metrics to the Kafka Connect
> > framework:
> > > > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > > > > > 196%3A+Add+metrics+to+Kafka+Connect+framework
> > > > > >
> > > > > > The KIP approval deadline is looming, so if you're interested in
> > > Kafka
> > > > > > Connect metrics please review and provide feedback as soon as
> > > possible.
> > > > > I'm
> > > > > > interested not only in whether the metrics are sufficient and
> > > > > appropriate,
> > > > > > but also in whether the MBean naming conventions are okay.
> > > > > >
> > > > > > Best regards,
> > > > > >
> > > > > > Randall
> > > > > >
> > > > >
> > > > >
> > > > >
> > > > > --
> > > > > *Gwen Shapira*
> > > > > Product Manager | Confluent
> > > > > 650.450.2760 | @gwenshap
> > > > > Follow us: Twitter <https://twitter.com/ConfluentInc> | blog
> > > > > <http://www.confluent.io/blog>
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-196: Add metrics to Kafka Connect framework

Posted by Ewen Cheslack-Postava <ew...@confluent.io>.
Randall,

A couple of questions:

* Some metrics don't seem to have unique names? e.g.
source-record-produce-rate and source-record-produce-total seem like they
are duplicated. Looks like maybe just an oversight that the second ones
should be changed from "produce" to "write".
* I think there's a stray extra character in a couple of
places: kafka.connect:type=source-task-metrics,name=source-record-produce-total,worker=([-.\w]+)l,connector=([-.\w]+),task=([\d]+)
has an extra char after the worker name.
* Are the produce totals actually useful given rebalancing would cancel
them out anyway? Doesn't seem like you could do much with them.
* Why do transformations get their own metric but not converters? And are
we concerned at all about the performance impact of getting such fine
grained info? Getting current time isn't free and we've seen before that we
ended up w/ accidental performance regressions as we tried to check it too
frequently to enforce timeouts fine grained in the producer (iirc).
Batching helps w/ this, but on the consumer side, a max.poll.records=1
setting could put you in a bad place, especially since transforms might be
very lightweight (or nothing) and converters are expected to be relatively
cheap as well.
* If we include the worker id everywhere and don't have metrics without
that included, isn't that a pain for users that dump this data into some
other system? They have to know which worker the connector/task is
currently on *or* need to do extra work to merge the metrics from across
machines. Including versions with the worker ID can make sense for
completeness and accuracy (e.g. technically there are still very slim risks
of having a task running twice due to zombies), but it seems like bad
usability for the common case.
* Is aggregating things like source record rate at the (worker, connector)
level really useful since you're just going to need to do additional
aggregation anyway once you've collected metrics across all workers? I'd
rather add a smaller number of metrics w/ clear use cases than just try to
be exhaustive and then have to maintain stuff that nobody actually uses.
* You have status for connectors but not for tasks. Any reason why? Seems
like it'd make sense to expose both, especially since users generally care
about task status more than connector status (not many connectors actually
run a monitoring thread.)
* Is number of tasks for each connector a useful metric? Not sure whether
someone would find this useful or not. Probably not for alerts, but might
be useful to be able to check it via your metrics dashboard.
* Same questions re: granularity of sink tasks/connectors timing and
whether the connectors need all the roll-ups of individual (worker, task)
values to (worker, connector) level.
* If we expose the who the worker currently thinks is leader, it might also
make sense to expose the underlying epoch. Not actually sure if we expose
that for the consumer today, but it's an indicator of who is properly up to
date.
* Why worker-level offset commit stats? It's not clear to me that these are
useful without considering the specific connector.

-Ewen


On Mon, Sep 11, 2017 at 9:43 AM, Randall Hauch <rh...@gmail.com> wrote:

> Thanks for reviewing. Responses inline below.
>
> On Mon, Sep 11, 2017 at 11:22 AM, Roger Hoover <ro...@gmail.com>
> wrote:
>
> > Randall,
> >
> > Thank you for the KIP.  This should improve visibility greatly.  I had a
> > few questions/ideas for more metrics.
> >
> >
> >    1. What's the relationship between the worker state and the connector
> >    status?  Does the 'paused' status at the Connector level include the
> > time
> >    that worker is 'rebalancing'?
> >
>
> The worker state metric simply reports whether the worker is running or
> rebalancing. This state is independent of how many connectors are
> deployed/running/paused. During a rebalance, the connectors are being
> stopped and restarted but are effectively not running.
>
>
> >    2. Are the "Source Connector" metrics like record rate an aggregation
> of
> >    the "Source Task" metrics?
> >
>
> Yes.
>
>
> >       - How much value is there is monitoring at the "Source Connector"
> >       level (other than status) if the number of constituent tasks may
> > change
> >       over time?
> >
>
> The task metrics allow you to know whether the tasks are evenly loaded and
> each making progress. The aggregate connector metrics tell you how much
> work has been performed by all the tasks in that worker. Both are useful
> IMO.
>
>
> >       - I'm imagining that it's most useful to collect metrics at the
> task
> >       level as the task-level metrics should be stable regardless of
> tasks
> >       shifting to different workers
> >
>
> Correct, this is where the most value is because it is the most fine
> grained.
>
>
> >       - If so, can we duplicate the Connector Status down at the task
> level
> >          so that all important metrics can be tracked by task?
> >
>
> Possibly. The challenge is that the threads running the tasks are blocked
> when a connector is paused.
>
>
> >          3. For the Sink Task metrics
> >       - Can we add offset lag and timestamp lag on commit?
> >          - After records are flushed/committed
> >             - what is the diff between the record timestamps and commit
> >             time (histogram)?  this is a measure of end-to-end pipeline
> > latency
> >             - what is the diff between record offsets and latest offset
> of
> >             their partition at commit time (histogram)? this is a
> > measure of whether
> >             this particular task is keeping up
> >
>
> Yeah, possibly. Will have to compare with the consumer metrics to see what
> we can get.
>
>
> >          - How about flush error rate?  Assuming the sink connectors are
> >       using retries, it would be helpful to know how many errors they're
> > seeing
> >
>
> We could add a metric to track how many times the framework receives a
> retry exception and then retries, but the connectors may also do this on
> their own.
>
>
> >       - Can we tell at the framework level how many records were inserted
> >       vs updated vs deleted?
> >
>
> No, there's no distinction in the Connect framework.
>
>
> >       - Batching stats
> >          - Histogram of flush batch size
> >          - Counts of flush trigger method (time vs max batch size)
> >
>
> Should be able to add these.
>
>
> >
> > Cheers,
> >
> > Roger
> >
> > On Sun, Sep 10, 2017 at 8:45 AM, Randall Hauch <rh...@gmail.com> wrote:
> >
> > > Thanks, Gwen.
> > >
> > > That's a great idea, so I've changed the KIP to add those metrics. I've
> > > also made a few other changes:
> > >
> > >
> > >    1. The context of all metrics is limited to the activity within the
> > >    worker. This wasn't clear before, so I changed the motivation and
> > metric
> > >    descriptions to explicitly state this.
> > >    2. Added the worker ID to all MBean attributes. In addition to
> > hopefully
> > >    making this same scope obvious from within JMX or other metric
> > reporting
> > >    system. This is also similar to how the Kafka producer and consumer
> > > metrics
> > >    include the client ID in their MBean attributes. Hopefully this does
> > not
> > >    negatively impact or complicate how external reporting systems'
> > > aggregate
> > >    metrics from multiple workers.
> > >    3. Stated explicitly that aggregating metrics across workers was out
> > of
> > >    scope of this KIP.
> > >    4. Added metrics to report the connector class and version for both
> > sink
> > >    and source connectors.
> > >
> > > Check this KIP's history for details of these changes.
> > >
> > > Please let me know if you have any other suggestions. I hope to start
> the
> > > voting soon!
> > >
> > > Best regards,
> > >
> > > Randall
> > >
> > > On Thu, Sep 7, 2017 at 9:35 PM, Gwen Shapira <gw...@confluent.io>
> wrote:
> > >
> > > > Thanks for the KIP, Randall. Those are badly needed!
> > > >
> > > > Can we have two metrics with record rate per task? One before SMT and
> > one
> > > > after?
> > > > We can have cases where we read 5000 rows from JDBC but write 5 to
> > Kafka,
> > > > or read 5000 records from Kafka and write 5 due to filtering. I think
> > its
> > > > important to know both numbers.
> > > >
> > > >
> > > > Gwen
> > > >
> > > > On Thu, Sep 7, 2017 at 7:50 PM, Randall Hauch <rh...@gmail.com>
> > wrote:
> > > >
> > > > > Hi everyone.
> > > > >
> > > > > I've created a new KIP to add metrics to the Kafka Connect
> framework:
> > > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > > > > 196%3A+Add+metrics+to+Kafka+Connect+framework
> > > > >
> > > > > The KIP approval deadline is looming, so if you're interested in
> > Kafka
> > > > > Connect metrics please review and provide feedback as soon as
> > possible.
> > > > I'm
> > > > > interested not only in whether the metrics are sufficient and
> > > > appropriate,
> > > > > but also in whether the MBean naming conventions are okay.
> > > > >
> > > > > Best regards,
> > > > >
> > > > > Randall
> > > > >
> > > >
> > > >
> > > >
> > > > --
> > > > *Gwen Shapira*
> > > > Product Manager | Confluent
> > > > 650.450.2760 | @gwenshap
> > > > Follow us: Twitter <https://twitter.com/ConfluentInc> | blog
> > > > <http://www.confluent.io/blog>
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-196: Add metrics to Kafka Connect framework

Posted by Randall Hauch <rh...@gmail.com>.
Thanks for reviewing. Responses inline below.

On Mon, Sep 11, 2017 at 11:22 AM, Roger Hoover <ro...@gmail.com>
wrote:

> Randall,
>
> Thank you for the KIP.  This should improve visibility greatly.  I had a
> few questions/ideas for more metrics.
>
>
>    1. What's the relationship between the worker state and the connector
>    status?  Does the 'paused' status at the Connector level include the
> time
>    that worker is 'rebalancing'?
>

The worker state metric simply reports whether the worker is running or
rebalancing. This state is independent of how many connectors are
deployed/running/paused. During a rebalance, the connectors are being
stopped and restarted but are effectively not running.


>    2. Are the "Source Connector" metrics like record rate an aggregation of
>    the "Source Task" metrics?
>

Yes.


>       - How much value is there is monitoring at the "Source Connector"
>       level (other than status) if the number of constituent tasks may
> change
>       over time?
>

The task metrics allow you to know whether the tasks are evenly loaded and
each making progress. The aggregate connector metrics tell you how much
work has been performed by all the tasks in that worker. Both are useful
IMO.


>       - I'm imagining that it's most useful to collect metrics at the task
>       level as the task-level metrics should be stable regardless of tasks
>       shifting to different workers
>

Correct, this is where the most value is because it is the most fine
grained.


>       - If so, can we duplicate the Connector Status down at the task level
>          so that all important metrics can be tracked by task?
>

Possibly. The challenge is that the threads running the tasks are blocked
when a connector is paused.


>          3. For the Sink Task metrics
>       - Can we add offset lag and timestamp lag on commit?
>          - After records are flushed/committed
>             - what is the diff between the record timestamps and commit
>             time (histogram)?  this is a measure of end-to-end pipeline
> latency
>             - what is the diff between record offsets and latest offset of
>             their partition at commit time (histogram)? this is a
> measure of whether
>             this particular task is keeping up
>

Yeah, possibly. Will have to compare with the consumer metrics to see what
we can get.


>          - How about flush error rate?  Assuming the sink connectors are
>       using retries, it would be helpful to know how many errors they're
> seeing
>

We could add a metric to track how many times the framework receives a
retry exception and then retries, but the connectors may also do this on
their own.


>       - Can we tell at the framework level how many records were inserted
>       vs updated vs deleted?
>

No, there's no distinction in the Connect framework.


>       - Batching stats
>          - Histogram of flush batch size
>          - Counts of flush trigger method (time vs max batch size)
>

Should be able to add these.


>
> Cheers,
>
> Roger
>
> On Sun, Sep 10, 2017 at 8:45 AM, Randall Hauch <rh...@gmail.com> wrote:
>
> > Thanks, Gwen.
> >
> > That's a great idea, so I've changed the KIP to add those metrics. I've
> > also made a few other changes:
> >
> >
> >    1. The context of all metrics is limited to the activity within the
> >    worker. This wasn't clear before, so I changed the motivation and
> metric
> >    descriptions to explicitly state this.
> >    2. Added the worker ID to all MBean attributes. In addition to
> hopefully
> >    making this same scope obvious from within JMX or other metric
> reporting
> >    system. This is also similar to how the Kafka producer and consumer
> > metrics
> >    include the client ID in their MBean attributes. Hopefully this does
> not
> >    negatively impact or complicate how external reporting systems'
> > aggregate
> >    metrics from multiple workers.
> >    3. Stated explicitly that aggregating metrics across workers was out
> of
> >    scope of this KIP.
> >    4. Added metrics to report the connector class and version for both
> sink
> >    and source connectors.
> >
> > Check this KIP's history for details of these changes.
> >
> > Please let me know if you have any other suggestions. I hope to start the
> > voting soon!
> >
> > Best regards,
> >
> > Randall
> >
> > On Thu, Sep 7, 2017 at 9:35 PM, Gwen Shapira <gw...@confluent.io> wrote:
> >
> > > Thanks for the KIP, Randall. Those are badly needed!
> > >
> > > Can we have two metrics with record rate per task? One before SMT and
> one
> > > after?
> > > We can have cases where we read 5000 rows from JDBC but write 5 to
> Kafka,
> > > or read 5000 records from Kafka and write 5 due to filtering. I think
> its
> > > important to know both numbers.
> > >
> > >
> > > Gwen
> > >
> > > On Thu, Sep 7, 2017 at 7:50 PM, Randall Hauch <rh...@gmail.com>
> wrote:
> > >
> > > > Hi everyone.
> > > >
> > > > I've created a new KIP to add metrics to the Kafka Connect framework:
> > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > > > 196%3A+Add+metrics+to+Kafka+Connect+framework
> > > >
> > > > The KIP approval deadline is looming, so if you're interested in
> Kafka
> > > > Connect metrics please review and provide feedback as soon as
> possible.
> > > I'm
> > > > interested not only in whether the metrics are sufficient and
> > > appropriate,
> > > > but also in whether the MBean naming conventions are okay.
> > > >
> > > > Best regards,
> > > >
> > > > Randall
> > > >
> > >
> > >
> > >
> > > --
> > > *Gwen Shapira*
> > > Product Manager | Confluent
> > > 650.450.2760 | @gwenshap
> > > Follow us: Twitter <https://twitter.com/ConfluentInc> | blog
> > > <http://www.confluent.io/blog>
> > >
> >
>

Re: [DISCUSS] KIP-196: Add metrics to Kafka Connect framework

Posted by Roger Hoover <ro...@gmail.com>.
Randall,

Thank you for the KIP.  This should improve visibility greatly.  I had a
few questions/ideas for more metrics.


   1. What's the relationship between the worker state and the connector
   status?  Does the 'paused' status at the Connector level include the time
   that worker is 'rebalancing'?
   2. Are the "Source Connector" metrics like record rate an aggregation of
   the "Source Task" metrics?
      - How much value is there is monitoring at the "Source Connector"
      level (other than status) if the number of constituent tasks may change
      over time?
      - I'm imagining that it's most useful to collect metrics at the task
      level as the task-level metrics should be stable regardless of tasks
      shifting to different workers
      - If so, can we duplicate the Connector Status down at the task level
         so that all important metrics can be tracked by task?
         3. For the Sink Task metrics
      - Can we add offset lag and timestamp lag on commit?
         - After records are flushed/committed
            - what is the diff between the record timestamps and commit
            time (histogram)?  this is a measure of end-to-end pipeline latency
            - what is the diff between record offsets and latest offset of
            their partition at commit time (histogram)? this is a
measure of whether
            this particular task is keeping up
         - How about flush error rate?  Assuming the sink connectors are
      using retries, it would be helpful to know how many errors they're seeing
      - Can we tell at the framework level how many records were inserted
      vs updated vs deleted?
      - Batching stats
         - Histogram of flush batch size
         - Counts of flush trigger method (time vs max batch size)

Cheers,

Roger

On Sun, Sep 10, 2017 at 8:45 AM, Randall Hauch <rh...@gmail.com> wrote:

> Thanks, Gwen.
>
> That's a great idea, so I've changed the KIP to add those metrics. I've
> also made a few other changes:
>
>
>    1. The context of all metrics is limited to the activity within the
>    worker. This wasn't clear before, so I changed the motivation and metric
>    descriptions to explicitly state this.
>    2. Added the worker ID to all MBean attributes. In addition to hopefully
>    making this same scope obvious from within JMX or other metric reporting
>    system. This is also similar to how the Kafka producer and consumer
> metrics
>    include the client ID in their MBean attributes. Hopefully this does not
>    negatively impact or complicate how external reporting systems'
> aggregate
>    metrics from multiple workers.
>    3. Stated explicitly that aggregating metrics across workers was out of
>    scope of this KIP.
>    4. Added metrics to report the connector class and version for both sink
>    and source connectors.
>
> Check this KIP's history for details of these changes.
>
> Please let me know if you have any other suggestions. I hope to start the
> voting soon!
>
> Best regards,
>
> Randall
>
> On Thu, Sep 7, 2017 at 9:35 PM, Gwen Shapira <gw...@confluent.io> wrote:
>
> > Thanks for the KIP, Randall. Those are badly needed!
> >
> > Can we have two metrics with record rate per task? One before SMT and one
> > after?
> > We can have cases where we read 5000 rows from JDBC but write 5 to Kafka,
> > or read 5000 records from Kafka and write 5 due to filtering. I think its
> > important to know both numbers.
> >
> >
> > Gwen
> >
> > On Thu, Sep 7, 2017 at 7:50 PM, Randall Hauch <rh...@gmail.com> wrote:
> >
> > > Hi everyone.
> > >
> > > I've created a new KIP to add metrics to the Kafka Connect framework:
> > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > > 196%3A+Add+metrics+to+Kafka+Connect+framework
> > >
> > > The KIP approval deadline is looming, so if you're interested in Kafka
> > > Connect metrics please review and provide feedback as soon as possible.
> > I'm
> > > interested not only in whether the metrics are sufficient and
> > appropriate,
> > > but also in whether the MBean naming conventions are okay.
> > >
> > > Best regards,
> > >
> > > Randall
> > >
> >
> >
> >
> > --
> > *Gwen Shapira*
> > Product Manager | Confluent
> > 650.450.2760 | @gwenshap
> > Follow us: Twitter <https://twitter.com/ConfluentInc> | blog
> > <http://www.confluent.io/blog>
> >
>

Re: [DISCUSS] KIP-196: Add metrics to Kafka Connect framework

Posted by Randall Hauch <rh...@gmail.com>.
Thanks, Gwen.

That's a great idea, so I've changed the KIP to add those metrics. I've
also made a few other changes:


   1. The context of all metrics is limited to the activity within the
   worker. This wasn't clear before, so I changed the motivation and metric
   descriptions to explicitly state this.
   2. Added the worker ID to all MBean attributes. In addition to hopefully
   making this same scope obvious from within JMX or other metric reporting
   system. This is also similar to how the Kafka producer and consumer metrics
   include the client ID in their MBean attributes. Hopefully this does not
   negatively impact or complicate how external reporting systems' aggregate
   metrics from multiple workers.
   3. Stated explicitly that aggregating metrics across workers was out of
   scope of this KIP.
   4. Added metrics to report the connector class and version for both sink
   and source connectors.

Check this KIP's history for details of these changes.

Please let me know if you have any other suggestions. I hope to start the
voting soon!

Best regards,

Randall

On Thu, Sep 7, 2017 at 9:35 PM, Gwen Shapira <gw...@confluent.io> wrote:

> Thanks for the KIP, Randall. Those are badly needed!
>
> Can we have two metrics with record rate per task? One before SMT and one
> after?
> We can have cases where we read 5000 rows from JDBC but write 5 to Kafka,
> or read 5000 records from Kafka and write 5 due to filtering. I think its
> important to know both numbers.
>
>
> Gwen
>
> On Thu, Sep 7, 2017 at 7:50 PM, Randall Hauch <rh...@gmail.com> wrote:
>
> > Hi everyone.
> >
> > I've created a new KIP to add metrics to the Kafka Connect framework:
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > 196%3A+Add+metrics+to+Kafka+Connect+framework
> >
> > The KIP approval deadline is looming, so if you're interested in Kafka
> > Connect metrics please review and provide feedback as soon as possible.
> I'm
> > interested not only in whether the metrics are sufficient and
> appropriate,
> > but also in whether the MBean naming conventions are okay.
> >
> > Best regards,
> >
> > Randall
> >
>
>
>
> --
> *Gwen Shapira*
> Product Manager | Confluent
> 650.450.2760 | @gwenshap
> Follow us: Twitter <https://twitter.com/ConfluentInc> | blog
> <http://www.confluent.io/blog>
>

Re: [DISCUSS] KIP-196: Add metrics to Kafka Connect framework

Posted by Gwen Shapira <gw...@confluent.io>.
Thanks for the KIP, Randall. Those are badly needed!

Can we have two metrics with record rate per task? One before SMT and one
after?
We can have cases where we read 5000 rows from JDBC but write 5 to Kafka,
or read 5000 records from Kafka and write 5 due to filtering. I think its
important to know both numbers.


Gwen

On Thu, Sep 7, 2017 at 7:50 PM, Randall Hauch <rh...@gmail.com> wrote:

> Hi everyone.
>
> I've created a new KIP to add metrics to the Kafka Connect framework:
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> 196%3A+Add+metrics+to+Kafka+Connect+framework
>
> The KIP approval deadline is looming, so if you're interested in Kafka
> Connect metrics please review and provide feedback as soon as possible. I'm
> interested not only in whether the metrics are sufficient and appropriate,
> but also in whether the MBean naming conventions are okay.
>
> Best regards,
>
> Randall
>



-- 
*Gwen Shapira*
Product Manager | Confluent
650.450.2760 | @gwenshap
Follow us: Twitter <https://twitter.com/ConfluentInc> | blog
<http://www.confluent.io/blog>