Versions Compared

Key

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

...

Three new task-level JMX properties will be added:

MBean namekafka.connect:type=source-task-metrics,connector=([-.\w]+),task=([\d]+)

Metric nameDescription
transaction-size-min The number of records in the smallest transaction the task has committed so far.
transaction-size-max The number of records in the largest transaction the task has committed so far.
transaction-size-avg The average number of records in the transactions the task has committed so far.

Proposed Changes

Atomic offset writes

...

Before upgrading a worker to 3.0, its producer principal must be given the following permissions on the Kafka cluster it writes to:

Operation

Resource Type

Resource Name

Write

TransactionalId

connect-cluster-${groupId} , where ${groupId } is the group ID of the cluster

Describe

TransactionalId

connect-cluster-${groupId} , where ${groupId } is the group ID of the cluster

IdempotentWrite

Cluster

Kafka cluster targeted by the Connect cluster

These new ACLs will be required regardless of whether exactly once source support is enabled on the worker.

...

Operation

Resource Type

Resource Name

Write

TransactionalId

${groupId}-${connector}-${taskId}, for each task that the connector will create, where ${groupId} is the group ID of the Connect cluster, ${connector} is the name of the connector, and ${taskId} is the ID of the task (starting from zero). A wildcarded prefix of ${groupId}-${connector}* can be used for convenience if there is no risk of conflict with other transactional IDs or if conflicts are acceptable to the user.

Describe

TransactionalId

${groupId}-${connector}-${taskId}, for each task that the connector will create, where ${groupId} is the group ID of the Connect cluster, ${connector} is the name of the connector, and ${taskId} is the ID of the task (starting from zero). A wildcarded prefix of ${groupId}-${connector}* can be used for convenience if there is no risk of conflict with other transactional IDs or if conflicts are acceptable to the user.

Write

TransactionalId

${groupId}-${connector}. Only necessary if the connector uses a separate offsets topic.

Describe

TransactionalId

${groupId}-${connector}. Only necessary if the connector uses a separate offsets topic.

IdempotentWrite

Cluster

Kafka cluster targeted by the connector.

Write

Topic

Offsets topic used by the connector, which is either the value of the offsets.storage.topic property in the connector’s configuration if provided, or the value of the offsets.storage.topic property in the worker’s configuration if not.

...

If the (implicitly- or explicitly-configured) offsets topic for a connector does not exist yet, its admin principal must be given the following permissions on the Kafka cluster that will host the offsets topic:

Create

Topic

Offsets topic used by the connector, which is the value of the offsets.storage.topic property in the connector’s configuration. If not provided and the connector targets the same Kafka cluster that the worker uses for its internal topics, this ACL is not needed, as the worker’s shared offsets topic should be created automatically before any connectors are started.

Additionally, the connector's admin principal must be given the following permissions on the Kafka cluster it reads offsets from no matter what:

...