org.apache.kafka.connect.storage.KafkaConfigBackingStore Maven / Gradle / Ivy
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.kafka.connect.storage;
import org.apache.kafka.clients.admin.NewTopic;
import org.apache.kafka.clients.consumer.ConsumerConfig;
import org.apache.kafka.clients.consumer.ConsumerRecord;
import org.apache.kafka.clients.producer.ProducerConfig;
import org.apache.kafka.common.config.ConfigException;
import org.apache.kafka.common.serialization.ByteArrayDeserializer;
import org.apache.kafka.common.serialization.ByteArraySerializer;
import org.apache.kafka.common.serialization.StringDeserializer;
import org.apache.kafka.common.serialization.StringSerializer;
import org.apache.kafka.common.utils.Time;
import org.apache.kafka.common.utils.Utils;
import org.apache.kafka.connect.data.Schema;
import org.apache.kafka.connect.data.SchemaAndValue;
import org.apache.kafka.connect.data.SchemaBuilder;
import org.apache.kafka.connect.data.Struct;
import org.apache.kafka.connect.errors.ConnectException;
import org.apache.kafka.connect.errors.DataException;
import org.apache.kafka.connect.runtime.TargetState;
import org.apache.kafka.connect.runtime.WorkerConfig;
import org.apache.kafka.connect.runtime.WorkerConfigTransformer;
import org.apache.kafka.connect.runtime.distributed.ClusterConfigState;
import org.apache.kafka.connect.runtime.distributed.DistributedConfig;
import org.apache.kafka.connect.util.Callback;
import org.apache.kafka.connect.util.ConnectorTaskId;
import org.apache.kafka.connect.util.KafkaBasedLog;
import org.apache.kafka.connect.util.TopicAdmin;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.TreeSet;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
/**
*
* Provides persistent storage of Kafka Connect connector configurations in a Kafka topic.
*
*
* This class manages both connector and task configurations. It tracks three types of configuration entries:
*
* 1. Connector config: map of string -> string configurations passed to the Connector class, with support for
* expanding this format if necessary. (Kafka key: connector-[connector-id]).
* These configs are *not* ephemeral. They represent the source of truth. If the entire Connect
* cluster goes down, this is all that is really needed to recover.
* 2. Task configs: map of string -> string configurations passed to the Task class, with support for expanding
* this format if necessary. (Kafka key: task-[connector-id]-[task-id]).
* These configs are ephemeral; they are stored here to a) disseminate them to all workers while
* ensuring agreement and b) to allow faster cluster/worker recovery since the common case
* of recovery (restoring a connector) will simply result in the same configuration as before
* the failure.
* 3. Task commit "configs": records indicating that previous task config entries should be committed and all task
* configs for a connector can be applied. (Kafka key: commit-[connector-id].
* This config has two effects. First, it records the number of tasks the connector is currently
* running (and can therefore increase/decrease parallelism). Second, because each task config
* is stored separately but they need to be applied together to ensure each partition is assigned
* to a single task, this record also indicates that task configs for the specified connector
* can be "applied" or "committed".
*
*
* This configuration is expected to be stored in a *single partition* and *compacted* topic. Using a single partition
* ensures we can enforce ordering on messages, allowing Kafka to be used as a write ahead log. Compaction allows
* us to clean up outdated configurations over time. However, this combination has some important implications for
* the implementation of this class and the configuration state that it may expose.
*
*
* Connector configurations are independent of all other configs, so they are handled easily. Writing a single record
* is already atomic, so these can be applied as soon as they are read. One connectors config does not affect any
* others, and they do not need to coordinate with the connector's task configuration at all.
*
*
* The most obvious implication for task configs is the need for the commit messages. Because Kafka does not
* currently have multi-record transactions or support atomic batch record writes, task commit messages are required
* to ensure that readers do not end up using inconsistent configs. For example, consider if a connector wrote configs
* for its tasks, then was reconfigured and only managed to write updated configs for half its tasks. If task configs
* were applied immediately you could be using half the old configs and half the new configs. In that condition, some
* partitions may be double-assigned because the old config and new config may use completely different assignments.
* Therefore, when reading the log, we must buffer config updates for a connector's tasks and only apply atomically them
* once a commit message has been read.
*
*
* However, there are also further challenges. This simple buffering approach would work fine as long as the entire log was
* always available, but we would like to be able to enable compaction so our configuration topic does not grow
* indefinitely. Compaction may break a normal log because old entries will suddenly go missing. A new worker reading
* from the beginning of the log in order to build up the full current configuration will see task commits, but some
* records required for those commits will have been removed because the same keys have subsequently been rewritten.
* For example, if you have a sequence of record keys [connector-foo-config, task-foo-1-config, task-foo-2-config,
* commit-foo (2 tasks), task-foo-1-config, commit-foo (1 task)], we can end up with a compacted log containing
* [connector-foo-config, task-foo-2-config, commit-foo (2 tasks), task-foo-1-config, commit-foo (1 task)]. When read
* back, the first commit will see an invalid state because the first task-foo-1-config has been cleaned up.
*
*
* Compaction can further complicate things if writing new task configs fails mid-write. Consider a similar scenario
* as the previous one, but in this case both the first and second update will write 2 task configs. However, the
* second write fails half of the way through:
* [connector-foo-config, task-foo-1-config, task-foo-2-config, commit-foo (2 tasks), task-foo-1-config]. Now compaction
* occurs and we're left with
* [connector-foo-config, task-foo-2-config, commit-foo (2 tasks), task-foo-1-config]. At the first commit, we don't
* have a complete set of configs. And because of the failure, there is no second commit. We are left in an inconsistent
* state with no obvious way to resolve the issue -- we can try to keep on reading, but the failed node may never
* recover and write the updated config. Meanwhile, other workers may have seen the entire log; they will see the second
* task-foo-1-config waiting to be applied, but will otherwise think everything is ok -- they have a valid set of task
* configs for connector "foo".
*
*
* Because we can encounter these inconsistencies and addressing them requires support from the rest of the system
* (resolving the task configuration inconsistencies requires support from the connector instance to regenerate updated
* configs), this class exposes not only the current set of configs, but also which connectors have inconsistent data.
* This allows users of this class (i.e., Herder implementations) to take action to resolve any inconsistencies. These
* inconsistencies should be rare (as described above, due to compaction combined with leader failures in the middle
* of updating task configurations).
*
*
* Note that the expectation is that this config storage system has only a single writer at a time.
* The caller (Herder) must ensure this is the case. In distributed mode this will require forwarding config change
* requests to the leader in the cluster (i.e. the worker group coordinated by the Kafka broker).
*
*
* Since processing of the config log occurs in a background thread, callers must take care when using accessors.
* To simplify handling this correctly, this class only exposes a mechanism to snapshot the current state of the cluster.
* Updates may continue to be applied (and callbacks invoked) in the background. Callers must take care that they are
* using a consistent snapshot and only update when it is safe. In particular, if task configs are updated which require
* synchronization across workers to commit offsets and update the configuration, callbacks and updates during the
* rebalance must be deferred.
*
*/
public class KafkaConfigBackingStore implements ConfigBackingStore {
private static final Logger log = LoggerFactory.getLogger(KafkaConfigBackingStore.class);
public static final String TARGET_STATE_PREFIX = "target-state-";
public static String TARGET_STATE_KEY(String connectorName) {
return TARGET_STATE_PREFIX + connectorName;
}
public static final String CONNECTOR_PREFIX = "connector-";
public static String CONNECTOR_KEY(String connectorName) {
return CONNECTOR_PREFIX + connectorName;
}
public static final String TASK_PREFIX = "task-";
public static String TASK_KEY(ConnectorTaskId taskId) {
return TASK_PREFIX + taskId.connector() + "-" + taskId.task();
}
public static final String COMMIT_TASKS_PREFIX = "commit-";
public static String COMMIT_TASKS_KEY(String connectorName) {
return COMMIT_TASKS_PREFIX + connectorName;
}
// Note that while using real serialization for values as we have here, but ad hoc string serialization for keys,
// isn't ideal, we use this approach because it avoids any potential problems with schema evolution or
// converter/serializer changes causing keys to change. We need to absolutely ensure that the keys remain precisely
// the same.
public static final Schema CONNECTOR_CONFIGURATION_V0 = SchemaBuilder.struct()
.field("properties", SchemaBuilder.map(Schema.STRING_SCHEMA, Schema.OPTIONAL_STRING_SCHEMA).build())
.build();
public static final Schema TASK_CONFIGURATION_V0 = CONNECTOR_CONFIGURATION_V0;
public static final Schema CONNECTOR_TASKS_COMMIT_V0 = SchemaBuilder.struct()
.field("tasks", Schema.INT32_SCHEMA)
.build();
public static final Schema TARGET_STATE_V0 = SchemaBuilder.struct()
.field("state", Schema.STRING_SCHEMA)
.build();
private static final long READ_TO_END_TIMEOUT_MS = 30000;
private final Object lock;
private final Converter converter;
private volatile boolean started;
// Although updateListener is not final, it's guaranteed to be visible to any thread after its
// initialization as long as we always read the volatile variable "started" before we access the listener.
private UpdateListener updateListener;
private final String topic;
// Data is passed to the log already serialized. We use a converter to handle translating to/from generic Connect
// format to serialized form
private final KafkaBasedLog configLog;
// Connector -> # of tasks
private final Map connectorTaskCounts = new HashMap<>();
// Connector and task configs: name or id -> config map
private final Map> connectorConfigs = new HashMap<>();
private final Map> taskConfigs = new HashMap<>();
// Set of connectors where we saw a task commit with an incomplete set of task config updates, indicating the data
// is in an inconsistent state and we cannot safely use them until they have been refreshed.
private final Set inconsistent = new HashSet<>();
// The most recently read offset. This does not take into account deferred task updates/commits, so we may have
// outstanding data to be applied.
private volatile long offset;
// Connector -> Map[ConnectorTaskId -> Configs]
private final Map>> deferredTaskUpdates = new HashMap<>();
private final Map connectorTargetStates = new HashMap<>();
private final WorkerConfigTransformer configTransformer;
public KafkaConfigBackingStore(Converter converter, WorkerConfig config, WorkerConfigTransformer configTransformer) {
this.lock = new Object();
this.started = false;
this.converter = converter;
this.offset = -1;
this.topic = config.getString(DistributedConfig.CONFIG_TOPIC_CONFIG);
if (this.topic == null || this.topic.trim().length() == 0)
throw new ConfigException("Must specify topic for connector configuration.");
configLog = setupAndCreateKafkaBasedLog(this.topic, config);
this.configTransformer = configTransformer;
}
@Override
public void setUpdateListener(UpdateListener listener) {
this.updateListener = listener;
}
@Override
public void start() {
log.info("Starting KafkaConfigBackingStore");
// Before startup, callbacks are *not* invoked. You can grab a snapshot after starting -- just take care that
// updates can continue to occur in the background
configLog.start();
started = true;
log.info("Started KafkaConfigBackingStore");
}
@Override
public void stop() {
log.info("Closing KafkaConfigBackingStore");
configLog.stop();
log.info("Closed KafkaConfigBackingStore");
}
/**
* Get a snapshot of the current state of the cluster.
*/
@Override
public ClusterConfigState snapshot() {
synchronized (lock) {
// Doing a shallow copy of the data is safe here because the complex nested data that is copied should all be
// immutable configs
return new ClusterConfigState(
offset,
new HashMap<>(connectorTaskCounts),
new HashMap<>(connectorConfigs),
new HashMap<>(connectorTargetStates),
new HashMap<>(taskConfigs),
new HashSet<>(inconsistent),
configTransformer
);
}
}
@Override
public boolean contains(String connector) {
synchronized (lock) {
return connectorConfigs.containsKey(connector);
}
}
/**
* Write this connector configuration to persistent storage and wait until it has been acknowledged and read back by
* tailing the Kafka log with a consumer.
*
* @param connector name of the connector to write data for
* @param properties the configuration to write
*/
@Override
public void putConnectorConfig(String connector, Map properties) {
log.debug("Writing connector configuration for connector '{}'", connector);
Struct connectConfig = new Struct(CONNECTOR_CONFIGURATION_V0);
connectConfig.put("properties", properties);
byte[] serializedConfig = converter.fromConnectData(topic, CONNECTOR_CONFIGURATION_V0, connectConfig);
updateConnectorConfig(connector, serializedConfig);
}
/**
* Remove configuration for a given connector.
* @param connector name of the connector to remove
*/
@Override
public void removeConnectorConfig(String connector) {
log.debug("Removing connector configuration for connector '{}'", connector);
try {
configLog.send(CONNECTOR_KEY(connector), null);
configLog.send(TARGET_STATE_KEY(connector), null);
configLog.readToEnd().get(READ_TO_END_TIMEOUT_MS, TimeUnit.MILLISECONDS);
} catch (InterruptedException | ExecutionException | TimeoutException e) {
log.error("Failed to remove connector configuration from Kafka: ", e);
throw new ConnectException("Error removing connector configuration from Kafka", e);
}
}
@Override
public void removeTaskConfigs(String connector) {
throw new UnsupportedOperationException("Removal of tasks is not currently supported");
}
private void updateConnectorConfig(String connector, byte[] serializedConfig) {
try {
configLog.send(CONNECTOR_KEY(connector), serializedConfig);
configLog.readToEnd().get(READ_TO_END_TIMEOUT_MS, TimeUnit.MILLISECONDS);
} catch (InterruptedException | ExecutionException | TimeoutException e) {
log.error("Failed to write connector configuration to Kafka: ", e);
throw new ConnectException("Error writing connector configuration to Kafka", e);
}
}
/**
* Write these task configurations and associated commit messages, unless an inconsistency is found that indicates
* that we would be leaving one of the referenced connectors with an inconsistent state.
*
* @param connector the connector to write task configuration
* @param configs list of task configurations for the connector
* @throws ConnectException if the task configurations do not resolve inconsistencies found in the existing root
* and task configurations.
*/
@Override
public void putTaskConfigs(String connector, List