You are viewing an old version of this page. View the current version.

Compare with Current View Page History

« Previous Version 4 Next »

Status

Current state: Under Discussion

Discussion thread: here

JIRA: here

Please keep the discussion on the mailing list rather than commenting on the wiki (wiki discussions get unwieldy fast).

Motivation

MirrorMaker2 distributed mode is capable of running multiple Connect workers in a single process, possibly connecting to a set of different Kafka clusters. This simplifies the operation of async replication compared to operating separate Kafka Connect clusters.

At the same time, diagnosing issues in MM2 distributed mode can be quite challenging. Each flow has a dedicated Connect group, and for each flow, the Connectors have the same name. This causes the following monitoring and diagnostic issues with MM2:

  1. The Connector log context provided by Connect contains the same Connector name/Task ID in each flow. In the MM2 logs, lines coming from inside the context cannot be distinguished easily/at all.
  2. The internal Connect thread names only contain the Connector name/Task ID, and those cannot be distinguished easily/at all.

MM2 distributed mode should also provide the replication flow in the the logs and the thread names to address this problem.

Public Interfaces

MM2 distributed mode logging change

  • New MDC context key available: flow.context

MM2 distributed mode client.id and log change

  • The flow context (named "flow.context" in the MDC) will be exported, and it will contain the flow using the following pattern: <SOURCE>-><TARGET> e.g. "primary→backup".
  • The Connect internal thread names will contain the flow next to the Connector name/Task ID.

Proposed Changes

Connect code changes

  • LoggingContext: a new context-creator static method: LoggingContext customContext(Map<String, String> contexts) which exports an arbitrary number of key-value pairs into MDC.
  • WorkerConfig: add 2 new methods with the following signatures:
    • Map<String, String> customContexts()
    • String threadNamePrefix()
    • The implementations in WorkerConfig will return null, DistributedConfig will not override it either.
  • Worker: if WorkerConfig.customContexts returns non-null, export it in the MDC in all places where LoggingContext is used
  • Worker: if WorkerConfig.threadNamePrefix returns non-null, add it to all internal thread names.
  • Important note: since all of the above relies on WorkerConfig.customContexts and WorkerConfig.threadNamePrefix returning non-null, Connect itself will not change.

MM2 code changes

  • New class: MirrorMakerWorkerConfig, subclass of DistributedConfig - the only added logic is that MirrorMakerWorkerConfig has a String flowContext field (set in its constructor), and
    • in the customContexts() implementation, returns "flow.context" → this.flowContext.
    • in the threadNamePrefix() implementation, returns "flowContext|"
  • MirrorMaker: instead of using DistributedConfig, use MirrorMakerWorkerConfig. Pass the flow (SourceAndTarget.toString) to the MirrorMakerWorkerConfig constructor.

Compatibility, Deprecation, and Migration Plan

  • Connect is not impacted at all.
  • MM2 distributed mode impact is minimal:
    • Connect internal thread names will be changed.
    • The flow context is only added if the user opts-in by referring to the flow.context MDC key in their logging configuration.

Test Plan

Unit tests.

Rejected Alternatives

Supporting the same feature in Connect, making it configurable

To have a unified feature set, we could also support the same prefixing strategy in Connect. This would require an extra Connect configuration to specify the prefix.

In the context of Connect, this doesn't make sense - in a single Connect cluster, Connector names are unique, and the existing client.id and logging context ensures that the diagnostic information is distinguishable.

  • No labels