Versions Compared

Key

  • This line was added.
  • This line was removed.
  • Formatting was changed.

...

  • DistributedHerder internal threads - their names already contain the client.id, which contains the flow in MM2 dedicated mode.

Proposed Changes


BeforeAfter (change is in bold)
Log pattern[%d] %p [%t] %X{connector.context}%m (%c:%L)%n[%d] %p [%t] %X{flow.context}%X{connector.context}%m (%c:%L)%n
MirrorSourceTask startup[2023-06-12 11:07:42,246] INFO [task-thread-MirrorSourceConnector-0] [MirrorSourceConnector|task-0] Starting with 2 previously uncommitted partitions. (org.apache.kafka.connect.mirror.MirrorSourceTask:108)[2023-06-12 11:07:42,246] INFO [task-thread-MirrorSourceConnector-0|A->B] [A->B] [MirrorSourceConnector|task-0] Starting with 2 previously uncommitted partitions. (org.apache.kafka.connect.mirror.MirrorSourceTask:108)
Connector logs[2023-06-12 11:10:51,829] INFO [connector-thread-MirrorHeartbeatConnector] [MirrorHeartbeatConnector|worker] MirrorHeartbeatConfig values:[2023-06-12 11:10:51,829] INFO [connector-thread-MirrorHeartbeatConnector|A->B] [A->B] [MirrorHeartbeatConnector|worker] MirrorHeartbeatConfig values:
Framework internal logs[2023-06-12 11:10:51,975] INFO [connector-thread-MirrorSourceConnector] [MirrorSourceConnector|worker] Completed shutdown for WorkerConnector{id=MirrorSourceConnector} (org.apache.kafka.connect.runtime.WorkerConnector:287)[2023-06-12 11:10:51,975] INFO [connector-thread-MirrorSourceConnector|A->B] [A->B] [MirrorSourceConnector|worker] Completed shutdown for WorkerConnector{id=MirrorSourceConnector} (org.apache.kafka.connect.runtime.WorkerConnector:287)

As seen in the example log lines, the Before lines do not provide the full context, the lines cannot be differentiated among the different flows. Lines after the change will contain the flow information in both the flow.context and the thread names.

There is overlap between the thread names and the flow.context. This shouldn't be a problem - many use-cases do not use the thread name in logging, for those cases, the flow.context is still useful. The thread name change also improves diagnostics in other areas, such as when generating thread dumps, or debugging a process.TBD: log line examples before/after change

Compatibility, Deprecation, and Migration Plan

...