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

Compare with Current View Page History

« Previous Version 3 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 producers and admins managed by the Connect framework for the tasks will get the exact same client.ids in each flow. In the MM2 logs, the logs of such Kafka clients cannot be distinguished easily/at all.
  2. 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.
  3. The toString method of some entities (such as WorkerConnector, WorkerTask) used in logs only contain the Connector name/Task ID, and those logs cannot be distinguished easily/at all.

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

Public Interfaces

MM2 distributed mode configuration change

  • New configuration in MM2 distributed mode: add.flow.context (boolean, default: false) - when enabled the following changes are applied

MM2 distributed mode client.id and log change

  • The client.id configuration of the Connect-managed Kafka clients (used by the Tasks) will be prefixed with the flow, using the following pattern: <SOURCE>-><TARGET>| e.g. "primary→backup|".
  • Entities having a toString implementation used in logs and relying on Connector name/Task ID will also contain the flow. (WorkerConnector, WorkerTask and all their subclasses)
  • The Connector context (named "connector.context" in the MDC) will be also prefixed with the flow, using the following pattern: <SOURCE>-><TARGET>| e.g. "primary→backup|".

Proposed Changes

Connect code changes

  • LoggingContext: all context-creator methods will now accept a nullable "String prefix" parameter, which if not null, will be prepended to the connector.context value with the "|" separator.
  • WorkerConfig: add a new method with the following signature: String contextPrefix()
    The implementation in WorkerConfig will return null, DistributedConfig will not override it either.
  • Worker: if WorkerConfig.contextPrefix returns non-null, use it to prefix the client.ids of the managed Kafka clients with the "|" separator.
  • WorkerConnector, WorkerTask and their subclasses: store contextPrefix, and if not null, toString also contains the field
  • Important note: since all of the above relies on WorkerConfig.contextPrefix 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 contextPrefix field (set in its constructor), and in its String contextPrefix() implementation, returns the field value.
  • MirrorMakerConfig: new supported config flag add.flow.context
  • MirrorMaker: instead of using DistributedConfig, use MirrorMakerWorkerConfig. If add.flow.context is true, pass the flow (SourceAndTarget.toString) to the MirrorMakerWorkerConfig constructor.

Compatibility, Deprecation, and Migration Plan

  • Connect is not impacted at all.
  • MM2 distributed mode is not impacted, as the new functionality is protected with a feature flag. Users need to opt-in after the upgrade by setting add.flow.context to true.

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.

Tracking the flow under a separate MDC context key

Instead of modifying the existing "connector.context", an extra context key could be introduced. This extra context value could be exported in the same scope as the connector.context. This would allow opting in through changing the logger configuration instead of changing the MM2 configs.

This would take the same amount of work as the proposed solution, only the configuration method would be different.

  • No labels