KAFKA-2372: Add Kafka-backed storage of Copycat configs.

This also adds some other needed infrastructure for distributed Copycat, most
importantly the DistributedHerder, and refactors some code for handling
Kafka-backed logs into KafkaBasedLog since this is shared betweeen offset and
config storage.

Author: Ewen Cheslack-Postava <me@ewencp.org>

Reviewers: Gwen Shapira, James Cheng

Closes #241 from ewencp/kafka-2372-copycat-distributed-config
This commit is contained in:
Ewen Cheslack-Postava 2015-10-13 10:23:21 -07:00 committed by Gwen Shapira
parent e2ec02e1d1
commit 36d4469326
31 changed files with 3019 additions and 609 deletions

View File

@ -26,9 +26,11 @@ import java.nio.MappedByteBuffer;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.regex.Matcher;
import java.util.regex.Pattern;
import java.util.Properties;
@ -419,6 +421,17 @@ public class Utils {
return props;
}
/**
* Converts a Properties object to a Map<String, String>, calling {@link #toString} to ensure all keys and values
* are Strings.
*/
public static Map<String, String> propsToStringMap(Properties props) {
Map<String, String> result = new HashMap<>();
for (Map.Entry<Object, Object> entry : props.entrySet())
result.put(entry.getKey().toString(), entry.getValue().toString());
return result;
}
/**
* Get the stack trace from an exception as a string
*/

View File

@ -27,13 +27,14 @@ value.converter=org.apache.kafka.copycat.json.JsonConverter
key.converter.schemas.enable=true
value.converter.schemas.enable=true
# The offset converter is configurable and must be specified, but most users will always want to use the built-in default.
# Offset data is never visible outside of Copcyat.
offset.key.converter=org.apache.kafka.copycat.json.JsonConverter
offset.value.converter=org.apache.kafka.copycat.json.JsonConverter
offset.key.converter.schemas.enable=false
offset.value.converter.schemas.enable=false
# The internal converter used for offsets and config data is configurable and must be specified, but most users will
# always want to use the built-in default. Offset and config data is never visible outside of Copcyat in this format.
internal.key.converter=org.apache.kafka.copycat.json.JsonConverter
internal.value.converter=org.apache.kafka.copycat.json.JsonConverter
internal.key.converter.schemas.enable=false
internal.value.converter.schemas.enable=false
offset.storage.topic=copycat-offsets
# Flush much faster than normal, which is useful for testing/debugging
offset.flush.interval.ms=10000
config.storage.topic=copycat-configs

View File

@ -25,12 +25,12 @@ value.converter=org.apache.kafka.copycat.json.JsonConverter
key.converter.schemas.enable=true
value.converter.schemas.enable=true
# The offset converter is configurable and must be specified, but most users will always want to use the built-in default.
# Offset data is never visible outside of Copcyat.
offset.key.converter=org.apache.kafka.copycat.json.JsonConverter
offset.value.converter=org.apache.kafka.copycat.json.JsonConverter
offset.key.converter.schemas.enable=false
offset.value.converter.schemas.enable=false
# The internal converter used for offsets and config data is configurable and must be specified, but most users will
# always want to use the built-in default. Offset and config data is never visible outside of Copcyat in this format.
internal.key.converter=org.apache.kafka.copycat.json.JsonConverter
internal.value.converter=org.apache.kafka.copycat.json.JsonConverter
internal.key.converter.schemas.enable=false
internal.value.converter.schemas.enable=false
offset.storage.file.filename=/tmp/copycat.offsets
# Flush much faster than normal, which is useful for testing/debugging

View File

@ -20,9 +20,8 @@ package org.apache.kafka.copycat.cli;
import org.apache.kafka.common.annotation.InterfaceStability;
import org.apache.kafka.common.utils.Utils;
import org.apache.kafka.copycat.runtime.Copycat;
import org.apache.kafka.copycat.runtime.Herder;
import org.apache.kafka.copycat.runtime.Worker;
import org.apache.kafka.copycat.runtime.standalone.StandaloneHerder;
import org.apache.kafka.copycat.runtime.distributed.DistributedHerder;
import org.apache.kafka.copycat.storage.KafkaOffsetBackingStore;
import org.apache.kafka.copycat.util.Callback;
import org.apache.kafka.copycat.util.FutureCallback;
@ -59,7 +58,8 @@ public class CopycatDistributed {
WorkerConfig workerConfig = new WorkerConfig(workerProps);
Worker worker = new Worker(workerConfig, new KafkaOffsetBackingStore());
Herder herder = new StandaloneHerder(worker);
DistributedHerder herder = new DistributedHerder(worker);
herder.configure(workerConfig.originals());
final Copycat copycat = new Copycat(worker, herder);
copycat.start();
@ -73,7 +73,7 @@ public class CopycatDistributed {
log.error("Failed to create job for {}", connectorPropsFile);
}
});
herder.addConnector(connectorProps, cb);
herder.addConnector(Utils.propsToStringMap(connectorProps), cb);
cb.get();
}
} catch (Throwable t) {

View File

@ -75,7 +75,7 @@ public class CopycatStandalone {
log.error("Failed to create job for {}", connectorPropsFile);
}
});
herder.addConnector(connectorProps, cb);
herder.addConnector(Utils.propsToStringMap(connectorProps), cb);
cb.get();
}
} catch (Throwable t) {

View File

@ -57,13 +57,13 @@ public class WorkerConfig extends AbstractConfig {
public static final String VALUE_CONVERTER_CLASS_DOC =
"Converter class for value Copycat data that implements the <code>Converter</code> interface.";
public static final String OFFSET_KEY_CONVERTER_CLASS_CONFIG = "offset.key.converter";
public static final String OFFSET_KEY_CONVERTER_CLASS_DOC =
"Converter class for offset key Copycat data that implements the <code>Converter</code> interface.";
public static final String INTERNAL_KEY_CONVERTER_CLASS_CONFIG = "internal.key.converter";
public static final String INTERNAL_KEY_CONVERTER_CLASS_DOC =
"Converter class for internal key Copycat data that implements the <code>Converter</code> interface. Used for converting data like offsets and configs.";
public static final String OFFSET_VALUE_CONVERTER_CLASS_CONFIG = "offset.value.converter";
public static final String OFFSET_VALUE_CONVERTER_CLASS_DOC =
"Converter class for offset value Copycat data that implements the <code>Converter</code> interface.";
public static final String INTERNAL_VALUE_CONVERTER_CLASS_CONFIG = "internal.value.converter";
public static final String INTERNAL_VALUE_CONVERTER_CLASS_DOC =
"Converter class for offset value Copycat data that implements the <code>Converter</code> interface. Used for converting data like offsets and configs.";
public static final String TASK_SHUTDOWN_GRACEFUL_TIMEOUT_MS_CONFIG
= "task.shutdown.graceful.timeout.ms";
@ -95,10 +95,10 @@ public class WorkerConfig extends AbstractConfig {
Importance.HIGH, KEY_CONVERTER_CLASS_DOC)
.define(VALUE_CONVERTER_CLASS_CONFIG, Type.CLASS,
Importance.HIGH, VALUE_CONVERTER_CLASS_DOC)
.define(OFFSET_KEY_CONVERTER_CLASS_CONFIG, Type.CLASS,
Importance.HIGH, OFFSET_KEY_CONVERTER_CLASS_DOC)
.define(OFFSET_VALUE_CONVERTER_CLASS_CONFIG, Type.CLASS,
Importance.HIGH, OFFSET_VALUE_CONVERTER_CLASS_DOC)
.define(INTERNAL_KEY_CONVERTER_CLASS_CONFIG, Type.CLASS,
Importance.HIGH, INTERNAL_KEY_CONVERTER_CLASS_DOC)
.define(INTERNAL_VALUE_CONVERTER_CLASS_CONFIG, Type.CLASS,
Importance.HIGH, INTERNAL_VALUE_CONVERTER_CLASS_DOC)
.define(TASK_SHUTDOWN_GRACEFUL_TIMEOUT_MS_CONFIG, Type.LONG,
TASK_SHUTDOWN_GRACEFUL_TIMEOUT_MS_DEFAULT, Importance.LOW,
TASK_SHUTDOWN_GRACEFUL_TIMEOUT_MS_DOC)

View File

@ -22,7 +22,8 @@ import org.apache.kafka.common.config.ConfigDef;
import org.apache.kafka.common.config.ConfigDef.Importance;
import org.apache.kafka.common.config.ConfigDef.Type;
import java.util.Properties;
import java.util.HashMap;
import java.util.Map;
/**
* <p>
@ -64,10 +65,10 @@ public class ConnectorConfig extends AbstractConfig {
}
public ConnectorConfig() {
this(new Properties());
this(new HashMap<String, String>());
}
public ConnectorConfig(Properties props) {
public ConnectorConfig(Map<String, String> props) {
super(config, props);
}
}

View File

@ -19,7 +19,7 @@ package org.apache.kafka.copycat.runtime;
import org.apache.kafka.copycat.util.Callback;
import java.util.Properties;
import java.util.Map;
/**
* <p>
@ -53,15 +53,24 @@ public interface Herder {
* the leader herder if necessary.
*
* @param connectorProps user-specified properties for this job
* @param callback callback to invoke when the request completes
* @param callback callback to invoke when the request completes
*/
void addConnector(Properties connectorProps, Callback<String> callback);
void addConnector(Map<String, String> connectorProps, Callback<String> callback);
/**
* Delete a connector job by name.
*
* @param name name of the connector job to shutdown and delete
* @param name name of the connector job to shutdown and delete
* @param callback callback to invoke when the request completes
*/
void deleteConnector(String name, Callback<Void> callback);
}
/**
* Requests reconfiguration of the task. This should only be triggered by
* {@link HerderConnectorContext}.
*
* @param connName name of the connector that should be reconfigured
*/
void requestTaskReconfiguration(String connName);
}

View File

@ -15,7 +15,7 @@
* limitations under the License.
**/
package org.apache.kafka.copycat.runtime.standalone;
package org.apache.kafka.copycat.runtime;
import org.apache.kafka.copycat.connector.ConnectorContext;
@ -23,12 +23,12 @@ import org.apache.kafka.copycat.connector.ConnectorContext;
* ConnectorContext for use with the StandaloneHerder, which maintains all connectors and tasks
* in a single process.
*/
class StandaloneConnectorContext implements ConnectorContext {
public class HerderConnectorContext implements ConnectorContext {
private StandaloneHerder herder;
private Herder herder;
private String connectorName;
public StandaloneConnectorContext(StandaloneHerder herder, String connectorName) {
public HerderConnectorContext(Herder herder, String connectorName) {
this.herder = herder;
this.connectorName = connectorName;
}

View File

@ -52,8 +52,8 @@ public class Worker {
private WorkerConfig config;
private Converter keyConverter;
private Converter valueConverter;
private Converter offsetKeyConverter;
private Converter offsetValueConverter;
private Converter internalKeyConverter;
private Converter internalValueConverter;
private OffsetBackingStore offsetBackingStore;
private HashMap<ConnectorTaskId, WorkerTask> tasks = new HashMap<>();
private KafkaProducer<byte[], byte[]> producer;
@ -71,10 +71,10 @@ public class Worker {
this.keyConverter.configure(config.originalsWithPrefix("key.converter."), true);
this.valueConverter = config.getConfiguredInstance(WorkerConfig.VALUE_CONVERTER_CLASS_CONFIG, Converter.class);
this.valueConverter.configure(config.originalsWithPrefix("value.converter."), false);
this.offsetKeyConverter = config.getConfiguredInstance(WorkerConfig.OFFSET_KEY_CONVERTER_CLASS_CONFIG, Converter.class);
this.offsetKeyConverter.configure(config.originalsWithPrefix("offset.key.converter."), true);
this.offsetValueConverter = config.getConfiguredInstance(WorkerConfig.OFFSET_VALUE_CONVERTER_CLASS_CONFIG, Converter.class);
this.offsetValueConverter.configure(config.originalsWithPrefix("offset.value.converter."), false);
this.internalKeyConverter = config.getConfiguredInstance(WorkerConfig.INTERNAL_KEY_CONVERTER_CLASS_CONFIG, Converter.class);
this.internalKeyConverter.configure(config.originalsWithPrefix("internal.key.converter."), true);
this.internalValueConverter = config.getConfiguredInstance(WorkerConfig.INTERNAL_VALUE_CONVERTER_CLASS_CONFIG, Converter.class);
this.internalValueConverter.configure(config.originalsWithPrefix("internal.value.converter."), false);
this.offsetBackingStore = offsetBackingStore;
this.offsetBackingStore.configure(config.originals());
@ -157,9 +157,9 @@ public class Worker {
if (task instanceof SourceTask) {
SourceTask sourceTask = (SourceTask) task;
OffsetStorageReader offsetReader = new OffsetStorageReaderImpl(offsetBackingStore, id.connector(),
offsetKeyConverter, offsetValueConverter);
internalKeyConverter, internalValueConverter);
OffsetStorageWriter offsetWriter = new OffsetStorageWriter(offsetBackingStore, id.connector(),
offsetKeyConverter, offsetValueConverter);
internalKeyConverter, internalValueConverter);
workerTask = new WorkerSourceTask(id, sourceTask, keyConverter, valueConverter, producer,
offsetReader, offsetWriter, config, time);
} else if (task instanceof SinkTask) {
@ -201,4 +201,11 @@ public class Worker {
return task;
}
public Converter getInternalKeyConverter() {
return internalKeyConverter;
}
public Converter getInternalValueConverter() {
return internalValueConverter;
}
}

View File

@ -0,0 +1,122 @@
/**
* 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.copycat.runtime.distributed;
import org.apache.kafka.copycat.util.ConnectorTaskId;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import java.util.Set;
/**
* An immutable snapshot of the configuration state of connectors and tasks in a Copycat cluster.
*/
public class ClusterConfigState {
private final long offset;
private final Map<String, Integer> connectorTaskCounts;
private final Map<String, Map<String, String>> connectorConfigs;
private final Map<ConnectorTaskId, Map<String, String>> taskConfigs;
private final Set<String> inconsistentConnectors;
public ClusterConfigState(long offset,
Map<String, Integer> connectorTaskCounts,
Map<String, Map<String, String>> connectorConfigs,
Map<ConnectorTaskId, Map<String, String>> taskConfigs,
Set<String> inconsistentConnectors) {
this.offset = offset;
this.connectorTaskCounts = connectorTaskCounts;
this.connectorConfigs = connectorConfigs;
this.taskConfigs = taskConfigs;
this.inconsistentConnectors = inconsistentConnectors;
}
/**
* Get the last offset read to generate this config state. This offset is not guaranteed to be perfectly consistent
* with the recorded state because some partial updates to task configs may have been read.
* @return the latest config offset
*/
public long offset() {
return offset;
}
/**
* Get a list of the connectors in this configuration
*/
public Collection<String> connectors() {
return connectorTaskCounts.keySet();
}
/**
* Get the configuration for a connector.
* @param connector name of the connector
* @return a map containing configuration parameters
*/
public Map<String, String> connectorConfig(String connector) {
return connectorConfigs.get(connector);
}
/**
* Get the configuration for a task.
* @param task id of the task
* @return a map containing configuration parameters
*/
public Map<String, String> taskConfig(ConnectorTaskId task) {
return taskConfigs.get(task);
}
/**
* Get the current set of task IDs for the specified connector.
* @param connectorName the name of the connector to look up task configs for
* @return the current set of connector task IDs
*/
public Collection<ConnectorTaskId> tasks(String connectorName) {
if (inconsistentConnectors.contains(connectorName))
return Collections.EMPTY_LIST;
Integer numTasks = connectorTaskCounts.get(connectorName);
if (numTasks == null)
throw new IllegalArgumentException("Connector does not exist in current configuration.");
List<ConnectorTaskId> taskIds = new ArrayList<>();
for (int taskIndex = 0; taskIndex < numTasks; taskIndex++) {
ConnectorTaskId taskId = new ConnectorTaskId(connectorName, taskIndex);
taskIds.add(taskId);
}
return taskIds;
}
/**
* Get the set of connectors which have inconsistent data in this snapshot. These inconsistencies can occur due to
* partially completed writes combined with log compaction.
*
* Connectors in this set will appear in the output of {@link #connectors()} since their connector configuration is
* available, but not in the output of {@link #taskConfig(ConnectorTaskId)} since the task configs are incomplete.
*
* When a worker detects a connector in this state, it should request that the connector regenerate its task
* configurations.
*
* @return the set of inconsistent connectors
*/
public Set<String> inconsistentConnectors() {
return inconsistentConnectors;
}
}

View File

@ -0,0 +1,320 @@
/**
* 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.copycat.runtime.distributed;
import org.apache.kafka.common.utils.Utils;
import org.apache.kafka.copycat.connector.Connector;
import org.apache.kafka.copycat.errors.CopycatException;
import org.apache.kafka.copycat.runtime.ConnectorConfig;
import org.apache.kafka.copycat.runtime.Herder;
import org.apache.kafka.copycat.runtime.HerderConnectorContext;
import org.apache.kafka.copycat.runtime.Worker;
import org.apache.kafka.copycat.sink.SinkConnector;
import org.apache.kafka.copycat.sink.SinkTask;
import org.apache.kafka.copycat.storage.KafkaConfigStorage;
import org.apache.kafka.copycat.util.Callback;
import org.apache.kafka.copycat.util.ConnectorTaskId;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.util.*;
/**
* Distributed "herder" that coordinates with other workers to spread work across multiple processes.
*/
public class DistributedHerder implements Herder {
private static final Logger log = LoggerFactory.getLogger(DistributedHerder.class);
private Worker worker;
private KafkaConfigStorage configStorage;
private ClusterConfigState configState;
private HashMap<String, ConnectorState> connectors = new HashMap<>();
public DistributedHerder(Worker worker) {
this.worker = worker;
this.configStorage = new KafkaConfigStorage(worker.getInternalValueConverter(),
new ConnectorConfigCallback(), new TaskConfigCallback());
}
// Public for testing (mock KafkaConfigStorage)
public DistributedHerder(Worker worker, KafkaConfigStorage configStorage) {
this.worker = worker;
this.configStorage = configStorage;
}
public synchronized void configure(Map<String, ?> configs) {
configStorage.configure(configs);
}
public synchronized void start() {
log.info("Herder starting");
configStorage.start();
log.info("Restoring connectors from stored configs");
restoreConnectors();
log.info("Herder started");
}
public synchronized void stop() {
log.info("Herder stopping");
// There's no coordination/hand-off to do here since this is all standalone. Instead, we
// should just clean up the stuff we normally would, i.e. cleanly checkpoint and shutdown all
// the tasks.
for (Map.Entry<String, ConnectorState> entry : connectors.entrySet()) {
ConnectorState state = entry.getValue();
stopConnector(state);
}
connectors.clear();
if (configStorage != null) {
configStorage.stop();
configStorage = null;
}
log.info("Herder stopped");
}
@Override
public synchronized void addConnector(Map<String, String> connectorProps,
Callback<String> callback) {
try {
// Ensure the config is written to storage first
ConnectorConfig connConfig = new ConnectorConfig(connectorProps);
String connName = connConfig.getString(ConnectorConfig.NAME_CONFIG);
configStorage.putConnectorConfig(connName, connectorProps);
ConnectorState connState = createConnector(connConfig);
if (callback != null)
callback.onCompletion(null, connState.name);
// This should always be a new job, create jobs from scratch
createConnectorTasks(connState);
} catch (CopycatException e) {
if (callback != null)
callback.onCompletion(e, null);
}
}
@Override
public synchronized void deleteConnector(String name, Callback<Void> callback) {
try {
destroyConnector(name);
if (callback != null)
callback.onCompletion(null, null);
} catch (CopycatException e) {
if (callback != null)
callback.onCompletion(e, null);
}
}
@Override
public synchronized void requestTaskReconfiguration(String connName) {
ConnectorState state = connectors.get(connName);
if (state == null) {
log.error("Task that requested reconfiguration does not exist: {}", connName);
return;
}
updateConnectorTasks(state);
}
// Creates and configures the connector. Does not setup any tasks
private ConnectorState createConnector(ConnectorConfig connConfig) {
String connName = connConfig.getString(ConnectorConfig.NAME_CONFIG);
String className = connConfig.getString(ConnectorConfig.CONNECTOR_CLASS_CONFIG);
log.info("Creating connector {} of type {}", connName, className);
int maxTasks = connConfig.getInt(ConnectorConfig.TASKS_MAX_CONFIG);
List<String> topics = connConfig.getList(ConnectorConfig.TOPICS_CONFIG); // Sinks only
Properties configs = connConfig.unusedProperties();
if (connectors.containsKey(connName)) {
log.error("Ignoring request to create connector due to conflicting connector name");
throw new CopycatException("Connector with name " + connName + " already exists");
}
final Connector connector;
try {
connector = instantiateConnector(className);
} catch (Throwable t) {
// Catches normal exceptions due to instantiation errors as well as any runtime errors that
// may be caused by user code
throw new CopycatException("Failed to create connector instance", t);
}
connector.initialize(new HerderConnectorContext(this, connName));
try {
connector.start(configs);
} catch (CopycatException e) {
throw new CopycatException("Connector threw an exception while starting", e);
}
ConnectorState state = new ConnectorState(connName, connector, maxTasks, topics);
connectors.put(connName, state);
log.info("Finished creating connector {}", connName);
return state;
}
private static Connector instantiateConnector(String className) {
try {
return Utils.newInstance(className, Connector.class);
} catch (ClassNotFoundException e) {
throw new CopycatException("Couldn't instantiate connector class", e);
}
}
private void destroyConnector(String connName) {
log.info("Destroying connector {}", connName);
ConnectorState state = connectors.get(connName);
if (state == null) {
log.error("Failed to destroy connector {} because it does not exist", connName);
throw new CopycatException("Connector does not exist");
}
stopConnector(state);
configStorage.putConnectorConfig(state.name, null);
connectors.remove(state.name);
log.info("Finished destroying connector {}", connName);
}
// Stops a connectors tasks, then the connector
private void stopConnector(ConnectorState state) {
removeConnectorTasks(state);
try {
state.connector.stop();
} catch (CopycatException e) {
log.error("Error shutting down connector {}: ", state.connector, e);
}
}
private void createConnectorTasks(ConnectorState state) {
String taskClassName = state.connector.taskClass().getName();
log.info("Creating tasks for connector {} of type {}", state.name, taskClassName);
List<Properties> taskConfigs = state.connector.taskConfigs(state.maxTasks);
// Generate the final configs, including framework provided settings
Map<ConnectorTaskId, Properties> taskProps = new HashMap<>();
for (int i = 0; i < taskConfigs.size(); i++) {
ConnectorTaskId taskId = new ConnectorTaskId(state.name, i);
Properties config = taskConfigs.get(i);
// TODO: This probably shouldn't be in the Herder. It's nice to have Copycat ensure the list of topics
// is automatically provided to tasks since it is required by the framework, but this
String subscriptionTopics = Utils.join(state.inputTopics, ",");
if (state.connector instanceof SinkConnector) {
// Make sure we don't modify the original since the connector may reuse it internally
Properties configForSink = new Properties();
configForSink.putAll(config);
configForSink.setProperty(SinkTask.TOPICS_CONFIG, subscriptionTopics);
config = configForSink;
}
taskProps.put(taskId, config);
}
// And initiate the tasks
for (int i = 0; i < taskConfigs.size(); i++) {
ConnectorTaskId taskId = new ConnectorTaskId(state.name, i);
Properties config = taskProps.get(taskId);
try {
worker.addTask(taskId, taskClassName, config);
// We only need to store the task IDs so we can clean up.
state.tasks.add(taskId);
} catch (Throwable e) {
log.error("Failed to add task {}: ", taskId, e);
// Swallow this so we can continue updating the rest of the tasks
// FIXME what's the proper response? Kill all the tasks? Consider this the same as a task
// that died after starting successfully.
}
}
}
private void removeConnectorTasks(ConnectorState state) {
Iterator<ConnectorTaskId> taskIter = state.tasks.iterator();
while (taskIter.hasNext()) {
ConnectorTaskId taskId = taskIter.next();
try {
worker.stopTask(taskId);
taskIter.remove();
} catch (CopycatException e) {
log.error("Failed to stop task {}: ", taskId, e);
// Swallow this so we can continue stopping the rest of the tasks
// FIXME: Forcibly kill the task?
}
}
}
private void updateConnectorTasks(ConnectorState state) {
removeConnectorTasks(state);
createConnectorTasks(state);
}
private void restoreConnectors() {
configState = configStorage.snapshot();
Collection<String> connNames = configState.connectors();
for (String connName : connNames) {
log.info("Restoring connector {}", connName);
Map<String, String> connProps = configState.connectorConfig(connName);
ConnectorConfig connConfig = new ConnectorConfig(connProps);
ConnectorState connState = createConnector(connConfig);
// Because this coordinator is standalone, connectors are only restored when this process
// starts and we know there can't be any existing tasks. So in this special case we're able
// to just create the tasks rather than having to check for existing tasks and sort out
// whether they need to be reconfigured.
createConnectorTasks(connState);
}
}
private static class ConnectorState {
public String name;
public Connector connector;
public int maxTasks;
public List<String> inputTopics;
Set<ConnectorTaskId> tasks;
public ConnectorState(String name, Connector connector, int maxTasks,
List<String> inputTopics) {
this.name = name;
this.connector = connector;
this.maxTasks = maxTasks;
this.inputTopics = inputTopics;
this.tasks = new HashSet<>();
}
}
private class ConnectorConfigCallback implements Callback<String> {
@Override
public void onCompletion(Throwable error, String result) {
configState = configStorage.snapshot();
// FIXME
}
}
private class TaskConfigCallback implements Callback<List<ConnectorTaskId>> {
@Override
public void onCompletion(Throwable error, List<ConnectorTaskId> result) {
configState = configStorage.snapshot();
// FIXME
}
}
}

View File

@ -22,6 +22,7 @@ import org.apache.kafka.copycat.connector.Connector;
import org.apache.kafka.copycat.errors.CopycatException;
import org.apache.kafka.copycat.runtime.ConnectorConfig;
import org.apache.kafka.copycat.runtime.Herder;
import org.apache.kafka.copycat.runtime.HerderConnectorContext;
import org.apache.kafka.copycat.runtime.Worker;
import org.apache.kafka.copycat.sink.SinkConnector;
import org.apache.kafka.copycat.sink.SinkTask;
@ -65,7 +66,7 @@ public class StandaloneHerder implements Herder {
}
@Override
public synchronized void addConnector(Properties connectorProps,
public synchronized void addConnector(Map<String, String> connectorProps,
Callback<String> callback) {
try {
ConnectorState connState = createConnector(connectorProps);
@ -91,8 +92,18 @@ public class StandaloneHerder implements Herder {
}
}
// Creates the and configures the connector. Does not setup any tasks
private ConnectorState createConnector(Properties connectorProps) {
@Override
public synchronized void requestTaskReconfiguration(String connName) {
ConnectorState state = connectors.get(connName);
if (state == null) {
log.error("Task that requested reconfiguration does not exist: {}", connName);
return;
}
updateConnectorTasks(state);
}
// Creates and configures the connector. Does not setup any tasks
private ConnectorState createConnector(Map<String, String> connectorProps) {
ConnectorConfig connConfig = new ConnectorConfig(connectorProps);
String connName = connConfig.getString(ConnectorConfig.NAME_CONFIG);
String className = connConfig.getString(ConnectorConfig.CONNECTOR_CLASS_CONFIG);
@ -114,7 +125,7 @@ public class StandaloneHerder implements Herder {
// may be caused by user code
throw new CopycatException("Failed to create connector instance", t);
}
connector.initialize(new StandaloneConnectorContext(this, connName));
connector.initialize(new HerderConnectorContext(this, connName));
try {
connector.start(configs);
} catch (CopycatException e) {
@ -222,21 +233,6 @@ public class StandaloneHerder implements Herder {
createConnectorTasks(state);
}
/**
* Requests reconfiguration of the task. This should only be triggered by
* {@link StandaloneConnectorContext}.
*
* @param connName name of the connector that should be reconfigured
*/
public synchronized void requestTaskReconfiguration(String connName) {
ConnectorState state = connectors.get(connName);
if (state == null) {
log.error("Task that requested reconfiguration does not exist: {}", connName);
return;
}
updateConnectorTasks(state);
}
private static class ConnectorState {
public String name;

View File

@ -0,0 +1,546 @@
/**
* 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
* <p/>
* http://www.apache.org/licenses/LICENSE-2.0
* <p/>
* 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.copycat.storage;
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.utils.SystemTime;
import org.apache.kafka.common.utils.Utils;
import org.apache.kafka.copycat.data.Schema;
import org.apache.kafka.copycat.data.SchemaAndValue;
import org.apache.kafka.copycat.data.SchemaBuilder;
import org.apache.kafka.copycat.data.Struct;
import org.apache.kafka.copycat.errors.CopycatException;
import org.apache.kafka.copycat.errors.DataException;
import org.apache.kafka.copycat.runtime.distributed.ClusterConfigState;
import org.apache.kafka.copycat.util.Callback;
import org.apache.kafka.copycat.util.ConnectorTaskId;
import org.apache.kafka.copycat.util.KafkaBasedLog;
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;
/**
* <p>
* Provides persistent storage of Copycat connector configurations in a Kafka topic.
* </p>
* <p>
* This class manages both connector and task configurations. It tracks three types of configuration entries:
* <p/>
* 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 Copycat
* 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".
* </p>
* <p>
* 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.
* </p>
* <p>
* 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.
* </p>
* <p>
* 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.
* </p>
* <p>
* 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.
* </p>
* <p>
* 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".
* </p>
* <p>
* 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).
* </p>
* <p>
* 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).
* </p>
* <p>
* 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.
* </p>
*/
public class KafkaConfigStorage {
private static final Logger log = LoggerFactory.getLogger(KafkaConfigStorage.class);
public static final String CONFIG_TOPIC_CONFIG = "config.storage.topic";
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();
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();
private static final long READ_TO_END_TIMEOUT_MS = 30000;
private final Object lock;
private boolean starting;
private final Converter converter;
private final Callback<String> connectorConfigCallback;
private final Callback<List<ConnectorTaskId>> tasksConfigCallback;
private String topic;
// Data is passed to the log already serialized. We use a converter to handle translating to/from generic Copycat
// format to serialized form
private KafkaBasedLog<String, byte[]> configLog;
// Connector -> # of tasks
private Map<String, Integer> connectorTaskCounts = new HashMap<>();
// Connector and task configs: name or id -> config map
private Map<String, Map<String, String>> connectorConfigs = new HashMap<>();
private Map<ConnectorTaskId, Map<String, String>> 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 Set<String> 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 long offset;
// Connector -> Map[ConnectorTaskId -> Configs]
private Map<String, Map<ConnectorTaskId, Map<String, String>>> deferredTaskUpdates = new HashMap<>();
public KafkaConfigStorage(Converter converter, Callback<String> connectorConfigCallback, Callback<List<ConnectorTaskId>> tasksConfigCallback) {
this.lock = new Object();
this.starting = false;
this.converter = converter;
this.connectorConfigCallback = connectorConfigCallback;
this.tasksConfigCallback = tasksConfigCallback;
offset = -1;
}
public void configure(Map<String, ?> configs) {
if (configs.get(CONFIG_TOPIC_CONFIG) == null)
throw new CopycatException("Must specify topic for Copycat connector configuration.");
topic = (String) configs.get(CONFIG_TOPIC_CONFIG);
Map<String, Object> producerProps = new HashMap<>();
producerProps.putAll(configs);
producerProps.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.StringSerializer");
producerProps.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer");
producerProps.put(ProducerConfig.ACKS_CONFIG, "all");
Map<String, Object> consumerProps = new HashMap<>();
consumerProps.putAll(configs);
consumerProps.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.StringDeserializer");
consumerProps.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArrayDeserializer");
configLog = createKafkaBasedLog(topic, producerProps, consumerProps, consumedCallback);
}
public void start() {
log.info("Starting KafkaConfigStorage");
// During startup, callbacks are *not* invoked. You can grab a snapshot after starting -- just take care that
// updates can continue to occur in the background
starting = true;
configLog.start();
starting = false;
log.info("Started KafkaConfigStorage");
}
public void stop() {
log.info("Closing KafkaConfigStorage");
configLog.stop();
log.info("Closed KafkaConfigStorage");
}
/**
* Get a snapshot of the current state of the cluster.
*/
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<>(taskConfigs),
new HashSet<>(inconsistent)
);
}
}
/**
* Write this connector configuration to persistent storage and wait until it has been acknowledge 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
*/
public void putConnectorConfig(String connector, Map<String, String> properties) {
Struct copycatConfig = new Struct(CONNECTOR_CONFIGURATION_V0);
copycatConfig.put("properties", properties);
byte[] serializedConfig = converter.fromCopycatData(topic, CONNECTOR_CONFIGURATION_V0, copycatConfig);
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 CopycatException("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 configs map containing task configurations
* @throws CopycatException if the task configurations do not resolve inconsistencies found in the existing root
* and task configurations.
*/
public void putTaskConfigs(Map<ConnectorTaskId, Map<String, String>> configs) {
// Make sure we're at the end of the log. We should be the only writer, but we want to make sure we don't have
// any outstanding lagging data to consume.
try {
configLog.readToEnd().get(READ_TO_END_TIMEOUT_MS, TimeUnit.MILLISECONDS);
} catch (InterruptedException | ExecutionException | TimeoutException e) {
log.error("Failed to write root configuration to Kafka: ", e);
throw new CopycatException("Error writing root configuration to Kafka", e);
}
// In theory, there is only a single writer and we shouldn't need this lock since the background thread should
// not invoke any callbacks that would conflict, but in practice this guards against inconsistencies due to
// the root config being updated.
Map<String, Integer> newTaskCounts = new HashMap<>();
synchronized (lock) {
// Validate tasks in this assignment. Any task configuration updates should include updates for *all* tasks
// in the connector -- we should have all task IDs 0 - N-1 within a connector if any task is included here
Map<String, Set<Integer>> updatedConfigIdsByConnector = taskIdsByConnector(configs);
for (Map.Entry<String, Set<Integer>> taskConfigSetEntry : updatedConfigIdsByConnector.entrySet()) {
if (!completeTaskIdSet(taskConfigSetEntry.getValue(), taskConfigSetEntry.getValue().size())) {
log.error("Submitted task configuration contain invalid range of task IDs, ignoring this submission");
throw new CopycatException("Error writing task configurations: found some connectors with invalid connectors");
}
newTaskCounts.put(taskConfigSetEntry.getKey(), taskConfigSetEntry.getValue().size());
}
}
// Start sending all the individual updates
for (Map.Entry<ConnectorTaskId, Map<String, String>> taskConfigEntry : configs.entrySet()) {
Struct copycatConfig = new Struct(TASK_CONFIGURATION_V0);
copycatConfig.put("properties", taskConfigEntry.getValue());
byte[] serializedConfig = converter.fromCopycatData(topic, TASK_CONFIGURATION_V0, copycatConfig);
configLog.send(TASK_KEY(taskConfigEntry.getKey()), serializedConfig);
}
// Finally, send the commit to update the number of tasks and apply the new configs, then wait until we read to
// the end of the log
try {
// Read to end to ensure all the task configs have been written
configLog.readToEnd().get(READ_TO_END_TIMEOUT_MS, TimeUnit.MILLISECONDS);
// Write all the commit messages
for (Map.Entry<String, Integer> taskCountEntry : newTaskCounts.entrySet()) {
Struct copycatConfig = new Struct(CONNECTOR_TASKS_COMMIT_V0);
copycatConfig.put("tasks", taskCountEntry.getValue());
byte[] serializedConfig = converter.fromCopycatData(topic, CONNECTOR_TASKS_COMMIT_V0, copycatConfig);
configLog.send(COMMIT_TASKS_KEY(taskCountEntry.getKey()), serializedConfig);
}
// Read to end to ensure all the commit messages have been written
configLog.readToEnd().get(READ_TO_END_TIMEOUT_MS, TimeUnit.MILLISECONDS);
} catch (InterruptedException | ExecutionException | TimeoutException e) {
log.error("Failed to write root configuration to Kafka: ", e);
throw new CopycatException("Error writing root configuration to Kafka", e);
}
}
private KafkaBasedLog<String, byte[]> createKafkaBasedLog(String topic, Map<String, Object> producerProps,
Map<String, Object> consumerProps, Callback<ConsumerRecord<String, byte[]>> consumedCallback) {
return new KafkaBasedLog<>(topic, producerProps, consumerProps, consumedCallback, new SystemTime());
}
private final Callback<ConsumerRecord<String, byte[]>> consumedCallback = new Callback<ConsumerRecord<String, byte[]>>() {
@Override
public void onCompletion(Throwable error, ConsumerRecord<String, byte[]> record) {
if (error != null) {
log.error("Unexpected in consumer callback for KafkaConfigStorage: ", error);
return;
}
final SchemaAndValue value;
try {
value = converter.toCopycatData(topic, record.value());
} catch (DataException e) {
log.error("Failed to convert config data to Copycat format: ", e);
return;
}
offset = record.offset();
if (record.key().startsWith(CONNECTOR_PREFIX)) {
String connectorName = record.key().substring(CONNECTOR_PREFIX.length());
synchronized (lock) {
// Connector configs can be applied and callbacks invoked immediately
if (!(value.value() instanceof Map)) {
log.error("Found connector configuration (" + record.key() + ") in wrong format: " + value.value().getClass());
return;
}
Object newConnectorConfig = ((Map<String, Object>) value.value()).get("properties");
if (!(newConnectorConfig instanceof Map)) {
log.error("Invalid data for connector config: properties filed should be a Map but is " + newConnectorConfig.getClass());
return;
}
connectorConfigs.put(connectorName, (Map<String, String>) newConnectorConfig);
}
if (!starting)
connectorConfigCallback.onCompletion(null, connectorName);
} else if (record.key().startsWith(TASK_PREFIX)) {
synchronized (lock) {
ConnectorTaskId taskId = parseTaskId(record.key());
if (taskId == null) {
log.error("Ignoring task configuration because " + record.key() + " couldn't be parsed as a task config key");
return;
}
if (!(value.value() instanceof Map)) {
log.error("Ignoring task configuration because it is in the wrong format: " + value.value());
return;
}
Object newTaskConfig = ((Map<String, Object>) value.value()).get("properties");
if (!(newTaskConfig instanceof Map)) {
log.error("Invalid data for task config: properties filed should be a Map but is " + newTaskConfig.getClass());
return;
}
Map<ConnectorTaskId, Map<String, String>> deferred = deferredTaskUpdates.get(taskId.connector());
if (deferred == null) {
deferred = new HashMap<>();
deferredTaskUpdates.put(taskId.connector(), deferred);
}
deferred.put(taskId, (Map<String, String>) newTaskConfig);
}
} else if (record.key().startsWith(COMMIT_TASKS_PREFIX)) {
String connectorName = record.key().substring(COMMIT_TASKS_PREFIX.length());
List<ConnectorTaskId> updatedTasks = new ArrayList<>();
synchronized (lock) {
// Apply any outstanding deferred task updates for the given connector. Note that just because we
// encounter a commit message does not mean it will result in consistent output. In particular due to
// compaction, there may be cases where . For example if we have the following sequence of writes:
//
// 1. Write connector "foo"'s config
// 2. Write connector "foo", task 1's config <-- compacted
// 3. Write connector "foo", task 2's config
// 4. Write connector "foo" task commit message
// 5. Write connector "foo", task 1's config
// 6. Write connector "foo", task 2's config
// 7. Write connector "foo" task commit message
//
// then when a new worker starts up, if message 2 had been compacted, then when message 4 is applied
// "foo" will not have a complete set of configs. Only when message 7 is applied will the complete
// configuration be available. Worse, if the leader died while writing messages 5, 6, and 7 such that
// only 5 was written, then there may be nothing that will finish writing the configs and get the
// log back into a consistent state.
//
// It is expected that the user of this class (i.e., the Herder) will take the necessary action to
// resolve this (i.e., get the connector to recommit its configuration). This inconsistent state is
// exposed in the snapshots provided via ClusterConfigState so they are easy to handle.
if (!(value.value() instanceof Map)) { // Schema-less, so we get maps instead of structs
log.error("Ignoring connector tasks configuration commit because it is in the wrong format: " + value.value());
return;
}
Map<ConnectorTaskId, Map<String, String>> deferred = deferredTaskUpdates.get(connectorName);
Object newTaskCountObj = ((Map<String, Object>) value.value()).get("tasks");
Integer newTaskCount = (Integer) newTaskCountObj;
// Validate the configs we're supposed to update to ensure we're getting a complete configuration
// update of all tasks that are expected based on the number of tasks in the commit message.
Map<String, Set<Integer>> updatedConfigIdsByConnector = taskIdsByConnector(deferred);
Set<Integer> taskIdSet = updatedConfigIdsByConnector.get(connectorName);
if (!completeTaskIdSet(taskIdSet, newTaskCount)) {
// Given the logic for writing commit messages, we should only hit this condition due to compacted
// historical data, in which case we would not have applied any updates yet and there will be no
// task config data already committed for the connector, so we shouldn't have to clear any data
// out. All we need to do is add the flag marking it inconsistent.
inconsistent.add(connectorName);
} else {
if (deferred != null) {
taskConfigs.putAll(deferred);
updatedTasks.addAll(taskConfigs.keySet());
}
inconsistent.remove(connectorName);
}
// Always clear the deferred entries, even if we didn't apply them. If they represented an inconsistent
// update, then we need to see a completely fresh set of configs after this commit message, so we don't
// want any of these outdated configs
if (deferred != null)
deferred.clear();
connectorTaskCounts.put(connectorName, newTaskCount);
}
if (!starting)
tasksConfigCallback.onCompletion(null, updatedTasks);
} else {
log.error("Discarding config update record with invalid key: " + record.key());
}
}
};
private ConnectorTaskId parseTaskId(String key) {
String[] parts = key.split("-");
if (parts.length < 3) return null;
try {
int taskNum = Integer.parseInt(parts[parts.length - 1]);
String connectorName = Utils.join(Arrays.copyOfRange(parts, 1, parts.length - 1), "-");
return new ConnectorTaskId(connectorName, taskNum);
} catch (NumberFormatException e) {
return null;
}
}
/**
* Given task configurations, get a set of integer task IDs organized by connector name.
*/
private Map<String, Set<Integer>> taskIdsByConnector(Map<ConnectorTaskId, Map<String, String>> configs) {
Map<String, Set<Integer>> connectorTaskIds = new HashMap<>();
if (configs == null)
return connectorTaskIds;
for (Map.Entry<ConnectorTaskId, Map<String, String>> taskConfigEntry : configs.entrySet()) {
ConnectorTaskId taskId = taskConfigEntry.getKey();
if (!connectorTaskIds.containsKey(taskId.connector()))
connectorTaskIds.put(taskId.connector(), new TreeSet<Integer>());
connectorTaskIds.get(taskId.connector()).add(taskId.task());
}
return connectorTaskIds;
}
private boolean completeTaskIdSet(Set<Integer> idSet, int expectedSize) {
// Note that we do *not* check for the exact set. This is an important implication of compaction. If we start out
// with 2 tasks, then reduce to 1, we'll end up with log entries like:
//
// 1. Connector "foo" config
// 2. Connector "foo", task 1 config
// 3. Connector "foo", task 2 config
// 4. Connector "foo", commit 2 tasks
// 5. Connector "foo", task 1 config
// 6. Connector "foo", commit 1 tasks
//
// However, due to compaction we could end up with a log that looks like this:
//
// 1. Connector "foo" config
// 3. Connector "foo", task 2 config
// 5. Connector "foo", task 1 config
// 6. Connector "foo", commit 1 tasks
//
// which isn't incorrect, but would appear in this code to have an extra task configuration. Instead, we just
// validate that all the configs specified by the commit message are present. This should be fine because the
// logic for writing configs ensures all the task configs are written (and reads them back) before writing the
// commit message.
if (idSet.size() < expectedSize)
return false;
for (int i = 0; i < expectedSize; i++)
if (!idSet.contains(i))
return false;
return true;
}
}

View File

@ -17,39 +17,22 @@
package org.apache.kafka.copycat.storage;
import org.apache.kafka.clients.consumer.Consumer;
import org.apache.kafka.clients.consumer.ConsumerConfig;
import org.apache.kafka.clients.consumer.ConsumerRecord;
import org.apache.kafka.clients.consumer.ConsumerRecords;
import org.apache.kafka.clients.consumer.ConsumerWakeupException;
import org.apache.kafka.clients.consumer.KafkaConsumer;
import org.apache.kafka.clients.producer.KafkaProducer;
import org.apache.kafka.clients.producer.Producer;
import org.apache.kafka.clients.producer.ProducerConfig;
import org.apache.kafka.clients.producer.ProducerRecord;
import org.apache.kafka.clients.producer.RecordMetadata;
import org.apache.kafka.common.KafkaException;
import org.apache.kafka.common.PartitionInfo;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.utils.SystemTime;
import org.apache.kafka.common.utils.Time;
import org.apache.kafka.common.utils.Utils;
import org.apache.kafka.copycat.errors.CopycatException;
import org.apache.kafka.copycat.util.Callback;
import org.apache.kafka.copycat.util.ConvertingFutureCallback;
import org.apache.kafka.copycat.util.KafkaBasedLog;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.nio.ByteBuffer;
import java.util.ArrayDeque;
import java.util.ArrayList;
import java.util.Collection;
import java.util.HashMap;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Queue;
import java.util.Set;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.Future;
import java.util.concurrent.TimeUnit;
@ -70,99 +53,42 @@ public class KafkaOffsetBackingStore implements OffsetBackingStore {
public final static String OFFSET_STORAGE_TOPIC_CONFIG = "offset.storage.topic";
private final static long CREATE_TOPIC_TIMEOUT_MS = 30000;
private Time time;
private Map<String, ?> configs;
private String topic;
private Consumer<byte[], byte[]> consumer;
private Producer<byte[], byte[]> producer;
private KafkaBasedLog<byte[], byte[]> offsetLog;
private HashMap<ByteBuffer, ByteBuffer> data;
private Thread thread;
private boolean stopRequested;
private Queue<Callback<Void>> readLogEndOffsetCallbacks;
public KafkaOffsetBackingStore() {
this(new SystemTime());
}
public KafkaOffsetBackingStore(Time time) {
this.time = time;
}
@Override
public void configure(Map<String, ?> configs) {
this.configs = configs;
topic = (String) configs.get(OFFSET_STORAGE_TOPIC_CONFIG);
String topic = (String) configs.get(OFFSET_STORAGE_TOPIC_CONFIG);
if (topic == null)
throw new CopycatException("Offset storage topic must be specified");
data = new HashMap<>();
stopRequested = false;
readLogEndOffsetCallbacks = new ArrayDeque<>();
Map<String, Object> producerProps = new HashMap<>();
producerProps.putAll(configs);
producerProps.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer");
producerProps.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer");
Map<String, Object> consumerProps = new HashMap<>();
consumerProps.putAll(configs);
consumerProps.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArrayDeserializer");
consumerProps.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArrayDeserializer");
offsetLog = createKafkaBasedLog(topic, producerProps, consumerProps, consumedCallback);
}
@Override
public void start() {
log.info("Starting KafkaOffsetBackingStore with topic " + topic);
producer = createProducer();
consumer = createConsumer();
List<TopicPartition> partitions = new ArrayList<>();
// Until we have admin utilities we can use to check for the existence of this topic and create it if it is missing,
// we rely on topic auto-creation
List<PartitionInfo> partitionInfos = null;
long started = time.milliseconds();
while (partitionInfos == null && time.milliseconds() - started < CREATE_TOPIC_TIMEOUT_MS) {
partitionInfos = consumer.partitionsFor(topic);
Utils.sleep(Math.min(time.milliseconds() - started, 1000));
}
if (partitionInfos == null)
throw new CopycatException("Could not look up partition metadata for offset backing store topic in" +
" allotted period. This could indicate a connectivity issue, unavailable topic partitions, or if" +
" this is your first use of the topic it may have taken too long to create.");
for (PartitionInfo partition : partitionInfos)
partitions.add(new TopicPartition(partition.topic(), partition.partition()));
consumer.assign(partitions);
readToLogEnd();
thread = new WorkThread();
thread.start();
log.info("Starting KafkaOffsetBackingStore");
offsetLog.start();
log.info("Finished reading offsets topic and starting KafkaOffsetBackingStore");
}
@Override
public void stop() {
log.info("Stopping KafkaOffsetBackingStore");
synchronized (this) {
stopRequested = true;
consumer.wakeup();
}
try {
thread.join();
} catch (InterruptedException e) {
throw new CopycatException("Failed to stop KafkaOffsetBackingStore. Exiting without cleanly shutting " +
"down it's producer and consumer.", e);
}
try {
producer.close();
} catch (KafkaException e) {
log.error("Failed to close KafkaOffsetBackingStore producer", e);
}
try {
consumer.close();
} catch (KafkaException e) {
log.error("Failed to close KafkaOffsetBackingStore consumer", e);
}
offsetLog.stop();
log.info("Stopped KafkaOffsetBackingStore");
}
@Override
@ -172,15 +98,16 @@ public class KafkaOffsetBackingStore implements OffsetBackingStore {
@Override
public Map<ByteBuffer, ByteBuffer> convert(Void result) {
Map<ByteBuffer, ByteBuffer> values = new HashMap<>();
synchronized (KafkaOffsetBackingStore.this) {
for (ByteBuffer key : keys)
values.put(key, data.get(key));
}
for (ByteBuffer key : keys)
values.put(key, data.get(key));
return values;
}
};
readLogEndOffsetCallbacks.add(future);
consumer.wakeup();
// This operation may be relatively (but not too) expensive since it always requires checking end offsets, even
// if we've already read up to the end. However, it also should not be common (offsets should only be read when
// resetting a task). Always requiring that we read to the end is simpler than trying to differentiate when it
// is safe not to (which should only be if we *know* we've maintained ownership since the last write).
offsetLog.readToEnd(future);
return future;
}
@ -188,95 +115,26 @@ public class KafkaOffsetBackingStore implements OffsetBackingStore {
public Future<Void> set(final Map<ByteBuffer, ByteBuffer> values, final Callback<Void> callback) {
SetCallbackFuture producerCallback = new SetCallbackFuture(values.size(), callback);
for (Map.Entry<ByteBuffer, ByteBuffer> entry : values.entrySet()) {
producer.send(new ProducerRecord<>(topic, entry.getKey().array(), entry.getValue().array()), producerCallback);
}
for (Map.Entry<ByteBuffer, ByteBuffer> entry : values.entrySet())
offsetLog.send(entry.getKey().array(), entry.getValue().array(), producerCallback);
return producerCallback;
}
private Producer<byte[], byte[]> createProducer() {
Map<String, Object> producerProps = new HashMap<>();
producerProps.putAll(configs);
producerProps.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer");
producerProps.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer");
producerProps.put(ProducerConfig.ACKS_CONFIG, "all");
return new KafkaProducer<>(producerProps);
}
private Consumer<byte[], byte[]> createConsumer() {
Map<String, Object> consumerConfig = new HashMap<>();
consumerConfig.putAll(configs);
consumerConfig.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, false);
consumerConfig.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
consumerConfig.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArrayDeserializer");
consumerConfig.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArrayDeserializer");
return new KafkaConsumer<>(consumerConfig);
}
private void poll(long timeoutMs) {
try {
ConsumerRecords<byte[], byte[]> records = consumer.poll(timeoutMs);
for (ConsumerRecord record : records) {
ByteBuffer key = record.key() != null ? ByteBuffer.wrap((byte[]) record.key()) : null;
ByteBuffer value = record.value() != null ? ByteBuffer.wrap((byte[]) record.value()) : null;
data.put(key, value);
}
} catch (ConsumerWakeupException e) {
// Expected on get() or stop(). The calling code should handle this
throw e;
} catch (KafkaException e) {
log.error("Error polling: " + e);
private final Callback<ConsumerRecord<byte[], byte[]>> consumedCallback = new Callback<ConsumerRecord<byte[], byte[]>>() {
@Override
public void onCompletion(Throwable error, ConsumerRecord<byte[], byte[]> record) {
ByteBuffer key = record.key() != null ? ByteBuffer.wrap(record.key()) : null;
ByteBuffer value = record.value() != null ? ByteBuffer.wrap(record.value()) : null;
data.put(key, value);
}
};
private KafkaBasedLog<byte[], byte[]> createKafkaBasedLog(String topic, Map<String, Object> producerProps,
Map<String, Object> consumerProps, Callback<ConsumerRecord<byte[], byte[]>> consumedCallback) {
return new KafkaBasedLog<>(topic, producerProps, consumerProps, consumedCallback, new SystemTime());
}
private void readToLogEnd() {
log.trace("Reading to end of offset log");
Set<TopicPartition> assignment = consumer.assignment();
// This approach to getting the current end offset is hacky until we have an API for looking these up directly
Map<TopicPartition, Long> offsets = new HashMap<>();
for (TopicPartition tp : assignment) {
long offset = consumer.position(tp);
offsets.put(tp, offset);
consumer.seekToEnd(tp);
}
Map<TopicPartition, Long> endOffsets = new HashMap<>();
try {
poll(0);
} finally {
// If there is an exception, even a possibly expected one like ConsumerWakeupException, we need to make sure
// the consumers position is reset or it'll get into an inconsistent state.
for (TopicPartition tp : assignment) {
long startOffset = offsets.get(tp);
long endOffset = consumer.position(tp);
if (endOffset > startOffset) {
endOffsets.put(tp, endOffset);
consumer.seek(tp, startOffset);
}
log.trace("Reading to end of log for {}: starting offset {} to ending offset {}", tp, startOffset, endOffset);
}
}
while (!endOffsets.isEmpty()) {
poll(Integer.MAX_VALUE);
Iterator<Map.Entry<TopicPartition, Long>> it = endOffsets.entrySet().iterator();
while (it.hasNext()) {
Map.Entry<TopicPartition, Long> entry = it.next();
if (consumer.position(entry.getKey()) >= entry.getValue())
it.remove();
else
break;
}
}
}
private static class SetCallbackFuture implements org.apache.kafka.clients.producer.Callback, Future<Void> {
private int numLeft;
private boolean completed = false;
@ -349,45 +207,5 @@ public class KafkaOffsetBackingStore implements OffsetBackingStore {
}
}
private class WorkThread extends Thread {
@Override
public void run() {
try {
while (true) {
int numCallbacks;
synchronized (KafkaOffsetBackingStore.this) {
if (stopRequested)
break;
numCallbacks = readLogEndOffsetCallbacks.size();
}
if (numCallbacks > 0) {
try {
readToLogEnd();
} catch (ConsumerWakeupException e) {
// Either received another get() call and need to retry reading to end of log or stop() was
// called. Both are handled by restarting this loop.
continue;
}
}
synchronized (KafkaOffsetBackingStore.this) {
for (int i = 0; i < numCallbacks; i++) {
Callback<Void> cb = readLogEndOffsetCallbacks.poll();
cb.onCompletion(null, null);
}
}
try {
poll(Integer.MAX_VALUE);
} catch (ConsumerWakeupException e) {
// See previous comment, both possible causes of this wakeup are handled by starting this loop again
continue;
}
}
} catch (Throwable t) {
log.error("Unexpected exception in KafkaOffsetBackingStore's work thread", t);
}
}
}
}

View File

@ -92,7 +92,7 @@ public class OffsetStorageReaderImpl implements OffsetStorageReader {
continue;
}
Map<String, T> origKey = serializedToOriginal.get(rawEntry.getKey());
SchemaAndValue deserializedSchemaAndValue = valueConverter.toCopycatData(namespace, rawEntry.getValue().array());
SchemaAndValue deserializedSchemaAndValue = valueConverter.toCopycatData(namespace, rawEntry.getValue() != null ? rawEntry.getValue().array() : null);
Object deserializedValue = deserializedSchemaAndValue.value();
OffsetUtils.validateFormat(deserializedValue);

View File

@ -25,6 +25,9 @@ import java.util.Map;
public class OffsetUtils {
public static void validateFormat(Object offsetData) {
if (offsetData == null)
return;
if (!(offsetData instanceof Map))
throw new DataException("Offsets must be specified as a Map");
validateFormat((Map<Object, Object>) offsetData);

View File

@ -0,0 +1,331 @@
/**
* 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.copycat.util;
import org.apache.kafka.clients.consumer.Consumer;
import org.apache.kafka.clients.consumer.ConsumerConfig;
import org.apache.kafka.clients.consumer.ConsumerRecord;
import org.apache.kafka.clients.consumer.ConsumerRecords;
import org.apache.kafka.clients.consumer.ConsumerWakeupException;
import org.apache.kafka.clients.consumer.KafkaConsumer;
import org.apache.kafka.clients.producer.KafkaProducer;
import org.apache.kafka.clients.producer.Producer;
import org.apache.kafka.clients.producer.ProducerConfig;
import org.apache.kafka.clients.producer.ProducerRecord;
import org.apache.kafka.common.KafkaException;
import org.apache.kafka.common.PartitionInfo;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.utils.Time;
import org.apache.kafka.common.utils.Utils;
import org.apache.kafka.copycat.errors.CopycatException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.util.ArrayDeque;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Queue;
import java.util.Set;
import java.util.concurrent.Future;
/**
* <p>
* KafkaBasedLog provides a generic implementation of a shared, compacted log of records stored in Kafka that all
* clients need to consume and, at times, agree on their offset / that they have read to the end of the log.
* </p>
* <p>
* This functionality is useful for storing different types of data that all clients may need to agree on --
* offsets or config for example. This class runs a consumer in a background thread to continuously tail the target
* topic, accepts write requests which it writes to the topic using an internal producer, and provides some helpful
* utilities like checking the current log end offset and waiting until the current end of the log is reached.
* </p>
* <p>
* To support different use cases, this class works with either single- or multi-partition topics.
* </p>
* <p>
* Since this class is generic, it delegates the details of data storage via a callback that is invoked for each
* record that is consumed from the topic. The invocation of callbacks is guaranteed to be serialized -- if the
* calling class keeps track of state based on the log and only writes to it when consume callbacks are invoked
* and only reads it in {@link #readToEnd(Callback)} callbacks then no additional synchronization will be required.
* </p>
*/
public class KafkaBasedLog<K, V> {
private static final Logger log = LoggerFactory.getLogger(KafkaBasedLog.class);
private static final long CREATE_TOPIC_TIMEOUT_MS = 30000;
private Time time;
private final String topic;
private final Map<String, Object> producerConfigs;
private final Map<String, Object> consumerConfigs;
private final Callback<ConsumerRecord<K, V>> consumedCallback;
private Consumer<K, V> consumer;
private Producer<K, V> producer;
private Thread thread;
private boolean stopRequested;
private Queue<Callback<Void>> readLogEndOffsetCallbacks;
/**
* Create a new KafkaBasedLog object. This does not start reading the log and writing is not permitted until
* {@link #start()} is invoked.
*
* @param topic the topic to treat as a log
* @param producerConfigs configuration options to use when creating the internal producer. At a minimum this must
* contain compatible serializer settings for the generic types used on this class. Some
* setting, such as the number of acks, will be overridden to ensure correct behavior of this
* class.
* @param consumerConfigs configuration options to use when creating the internal consumer. At a minimum this must
* contain compatible serializer settings for the generic types used on this class. Some
* setting, such as the auto offset reset policy, will be overridden to ensure correct
* behavior of this class.
* @param consumedCallback callback to invoke for each {@link ConsumerRecord} consumed when tailing the log
* @param time Time interface
*/
public KafkaBasedLog(String topic, Map<String, Object> producerConfigs, Map<String, Object> consumerConfigs,
Callback<ConsumerRecord<K, V>> consumedCallback, Time time) {
this.topic = topic;
this.producerConfigs = producerConfigs;
this.consumerConfigs = consumerConfigs;
this.consumedCallback = consumedCallback;
this.stopRequested = false;
this.readLogEndOffsetCallbacks = new ArrayDeque<>();
this.time = time;
}
public void start() {
log.info("Starting KafkaBasedLog with topic " + topic);
producer = createProducer();
consumer = createConsumer();
List<TopicPartition> partitions = new ArrayList<>();
// Until we have admin utilities we can use to check for the existence of this topic and create it if it is missing,
// we rely on topic auto-creation
List<PartitionInfo> partitionInfos = null;
long started = time.milliseconds();
while (partitionInfos == null && time.milliseconds() - started < CREATE_TOPIC_TIMEOUT_MS) {
partitionInfos = consumer.partitionsFor(topic);
Utils.sleep(Math.min(time.milliseconds() - started, 1000));
}
if (partitionInfos == null)
throw new CopycatException("Could not look up partition metadata for offset backing store topic in" +
" allotted period. This could indicate a connectivity issue, unavailable topic partitions, or if" +
" this is your first use of the topic it may have taken too long to create.");
for (PartitionInfo partition : partitionInfos)
partitions.add(new TopicPartition(partition.topic(), partition.partition()));
consumer.assign(partitions);
readToLogEnd();
thread = new WorkThread();
thread.start();
log.info("Finished reading KafakBasedLog for topic " + topic);
log.info("Started KafakBasedLog for topic " + topic);
}
public void stop() {
log.info("Stopping KafkaBasedLog for topic " + topic);
synchronized (this) {
stopRequested = true;
}
consumer.wakeup();
try {
thread.join();
} catch (InterruptedException e) {
throw new CopycatException("Failed to stop KafkaBasedLog. Exiting without cleanly shutting " +
"down it's producer and consumer.", e);
}
try {
producer.close();
} catch (KafkaException e) {
log.error("Failed to stop KafkaBasedLog producer", e);
}
try {
consumer.close();
} catch (KafkaException e) {
log.error("Failed to stop KafkaBasedLog consumer", e);
}
log.info("Stopped KafkaBasedLog for topic " + topic);
}
/**
* Flushes any outstanding writes and then reads to the current end of the log and invokes the specified callback.
* Note that this checks the current, offsets, reads to them, and invokes the callback regardless of whether
* additional records have been written to the log. If the caller needs to ensure they have truly reached the end
* of the log, they must ensure there are no other writers during this period.
*
* This waits until the end of all partitions has been reached.
*
* This method is asynchronous. If you need a synchronous version, pass an instance of
* {@link org.apache.kafka.copycat.util.FutureCallback} as the {@param callback} parameter and wait on it to block.
*
* @param callback the callback to invoke once the end of the log has been reached.
*/
public void readToEnd(Callback<Void> callback) {
producer.flush();
synchronized (this) {
readLogEndOffsetCallbacks.add(callback);
}
consumer.wakeup();
}
/**
* Same as {@link #readToEnd(Callback)} but provides a {@link Future} instead of using a callback.
* @return the future associated with the operation
*/
public Future<Void> readToEnd() {
FutureCallback<Void> future = new FutureCallback<>(null);
readToEnd(future);
return future;
}
public void send(K key, V value) {
send(key, value, null);
}
public void send(K key, V value, org.apache.kafka.clients.producer.Callback callback) {
producer.send(new ProducerRecord<>(topic, key, value), callback);
}
private Producer<K, V> createProducer() {
// Always require producer acks to all to ensure durable writes
producerConfigs.put(ProducerConfig.ACKS_CONFIG, "all");
return new KafkaProducer<>(producerConfigs);
}
private Consumer<K, V> createConsumer() {
// Always force reset to the beginning of the log since this class wants to consume all available log data
consumerConfigs.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
return new KafkaConsumer<>(consumerConfigs);
}
private void poll(long timeoutMs) {
try {
ConsumerRecords<K, V> records = consumer.poll(timeoutMs);
for (ConsumerRecord<K, V> record : records)
consumedCallback.onCompletion(null, record);
} catch (ConsumerWakeupException e) {
// Expected on get() or stop(). The calling code should handle this
throw e;
} catch (KafkaException e) {
log.error("Error polling: " + e);
}
}
private void readToLogEnd() {
log.trace("Reading to end of offset log");
Set<TopicPartition> assignment = consumer.assignment();
// This approach to getting the current end offset is hacky until we have an API for looking these up directly
Map<TopicPartition, Long> offsets = new HashMap<>();
for (TopicPartition tp : assignment) {
long offset = consumer.position(tp);
offsets.put(tp, offset);
consumer.seekToEnd(tp);
}
Map<TopicPartition, Long> endOffsets = new HashMap<>();
try {
poll(0);
} finally {
// If there is an exception, even a possibly expected one like ConsumerWakeupException, we need to make sure
// the consumers position is reset or it'll get into an inconsistent state.
for (TopicPartition tp : assignment) {
long startOffset = offsets.get(tp);
long endOffset = consumer.position(tp);
if (endOffset > startOffset) {
endOffsets.put(tp, endOffset);
consumer.seek(tp, startOffset);
}
log.trace("Reading to end of log for {}: starting offset {} to ending offset {}", tp, startOffset, endOffset);
}
}
while (!endOffsets.isEmpty()) {
poll(Integer.MAX_VALUE);
Iterator<Map.Entry<TopicPartition, Long>> it = endOffsets.entrySet().iterator();
while (it.hasNext()) {
Map.Entry<TopicPartition, Long> entry = it.next();
if (consumer.position(entry.getKey()) >= entry.getValue())
it.remove();
else
break;
}
}
}
private class WorkThread extends Thread {
@Override
public void run() {
try {
while (true) {
int numCallbacks;
synchronized (KafkaBasedLog.this) {
if (stopRequested)
break;
numCallbacks = readLogEndOffsetCallbacks.size();
}
if (numCallbacks > 0) {
try {
readToLogEnd();
} catch (ConsumerWakeupException e) {
// Either received another get() call and need to retry reading to end of log or stop() was
// called. Both are handled by restarting this loop.
continue;
}
}
synchronized (KafkaBasedLog.this) {
// Only invoke exactly the number of callbacks we found before triggering the read to log end
// since it is possible for another write + readToEnd to sneak in in the meantime
for (int i = 0; i < numCallbacks; i++) {
Callback<Void> cb = readLogEndOffsetCallbacks.poll();
cb.onCompletion(null, null);
}
}
try {
poll(Integer.MAX_VALUE);
} catch (ConsumerWakeupException e) {
// See previous comment, both possible causes of this wakeup are handled by starting this loop again
continue;
}
}
} catch (Throwable t) {
log.error("Unexpected exception in KafkaBasedLog's work thread", t);
}
}
}
}

View File

@ -97,10 +97,10 @@ public class WorkerSinkTaskTest extends ThreadedTest {
Properties workerProps = new Properties();
workerProps.setProperty("key.converter", "org.apache.kafka.copycat.json.JsonConverter");
workerProps.setProperty("value.converter", "org.apache.kafka.copycat.json.JsonConverter");
workerProps.setProperty("offset.key.converter", "org.apache.kafka.copycat.json.JsonConverter");
workerProps.setProperty("offset.value.converter", "org.apache.kafka.copycat.json.JsonConverter");
workerProps.setProperty("offset.key.converter.schemas.enable", "false");
workerProps.setProperty("offset.value.converter.schemas.enable", "false");
workerProps.setProperty("internal.key.converter", "org.apache.kafka.copycat.json.JsonConverter");
workerProps.setProperty("internal.value.converter", "org.apache.kafka.copycat.json.JsonConverter");
workerProps.setProperty("internal.key.converter.schemas.enable", "false");
workerProps.setProperty("internal.value.converter.schemas.enable", "false");
workerConfig = new WorkerConfig(workerProps);
workerTask = PowerMock.createPartialMock(
WorkerSinkTask.class, new String[]{"createConsumer", "createWorkerThread"},

View File

@ -92,10 +92,10 @@ public class WorkerSourceTaskTest extends ThreadedTest {
Properties workerProps = new Properties();
workerProps.setProperty("key.converter", "org.apache.kafka.copycat.json.JsonConverter");
workerProps.setProperty("value.converter", "org.apache.kafka.copycat.json.JsonConverter");
workerProps.setProperty("offset.key.converter", "org.apache.kafka.copycat.json.JsonConverter");
workerProps.setProperty("offset.value.converter", "org.apache.kafka.copycat.json.JsonConverter");
workerProps.setProperty("offset.key.converter.schemas.enable", "false");
workerProps.setProperty("offset.value.converter.schemas.enable", "false");
workerProps.setProperty("internal.key.converter", "org.apache.kafka.copycat.json.JsonConverter");
workerProps.setProperty("internal.value.converter", "org.apache.kafka.copycat.json.JsonConverter");
workerProps.setProperty("internal.key.converter.schemas.enable", "false");
workerProps.setProperty("internal.value.converter.schemas.enable", "false");
config = new WorkerConfig(workerProps);
producerCallbacks = EasyMock.newCapture();
}

View File

@ -57,10 +57,10 @@ public class WorkerTest extends ThreadedTest {
Properties workerProps = new Properties();
workerProps.setProperty("key.converter", "org.apache.kafka.copycat.json.JsonConverter");
workerProps.setProperty("value.converter", "org.apache.kafka.copycat.json.JsonConverter");
workerProps.setProperty("offset.key.converter", "org.apache.kafka.copycat.json.JsonConverter");
workerProps.setProperty("offset.value.converter", "org.apache.kafka.copycat.json.JsonConverter");
workerProps.setProperty("offset.key.converter.schemas.enable", "false");
workerProps.setProperty("offset.value.converter.schemas.enable", "false");
workerProps.setProperty("internal.key.converter", "org.apache.kafka.copycat.json.JsonConverter");
workerProps.setProperty("internal.value.converter", "org.apache.kafka.copycat.json.JsonConverter");
workerProps.setProperty("internal.key.converter.schemas.enable", "false");
workerProps.setProperty("internal.value.converter.schemas.enable", "false");
config = new WorkerConfig(workerProps);
}

View File

@ -0,0 +1,289 @@
/**
* 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.copycat.runtime.distributed;
import org.apache.kafka.copycat.connector.Connector;
import org.apache.kafka.copycat.connector.Task;
import org.apache.kafka.copycat.runtime.ConnectorConfig;
import org.apache.kafka.copycat.runtime.HerderConnectorContext;
import org.apache.kafka.copycat.runtime.Worker;
import org.apache.kafka.copycat.sink.SinkConnector;
import org.apache.kafka.copycat.sink.SinkTask;
import org.apache.kafka.copycat.source.SourceConnector;
import org.apache.kafka.copycat.source.SourceTask;
import org.apache.kafka.copycat.storage.KafkaConfigStorage;
import org.apache.kafka.copycat.util.Callback;
import org.apache.kafka.copycat.util.ConnectorTaskId;
import org.apache.kafka.copycat.util.FutureCallback;
import org.easymock.EasyMock;
import org.junit.Before;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.powermock.api.easymock.PowerMock;
import org.powermock.api.easymock.annotation.Mock;
import org.powermock.core.classloader.annotations.PowerMockIgnore;
import org.powermock.core.classloader.annotations.PrepareForTest;
import org.powermock.modules.junit4.PowerMockRunner;
import java.util.*;
import java.util.concurrent.TimeUnit;
@RunWith(PowerMockRunner.class)
@PrepareForTest({DistributedHerder.class})
@PowerMockIgnore("javax.management.*")
public class DistributedHerderTest {
private static final List<String> CONNECTOR_NAMES = Arrays.asList("source-test1", "source-test2", "sink-test3");
private static final List<String> SOURCE_CONNECTOR_NAMES = Arrays.asList("source-test1", "source-test2");
private static final List<String> SINK_CONNECTOR_NAMES = Arrays.asList("sink-test3");
private static final String TOPICS_LIST_STR = "topic1,topic2";
private static final Map<String, String> CONFIG_STORAGE_CONFIG = Collections.singletonMap(KafkaConfigStorage.CONFIG_TOPIC_CONFIG, "config-topic");
@Mock private KafkaConfigStorage configStorage;
private DistributedHerder herder;
@Mock private Worker worker;
@Mock private Callback<String> createCallback;
private Map<String, Map<String, String>> connectorProps;
private Map<String, Class<? extends Connector>> connectorClasses;
private Map<String, Class<? extends Task>> connectorTaskClasses;
private Map<String, Connector> connectors;
private Properties taskProps;
@Before
public void setUp() {
worker = PowerMock.createMock(Worker.class);
herder = new DistributedHerder(worker, configStorage);
connectorProps = new HashMap<>();
connectorClasses = new HashMap<>();
connectorTaskClasses = new HashMap<>();
connectors = new HashMap<>();
for (String connectorName : CONNECTOR_NAMES) {
Class<? extends Connector> connectorClass = connectorName.contains("source") ? BogusSourceConnector.class : BogusSinkConnector.class;
Class<? extends Task> taskClass = connectorName.contains("source") ? BogusSourceTask.class : BogusSinkTask.class;
Connector connector = connectorName.contains("source") ? PowerMock.createMock(BogusSourceConnector.class) : PowerMock.createMock(BogusSinkConnector.class);
Map<String, String> props = new HashMap<>();
props.put(ConnectorConfig.NAME_CONFIG, connectorName);
props.put(SinkConnector.TOPICS_CONFIG, TOPICS_LIST_STR);
props.put(ConnectorConfig.CONNECTOR_CLASS_CONFIG, connectorClass.getName());
connectorProps.put(connectorName, props);
connectorClasses.put(connectorName, connectorClass);
connectorTaskClasses.put(connectorName, taskClass);
connectors.put(connectorName, connector);
}
PowerMock.mockStatic(DistributedHerder.class);
// These can be anything since connectors can pass along whatever they want.
taskProps = new Properties();
taskProps.setProperty("foo", "bar");
}
@Test
public void testCreateSourceConnector() throws Exception {
String connectorName = SOURCE_CONNECTOR_NAMES.get(0);
expectConfigStorageConfigureStart();
expectEmptyRestore();
expectAdd(connectorName);
PowerMock.replayAll();
herder.configure(CONFIG_STORAGE_CONFIG);
herder.start();
herder.addConnector(connectorProps.get(connectorName), createCallback);
PowerMock.verifyAll();
}
@Test
public void testCreateSinkConnector() throws Exception {
String connectorName = SINK_CONNECTOR_NAMES.get(0);
expectConfigStorageConfigureStart();
expectEmptyRestore();
expectAdd(connectorName);
PowerMock.replayAll();
herder.configure(CONFIG_STORAGE_CONFIG);
herder.start();
herder.addConnector(connectorProps.get(connectorName), createCallback);
PowerMock.verifyAll();
}
@Test
public void testDestroyConnector() throws Exception {
String connectorName = SOURCE_CONNECTOR_NAMES.get(0);
expectConfigStorageConfigureStart();
expectEmptyRestore();
expectAdd(connectorName);
expectDestroy(connectorName);
PowerMock.replayAll();
herder.configure(CONFIG_STORAGE_CONFIG);
herder.start();
herder.addConnector(connectorProps.get(connectorName), createCallback);
FutureCallback<Void> futureCb = new FutureCallback<>(new Callback<Void>() {
@Override
public void onCompletion(Throwable error, Void result) {
}
});
herder.deleteConnector(CONNECTOR_NAMES.get(0), futureCb);
futureCb.get(1000L, TimeUnit.MILLISECONDS);
PowerMock.verifyAll();
}
@Test
public void testCreateAndStop() throws Exception {
String connectorName = SOURCE_CONNECTOR_NAMES.get(0);
expectConfigStorageConfigureStart();
expectEmptyRestore();
expectAdd(connectorName);
PowerMock.replayAll();
herder.configure(CONFIG_STORAGE_CONFIG);
herder.start();
herder.addConnector(connectorProps.get(connectorName), createCallback);
PowerMock.verifyAll();
}
@Test
public void testRestoreAndStop() throws Exception {
String restoreConnectorName1 = SOURCE_CONNECTOR_NAMES.get(0);
String restoreConnectorName2 = SINK_CONNECTOR_NAMES.get(0);
String additionalConnectorName = SOURCE_CONNECTOR_NAMES.get(1);
expectConfigStorageConfigureStart();
expectRestore(Arrays.asList(restoreConnectorName1, restoreConnectorName2));
expectAdd(additionalConnectorName);
// Stopping the herder should correctly stop all restored and new connectors
expectStop(restoreConnectorName1);
expectStop(restoreConnectorName2);
expectStop(additionalConnectorName);
configStorage.stop();
PowerMock.expectLastCall();
PowerMock.replayAll();
herder.configure(CONFIG_STORAGE_CONFIG);
herder.start();
herder.addConnector(connectorProps.get(additionalConnectorName), createCallback);
herder.stop();
PowerMock.verifyAll();
}
private void expectConfigStorageConfigureStart() {
configStorage.configure(CONFIG_STORAGE_CONFIG);
PowerMock.expectLastCall();
configStorage.start();
PowerMock.expectLastCall();
}
private void expectAdd(String connectorName) throws Exception {
configStorage.putConnectorConfig(connectorName, connectorProps.get(connectorName));
PowerMock.expectLastCall();
expectInstantiateConnector(connectorName, true);
}
private void expectEmptyRestore() throws Exception {
expectRestore(Collections.<String>emptyList());
}
private void expectRestore(List<String> connectorNames) throws Exception {
Map<String, Integer> rootConfig = new HashMap<>();
Map<String, Map<String, String>> connectorConfigs = new HashMap<>();
for (String connName : connectorNames) {
rootConfig.put(connName, 0);
connectorConfigs.put(connName, connectorProps.get(connName));
}
EasyMock.expect(configStorage.snapshot())
.andReturn(new ClusterConfigState(1, rootConfig, connectorConfigs, Collections.EMPTY_MAP, Collections.EMPTY_SET));
// Restore never uses a callback
for (String connectorName : connectorNames)
expectInstantiateConnector(connectorName, false);
}
private void expectInstantiateConnector(String connectorName, boolean expectCallback) throws Exception {
PowerMock.expectPrivate(DistributedHerder.class, "instantiateConnector", connectorClasses.get(connectorName).getName())
.andReturn(connectors.get(connectorName));
if (expectCallback) {
createCallback.onCompletion(null, connectorName);
PowerMock.expectLastCall();
}
Connector connector = connectors.get(connectorName);
connector.initialize(EasyMock.anyObject(HerderConnectorContext.class));
PowerMock.expectLastCall();
connector.start(new Properties());
PowerMock.expectLastCall();
// Just return the connector properties for the individual task we generate by default
EasyMock.<Class<? extends Task>>expect(connector.taskClass()).andReturn(connectorTaskClasses.get(connectorName));
EasyMock.expect(connector.taskConfigs(ConnectorConfig.TASKS_MAX_DEFAULT))
.andReturn(Arrays.asList(taskProps));
// And we should instantiate the tasks. For a sink task, we should see added properties for
// the input topic partitions
Properties generatedTaskProps = new Properties();
generatedTaskProps.putAll(taskProps);
if (connectorName.contains("sink"))
generatedTaskProps.setProperty(SinkTask.TOPICS_CONFIG, TOPICS_LIST_STR);
ConnectorTaskId taskId = new ConnectorTaskId(connectorName, 0);
worker.addTask(taskId, connectorTaskClasses.get(connectorName).getName(), generatedTaskProps);
PowerMock.expectLastCall();
}
private void expectStop(String connectorName) {
worker.stopTask(new ConnectorTaskId(connectorName, 0));
EasyMock.expectLastCall();
Connector connector = connectors.get(connectorName);
connector.stop();
EasyMock.expectLastCall();
}
private void expectDestroy(String connectorName) {
expectStop(connectorName);
configStorage.putConnectorConfig(connectorName, null);
PowerMock.expectLastCall();
}
// We need to use a real class here due to some issue with mocking java.lang.Class
private abstract class BogusSourceConnector extends SourceConnector {
}
private abstract class BogusSourceTask extends SourceTask {
}
private abstract class BogusSinkConnector extends SinkConnector {
}
private abstract class BogusSinkTask extends SourceTask {
}
}

View File

@ -20,6 +20,7 @@ package org.apache.kafka.copycat.runtime.standalone;
import org.apache.kafka.copycat.connector.Connector;
import org.apache.kafka.copycat.connector.Task;
import org.apache.kafka.copycat.runtime.ConnectorConfig;
import org.apache.kafka.copycat.runtime.HerderConnectorContext;
import org.apache.kafka.copycat.runtime.Worker;
import org.apache.kafka.copycat.sink.SinkConnector;
import org.apache.kafka.copycat.sink.SinkTask;
@ -39,6 +40,8 @@ import org.powermock.core.classloader.annotations.PrepareForTest;
import org.powermock.modules.junit4.PowerMockRunner;
import java.util.Arrays;
import java.util.HashMap;
import java.util.Map;
import java.util.Properties;
import java.util.concurrent.TimeUnit;
@ -54,7 +57,7 @@ public class StandaloneHerderTest {
private Connector connector;
@Mock protected Callback<String> createCallback;
private Properties connectorProps;
private Map<String, String> connectorProps;
private Properties taskProps;
@Before
@ -62,9 +65,9 @@ public class StandaloneHerderTest {
worker = PowerMock.createMock(Worker.class);
herder = new StandaloneHerder(worker);
connectorProps = new Properties();
connectorProps.setProperty(ConnectorConfig.NAME_CONFIG, CONNECTOR_NAME);
connectorProps.setProperty(SinkConnector.TOPICS_CONFIG, TOPICS_LIST_STR);
connectorProps = new HashMap<>();
connectorProps.put(ConnectorConfig.NAME_CONFIG, CONNECTOR_NAME);
connectorProps.put(SinkConnector.TOPICS_CONFIG, TOPICS_LIST_STR);
PowerMock.mockStatic(StandaloneHerder.class);
// These can be anything since connectors can pass along whatever they want.
@ -74,8 +77,8 @@ public class StandaloneHerderTest {
@Test
public void testCreateSourceConnector() throws Exception {
connector = PowerMock.createMock(BogusSourceClass.class);
expectAdd(BogusSourceClass.class, BogusSourceTask.class, false);
connector = PowerMock.createMock(BogusSourceConnector.class);
expectAdd(BogusSourceConnector.class, BogusSourceTask.class, false);
PowerMock.replayAll();
herder.addConnector(connectorProps, createCallback);
@ -85,8 +88,8 @@ public class StandaloneHerderTest {
@Test
public void testCreateSinkConnector() throws Exception {
connector = PowerMock.createMock(BogusSinkClass.class);
expectAdd(BogusSinkClass.class, BogusSinkTask.class, true);
connector = PowerMock.createMock(BogusSinkConnector.class);
expectAdd(BogusSinkConnector.class, BogusSinkTask.class, true);
PowerMock.replayAll();
@ -97,8 +100,8 @@ public class StandaloneHerderTest {
@Test
public void testDestroyConnector() throws Exception {
connector = PowerMock.createMock(BogusSourceClass.class);
expectAdd(BogusSourceClass.class, BogusSourceTask.class, false);
connector = PowerMock.createMock(BogusSourceConnector.class);
expectAdd(BogusSourceConnector.class, BogusSourceTask.class, false);
expectDestroy();
PowerMock.replayAll();
@ -114,32 +117,31 @@ public class StandaloneHerderTest {
PowerMock.verifyAll();
}
@Test
public void testCreateAndStop() throws Exception {
connector = PowerMock.createMock(BogusSourceConnector.class);
expectAdd(BogusSourceConnector.class, BogusSourceTask.class, false);
expectStop();
PowerMock.replayAll();
herder.addConnector(connectorProps, createCallback);
herder.stop();
PowerMock.verifyAll();
}
private void expectAdd(Class<? extends Connector> connClass,
Class<? extends Task> taskClass,
boolean sink) throws Exception {
expectCreate(connClass, taskClass, sink, true);
}
private void expectRestore(Class<? extends Connector> connClass,
Class<? extends Task> taskClass) throws Exception {
// Restore never uses a callback. These tests always use sources
expectCreate(connClass, taskClass, false, false);
}
private void expectCreate(Class<? extends Connector> connClass,
Class<? extends Task> taskClass,
boolean sink, boolean expectCallback) throws Exception {
connectorProps.setProperty(ConnectorConfig.CONNECTOR_CLASS_CONFIG, connClass.getName());
Class<? extends Task> taskClass,
boolean sink) throws Exception {
connectorProps.put(ConnectorConfig.CONNECTOR_CLASS_CONFIG, connClass.getName());
PowerMock.expectPrivate(StandaloneHerder.class, "instantiateConnector", connClass.getName())
.andReturn(connector);
if (expectCallback) {
createCallback.onCompletion(null, CONNECTOR_NAME);
PowerMock.expectLastCall();
}
connector.initialize(EasyMock.anyObject(StandaloneConnectorContext.class));
createCallback.onCompletion(null, CONNECTOR_NAME);
PowerMock.expectLastCall();
connector.initialize(EasyMock.anyObject(HerderConnectorContext.class));
PowerMock.expectLastCall();
connector.start(new Properties());
PowerMock.expectLastCall();
@ -171,13 +173,13 @@ public class StandaloneHerderTest {
}
// We need to use a real class here due to some issue with mocking java.lang.Class
private abstract class BogusSourceClass extends SourceConnector {
private abstract class BogusSourceConnector extends SourceConnector {
}
private abstract class BogusSourceTask extends SourceTask {
}
private abstract class BogusSinkClass extends SinkConnector {
private abstract class BogusSinkConnector extends SinkConnector {
}
private abstract class BogusSinkTask extends SourceTask {

View File

@ -0,0 +1,508 @@
/**
* 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.copycat.storage;
import org.apache.kafka.clients.CommonClientConfigs;
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.KafkaException;
import org.apache.kafka.copycat.data.Field;
import org.apache.kafka.copycat.data.Schema;
import org.apache.kafka.copycat.data.SchemaAndValue;
import org.apache.kafka.copycat.data.Struct;
import org.apache.kafka.copycat.runtime.distributed.ClusterConfigState;
import org.apache.kafka.copycat.util.Callback;
import org.apache.kafka.copycat.util.ConnectorTaskId;
import org.apache.kafka.copycat.util.KafkaBasedLog;
import org.apache.kafka.copycat.util.TestFuture;
import org.easymock.Capture;
import org.easymock.EasyMock;
import org.easymock.IAnswer;
import org.junit.Before;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.powermock.api.easymock.PowerMock;
import org.powermock.api.easymock.annotation.Mock;
import org.powermock.core.classloader.annotations.PowerMockIgnore;
import org.powermock.core.classloader.annotations.PrepareForTest;
import org.powermock.modules.junit4.PowerMockRunner;
import org.powermock.reflect.Whitebox;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
import java.util.concurrent.Future;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNull;
import static org.junit.Assert.fail;
@RunWith(PowerMockRunner.class)
@PrepareForTest(KafkaConfigStorage.class)
@PowerMockIgnore("javax.management.*")
public class KafkaConfigStorageTest {
private static final String TOPIC = "copycat-configs";
private static final Map<String, String> DEFAULT_CONFIG_STORAGE_PROPS = new HashMap<>();
static {
DEFAULT_CONFIG_STORAGE_PROPS.put(KafkaConfigStorage.CONFIG_TOPIC_CONFIG, TOPIC);
DEFAULT_CONFIG_STORAGE_PROPS.put(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, "broker1:9092,broker2:9093");
}
private static final List<String> CONNECTOR_IDS = Arrays.asList("connector1", "connector2");
private static final List<String> CONNECTOR_CONFIG_KEYS = Arrays.asList("connector-connector1", "connector-connector2");
private static final List<String> COMMIT_TASKS_CONFIG_KEYS = Arrays.asList("commit-connector1", "commit-connector2");
// Need a) connector with multiple tasks and b) multiple connectors
private static final List<ConnectorTaskId> TASK_IDS = Arrays.asList(
new ConnectorTaskId("connector1", 0),
new ConnectorTaskId("connector1", 1),
new ConnectorTaskId("connector2", 0)
);
private static final List<String> TASK_CONFIG_KEYS = Arrays.asList("task-connector1-0", "task-connector1-1", "task-connector2-0");
// Need some placeholders -- the contents don't matter here, just that they are restored properly
private static final List<Map<String, String>> SAMPLE_CONFIGS = Arrays.asList(
Collections.singletonMap("config-key-one", "config-value-one"),
Collections.singletonMap("config-key-two", "config-value-two"),
Collections.singletonMap("config-key-three", "config-value-three")
);
private static final List<Struct> CONNECTOR_CONFIG_STRUCTS = Arrays.asList(
new Struct(KafkaConfigStorage.CONNECTOR_CONFIGURATION_V0).put("properties", SAMPLE_CONFIGS.get(0)),
new Struct(KafkaConfigStorage.CONNECTOR_CONFIGURATION_V0).put("properties", SAMPLE_CONFIGS.get(1)),
new Struct(KafkaConfigStorage.CONNECTOR_CONFIGURATION_V0).put("properties", SAMPLE_CONFIGS.get(2))
);
private static final List<Struct> TASK_CONFIG_STRUCTS = Arrays.asList(
new Struct(KafkaConfigStorage.TASK_CONFIGURATION_V0).put("properties", SAMPLE_CONFIGS.get(0)),
new Struct(KafkaConfigStorage.TASK_CONFIGURATION_V0).put("properties", SAMPLE_CONFIGS.get(1))
);
private static final Struct TASKS_COMMIT_STRUCT_TWO_TASK_CONNECTOR
= new Struct(KafkaConfigStorage.CONNECTOR_TASKS_COMMIT_V0).put("tasks", 2);
// The exact format doesn't matter here since both conversions are mocked
private static final List<byte[]> CONFIGS_SERIALIZED = Arrays.asList(
"config-bytes-1".getBytes(), "config-bytes-2".getBytes(), "config-bytes-3".getBytes(),
"config-bytes-4".getBytes(), "config-bytes-5".getBytes(), "config-bytes-6".getBytes(),
"config-bytes-7".getBytes(), "config-bytes-8".getBytes(), "config-bytes-9".getBytes()
);
@Mock
private Converter converter;
@Mock
private Callback<String> connectorReconfiguredCallback;
@Mock
private Callback<List<ConnectorTaskId>> tasksReconfiguredCallback;
@Mock
KafkaBasedLog<String, byte[]> storeLog;
private KafkaConfigStorage configStorage;
private Capture<String> capturedTopic = EasyMock.newCapture();
private Capture<Map<String, Object>> capturedProducerProps = EasyMock.newCapture();
private Capture<Map<String, Object>> capturedConsumerProps = EasyMock.newCapture();
private Capture<Callback<ConsumerRecord<String, byte[]>>> capturedConsumedCallback = EasyMock.newCapture();
private long logOffset = 0;
@Before
public void setUp() {
configStorage = PowerMock.createPartialMock(KafkaConfigStorage.class, new String[]{"createKafkaBasedLog"},
converter, connectorReconfiguredCallback, tasksReconfiguredCallback);
}
@Test
public void testStartStop() throws Exception {
expectConfigure();
expectStart(Collections.EMPTY_LIST, Collections.EMPTY_MAP);
expectStop();
PowerMock.replayAll();
configStorage.configure(DEFAULT_CONFIG_STORAGE_PROPS);
assertEquals(TOPIC, capturedTopic.getValue());
assertEquals("org.apache.kafka.common.serialization.StringSerializer", capturedProducerProps.getValue().get(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG));
assertEquals("org.apache.kafka.common.serialization.ByteArraySerializer", capturedProducerProps.getValue().get(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG));
assertEquals("org.apache.kafka.common.serialization.StringDeserializer", capturedConsumerProps.getValue().get(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG));
assertEquals("org.apache.kafka.common.serialization.ByteArrayDeserializer", capturedConsumerProps.getValue().get(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG));
configStorage.start();
configStorage.stop();
PowerMock.verifyAll();
}
@Test
public void testPutConnectorConfig() throws Exception {
expectConfigure();
expectStart(Collections.EMPTY_LIST, Collections.EMPTY_MAP);
expectConvertWriteAndRead(
CONNECTOR_CONFIG_KEYS.get(0), KafkaConfigStorage.CONNECTOR_CONFIGURATION_V0, CONFIGS_SERIALIZED.get(0),
"properties", SAMPLE_CONFIGS.get(0));
connectorReconfiguredCallback.onCompletion(null, CONNECTOR_IDS.get(0));
EasyMock.expectLastCall();
expectConvertWriteAndRead(
CONNECTOR_CONFIG_KEYS.get(1), KafkaConfigStorage.CONNECTOR_CONFIGURATION_V0, CONFIGS_SERIALIZED.get(1),
"properties", SAMPLE_CONFIGS.get(1));
connectorReconfiguredCallback.onCompletion(null, CONNECTOR_IDS.get(1));
EasyMock.expectLastCall();
expectStop();
PowerMock.replayAll();
configStorage.configure(DEFAULT_CONFIG_STORAGE_PROPS);
configStorage.start();
// Null before writing
ClusterConfigState configState = configStorage.snapshot();
assertEquals(-1, configState.offset());
assertNull(configState.connectorConfig(CONNECTOR_IDS.get(0)));
assertNull(configState.connectorConfig(CONNECTOR_IDS.get(1)));
// Writing should block until it is written and read back from Kafka
configStorage.putConnectorConfig(CONNECTOR_IDS.get(0), SAMPLE_CONFIGS.get(0));
configState = configStorage.snapshot();
assertEquals(0, configState.offset());
assertEquals(SAMPLE_CONFIGS.get(0), configState.connectorConfig(CONNECTOR_IDS.get(0)));
assertNull(configState.connectorConfig(CONNECTOR_IDS.get(1)));
// Second should also block and all configs should still be available
configStorage.putConnectorConfig(CONNECTOR_IDS.get(1), SAMPLE_CONFIGS.get(1));
configState = configStorage.snapshot();
assertEquals(1, configState.offset());
assertEquals(SAMPLE_CONFIGS.get(0), configState.connectorConfig(CONNECTOR_IDS.get(0)));
assertEquals(SAMPLE_CONFIGS.get(1), configState.connectorConfig(CONNECTOR_IDS.get(1)));
configStorage.stop();
PowerMock.verifyAll();
}
@Test
public void testPutTaskConfigs() throws Exception {
expectConfigure();
expectStart(Collections.EMPTY_LIST, Collections.EMPTY_MAP);
// Task configs should read to end, write to the log, read to end, write root, then read to end again
expectReadToEnd(new LinkedHashMap<String, byte[]>());
expectConvertWriteRead(
TASK_CONFIG_KEYS.get(0), KafkaConfigStorage.TASK_CONFIGURATION_V0, CONFIGS_SERIALIZED.get(0),
"properties", SAMPLE_CONFIGS.get(0));
expectConvertWriteRead(
TASK_CONFIG_KEYS.get(1), KafkaConfigStorage.TASK_CONFIGURATION_V0, CONFIGS_SERIALIZED.get(1),
"properties", SAMPLE_CONFIGS.get(1));
expectReadToEnd(new LinkedHashMap<String, byte[]>());
expectConvertWriteRead(
COMMIT_TASKS_CONFIG_KEYS.get(0), KafkaConfigStorage.CONNECTOR_TASKS_COMMIT_V0, CONFIGS_SERIALIZED.get(2),
"tasks", 2); // Starts with 0 tasks, after update has 2
// As soon as root is rewritten, we should see a callback notifying us that we reconfigured some tasks
tasksReconfiguredCallback.onCompletion(null, Arrays.asList(TASK_IDS.get(0), TASK_IDS.get(1)));
EasyMock.expectLastCall();
// Records to be read by consumer as it reads to the end of the log
LinkedHashMap<String, byte[]> serializedConfigs = new LinkedHashMap<>();
serializedConfigs.put(TASK_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(0));
serializedConfigs.put(TASK_CONFIG_KEYS.get(1), CONFIGS_SERIALIZED.get(1));
serializedConfigs.put(COMMIT_TASKS_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(2));
expectReadToEnd(serializedConfigs);
expectStop();
PowerMock.replayAll();
configStorage.configure(DEFAULT_CONFIG_STORAGE_PROPS);
configStorage.start();
// Bootstrap as if we had already added the connector, but no tasks had been added yet
whiteboxAddConnector(CONNECTOR_IDS.get(0), SAMPLE_CONFIGS.get(0), Collections.EMPTY_LIST);
// Null before writing
ClusterConfigState configState = configStorage.snapshot();
assertEquals(-1, configState.offset());
assertNull(configState.taskConfig(TASK_IDS.get(0)));
assertNull(configState.taskConfig(TASK_IDS.get(1)));
// Writing task task configs should block until all the writes have been performed and the root record update
// has completed
Map<ConnectorTaskId, Map<String, String>> taskConfigs = new HashMap<>();
taskConfigs.put(TASK_IDS.get(0), SAMPLE_CONFIGS.get(0));
taskConfigs.put(TASK_IDS.get(1), SAMPLE_CONFIGS.get(1));
configStorage.putTaskConfigs(taskConfigs);
// Validate root config by listing all connectors and tasks
configState = configStorage.snapshot();
assertEquals(2, configState.offset());
String connectorName = CONNECTOR_IDS.get(0);
assertEquals(Arrays.asList(connectorName), new ArrayList<>(configState.connectors()));
assertEquals(Arrays.asList(TASK_IDS.get(0), TASK_IDS.get(1)), configState.tasks(connectorName));
assertEquals(SAMPLE_CONFIGS.get(0), configState.taskConfig(TASK_IDS.get(0)));
assertEquals(SAMPLE_CONFIGS.get(1), configState.taskConfig(TASK_IDS.get(1)));
assertEquals(new HashSet<>(Collections.EMPTY_LIST), configState.inconsistentConnectors());
configStorage.stop();
PowerMock.verifyAll();
}
@Test
public void testRestore() throws Exception {
// Restoring data should notify only of the latest values after loading is complete. This also validates
// that inconsistent state is ignored.
expectConfigure();
// Overwrite each type at least once to ensure we see the latest data after loading
List<ConsumerRecord<String, byte[]>> existingRecords = Arrays.asList(
new ConsumerRecord<>(TOPIC, 0, 0, CONNECTOR_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(0)),
new ConsumerRecord<>(TOPIC, 0, 1, TASK_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(1)),
new ConsumerRecord<>(TOPIC, 0, 2, TASK_CONFIG_KEYS.get(1), CONFIGS_SERIALIZED.get(2)),
new ConsumerRecord<>(TOPIC, 0, 3, CONNECTOR_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(3)),
new ConsumerRecord<>(TOPIC, 0, 4, COMMIT_TASKS_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(4)),
// Connector after root update should make it through, task update shouldn't
new ConsumerRecord<>(TOPIC, 0, 5, CONNECTOR_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(5)),
new ConsumerRecord<>(TOPIC, 0, 6, TASK_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(6)));
LinkedHashMap<byte[], Struct> deserialized = new LinkedHashMap();
deserialized.put(CONFIGS_SERIALIZED.get(0), CONNECTOR_CONFIG_STRUCTS.get(0));
deserialized.put(CONFIGS_SERIALIZED.get(1), TASK_CONFIG_STRUCTS.get(0));
deserialized.put(CONFIGS_SERIALIZED.get(2), TASK_CONFIG_STRUCTS.get(0));
deserialized.put(CONFIGS_SERIALIZED.get(3), CONNECTOR_CONFIG_STRUCTS.get(1));
deserialized.put(CONFIGS_SERIALIZED.get(4), TASKS_COMMIT_STRUCT_TWO_TASK_CONNECTOR);
deserialized.put(CONFIGS_SERIALIZED.get(5), CONNECTOR_CONFIG_STRUCTS.get(2));
deserialized.put(CONFIGS_SERIALIZED.get(6), TASK_CONFIG_STRUCTS.get(1));
logOffset = 7;
expectStart(existingRecords, deserialized);
// Shouldn't see any callbacks since this is during startup
expectStop();
PowerMock.replayAll();
configStorage.configure(DEFAULT_CONFIG_STORAGE_PROPS);
configStorage.start();
// Should see a single connector and its config should be the last one seen anywhere in the log
ClusterConfigState configState = configStorage.snapshot();
assertEquals(6, configState.offset()); // Should always be last read, even if uncommitted
assertEquals(Arrays.asList(CONNECTOR_IDS.get(0)), new ArrayList<>(configState.connectors()));
// CONNECTOR_CONFIG_STRUCTS[2] -> SAMPLE_CONFIGS[2]
assertEquals(SAMPLE_CONFIGS.get(2), configState.connectorConfig(CONNECTOR_IDS.get(0)));
// Should see 2 tasks for that connector. Only config updates before the root key update should be reflected
assertEquals(Arrays.asList(TASK_IDS.get(0), TASK_IDS.get(1)), configState.tasks(CONNECTOR_IDS.get(0)));
// Both TASK_CONFIG_STRUCTS[0] -> SAMPLE_CONFIGS[0]
assertEquals(SAMPLE_CONFIGS.get(0), configState.taskConfig(TASK_IDS.get(0)));
assertEquals(SAMPLE_CONFIGS.get(0), configState.taskConfig(TASK_IDS.get(1)));
assertEquals(new HashSet<>(Collections.EMPTY_LIST), configState.inconsistentConnectors());
configStorage.stop();
PowerMock.verifyAll();
}
@Test
public void testPutTaskConfigsDoesNotResolveAllInconsistencies() throws Exception {
// Test a case where a failure and compaction has left us in an inconsistent state when reading the log.
// We start out by loading an initial configuration where we started to write a task update and failed before
// writing an the commit, and then compaction cleaned up the earlier record.
expectConfigure();
List<ConsumerRecord<String, byte[]>> existingRecords = Arrays.asList(
new ConsumerRecord<>(TOPIC, 0, 0, CONNECTOR_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(0)),
// This is the record that has been compacted:
//new ConsumerRecord<>(TOPIC, 0, 1, TASK_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(1)),
new ConsumerRecord<>(TOPIC, 0, 2, TASK_CONFIG_KEYS.get(1), CONFIGS_SERIALIZED.get(2)),
new ConsumerRecord<>(TOPIC, 0, 4, COMMIT_TASKS_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(4)),
new ConsumerRecord<>(TOPIC, 0, 5, TASK_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(5)));
LinkedHashMap<byte[], Struct> deserialized = new LinkedHashMap();
deserialized.put(CONFIGS_SERIALIZED.get(0), CONNECTOR_CONFIG_STRUCTS.get(0));
deserialized.put(CONFIGS_SERIALIZED.get(2), TASK_CONFIG_STRUCTS.get(0));
deserialized.put(CONFIGS_SERIALIZED.get(4), TASKS_COMMIT_STRUCT_TWO_TASK_CONNECTOR);
deserialized.put(CONFIGS_SERIALIZED.get(5), TASK_CONFIG_STRUCTS.get(1));
logOffset = 6;
expectStart(existingRecords, deserialized);
// One failed attempt to write new task configs
expectReadToEnd(new LinkedHashMap<String, byte[]>());
// Successful attempt to write new task config
expectReadToEnd(new LinkedHashMap<String, byte[]>());
expectConvertWriteRead(
TASK_CONFIG_KEYS.get(0), KafkaConfigStorage.TASK_CONFIGURATION_V0, CONFIGS_SERIALIZED.get(0),
"properties", SAMPLE_CONFIGS.get(0));
expectReadToEnd(new LinkedHashMap<String, byte[]>());
expectConvertWriteRead(
COMMIT_TASKS_CONFIG_KEYS.get(0), KafkaConfigStorage.CONNECTOR_TASKS_COMMIT_V0, CONFIGS_SERIALIZED.get(2),
"tasks", 1); // Updated to just 1 task
// As soon as root is rewritten, we should see a callback notifying us that we reconfigured some tasks
tasksReconfiguredCallback.onCompletion(null, Arrays.asList(TASK_IDS.get(0)));
EasyMock.expectLastCall();
// Records to be read by consumer as it reads to the end of the log
LinkedHashMap<String, byte[]> serializedConfigs = new LinkedHashMap<>();
serializedConfigs.put(TASK_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(0));
serializedConfigs.put(COMMIT_TASKS_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(2));
expectReadToEnd(serializedConfigs);
expectStop();
PowerMock.replayAll();
configStorage.configure(DEFAULT_CONFIG_STORAGE_PROPS);
configStorage.start();
// After reading the log, it should have been in an inconsistent state
ClusterConfigState configState = configStorage.snapshot();
assertEquals(5, configState.offset()); // Should always be last read, not last committed
assertEquals(Arrays.asList(CONNECTOR_IDS.get(0)), new ArrayList<>(configState.connectors()));
// Inconsistent data should leave us with no tasks listed for the connector and an entry in the inconsistent list
assertEquals(Collections.EMPTY_LIST, configState.tasks(CONNECTOR_IDS.get(0)));
// Both TASK_CONFIG_STRUCTS[0] -> SAMPLE_CONFIGS[0]
assertNull(configState.taskConfig(TASK_IDS.get(0)));
assertNull(configState.taskConfig(TASK_IDS.get(1)));
assertEquals(new HashSet<>(Arrays.asList(CONNECTOR_IDS.get(0))), configState.inconsistentConnectors());
// First try sending an invalid set of configs (can't possibly represent a valid config set for the tasks)
try {
configStorage.putTaskConfigs(Collections.singletonMap(TASK_IDS.get(1), SAMPLE_CONFIGS.get(2)));
fail("Should have failed due to incomplete task set.");
} catch (KafkaException e) {
// expected
}
// Next, issue a write that has everything that is needed and it should be accepted. Note that in this case
// we are going to shrink the number of tasks to 1
configStorage.putTaskConfigs(Collections.singletonMap(TASK_IDS.get(0), SAMPLE_CONFIGS.get(0)));
// Validate updated config
configState = configStorage.snapshot();
// This is only two more ahead of the last one because multiple calls fail, and so their configs are not written
// to the topic. Only the last call with 1 task config + 1 commit actually gets written.
assertEquals(7, configState.offset());
assertEquals(Arrays.asList(CONNECTOR_IDS.get(0)), new ArrayList<>(configState.connectors()));
assertEquals(Arrays.asList(TASK_IDS.get(0)), configState.tasks(CONNECTOR_IDS.get(0)));
assertEquals(SAMPLE_CONFIGS.get(0), configState.taskConfig(TASK_IDS.get(0)));
assertEquals(new HashSet<>(Collections.EMPTY_LIST), configState.inconsistentConnectors());
configStorage.stop();
PowerMock.verifyAll();
}
private void expectConfigure() throws Exception {
PowerMock.expectPrivate(configStorage, "createKafkaBasedLog",
EasyMock.capture(capturedTopic), EasyMock.capture(capturedProducerProps),
EasyMock.capture(capturedConsumerProps), EasyMock.capture(capturedConsumedCallback))
.andReturn(storeLog);
}
// If non-empty, deserializations should be a LinkedHashMap
private void expectStart(final List<ConsumerRecord<String, byte[]>> preexistingRecords,
final Map<byte[], Struct> deserializations) throws Exception {
storeLog.start();
PowerMock.expectLastCall().andAnswer(new IAnswer<Object>() {
@Override
public Object answer() throws Throwable {
for (ConsumerRecord<String, byte[]> rec : preexistingRecords)
capturedConsumedCallback.getValue().onCompletion(null, rec);
return null;
}
});
for (Map.Entry<byte[], Struct> deserializationEntry : deserializations.entrySet()) {
// Note null schema because default settings for internal serialization are schema-less
EasyMock.expect(converter.toCopycatData(EasyMock.eq(TOPIC), EasyMock.aryEq(deserializationEntry.getKey())))
.andReturn(new SchemaAndValue(null, structToMap(deserializationEntry.getValue())));
}
}
private void expectStop() {
storeLog.stop();
PowerMock.expectLastCall();
}
// Expect a conversion & write to the underlying log, followed by a subsequent read when the data is consumed back
// from the log. Validate the data that is captured when the conversion is performed matches the specified data
// (by checking a single field's value)
private void expectConvertWriteRead(final String configKey, final Schema valueSchema, final byte[] serialized,
final String dataFieldName, final Object dataFieldValue) {
final Capture<Struct> capturedRecord = EasyMock.newCapture();
EasyMock.expect(converter.fromCopycatData(EasyMock.eq(TOPIC), EasyMock.eq(valueSchema), EasyMock.capture(capturedRecord)))
.andReturn(serialized);
storeLog.send(EasyMock.eq(configKey), EasyMock.aryEq(serialized));
PowerMock.expectLastCall();
EasyMock.expect(converter.toCopycatData(EasyMock.eq(TOPIC), EasyMock.aryEq(serialized)))
.andAnswer(new IAnswer<SchemaAndValue>() {
@Override
public SchemaAndValue answer() throws Throwable {
assertEquals(dataFieldValue, capturedRecord.getValue().get(dataFieldName));
// Note null schema because default settings for internal serialization are schema-less
return new SchemaAndValue(null, structToMap(capturedRecord.getValue()));
}
});
}
// This map needs to maintain ordering
private void expectReadToEnd(final LinkedHashMap<String, byte[]> serializedConfigs) {
EasyMock.expect(storeLog.readToEnd())
.andAnswer(new IAnswer<Future<Void>>() {
@Override
public Future<Void> answer() throws Throwable {
TestFuture<Void> future = new TestFuture<Void>();
for (Map.Entry<String, byte[]> entry : serializedConfigs.entrySet())
capturedConsumedCallback.getValue().onCompletion(null, new ConsumerRecord<>(TOPIC, 0, logOffset++, entry.getKey(), entry.getValue()));
future.resolveOnGet((Void) null);
return future;
}
});
}
private void expectConvertWriteAndRead(final String configKey, final Schema valueSchema, final byte[] serialized,
final String dataFieldName, final Object dataFieldValue) {
expectConvertWriteRead(configKey, valueSchema, serialized, dataFieldName, dataFieldValue);
LinkedHashMap<String, byte[]> recordsToRead = new LinkedHashMap<>();
recordsToRead.put(configKey, serialized);
expectReadToEnd(recordsToRead);
}
// Manually insert a connector into config storage, updating the task configs, connector config, and root config
private void whiteboxAddConnector(String connectorName, Map<String, String> connectorConfig, List<Map<String, String>> taskConfigs) {
Map<ConnectorTaskId, Map<String, String>> storageTaskConfigs = Whitebox.getInternalState(configStorage, "taskConfigs");
for (int i = 0; i < taskConfigs.size(); i++)
storageTaskConfigs.put(new ConnectorTaskId(connectorName, i), taskConfigs.get(i));
Map<String, Map<String, String>> connectorConfigs = Whitebox.getInternalState(configStorage, "connectorConfigs");
connectorConfigs.put(connectorName, connectorConfig);
Whitebox.<Map<String, Integer>>getInternalState(configStorage, "connectorTaskCounts").put(connectorName, taskConfigs.size());
}
// Generates a Map representation of Struct. Only does shallow traversal, so nested structs are not converted
private Map<String, Object> structToMap(Struct struct) {
HashMap<String, Object> result = new HashMap<>();
for (Field field : struct.schema().fields())
result.put(field.name(), struct.get(field));
return result;
}
}

View File

@ -18,23 +18,16 @@
package org.apache.kafka.copycat.storage;
import org.apache.kafka.clients.CommonClientConfigs;
import org.apache.kafka.clients.consumer.ConsumerConfig;
import org.apache.kafka.clients.consumer.ConsumerRecord;
import org.apache.kafka.clients.consumer.MockConsumer;
import org.apache.kafka.clients.consumer.OffsetResetStrategy;
import org.apache.kafka.clients.producer.KafkaProducer;
import org.apache.kafka.clients.producer.ProducerRecord;
import org.apache.kafka.clients.producer.RecordMetadata;
import org.apache.kafka.clients.producer.ProducerConfig;
import org.apache.kafka.common.KafkaException;
import org.apache.kafka.common.Node;
import org.apache.kafka.common.PartitionInfo;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.errors.LeaderNotAvailableException;
import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.copycat.errors.CopycatException;
import org.apache.kafka.copycat.util.Callback;
import org.apache.kafka.copycat.util.TestFuture;
import org.apache.kafka.copycat.util.KafkaBasedLog;
import org.easymock.Capture;
import org.easymock.EasyMock;
import org.easymock.IAnswer;
import org.junit.Before;
import org.junit.Test;
import org.junit.runner.RunWith;
@ -49,70 +42,56 @@ import java.nio.ByteBuffer;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.Future;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicReference;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
import static org.apache.kafka.copycat.util.ByteArrayProducerRecordEquals.eqProducerRecord;
@RunWith(PowerMockRunner.class)
@PrepareForTest(KafkaOffsetBackingStore.class)
@PowerMockIgnore("javax.management.*")
public class KafkaOffsetBackingStoreTest {
private static final String TOPIC = "copycat-offsets";
private static final TopicPartition TP0 = new TopicPartition(TOPIC, 0);
private static final TopicPartition TP1 = new TopicPartition(TOPIC, 1);
private static final Map<String, String> DEFAULT_PROPS = new HashMap<>();
static {
DEFAULT_PROPS.put(KafkaOffsetBackingStore.OFFSET_STORAGE_TOPIC_CONFIG, TOPIC);
DEFAULT_PROPS.put(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, "broker1:9092,broker2:9093");
}
private static final Set<TopicPartition> CONSUMER_ASSIGNMENT = new HashSet<>(Arrays.asList(TP0, TP1));
private static final Map<ByteBuffer, ByteBuffer> FIRST_SET = new HashMap<>();
static {
FIRST_SET.put(buffer("key"), buffer("value"));
FIRST_SET.put(null, null);
}
private static final Node LEADER = new Node(1, "broker1", 9092);
private static final Node REPLICA = new Node(1, "broker2", 9093);
private static final PartitionInfo TPINFO0 = new PartitionInfo(TOPIC, 0, LEADER, new Node[]{REPLICA}, new Node[]{REPLICA});
private static final PartitionInfo TPINFO1 = new PartitionInfo(TOPIC, 1, LEADER, new Node[]{REPLICA}, new Node[]{REPLICA});
private static final ByteBuffer TP0_KEY = buffer("TP0KEY");
private static final ByteBuffer TP1_KEY = buffer("TP1KEY");
private static final ByteBuffer TP2_KEY = buffer("TP2KEY");
private static final ByteBuffer TP0_VALUE = buffer("VAL0");
private static final ByteBuffer TP1_VALUE = buffer("VAL1");
private static final ByteBuffer TP2_VALUE = buffer("VAL2");
private static final ByteBuffer TP0_VALUE_NEW = buffer("VAL0_NEW");
private static final ByteBuffer TP1_VALUE_NEW = buffer("VAL1_NEW");
@Mock
KafkaBasedLog<byte[], byte[]> storeLog;
private KafkaOffsetBackingStore store;
@Mock private KafkaProducer<byte[], byte[]> producer;
private MockConsumer<byte[], byte[]> consumer;
private Capture<String> capturedTopic = EasyMock.newCapture();
private Capture<Map<String, Object>> capturedProducerProps = EasyMock.newCapture();
private Capture<Map<String, Object>> capturedConsumerProps = EasyMock.newCapture();
private Capture<Callback<ConsumerRecord<byte[], byte[]>>> capturedConsumedCallback = EasyMock.newCapture();
@Before
public void setUp() throws Exception {
store = PowerMock.createPartialMockAndInvokeDefaultConstructor(KafkaOffsetBackingStore.class, new String[]{"createConsumer", "createProducer"});
consumer = new MockConsumer<>(OffsetResetStrategy.EARLIEST);
consumer.updatePartitions(TOPIC, Arrays.asList(TPINFO0, TPINFO1));
Map<TopicPartition, Long> beginningOffsets = new HashMap<>();
beginningOffsets.put(TP0, 0L);
beginningOffsets.put(TP1, 0L);
consumer.updateBeginningOffsets(beginningOffsets);
store = PowerMock.createPartialMockAndInvokeDefaultConstructor(KafkaOffsetBackingStore.class, new String[]{"createKafkaBasedLog"});
}
@Test(expected = CopycatException.class)
@ -123,142 +102,108 @@ public class KafkaOffsetBackingStoreTest {
@Test
public void testStartStop() throws Exception {
expectStart();
expectConfigure();
expectStart(Collections.EMPTY_LIST);
expectStop();
PowerMock.replayAll();
store.configure(DEFAULT_PROPS);
assertEquals(TOPIC, capturedTopic.getValue());
assertEquals("org.apache.kafka.common.serialization.ByteArraySerializer", capturedProducerProps.getValue().get(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG));
assertEquals("org.apache.kafka.common.serialization.ByteArraySerializer", capturedProducerProps.getValue().get(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG));
assertEquals("org.apache.kafka.common.serialization.ByteArrayDeserializer", capturedConsumerProps.getValue().get(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG));
assertEquals("org.apache.kafka.common.serialization.ByteArrayDeserializer", capturedConsumerProps.getValue().get(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG));
Map<TopicPartition, Long> endOffsets = new HashMap<>();
endOffsets.put(TP0, 0L);
endOffsets.put(TP1, 0L);
consumer.updateEndOffsets(endOffsets);
store.start();
assertEquals(CONSUMER_ASSIGNMENT, consumer.assignment());
store.stop();
assertFalse(Whitebox.<Thread>getInternalState(store, "thread").isAlive());
assertTrue(consumer.closed());
PowerMock.verifyAll();
}
@Test
public void testReloadOnStart() throws Exception {
expectStart();
expectConfigure();
expectStart(Arrays.asList(
new ConsumerRecord<>(TOPIC, 0, 0, TP0_KEY.array(), TP0_VALUE.array()),
new ConsumerRecord<>(TOPIC, 1, 0, TP1_KEY.array(), TP1_VALUE.array()),
new ConsumerRecord<>(TOPIC, 0, 1, TP0_KEY.array(), TP0_VALUE_NEW.array()),
new ConsumerRecord<>(TOPIC, 1, 1, TP1_KEY.array(), TP1_VALUE_NEW.array())
));
expectStop();
PowerMock.replayAll();
store.configure(DEFAULT_PROPS);
Map<TopicPartition, Long> endOffsets = new HashMap<>();
endOffsets.put(TP0, 1L);
endOffsets.put(TP1, 1L);
consumer.updateEndOffsets(endOffsets);
Thread startConsumerOpsThread = new Thread("start-consumer-ops-thread") {
@Override
public void run() {
// Needs to seek to end to find end offsets
consumer.waitForPoll(10000);
// Should keep polling until it reaches current log end offset for all partitions
consumer.waitForPollThen(new Runnable() {
@Override
public void run() {
consumer.addRecord(new ConsumerRecord<>(TOPIC, 0, 0, TP0_KEY.array(), TP0_VALUE.array()));
}
}, 10000);
consumer.waitForPollThen(new Runnable() {
@Override
public void run() {
consumer.addRecord(new ConsumerRecord<>(TOPIC, 1, 0, TP1_KEY.array(), TP1_VALUE.array()));
}
}, 10000);
}
};
startConsumerOpsThread.start();
store.start();
startConsumerOpsThread.join(10000);
assertFalse(startConsumerOpsThread.isAlive());
assertEquals(CONSUMER_ASSIGNMENT, consumer.assignment());
HashMap<ByteBuffer, ByteBuffer> data = Whitebox.getInternalState(store, "data");
assertEquals(TP0_VALUE, data.get(TP0_KEY));
assertEquals(TP1_VALUE, data.get(TP1_KEY));
assertEquals(TP0_VALUE_NEW, data.get(TP0_KEY));
assertEquals(TP1_VALUE_NEW, data.get(TP1_KEY));
store.stop();
assertFalse(Whitebox.<Thread>getInternalState(store, "thread").isAlive());
assertTrue(consumer.closed());
PowerMock.verifyAll();
}
@Test
public void testGetSet() throws Exception {
expectStart();
TestFuture<RecordMetadata> tp0Future = new TestFuture<>();
ProducerRecord<byte[], byte[]> tp0Record = new ProducerRecord<>(TOPIC, TP0_KEY.array(), TP0_VALUE.array());
Capture<org.apache.kafka.clients.producer.Callback> callback1 = EasyMock.newCapture();
EasyMock.expect(producer.send(eqProducerRecord(tp0Record), EasyMock.capture(callback1))).andReturn(tp0Future);
TestFuture<RecordMetadata> tp1Future = new TestFuture<>();
ProducerRecord<byte[], byte[]> tp1Record = new ProducerRecord<>(TOPIC, TP1_KEY.array(), TP1_VALUE.array());
Capture<org.apache.kafka.clients.producer.Callback> callback2 = EasyMock.newCapture();
EasyMock.expect(producer.send(eqProducerRecord(tp1Record), EasyMock.capture(callback2))).andReturn(tp1Future);
expectConfigure();
expectStart(Collections.EMPTY_LIST);
expectStop();
// First get() against an empty store
final Capture<Callback<Void>> firstGetReadToEndCallback = EasyMock.newCapture();
storeLog.readToEnd(EasyMock.capture(firstGetReadToEndCallback));
PowerMock.expectLastCall().andAnswer(new IAnswer<Object>() {
@Override
public Object answer() throws Throwable {
firstGetReadToEndCallback.getValue().onCompletion(null, null);
return null;
}
});
// Set offsets
Capture<org.apache.kafka.clients.producer.Callback> callback0 = EasyMock.newCapture();
storeLog.send(EasyMock.aryEq(TP0_KEY.array()), EasyMock.aryEq(TP0_VALUE.array()), EasyMock.capture(callback0));
PowerMock.expectLastCall();
Capture<org.apache.kafka.clients.producer.Callback> callback1 = EasyMock.newCapture();
storeLog.send(EasyMock.aryEq(TP1_KEY.array()), EasyMock.aryEq(TP1_VALUE.array()), EasyMock.capture(callback1));
PowerMock.expectLastCall();
// Second get() should get the produced data and return the new values
final Capture<Callback<Void>> secondGetReadToEndCallback = EasyMock.newCapture();
storeLog.readToEnd(EasyMock.capture(secondGetReadToEndCallback));
PowerMock.expectLastCall().andAnswer(new IAnswer<Object>() {
@Override
public Object answer() throws Throwable {
capturedConsumedCallback.getValue().onCompletion(null, new ConsumerRecord<>(TOPIC, 0, 0, TP0_KEY.array(), TP0_VALUE.array()));
capturedConsumedCallback.getValue().onCompletion(null, new ConsumerRecord<>(TOPIC, 1, 0, TP1_KEY.array(), TP1_VALUE.array()));
secondGetReadToEndCallback.getValue().onCompletion(null, null);
return null;
}
});
// Third get() should pick up data produced by someone else and return those values
final Capture<Callback<Void>> thirdGetReadToEndCallback = EasyMock.newCapture();
storeLog.readToEnd(EasyMock.capture(thirdGetReadToEndCallback));
PowerMock.expectLastCall().andAnswer(new IAnswer<Object>() {
@Override
public Object answer() throws Throwable {
capturedConsumedCallback.getValue().onCompletion(null, new ConsumerRecord<>(TOPIC, 0, 1, TP0_KEY.array(), TP0_VALUE_NEW.array()));
capturedConsumedCallback.getValue().onCompletion(null, new ConsumerRecord<>(TOPIC, 1, 1, TP1_KEY.array(), TP1_VALUE_NEW.array()));
thirdGetReadToEndCallback.getValue().onCompletion(null, null);
return null;
}
});
PowerMock.replayAll();
store.configure(DEFAULT_PROPS);
Map<TopicPartition, Long> endOffsets = new HashMap<>();
endOffsets.put(TP0, 0L);
endOffsets.put(TP1, 0L);
consumer.updateEndOffsets(endOffsets);
Thread startConsumerOpsThread = new Thread("start-consumer-ops-thread") {
@Override
public void run() {
// Should keep polling until it has partition info
consumer.waitForPollThen(new Runnable() {
@Override
public void run() {
consumer.seek(TP0, 0);
consumer.seek(TP1, 0);
}
}, 10000);
}
};
startConsumerOpsThread.start();
store.start();
startConsumerOpsThread.join(10000);
assertFalse(startConsumerOpsThread.isAlive());
assertEquals(CONSUMER_ASSIGNMENT, consumer.assignment());
Map<ByteBuffer, ByteBuffer> toSet = new HashMap<>();
toSet.put(TP0_KEY, TP0_VALUE);
toSet.put(TP1_KEY, TP1_VALUE);
final AtomicBoolean invoked = new AtomicBoolean(false);
Future<Void> setFuture = store.set(toSet, new Callback<Void>() {
@Override
public void onCompletion(Throwable error, Void result) {
invoked.set(true);
}
});
assertFalse(setFuture.isDone());
tp1Future.resolve((RecordMetadata) null); // Output not used, so safe to not return a real value for testing
assertFalse(setFuture.isDone());
tp0Future.resolve((RecordMetadata) null);
// Out of order callbacks shouldn't matter, should still require all to be invoked before invoking the callback
// for the store's set callback
callback2.getValue().onCompletion(null, null);
assertFalse(invoked.get());
callback1.getValue().onCompletion(null, null);
setFuture.get(10000, TimeUnit.MILLISECONDS);
assertTrue(invoked.get());
// Getting data should continue to return old data...
// Getting from empty store should return nulls
final AtomicBoolean getInvokedAndPassed = new AtomicBoolean(false);
store.get(Arrays.asList(TP0_KEY, TP1_KEY), new Callback<Map<ByteBuffer, ByteBuffer>>() {
@Override
@ -271,184 +216,138 @@ public class KafkaOffsetBackingStoreTest {
}).get(10000, TimeUnit.MILLISECONDS);
assertTrue(getInvokedAndPassed.get());
// Until the consumer gets the new data
Thread readNewDataThread = new Thread("read-new-data-thread") {
// Set some offsets
Map<ByteBuffer, ByteBuffer> toSet = new HashMap<>();
toSet.put(TP0_KEY, TP0_VALUE);
toSet.put(TP1_KEY, TP1_VALUE);
final AtomicBoolean invoked = new AtomicBoolean(false);
Future<Void> setFuture = store.set(toSet, new Callback<Void>() {
@Override
public void run() {
// Should keep polling until it reaches current log end offset for all partitions
consumer.waitForPollThen(new Runnable() {
@Override
public void run() {
consumer.addRecord(new ConsumerRecord<>(TOPIC, 0, 0, TP0_KEY.array(), TP0_VALUE_NEW.array()));
}
}, 10000);
consumer.waitForPollThen(new Runnable() {
@Override
public void run() {
consumer.addRecord(new ConsumerRecord<>(TOPIC, 1, 0, TP1_KEY.array(), TP1_VALUE_NEW.array()));
}
}, 10000);
public void onCompletion(Throwable error, Void result) {
invoked.set(true);
}
};
readNewDataThread.start();
readNewDataThread.join(10000);
assertFalse(readNewDataThread.isAlive());
});
assertFalse(setFuture.isDone());
// Out of order callbacks shouldn't matter, should still require all to be invoked before invoking the callback
// for the store's set callback
callback1.getValue().onCompletion(null, null);
assertFalse(invoked.get());
callback0.getValue().onCompletion(null, null);
setFuture.get(10000, TimeUnit.MILLISECONDS);
assertTrue(invoked.get());
// And now the new data should be returned
final AtomicBoolean finalGetInvokedAndPassed = new AtomicBoolean(false);
// Getting data should read to end of our published data and return it
final AtomicBoolean secondGetInvokedAndPassed = new AtomicBoolean(false);
store.get(Arrays.asList(TP0_KEY, TP1_KEY), new Callback<Map<ByteBuffer, ByteBuffer>>() {
@Override
public void onCompletion(Throwable error, Map<ByteBuffer, ByteBuffer> result) {
assertEquals(TP0_VALUE, result.get(TP0_KEY));
assertEquals(TP1_VALUE, result.get(TP1_KEY));
secondGetInvokedAndPassed.set(true);
}
}).get(10000, TimeUnit.MILLISECONDS);
assertTrue(secondGetInvokedAndPassed.get());
// Getting data should read to end of our published data and return it
final AtomicBoolean thirdGetInvokedAndPassed = new AtomicBoolean(false);
store.get(Arrays.asList(TP0_KEY, TP1_KEY), new Callback<Map<ByteBuffer, ByteBuffer>>() {
@Override
public void onCompletion(Throwable error, Map<ByteBuffer, ByteBuffer> result) {
assertEquals(TP0_VALUE_NEW, result.get(TP0_KEY));
assertEquals(TP1_VALUE_NEW, result.get(TP1_KEY));
finalGetInvokedAndPassed.set(true);
thirdGetInvokedAndPassed.set(true);
}
}).get(10000, TimeUnit.MILLISECONDS);
assertTrue(finalGetInvokedAndPassed.get());
assertTrue(thirdGetInvokedAndPassed.get());
store.stop();
assertFalse(Whitebox.<Thread>getInternalState(store, "thread").isAlive());
assertTrue(consumer.closed());
PowerMock.verifyAll();
}
@Test
public void testConsumerError() throws Exception {
expectStart();
public void testSetFailure() throws Exception {
expectConfigure();
expectStart(Collections.EMPTY_LIST);
expectStop();
PowerMock.replayAll();
store.configure(DEFAULT_PROPS);
Map<TopicPartition, Long> endOffsets = new HashMap<>();
endOffsets.put(TP0, 1L);
endOffsets.put(TP1, 1L);
consumer.updateEndOffsets(endOffsets);
Thread startConsumerOpsThread = new Thread("start-consumer-ops-thread") {
@Override
public void run() {
// Trigger exception
consumer.waitForPollThen(new Runnable() {
@Override
public void run() {
consumer.setException(Errors.CONSUMER_COORDINATOR_NOT_AVAILABLE.exception());
}
}, 10000);
// Should keep polling until it reaches current log end offset for all partitions
consumer.waitForPollThen(new Runnable() {
@Override
public void run() {
consumer.addRecord(new ConsumerRecord<>(TOPIC, 0, 0, TP0_KEY.array(), TP0_VALUE_NEW.array()));
consumer.addRecord(new ConsumerRecord<>(TOPIC, 1, 0, TP0_KEY.array(), TP0_VALUE_NEW.array()));
}
}, 10000);
}
};
startConsumerOpsThread.start();
store.start();
startConsumerOpsThread.join(10000);
assertFalse(startConsumerOpsThread.isAlive());
assertEquals(CONSUMER_ASSIGNMENT, consumer.assignment());
store.stop();
assertFalse(Whitebox.<Thread>getInternalState(store, "thread").isAlive());
assertTrue(consumer.closed());
PowerMock.verifyAll();
}
@Test
public void testProducerError() throws Exception {
expectStart();
TestFuture<RecordMetadata> tp0Future = new TestFuture<>();
ProducerRecord<byte[], byte[]> tp0Record = new ProducerRecord<>(TOPIC, TP0_KEY.array(), TP0_VALUE.array());
// Set offsets
Capture<org.apache.kafka.clients.producer.Callback> callback0 = EasyMock.newCapture();
storeLog.send(EasyMock.aryEq(TP0_KEY.array()), EasyMock.aryEq(TP0_VALUE.array()), EasyMock.capture(callback0));
PowerMock.expectLastCall();
Capture<org.apache.kafka.clients.producer.Callback> callback1 = EasyMock.newCapture();
EasyMock.expect(producer.send(eqProducerRecord(tp0Record), EasyMock.capture(callback1))).andReturn(tp0Future);
TestFuture<RecordMetadata> tp1Future = new TestFuture<>();
ProducerRecord<byte[], byte[]> tp1Record = new ProducerRecord<>(TOPIC, TP1_KEY.array(), TP1_VALUE.array());
storeLog.send(EasyMock.aryEq(TP1_KEY.array()), EasyMock.aryEq(TP1_VALUE.array()), EasyMock.capture(callback1));
PowerMock.expectLastCall();
Capture<org.apache.kafka.clients.producer.Callback> callback2 = EasyMock.newCapture();
EasyMock.expect(producer.send(eqProducerRecord(tp1Record), EasyMock.capture(callback2))).andReturn(tp1Future);
expectStop();
storeLog.send(EasyMock.aryEq(TP2_KEY.array()), EasyMock.aryEq(TP2_VALUE.array()), EasyMock.capture(callback2));
PowerMock.expectLastCall();
PowerMock.replayAll();
store.configure(DEFAULT_PROPS);
Map<TopicPartition, Long> endOffsets = new HashMap<>();
endOffsets.put(TP0, 0L);
endOffsets.put(TP1, 0L);
consumer.updateEndOffsets(endOffsets);
Thread startConsumerOpsThread = new Thread("start-consumer-ops-thread") {
@Override
public void run() {
// Should keep polling until it has partition info
consumer.waitForPollThen(new Runnable() {
@Override
public void run() {
consumer.seek(TP0, 0);
consumer.seek(TP1, 0);
}
}, 10000);
}
};
startConsumerOpsThread.start();
store.start();
startConsumerOpsThread.join(10000);
assertFalse(startConsumerOpsThread.isAlive());
assertEquals(CONSUMER_ASSIGNMENT, consumer.assignment());
// Set some offsets
Map<ByteBuffer, ByteBuffer> toSet = new HashMap<>();
toSet.put(TP0_KEY, TP0_VALUE);
toSet.put(TP1_KEY, TP1_VALUE);
final AtomicReference<Throwable> setException = new AtomicReference<>();
toSet.put(TP2_KEY, TP2_VALUE);
final AtomicBoolean invoked = new AtomicBoolean(false);
final AtomicBoolean invokedFailure = new AtomicBoolean(false);
Future<Void> setFuture = store.set(toSet, new Callback<Void>() {
@Override
public void onCompletion(Throwable error, Void result) {
assertNull(setException.get()); // Should only be invoked once
setException.set(error);
invoked.set(true);
if (error != null)
invokedFailure.set(true);
}
});
assertFalse(setFuture.isDone());
KafkaException exc = new LeaderNotAvailableException("Error");
tp1Future.resolve(exc);
callback2.getValue().onCompletion(null, exc);
// One failure should resolve the future immediately
// Out of order callbacks shouldn't matter, should still require all to be invoked before invoking the callback
// for the store's set callback
callback1.getValue().onCompletion(null, null);
assertFalse(invoked.get());
callback2.getValue().onCompletion(null, new KafkaException("bogus error"));
assertTrue(invoked.get());
assertTrue(invokedFailure.get());
callback0.getValue().onCompletion(null, null);
try {
setFuture.get(10000, TimeUnit.MILLISECONDS);
fail("Should have see ExecutionException");
fail("Should have seen KafkaException thrown when waiting on KafkaOffsetBackingStore.set() future");
} catch (ExecutionException e) {
// expected
assertNotNull(e.getCause());
assertTrue(e.getCause() instanceof KafkaException);
}
assertNotNull(setException.get());
// Callbacks can continue to arrive
tp0Future.resolve((RecordMetadata) null);
callback1.getValue().onCompletion(null, null);
store.stop();
assertFalse(Whitebox.<Thread>getInternalState(store, "thread").isAlive());
assertTrue(consumer.closed());
PowerMock.verifyAll();
}
private void expectConfigure() throws Exception {
PowerMock.expectPrivate(store, "createKafkaBasedLog", EasyMock.capture(capturedTopic), EasyMock.capture(capturedProducerProps),
EasyMock.capture(capturedConsumerProps), EasyMock.capture(capturedConsumedCallback))
.andReturn(storeLog);
}
private void expectStart() throws Exception {
PowerMock.expectPrivate(store, "createProducer")
.andReturn(producer);
PowerMock.expectPrivate(store, "createConsumer")
.andReturn(consumer);
private void expectStart(final List<ConsumerRecord<byte[], byte[]>> preexistingRecords) throws Exception {
storeLog.start();
PowerMock.expectLastCall().andAnswer(new IAnswer<Object>() {
@Override
public Object answer() throws Throwable {
for (ConsumerRecord<byte[], byte[]> rec : preexistingRecords)
capturedConsumedCallback.getValue().onCompletion(null, rec);
return null;
}
});
}
private void expectStop() {
producer.close();
storeLog.stop();
PowerMock.expectLastCall();
// MockConsumer close is checked after test.
}
private static ByteBuffer buffer(String v) {

View File

@ -0,0 +1,463 @@
/**
* 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.copycat.util;
import org.apache.kafka.clients.CommonClientConfigs;
import org.apache.kafka.clients.consumer.ConsumerConfig;
import org.apache.kafka.clients.consumer.ConsumerRecord;
import org.apache.kafka.clients.consumer.MockConsumer;
import org.apache.kafka.clients.consumer.OffsetResetStrategy;
import org.apache.kafka.clients.producer.KafkaProducer;
import org.apache.kafka.clients.producer.ProducerConfig;
import org.apache.kafka.clients.producer.ProducerRecord;
import org.apache.kafka.clients.producer.RecordMetadata;
import org.apache.kafka.common.KafkaException;
import org.apache.kafka.common.Node;
import org.apache.kafka.common.PartitionInfo;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.errors.LeaderNotAvailableException;
import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.utils.Time;
import org.easymock.Capture;
import org.easymock.EasyMock;
import org.junit.Before;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.powermock.api.easymock.PowerMock;
import org.powermock.api.easymock.annotation.Mock;
import org.powermock.core.classloader.annotations.PowerMockIgnore;
import org.powermock.core.classloader.annotations.PrepareForTest;
import org.powermock.modules.junit4.PowerMockRunner;
import org.powermock.reflect.Whitebox;
import java.nio.ByteBuffer;
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.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicReference;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertTrue;
@RunWith(PowerMockRunner.class)
@PrepareForTest(KafkaBasedLog.class)
@PowerMockIgnore("javax.management.*")
public class KafkaBasedLogTest {
private static final String TOPIC = "copycat-log";
private static final TopicPartition TP0 = new TopicPartition(TOPIC, 0);
private static final TopicPartition TP1 = new TopicPartition(TOPIC, 1);
private static final Map<String, Object> PRODUCER_PROPS = new HashMap<>();
static {
PRODUCER_PROPS.put(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, "broker1:9092,broker2:9093");
PRODUCER_PROPS.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.StringSerializer");
PRODUCER_PROPS.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.StringSerializer");
}
private static final Map<String, Object> CONSUMER_PROPS = new HashMap<>();
static {
CONSUMER_PROPS.put(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, "broker1:9092,broker2:9093");
CONSUMER_PROPS.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.StringDeserializer");
CONSUMER_PROPS.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.StringDeserializer");
}
private static final Set<TopicPartition> CONSUMER_ASSIGNMENT = new HashSet<>(Arrays.asList(TP0, TP1));
private static final Map<String, String> FIRST_SET = new HashMap<>();
static {
FIRST_SET.put("key", "value");
FIRST_SET.put(null, null);
}
private static final Node LEADER = new Node(1, "broker1", 9092);
private static final Node REPLICA = new Node(1, "broker2", 9093);
private static final PartitionInfo TPINFO0 = new PartitionInfo(TOPIC, 0, LEADER, new Node[]{REPLICA}, new Node[]{REPLICA});
private static final PartitionInfo TPINFO1 = new PartitionInfo(TOPIC, 1, LEADER, new Node[]{REPLICA}, new Node[]{REPLICA});
private static final String TP0_KEY = "TP0KEY";
private static final String TP1_KEY = "TP1KEY";
private static final String TP0_VALUE = "VAL0";
private static final String TP1_VALUE = "VAL1";
private static final String TP0_VALUE_NEW = "VAL0_NEW";
private static final String TP1_VALUE_NEW = "VAL1_NEW";
private Time time = new MockTime();
private KafkaBasedLog<String, String> store;
@Mock
private KafkaProducer<String, String> producer;
private MockConsumer<String, String> consumer;
private List<ConsumerRecord<String, String>> consumedRecords = new ArrayList<>();
private Callback<ConsumerRecord<String, String>> consumedCallback = new Callback<ConsumerRecord<String, String>>() {
@Override
public void onCompletion(Throwable error, ConsumerRecord<String, String> record) {
consumedRecords.add(record);
}
};
@Before
public void setUp() throws Exception {
store = PowerMock.createPartialMock(KafkaBasedLog.class, new String[]{"createConsumer", "createProducer"},
TOPIC, PRODUCER_PROPS, CONSUMER_PROPS, consumedCallback, time);
consumer = new MockConsumer<>(OffsetResetStrategy.EARLIEST);
consumer.updatePartitions(TOPIC, Arrays.asList(TPINFO0, TPINFO1));
Map<TopicPartition, Long> beginningOffsets = new HashMap<>();
beginningOffsets.put(TP0, 0L);
beginningOffsets.put(TP1, 0L);
consumer.updateBeginningOffsets(beginningOffsets);
}
@Test
public void testStartStop() throws Exception {
expectStart();
expectStop();
PowerMock.replayAll();
Map<TopicPartition, Long> endOffsets = new HashMap<>();
endOffsets.put(TP0, 0L);
endOffsets.put(TP1, 0L);
consumer.updateEndOffsets(endOffsets);
store.start();
assertEquals(CONSUMER_ASSIGNMENT, consumer.assignment());
store.stop();
assertFalse(Whitebox.<Thread>getInternalState(store, "thread").isAlive());
assertTrue(consumer.closed());
PowerMock.verifyAll();
}
@Test
public void testReloadOnStart() throws Exception {
expectStart();
expectStop();
PowerMock.replayAll();
Map<TopicPartition, Long> endOffsets = new HashMap<>();
endOffsets.put(TP0, 1L);
endOffsets.put(TP1, 1L);
consumer.updateEndOffsets(endOffsets);
Thread startConsumerOpsThread = new Thread("start-consumer-ops-thread") {
@Override
public void run() {
// Needs to seek to end to find end offsets
consumer.waitForPoll(10000);
// Should keep polling until it reaches current log end offset for all partitions
consumer.waitForPollThen(new Runnable() {
@Override
public void run() {
consumer.addRecord(new ConsumerRecord<>(TOPIC, 0, 0, TP0_KEY, TP0_VALUE));
}
}, 10000);
consumer.waitForPollThen(new Runnable() {
@Override
public void run() {
consumer.addRecord(new ConsumerRecord<>(TOPIC, 1, 0, TP1_KEY, TP1_VALUE));
}
}, 10000);
}
};
startConsumerOpsThread.start();
store.start();
startConsumerOpsThread.join(10000);
assertFalse(startConsumerOpsThread.isAlive());
assertEquals(CONSUMER_ASSIGNMENT, consumer.assignment());
assertEquals(2, consumedRecords.size());
assertEquals(TP0_VALUE, consumedRecords.get(0).value());
assertEquals(TP1_VALUE, consumedRecords.get(1).value());
store.stop();
assertFalse(Whitebox.<Thread>getInternalState(store, "thread").isAlive());
assertTrue(consumer.closed());
PowerMock.verifyAll();
}
@Test
public void testSendAndReadToEnd() throws Exception {
expectStart();
TestFuture<RecordMetadata> tp0Future = new TestFuture<>();
ProducerRecord<String, String> tp0Record = new ProducerRecord<>(TOPIC, TP0_KEY, TP0_VALUE);
Capture<org.apache.kafka.clients.producer.Callback> callback0 = EasyMock.newCapture();
EasyMock.expect(producer.send(EasyMock.eq(tp0Record), EasyMock.capture(callback0))).andReturn(tp0Future);
TestFuture<RecordMetadata> tp1Future = new TestFuture<>();
ProducerRecord<String, String> tp1Record = new ProducerRecord<>(TOPIC, TP1_KEY, TP1_VALUE);
Capture<org.apache.kafka.clients.producer.Callback> callback1 = EasyMock.newCapture();
EasyMock.expect(producer.send(EasyMock.eq(tp1Record), EasyMock.capture(callback1))).andReturn(tp1Future);
// Producer flushes when read to log end is called
producer.flush();
PowerMock.expectLastCall();
expectStop();
PowerMock.replayAll();
Map<TopicPartition, Long> endOffsets = new HashMap<>();
endOffsets.put(TP0, 0L);
endOffsets.put(TP1, 0L);
consumer.updateEndOffsets(endOffsets);
Thread startConsumerOpsThread = new Thread("start-consumer-ops-thread") {
@Override
public void run() {
// Should keep polling until it has partition info
consumer.waitForPollThen(new Runnable() {
@Override
public void run() {
consumer.seek(TP0, 0);
consumer.seek(TP1, 0);
}
}, 10000);
}
};
startConsumerOpsThread.start();
store.start();
startConsumerOpsThread.join(10000);
assertFalse(startConsumerOpsThread.isAlive());
assertEquals(CONSUMER_ASSIGNMENT, consumer.assignment());
// Set some keys
final AtomicInteger invoked = new AtomicInteger(0);
org.apache.kafka.clients.producer.Callback producerCallback = new org.apache.kafka.clients.producer.Callback() {
@Override
public void onCompletion(RecordMetadata metadata, Exception exception) {
invoked.incrementAndGet();
}
};
store.send(TP0_KEY, TP0_VALUE, producerCallback);
store.send(TP1_KEY, TP1_VALUE, producerCallback);
assertEquals(0, invoked.get());
tp1Future.resolve((RecordMetadata) null); // Output not used, so safe to not return a real value for testing
callback1.getValue().onCompletion(null, null);
assertEquals(1, invoked.get());
tp0Future.resolve((RecordMetadata) null);
callback0.getValue().onCompletion(null, null);
assertEquals(2, invoked.get());
// Now we should have to wait for the records to be read back when we call readToEnd()
final CountDownLatch startOffsetUpdateLatch = new CountDownLatch(1);
Thread readNewDataThread = new Thread("read-new-data-thread") {
@Override
public void run() {
// Needs to be woken up after calling readToEnd()
consumer.waitForPollThen(new Runnable() {
@Override
public void run() {
try {
startOffsetUpdateLatch.await();
} catch (InterruptedException e) {
throw new RuntimeException("Interrupted");
}
}
}, 10000);
// Needs to seek to end to find end offsets
consumer.waitForPollThen(new Runnable() {
@Override
public void run() {
try {
startOffsetUpdateLatch.await();
} catch (InterruptedException e) {
throw new RuntimeException("Interrupted");
}
Map<TopicPartition, Long> newEndOffsets = new HashMap<>();
newEndOffsets.put(TP0, 2L);
newEndOffsets.put(TP1, 2L);
consumer.updateEndOffsets(newEndOffsets);
}
}, 10000);
// Should keep polling until it reaches current log end offset for all partitions
consumer.waitForPollThen(new Runnable() {
@Override
public void run() {
consumer.addRecord(new ConsumerRecord<>(TOPIC, 0, 0, TP0_KEY, TP0_VALUE));
consumer.addRecord(new ConsumerRecord<>(TOPIC, 0, 1, TP0_KEY, TP0_VALUE_NEW));
consumer.addRecord(new ConsumerRecord<>(TOPIC, 1, 0, TP1_KEY, TP1_VALUE));
}
}, 10000);
consumer.waitForPollThen(new Runnable() {
@Override
public void run() {
consumer.addRecord(new ConsumerRecord<>(TOPIC, 1, 1, TP1_KEY, TP1_VALUE_NEW));
}
}, 10000);
}
};
readNewDataThread.start();
final AtomicBoolean getInvokedAndPassed = new AtomicBoolean(false);
FutureCallback<Void> readEndFutureCallback = new FutureCallback<>(new Callback<Void>() {
@Override
public void onCompletion(Throwable error, Void result) {
assertEquals(4, consumedRecords.size());
assertEquals(TP0_VALUE_NEW, consumedRecords.get(2).value());
assertEquals(TP1_VALUE_NEW, consumedRecords.get(3).value());
getInvokedAndPassed.set(true);
}
});
store.readToEnd(readEndFutureCallback);
startOffsetUpdateLatch.countDown();
readNewDataThread.join(10000);
assertFalse(readNewDataThread.isAlive());
readEndFutureCallback.get(10000, TimeUnit.MILLISECONDS);
assertTrue(getInvokedAndPassed.get());
// Cleanup
store.stop();
assertFalse(Whitebox.<Thread>getInternalState(store, "thread").isAlive());
assertTrue(consumer.closed());
PowerMock.verifyAll();
}
@Test
public void testConsumerError() throws Exception {
expectStart();
expectStop();
PowerMock.replayAll();
Map<TopicPartition, Long> endOffsets = new HashMap<>();
endOffsets.put(TP0, 1L);
endOffsets.put(TP1, 1L);
consumer.updateEndOffsets(endOffsets);
Thread startConsumerOpsThread = new Thread("start-consumer-ops-thread") {
@Override
public void run() {
// Trigger exception
consumer.waitForPollThen(new Runnable() {
@Override
public void run() {
consumer.setException(Errors.CONSUMER_COORDINATOR_NOT_AVAILABLE.exception());
}
}, 10000);
// Should keep polling until it reaches current log end offset for all partitions
consumer.waitForPollThen(new Runnable() {
@Override
public void run() {
consumer.addRecord(new ConsumerRecord<>(TOPIC, 0, 0, TP0_KEY, TP0_VALUE_NEW));
consumer.addRecord(new ConsumerRecord<>(TOPIC, 1, 0, TP0_KEY, TP0_VALUE_NEW));
}
}, 10000);
}
};
startConsumerOpsThread.start();
store.start();
startConsumerOpsThread.join(10000);
assertFalse(startConsumerOpsThread.isAlive());
assertEquals(CONSUMER_ASSIGNMENT, consumer.assignment());
store.stop();
assertFalse(Whitebox.<Thread>getInternalState(store, "thread").isAlive());
assertTrue(consumer.closed());
PowerMock.verifyAll();
}
@Test
public void testProducerError() throws Exception {
expectStart();
TestFuture<RecordMetadata> tp0Future = new TestFuture<>();
ProducerRecord<String, String> tp0Record = new ProducerRecord<>(TOPIC, TP0_KEY, TP0_VALUE);
Capture<org.apache.kafka.clients.producer.Callback> callback0 = EasyMock.newCapture();
EasyMock.expect(producer.send(EasyMock.eq(tp0Record), EasyMock.capture(callback0))).andReturn(tp0Future);
expectStop();
PowerMock.replayAll();
Map<TopicPartition, Long> endOffsets = new HashMap<>();
endOffsets.put(TP0, 0L);
endOffsets.put(TP1, 0L);
consumer.updateEndOffsets(endOffsets);
Thread startConsumerOpsThread = new Thread("start-consumer-ops-thread") {
@Override
public void run() {
// Should keep polling until it has partition info
consumer.waitForPollThen(new Runnable() {
@Override
public void run() {
consumer.seek(TP0, 0);
consumer.seek(TP1, 0);
}
}, 10000);
}
};
startConsumerOpsThread.start();
store.start();
startConsumerOpsThread.join(10000);
assertFalse(startConsumerOpsThread.isAlive());
assertEquals(CONSUMER_ASSIGNMENT, consumer.assignment());
final AtomicReference<Throwable> setException = new AtomicReference<>();
store.send(TP0_KEY, TP0_VALUE, new org.apache.kafka.clients.producer.Callback() {
@Override
public void onCompletion(RecordMetadata metadata, Exception exception) {
assertNull(setException.get()); // Should only be invoked once
setException.set(exception);
}
});
KafkaException exc = new LeaderNotAvailableException("Error");
tp0Future.resolve(exc);
callback0.getValue().onCompletion(null, exc);
assertNotNull(setException.get());
store.stop();
assertFalse(Whitebox.<Thread>getInternalState(store, "thread").isAlive());
assertTrue(consumer.closed());
PowerMock.verifyAll();
}
private void expectStart() throws Exception {
PowerMock.expectPrivate(store, "createProducer")
.andReturn(producer);
PowerMock.expectPrivate(store, "createConsumer")
.andReturn(consumer);
}
private void expectStop() {
producer.close();
PowerMock.expectLastCall();
// MockConsumer close is checked after test.
}
private static ByteBuffer buffer(String v) {
return ByteBuffer.wrap(v.getBytes());
}
}

View File

@ -17,6 +17,7 @@
package org.apache.kafka.copycat.util;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.Future;
import java.util.concurrent.TimeUnit;
@ -26,20 +27,35 @@ public class TestFuture<T> implements Future<T> {
private volatile boolean resolved;
private T result;
private Throwable exception;
private CountDownLatch getCalledLatch;
private volatile boolean resolveOnGet;
private T resolveOnGetResult;
private Throwable resolveOnGetException;
public TestFuture() {
resolved = false;
getCalledLatch = new CountDownLatch(1);
resolveOnGet = false;
resolveOnGetResult = null;
resolveOnGetException = null;
}
public void resolve(T val) {
this.result = val;
resolved = true;
synchronized (this) {
this.notifyAll();
}
}
public void resolve(Throwable t) {
exception = t;
resolved = true;
synchronized (this) {
this.notifyAll();
}
}
@Override
@ -59,6 +75,7 @@ public class TestFuture<T> implements Future<T> {
@Override
public T get() throws InterruptedException, ExecutionException {
getCalledLatch.countDown();
while (true) {
try {
return get(Integer.MAX_VALUE, TimeUnit.DAYS);
@ -70,12 +87,69 @@ public class TestFuture<T> implements Future<T> {
@Override
public T get(long timeout, TimeUnit unit) throws InterruptedException, ExecutionException, TimeoutException {
while (!resolved) {
this.wait(TimeUnit.MILLISECONDS.convert(timeout, unit));
getCalledLatch.countDown();
if (resolveOnGet) {
if (resolveOnGetException != null)
resolve(resolveOnGetException);
else
resolve(resolveOnGetResult);
}
synchronized (this) {
while (!resolved) {
this.wait(TimeUnit.MILLISECONDS.convert(timeout, unit));
}
}
if (exception != null)
throw new ExecutionException(exception);
return result;
}
/**
* Set a flag to resolve the future as soon as one of the get() methods has been called. Returns immediately.
* @param val the value to return from the future
*/
public void resolveOnGet(T val) {
resolveOnGet = true;
resolveOnGetResult = val;
}
/**
* Set a flag to resolve the future as soon as one of the get() methods has been called. Returns immediately.
* @param t the exception to return from the future
*/
public void resolveOnGet(Throwable t) {
resolveOnGet = true;
resolveOnGetException = t;
}
/**
* Block, waiting for another thread to call one of the get() methods, and then immediately resolve the future with
* the specified value.
* @param val the value to return from the future
*/
public void waitForGetAndResolve(T val) {
waitForGet();
resolve(val);
}
/**
* Block, waiting for another thread to call one of the get() methods, and then immediately resolve the future with
* the specified value.
* @param t the exception to use to resolve the future
*/
public void waitForGetAndResolve(Throwable t) {
waitForGet();
resolve(t);
}
private void waitForGet() {
try {
getCalledLatch.await();
} catch (InterruptedException e) {
throw new RuntimeException("Unexpected interruption: ", e);
}
}
}

View File

@ -104,9 +104,10 @@ class CopycatStandaloneService(CopycatServiceBase):
class CopycatDistributedService(CopycatServiceBase):
"""Runs Copycat in distributed mode."""
def __init__(self, context, num_nodes, kafka, files, offsets_topic="copycat-offsets"):
def __init__(self, context, num_nodes, kafka, files, offsets_topic="copycat-offsets", configs_topic="copycat-configs"):
super(CopycatDistributedService, self).__init__(context, num_nodes, kafka, files)
self.offsets_topic = offsets_topic
self.configs_topic = configs_topic
def set_configs(self, config_template, connector_config_template):
"""

View File

@ -29,6 +29,7 @@ class CopycatDistributedFileTest(KafkaTest):
TOPIC = "test"
OFFSETS_TOPIC = "copycat-offsets"
CONFIG_TOPIC = "copycat-configs"
FIRST_INPUT_LISTS = [["foo", "bar", "baz"], ["foo2", "bar2", "baz2"]]
FIRST_INPUTS = ["\n".join(input_list) + "\n" for input_list in FIRST_INPUT_LISTS]
@ -42,8 +43,13 @@ class CopycatDistributedFileTest(KafkaTest):
'test' : { 'partitions': 1, 'replication-factor': 1 }
})
self.source = CopycatDistributedService(test_context, 2, self.kafka, [self.INPUT_FILE])
self.sink = CopycatDistributedService(test_context, 2, self.kafka, [self.OUTPUT_FILE])
# FIXME these should have multiple nodes. However, currently the connectors are submitted via command line,
# which means we would get duplicates. Both would run, but they would have conflicting keys for offsets and
# configs. Until we have real distributed coordination of workers with unified connector submission, we need
# to restrict each of these to a single node.
self.num_nodes = 1
self.source = CopycatDistributedService(test_context, self.num_nodes, self.kafka, [self.INPUT_FILE])
self.sink = CopycatDistributedService(test_context, self.num_nodes, self.kafka, [self.OUTPUT_FILE])
def test_file_source_and_sink(self, converter="org.apache.kafka.copycat.json.JsonConverter", schemas=True):
assert converter != None, "converter type must be set"
@ -62,7 +68,7 @@ class CopycatDistributedFileTest(KafkaTest):
# Generating data on the source node should generate new records and create new output on the sink node
for node, input in zip(self.source.nodes, self.FIRST_INPUTS):
node.account.ssh("echo -e -n " + repr(input) + " >> " + self.INPUT_FILE)
wait_until(lambda: self.validate_output(self.FIRST_INPUT_LISTS), timeout_sec=60, err_msg="Data added to input file was not seen in the output file in a reasonable amount of time.")
wait_until(lambda: self.validate_output(self.FIRST_INPUT_LISTS[:self.num_nodes]), timeout_sec=60, err_msg="Data added to input file was not seen in the output file in a reasonable amount of time.")
# Restarting both should result in them picking up where they left off,
# only processing new data.
@ -71,7 +77,7 @@ class CopycatDistributedFileTest(KafkaTest):
for node, input in zip(self.source.nodes, self.SECOND_INPUTS):
node.account.ssh("echo -e -n " + repr(input) + " >> " + self.INPUT_FILE)
wait_until(lambda: self.validate_output(self.FIRST_INPUT_LISTS + self.SECOND_INPUT_LISTS), timeout_sec=60, err_msg="Sink output file never converged to the same state as the input file")
wait_until(lambda: self.validate_output(self.FIRST_INPUT_LISTS[:self.num_nodes] + self.SECOND_INPUT_LISTS[:self.num_nodes]), timeout_sec=60, err_msg="Sink output file never converged to the same state as the input file")
def validate_output(self, inputs):
try:

View File

@ -24,9 +24,10 @@ key.converter.schemas.enable={{ schemas|default(True)|string|lower }}
value.converter.schemas.enable={{ schemas|default(True)|string|lower }}
{% endif %}
offset.key.converter=org.apache.kafka.copycat.json.JsonConverter
offset.value.converter=org.apache.kafka.copycat.json.JsonConverter
offset.key.converter.schemas.enable=false
offset.value.converter.schemas.enable=false
internal.key.converter=org.apache.kafka.copycat.json.JsonConverter
internal.value.converter=org.apache.kafka.copycat.json.JsonConverter
internal.key.converter.schemas.enable=false
internal.value.converter.schemas.enable=false
offset.storage.topic={{ OFFSETS_TOPIC }}
config.storage.topic={{ CONFIG_TOPIC }}

View File

@ -24,9 +24,9 @@ key.converter.schemas.enable={{ schemas|default(True)|string|lower }}
value.converter.schemas.enable={{ schemas|default(True)|string|lower }}
{% endif %}
offset.key.converter=org.apache.kafka.copycat.json.JsonConverter
offset.value.converter=org.apache.kafka.copycat.json.JsonConverter
offset.key.converter.schemas.enable=false
offset.value.converter.schemas.enable=false
internal.key.converter=org.apache.kafka.copycat.json.JsonConverter
internal.value.converter=org.apache.kafka.copycat.json.JsonConverter
internal.key.converter.schemas.enable=false
internal.value.converter.schemas.enable=false
offset.storage.file.filename={{ OFFSETS_FILE }}