Versions Compared

Key

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

Status

Page properties


Discussion thread https://lists.apache.org/thread/vz7nw5qzvmxwnpktnofc9p13s1dzqm6z
Vote threadTBD
JIRA
Jira
serverASF JIRA
serverId5aa69414-a9e9-3523-82ec-879b028fb15b
keyFLINK-32197
ReleaseTBD


...

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

Motivation

Users in large infrastructure setups often need to process and/or join data that lives in different Kafka clusters in Flink. In addition, multiple Kafka clusters may be relevant when Kafka consumers need to migrate Kafka clusters.

...

This source will extend the KafkaSource to be able to read from multiple Kafka clusters within a single source and introduces an interface to enable more sophisticated automation/coordination between Kafka and Flink infrastructure.

Basic Idea

MultiClusterKafkaSource relies on metadata to determine what clusters and topics to subscribe to, and the metadata can change over time so the source will poll for new metadata and dynamically reconcile changes on an interval.

...

Other required functionality leverages and composes the existing KafkaSource implementation for discovering Kafka topic partition offsets and round robin split assignment per Kafka cluster. The diagrams below depicts how the source will reuse the code of the KafkaSource in order to achieve the requirements.

The Kafka Metadata Service

To provide the ability to dynamically change the underlying source components without job restart, there needs to exist a coordination mechanism to manage how the underlying KafkaSourceEnumerators and KafkaSources interact with multiple clusters and multiple topics. A KafkaMetadataService is the discovery mechanism by which the source components will reconcile metadata changes, and only the MultiClusterKafkaSourceEnumerator interacts with the KafkaMetadata Service. Periodic metadata discovery will be supported via source configuration just like topic partition discovery is supported via an interval in KafkaSource. It is possible to interpret this as a multi cluster extension of the AdminClient for KafkaSource, serving only cluster and topic metadata.

A default implementation will be provided so that native Kubernetes configmap can easily control the metadata (yaml/json file). This implementation is targeted for the basic use cases where external monitoring will inform how users change the metadata.

KafkaStream and KafkaClusterId

In addition, KafkaStream is part of the metadata returned by KafkaMetadataService and contains a logical mapping to physical Kafka clusters and Kafka topics, which can be transient and change dynamically. Changes in metadata are detected by the enumerator and propagated to readers via source events to reconcile the changes. In addition, Kafka clusters are uniquely identified by a string id since there could be multiple bootstrap servers lists that can read a certain Kafka cluster (e.g. kafka-server1:9092, kafka-server2:9092 and kafka-server1:9092).

Exactly Once Semantics and Consistency Guarantees

KafkaSource guarantees exactly once reading since offsets move forward only when checkpoint succeeds and MultiClusterKafkaSource inherits these properties since the source delegates the functionality from the KafkaSource components. Metadata is checkpointed and can be rebuilt from the reader split state. Exactly once guarantees can be maintained with the assumption that KafkaMetadataService does not decide to expire a cluster in which data still needs to be read. This can be solved by not destroying the old Kafka cluster until consumers are already drained (no more producer traffic and lag is 0)–in practice, a good strategy is to let data expire naturally via Kafka cluster retention. In Kafka migration switchover, the consumer would consume from both old and new clusters. With the regular KafkaSource, if Kafka deletes topic or a cluster is destroyed, the exactly once semantics are not preserved and the semantic is tightly coupled with storage. The design composes and delegates the responsibilities to KafkaSource components so it is limited to whatever KafkaSource can do for exactly once semantics–the KafkaMetadataService and source metadata reconciliation mechanism make it possible to automate migration and prevent data loss.

...

To the source more user friendly, a MultiClusterKafkaSourceBuilder will be provided (e.g. batch mode should not turn on KafkaMetadataService discovery, should only be done at startup).

Public Interfaces

The source will use the FLIP-27: Refactor Source Interface to integrate it with Flink and support both bounded and unbounded jobs.

...

Code Block
languagejava
titleBuilder Example
MultiClusterKafkaSource.<String>builder()
  // some default implementations will be provided (file based, statically defined streams)
  .setKafkaMetadataService(new KafkaMetadataServiceImpl())
  .setStreamIds(List.of("my-stream-1", "my-stream-2"))
  .setGroupId("myConsumerGroup")
  .setDeserializer(KafkaRecordDeserializationSchema.valueOnly(StringDeserializer.class))
  .setStartingOffsets(OffsetsInitializer.earliest())
  .setProperties(properties)
  .build();

// Can configure via properties
ConfigOptions.key("multi.cluster.kafka.source.kafka-metadata-service-discovery-internal-ms")
	.longType()
	.noDefaultValue()
	.withDescription(
		"The rate at which the Kafka metadata service will be polled in milliseconds.");

Proposed Changes

KafkaClusterIdentifier

This is logical abstraction is introduced since bootstrap servers may change although the "cluster" is still the same. Thus, name is used as a unique identifier, which also has the added benefit to use a short name for connector related metrics. Bootstrap server can be used as the name in simple usecases.

Code Block
languagejava
titleKafkaClusterIdentifier
@PublicEvolving 
public class KafkaClusterIdentifier implements Comparable<KafkaClusterIdentifier>, Serializable {
  private final String name;
  private final String bootstrapServers;

...
}

KafkaStream

It is possible that a Kafka stream is composed of multiple topics on multiple Kafka clusters. In addition, this flexible and general abstraction does not require any conventions on the topic naming but implementations can make assumptions to do so if desired. In the simplest case, a Kafka stream is a single topic on a single Kafka cluster.

Code Block
languagejava
titleKafkaStream
@PublicEvolving
public class KafkaStream implements Serializable {
  private final String streamId;
  private final Map<KafkaClusterIdentifier, Set<String>> kafkaClusterTopicMap;

  public KafkaStream(
      String streamId, Map<KafkaClusterIdentifier, Set<String>> kafkaClusterTopicMap) {
    this.streamId = streamId;
    this.kafkaClusterTopicMap = kafkaClusterTopicMap;
  }

...
}

KafkaMetadataService

This is responsible to resolve Kafka metadata from streams. This may be backed by an external service or simply something logical that is contained in memory. A config map file based implementation will be provided as well for convenience. Similarly to KafkaSource subscriber integration, the #getAllStreams() API is supported here to be able to filter out streams, for example, by a regex.

...

Code Block
languagejava
titleKafkaMetadataService
@PublicEvolving 
public interface KafkaMetadataService extends AutoCloseable, Serializable {
  /**
   * Get current metadata for all streams.
   *
   * @return set of all streams
   */
  Set<KafkaStream> getAllStreams();

  /**
   * Get current metadata for queried streams.
   *
   * @param streamIds stream full names
   * @return map of stream name to metadata
   */
  Map<String, KafkaStream> describeStreams(Collection<String> streamIds);

  /**
   * Check if the cluster is active.
   *
   * @param kafkaClusterIdentifier Kafka cluster identifier
   * @return boolean whether the cluster is active
   */
  boolean isClusterActive(KafkaClusterIdentifier kafkaClusterIdentifier);
}

KafkaStreamSubscriber

This is similar to KafkaSource's KafkaSubscriber. A regex subscriber will be provided to match streams by a regex pattern.

Code Block
languagejava
titleKafkaStreamSubscriber
@PublicEvolving
public interface KafkaStreamSubscriber extends Serializable {

  /** Get the set of subscribed streams. */
  Set<KafkaStream> getSubscribedStreams(KafkaMetadataService kafkaMetadataService);
}

MetadataUpdateEvent

This is a metadata update event containing the current metadata, sent from enumerator to reader. The metadata does not include stream information since it is not required by the reader, which does not directly interact with streams or the KafkaMetadataService.

...

Code Block
languagejava
titleMetadataUpdateEvent
@Internal
public class MetadataUpdateEvent implements SourceEvent {
  private final Map<KafkaClusterIdentifier, Set<String>> currentClusterTopics;

...
}

MultiClusterKafkaSourceEnumerator

This reader is responsible for discovering and assigning splits from 1+ cluster. At startup, the enumerator will invoke the KafkaStreamSubscriber and reconcile changes from state. Source events will be sent to the source reader to reconcile the metadata. This enumerator has the ability to poll the KafkaMetadataService, periodically for stream discovery. In addition, restarting enumerators involve clearing outdated metrics since clusters may be removed and so should their metrics.

Code Block
languagejava
titleMultiClusterKafkaSourceEnumerator
@PublicEvolving 
public class MultiClusterKafkaSourceEnumerator
    implements SplitEnumerator<MultiClusterKafkaSourceSplit, MultiClusterKafkaSourceEnumState> {
  
  private final Map<
          KafkaClusterIdentifier, SplitEnumerator<KafkaPartitionSplit, KafkaSourceEnumState>>
      clusterEnumeratorMap;
  private final Map<KafkaClusterIdentifier, StoppableKafkaEnumContextProxy> clusterEnumContextMap;
  private final KafkaStreamSubscriber kafkaStreamSubscriber;
  private final KafkaMetadataService kafkaMetadataService;
  private Map<KafkaClusterIdentifier, Set<String>> activeClusterTopicsMap;

  private void restartEnumerators(KafkaClusterIdentifier kafkaClusterId, Set<TopicPartition> enumeratorState) {}

...
}

StoppableKafkaEnumContextProxy

This enumerator context proxy facilitates the ability to close executors used by scheduled callables in the underlying KafkaSourceEnumerators and wraps the KafkaPartitionSplits with cluster information.

...

Code Block
languagejava
titleStoppableKafkaEnumContextProxy
@Internal
public class StoppableKafkaEnumContextProxy
    implements SplitEnumeratorContext<KafkaPartitionSplit>, AutoCloseable {

  private final KafkaClusterIdentifier kafkaClusterIdentifier;
  private final KafkaMetadataService kafkaMetadataService;
  private final SplitEnumeratorContext<MultiClusterKafkaSourceSplit> enumContext;
  private final ScheduledExecutorService subEnumeratorWorker;

  /** Wrap splits with cluster metadata. */
  public void assignSplits(SplitsAssignment<KafkaPartitionSplit> newSplitAssignments) {}

...
}

GetMetadataUpdateEvent

This is a metadata update event requesting the current metadata, sent from reader to enumerator.

...

Code Block
languagejava
titleGetMetadataUpdateEvent
@Internal
public class GetMetadataUpdateEvent implements SourceEvent {}

MultiClusterKafkaSourceReader

This reader is responsible for reading from 1+ clusters.

...

Code Block
languagejava
titleMultiClusterKafkaSourceReader
@PublicEvolving 
public class MultiClusterKafkaSourceReader<T>
    implements SourceReader<T, MultiClusterKafkaSourceSplit> {

  @VisibleForTesting
  final NavigableMap<KafkaClusterIdentifier, KafkaSourceReader<T>> clusterReaderMap;
  
  private void restartReader(
      KafkaClusterIdentifier kafkaClusterId, List<KafkaPartitionSplit> readerState) {}
...
}

MultiClusterKafkaSourceSplit

This extends KafkaSource's KafkaPartitionSplit to include cluster information.

Code Block
languagejava
titleMultiClusterKafkaSourceSplit
@PublicEvolving 
public class MultiClusterKafkaSourceSplit implements SourceSplit {

  private final KafkaClusterIdentifier kafkaClusterId;
  private final KafkaPartitionSplit kafkaPartitionSplit;

...
}

MultiClusterKafkaSource

Connecting it all together...

Code Block
languagejava
titleMultiClusterKafkaSource
@PublicEvolving 
public class MultiClusterKafkaSource<T>
    implements Source<T, MultiClusterKafkaSourceSplit, MultiClusterKafkaSourceEnumState>,
        ResultTypeQueryable<T> {

  private final KafkaStreamSubscriber kafkaStreamSubscriber;
  private final KafkaMetadataService kafkaMetadataService;
  private final KafkaRecordDeserializationSchema<T> deserializationSchema;
  private final OffsetsInitializer startingOffsetsInitializer;
  private final OffsetsInitializer stoppingOffsetsInitializer;
  private final Properties properties;
  private final Boundedness boundedness;

...
}

Compatibility, Deprecation, and Migration Plan

The source is opt in and would require users to implement code changes.

In the same vein as the migration from FlinkKafkaConsumer and KafkaSource, the source state is incompatible between KafkaSource and MultiClusterKafkaSource so it is recommended to reset all state or reset partial state by setting a different uid and starting the application from nonrestore state.

Test Plan

This will be tested by unit and integration tests. The work will extend existing KafkaSource test utilities in Flink to exercise multiple clusters.

The testcontainers utilities can be used to create multiple Kafka clusters and the file based implementation of Kafka metadata service would be essential in testing metadata changes.

Future Improvements

KafkaMetadataService can also coordinate the configurations of the source such as Kafka properties and offset initialization strategies. For now, the proposal only includes making 1 common configuration defined by the user. The Kafka properties is essential for cases such as enforcing the max message size configuration from the Kafka server and security configurations.

Rejected Alternatives

None