mirror of https://github.com/apache/kafka.git
KAFKA-10835: Replace Runnable and Callable overrides with lambdas in Connect (#9867)
Reviewers: Konstantine Karantasis <k.karantasis@gmail.com>
This commit is contained in:
parent
131d4753cf
commit
c19a35d1b7
|
@ -69,7 +69,8 @@ public class ConnectSchemaTest {
|
|||
|
||||
@Test
|
||||
public void testFieldsOnlyValidForStructs() {
|
||||
assertThrows(DataException.class, Schema.INT8_SCHEMA::fields);
|
||||
assertThrows(DataException.class,
|
||||
Schema.INT8_SCHEMA::fields);
|
||||
}
|
||||
|
||||
@Test
|
||||
|
|
|
@ -345,24 +345,16 @@ public class ConnectHeadersTest {
|
|||
}
|
||||
|
||||
protected HeaderTransform appendToKey(final String suffix) {
|
||||
return new HeaderTransform() {
|
||||
@Override
|
||||
public Header apply(Header header) {
|
||||
return header.rename(header.key() + suffix);
|
||||
}
|
||||
};
|
||||
return header -> header.rename(header.key() + suffix);
|
||||
}
|
||||
|
||||
protected HeaderTransform removeHeadersOfType(final Type type) {
|
||||
return new HeaderTransform() {
|
||||
@Override
|
||||
public Header apply(Header header) {
|
||||
Schema schema = header.schema();
|
||||
if (schema != null && schema.type() == type) {
|
||||
return null;
|
||||
}
|
||||
return header;
|
||||
return header -> {
|
||||
Schema schema = header.schema();
|
||||
if (schema != null && schema.type() == type) {
|
||||
return null;
|
||||
}
|
||||
return header;
|
||||
};
|
||||
}
|
||||
|
||||
|
|
|
@ -66,125 +66,73 @@ public class JsonConverter implements Converter, HeaderConverter {
|
|||
private static final Map<Schema.Type, JsonToConnectTypeConverter> TO_CONNECT_CONVERTERS = new EnumMap<>(Schema.Type.class);
|
||||
|
||||
static {
|
||||
TO_CONNECT_CONVERTERS.put(Schema.Type.BOOLEAN, new JsonToConnectTypeConverter() {
|
||||
@Override
|
||||
public Object convert(Schema schema, JsonNode value) {
|
||||
return value.booleanValue();
|
||||
TO_CONNECT_CONVERTERS.put(Schema.Type.BOOLEAN, (schema, value) -> value.booleanValue());
|
||||
TO_CONNECT_CONVERTERS.put(Schema.Type.INT8, (schema, value) -> (byte) value.intValue());
|
||||
TO_CONNECT_CONVERTERS.put(Schema.Type.INT16, (schema, value) -> (short) value.intValue());
|
||||
TO_CONNECT_CONVERTERS.put(Schema.Type.INT32, (schema, value) -> value.intValue());
|
||||
TO_CONNECT_CONVERTERS.put(Schema.Type.INT64, (schema, value) -> value.longValue());
|
||||
TO_CONNECT_CONVERTERS.put(Schema.Type.FLOAT32, (schema, value) -> value.floatValue());
|
||||
TO_CONNECT_CONVERTERS.put(Schema.Type.FLOAT64, (schema, value) -> value.doubleValue());
|
||||
TO_CONNECT_CONVERTERS.put(Schema.Type.BYTES, (schema, value) -> {
|
||||
try {
|
||||
return value.binaryValue();
|
||||
} catch (IOException e) {
|
||||
throw new DataException("Invalid bytes field", e);
|
||||
}
|
||||
});
|
||||
TO_CONNECT_CONVERTERS.put(Schema.Type.INT8, new JsonToConnectTypeConverter() {
|
||||
@Override
|
||||
public Object convert(Schema schema, JsonNode value) {
|
||||
return (byte) value.intValue();
|
||||
TO_CONNECT_CONVERTERS.put(Schema.Type.STRING, (schema, value) -> value.textValue());
|
||||
TO_CONNECT_CONVERTERS.put(Schema.Type.ARRAY, (schema, value) -> {
|
||||
Schema elemSchema = schema == null ? null : schema.valueSchema();
|
||||
ArrayList<Object> result = new ArrayList<>();
|
||||
for (JsonNode elem : value) {
|
||||
result.add(convertToConnect(elemSchema, elem));
|
||||
}
|
||||
return result;
|
||||
});
|
||||
TO_CONNECT_CONVERTERS.put(Schema.Type.INT16, new JsonToConnectTypeConverter() {
|
||||
@Override
|
||||
public Object convert(Schema schema, JsonNode value) {
|
||||
return (short) value.intValue();
|
||||
}
|
||||
});
|
||||
TO_CONNECT_CONVERTERS.put(Schema.Type.INT32, new JsonToConnectTypeConverter() {
|
||||
@Override
|
||||
public Object convert(Schema schema, JsonNode value) {
|
||||
return value.intValue();
|
||||
}
|
||||
});
|
||||
TO_CONNECT_CONVERTERS.put(Schema.Type.INT64, new JsonToConnectTypeConverter() {
|
||||
@Override
|
||||
public Object convert(Schema schema, JsonNode value) {
|
||||
return value.longValue();
|
||||
}
|
||||
});
|
||||
TO_CONNECT_CONVERTERS.put(Schema.Type.FLOAT32, new JsonToConnectTypeConverter() {
|
||||
@Override
|
||||
public Object convert(Schema schema, JsonNode value) {
|
||||
return value.floatValue();
|
||||
}
|
||||
});
|
||||
TO_CONNECT_CONVERTERS.put(Schema.Type.FLOAT64, new JsonToConnectTypeConverter() {
|
||||
@Override
|
||||
public Object convert(Schema schema, JsonNode value) {
|
||||
return value.doubleValue();
|
||||
}
|
||||
});
|
||||
TO_CONNECT_CONVERTERS.put(Schema.Type.BYTES, new JsonToConnectTypeConverter() {
|
||||
@Override
|
||||
public Object convert(Schema schema, JsonNode value) {
|
||||
try {
|
||||
return value.binaryValue();
|
||||
} catch (IOException e) {
|
||||
throw new DataException("Invalid bytes field", e);
|
||||
}
|
||||
}
|
||||
});
|
||||
TO_CONNECT_CONVERTERS.put(Schema.Type.STRING, new JsonToConnectTypeConverter() {
|
||||
@Override
|
||||
public Object convert(Schema schema, JsonNode value) {
|
||||
return value.textValue();
|
||||
}
|
||||
});
|
||||
TO_CONNECT_CONVERTERS.put(Schema.Type.ARRAY, new JsonToConnectTypeConverter() {
|
||||
@Override
|
||||
public Object convert(Schema schema, JsonNode value) {
|
||||
Schema elemSchema = schema == null ? null : schema.valueSchema();
|
||||
ArrayList<Object> result = new ArrayList<>();
|
||||
for (JsonNode elem : value) {
|
||||
result.add(convertToConnect(elemSchema, elem));
|
||||
}
|
||||
return result;
|
||||
}
|
||||
});
|
||||
TO_CONNECT_CONVERTERS.put(Schema.Type.MAP, new JsonToConnectTypeConverter() {
|
||||
@Override
|
||||
public Object convert(Schema schema, JsonNode value) {
|
||||
Schema keySchema = schema == null ? null : schema.keySchema();
|
||||
Schema valueSchema = schema == null ? null : schema.valueSchema();
|
||||
TO_CONNECT_CONVERTERS.put(Schema.Type.MAP, (schema, value) -> {
|
||||
Schema keySchema = schema == null ? null : schema.keySchema();
|
||||
Schema valueSchema = schema == null ? null : schema.valueSchema();
|
||||
|
||||
// If the map uses strings for keys, it should be encoded in the natural JSON format. If it uses other
|
||||
// primitive types or a complex type as a key, it will be encoded as a list of pairs. If we don't have a
|
||||
// schema, we default to encoding in a Map.
|
||||
Map<Object, Object> result = new HashMap<>();
|
||||
if (schema == null || keySchema.type() == Schema.Type.STRING) {
|
||||
if (!value.isObject())
|
||||
throw new DataException("Maps with string fields should be encoded as JSON objects, but found " + value.getNodeType());
|
||||
Iterator<Map.Entry<String, JsonNode>> fieldIt = value.fields();
|
||||
while (fieldIt.hasNext()) {
|
||||
Map.Entry<String, JsonNode> entry = fieldIt.next();
|
||||
result.put(entry.getKey(), convertToConnect(valueSchema, entry.getValue()));
|
||||
}
|
||||
} else {
|
||||
if (!value.isArray())
|
||||
throw new DataException("Maps with non-string fields should be encoded as JSON array of tuples, but found " + value.getNodeType());
|
||||
for (JsonNode entry : value) {
|
||||
if (!entry.isArray())
|
||||
throw new DataException("Found invalid map entry instead of array tuple: " + entry.getNodeType());
|
||||
if (entry.size() != 2)
|
||||
throw new DataException("Found invalid map entry, expected length 2 but found :" + entry.size());
|
||||
result.put(convertToConnect(keySchema, entry.get(0)),
|
||||
convertToConnect(valueSchema, entry.get(1)));
|
||||
}
|
||||
}
|
||||
return result;
|
||||
}
|
||||
});
|
||||
TO_CONNECT_CONVERTERS.put(Schema.Type.STRUCT, new JsonToConnectTypeConverter() {
|
||||
@Override
|
||||
public Object convert(Schema schema, JsonNode value) {
|
||||
// If the map uses strings for keys, it should be encoded in the natural JSON format. If it uses other
|
||||
// primitive types or a complex type as a key, it will be encoded as a list of pairs. If we don't have a
|
||||
// schema, we default to encoding in a Map.
|
||||
Map<Object, Object> result = new HashMap<>();
|
||||
if (schema == null || keySchema.type() == Schema.Type.STRING) {
|
||||
if (!value.isObject())
|
||||
throw new DataException("Structs should be encoded as JSON objects, but found " + value.getNodeType());
|
||||
|
||||
// We only have ISchema here but need Schema, so we need to materialize the actual schema. Using ISchema
|
||||
// avoids having to materialize the schema for non-Struct types but it cannot be avoided for Structs since
|
||||
// they require a schema to be provided at construction. However, the schema is only a SchemaBuilder during
|
||||
// translation of schemas to JSON; during the more common translation of data to JSON, the call to schema.schema()
|
||||
// just returns the schema Object and has no overhead.
|
||||
Struct result = new Struct(schema.schema());
|
||||
for (Field field : schema.fields())
|
||||
result.put(field, convertToConnect(field.schema(), value.get(field.name())));
|
||||
|
||||
return result;
|
||||
throw new DataException("Maps with string fields should be encoded as JSON objects, but found " + value.getNodeType());
|
||||
Iterator<Map.Entry<String, JsonNode>> fieldIt = value.fields();
|
||||
while (fieldIt.hasNext()) {
|
||||
Map.Entry<String, JsonNode> entry = fieldIt.next();
|
||||
result.put(entry.getKey(), convertToConnect(valueSchema, entry.getValue()));
|
||||
}
|
||||
} else {
|
||||
if (!value.isArray())
|
||||
throw new DataException("Maps with non-string fields should be encoded as JSON array of tuples, but found " + value.getNodeType());
|
||||
for (JsonNode entry : value) {
|
||||
if (!entry.isArray())
|
||||
throw new DataException("Found invalid map entry instead of array tuple: " + entry.getNodeType());
|
||||
if (entry.size() != 2)
|
||||
throw new DataException("Found invalid map entry, expected length 2 but found :" + entry.size());
|
||||
result.put(convertToConnect(keySchema, entry.get(0)),
|
||||
convertToConnect(valueSchema, entry.get(1)));
|
||||
}
|
||||
}
|
||||
return result;
|
||||
});
|
||||
TO_CONNECT_CONVERTERS.put(Schema.Type.STRUCT, (schema, value) -> {
|
||||
if (!value.isObject())
|
||||
throw new DataException("Structs should be encoded as JSON objects, but found " + value.getNodeType());
|
||||
|
||||
// We only have ISchema here but need Schema, so we need to materialize the actual schema. Using ISchema
|
||||
// avoids having to materialize the schema for non-Struct types but it cannot be avoided for Structs since
|
||||
// they require a schema to be provided at construction. However, the schema is only a SchemaBuilder during
|
||||
// translation of schemas to JSON; during the more common translation of data to JSON, the call to schema.schema()
|
||||
// just returns the schema Object and has no overhead.
|
||||
Struct result = new Struct(schema.schema());
|
||||
for (Field field : schema.fields())
|
||||
result.put(field, convertToConnect(field.schema(), value.get(field.name())));
|
||||
|
||||
return result;
|
||||
});
|
||||
}
|
||||
|
||||
|
|
|
@ -148,7 +148,7 @@ public class MirrorCheckpointConnector extends SourceConnector {
|
|||
List<String> findConsumerGroups()
|
||||
throws InterruptedException, ExecutionException {
|
||||
return listConsumerGroups().stream()
|
||||
.map(x -> x.groupId())
|
||||
.map(ConsumerGroupListing::groupId)
|
||||
.filter(this::shouldReplicate)
|
||||
.collect(Collectors.toList());
|
||||
}
|
||||
|
|
|
@ -235,7 +235,7 @@ public class MirrorCheckpointTask extends SourceTask {
|
|||
if (consumerGroupState.equals(ConsumerGroupState.EMPTY)) {
|
||||
idleConsumerGroupsOffset.put(group, targetAdminClient.listConsumerGroupOffsets(group)
|
||||
.partitionsToOffsetAndMetadata().get().entrySet().stream().collect(
|
||||
Collectors.toMap(e -> e.getKey(), e -> e.getValue())));
|
||||
Collectors.toMap(Entry::getKey, Entry::getValue)));
|
||||
}
|
||||
// new consumer upstream has state "DEAD" and will be identified during the offset sync-up
|
||||
} catch (InterruptedException | ExecutionException e) {
|
||||
|
|
|
@ -133,7 +133,7 @@ public class MirrorMaker {
|
|||
if (herderPairs.isEmpty()) {
|
||||
throw new IllegalArgumentException("No source->target replication flows.");
|
||||
}
|
||||
this.herderPairs.forEach(x -> addHerder(x));
|
||||
this.herderPairs.forEach(this::addHerder);
|
||||
shutdownHook = new ShutdownHook();
|
||||
}
|
||||
|
||||
|
@ -173,7 +173,7 @@ public class MirrorMaker {
|
|||
}
|
||||
}
|
||||
log.info("Configuring connectors...");
|
||||
herderPairs.forEach(x -> configureConnectors(x));
|
||||
herderPairs.forEach(this::configureConnectors);
|
||||
log.info("Kafka MirrorMaker started");
|
||||
}
|
||||
|
||||
|
|
|
@ -16,6 +16,7 @@
|
|||
*/
|
||||
package org.apache.kafka.connect.mirror;
|
||||
|
||||
import java.util.Map.Entry;
|
||||
import org.apache.kafka.common.utils.Utils;
|
||||
import org.apache.kafka.common.config.AbstractConfig;
|
||||
import org.apache.kafka.common.config.ConfigDef;
|
||||
|
@ -266,7 +267,7 @@ public class MirrorMakerConfig extends AbstractConfig {
|
|||
private Map<String, String> stringsWithPrefixStripped(String prefix) {
|
||||
return originalsStrings().entrySet().stream()
|
||||
.filter(x -> x.getKey().startsWith(prefix))
|
||||
.collect(Collectors.toMap(x -> x.getKey().substring(prefix.length()), x -> x.getValue()));
|
||||
.collect(Collectors.toMap(x -> x.getKey().substring(prefix.length()), Entry::getValue));
|
||||
}
|
||||
|
||||
private Map<String, String> stringsWithPrefix(String prefix) {
|
||||
|
@ -278,12 +279,12 @@ public class MirrorMakerConfig extends AbstractConfig {
|
|||
static Map<String, String> clusterConfigsWithPrefix(String prefix, Map<String, String> props) {
|
||||
return props.entrySet().stream()
|
||||
.filter(x -> !x.getKey().matches("(^consumer.*|^producer.*|^admin.*)"))
|
||||
.collect(Collectors.toMap(x -> prefix + x.getKey(), x -> x.getValue()));
|
||||
.collect(Collectors.toMap(x -> prefix + x.getKey(), Entry::getValue));
|
||||
}
|
||||
|
||||
static Map<String, String> clientConfigsWithPrefix(String prefix, Map<String, String> props) {
|
||||
return props.entrySet().stream()
|
||||
.filter(x -> x.getKey().matches("(^consumer.*|^producer.*|^admin.*)"))
|
||||
.collect(Collectors.toMap(x -> prefix + x.getKey(), x -> x.getValue()));
|
||||
.collect(Collectors.toMap(x -> prefix + x.getKey(), Entry::getValue));
|
||||
}
|
||||
}
|
||||
|
|
|
@ -115,7 +115,7 @@ class MirrorMetrics implements AutoCloseable {
|
|||
ReplicationPolicy replicationPolicy = taskConfig.replicationPolicy();
|
||||
partitionMetrics = taskConfig.taskTopicPartitions().stream()
|
||||
.map(x -> new TopicPartition(replicationPolicy.formatRemoteTopic(source, x.topic()), x.partition()))
|
||||
.collect(Collectors.toMap(x -> x, x -> new PartitionMetrics(x)));
|
||||
.collect(Collectors.toMap(x -> x, PartitionMetrics::new));
|
||||
|
||||
}
|
||||
|
||||
|
|
|
@ -16,6 +16,7 @@
|
|||
*/
|
||||
package org.apache.kafka.connect.mirror;
|
||||
|
||||
import java.util.Map.Entry;
|
||||
import org.apache.kafka.connect.connector.Task;
|
||||
import org.apache.kafka.connect.source.SourceConnector;
|
||||
import org.apache.kafka.common.config.ConfigDef;
|
||||
|
@ -269,7 +270,7 @@ public class MirrorSourceConnector extends SourceConnector {
|
|||
private Set<String> topicsBeingReplicated() {
|
||||
Set<String> knownTargetTopics = toTopics(knownTargetTopicPartitions);
|
||||
return knownSourceTopicPartitions.stream()
|
||||
.map(x -> x.topic())
|
||||
.map(TopicPartition::topic)
|
||||
.distinct()
|
||||
.filter(x -> knownTargetTopics.contains(formatRemoteTopic(x)))
|
||||
.collect(Collectors.toSet());
|
||||
|
@ -277,7 +278,7 @@ public class MirrorSourceConnector extends SourceConnector {
|
|||
|
||||
private Set<String> toTopics(Collection<TopicPartition> tps) {
|
||||
return tps.stream()
|
||||
.map(x -> x.topic())
|
||||
.map(TopicPartition::topic)
|
||||
.collect(Collectors.toSet());
|
||||
}
|
||||
|
||||
|
@ -309,8 +310,8 @@ public class MirrorSourceConnector extends SourceConnector {
|
|||
void computeAndCreateTopicPartitions()
|
||||
throws InterruptedException, ExecutionException {
|
||||
Map<String, Long> partitionCounts = knownSourceTopicPartitions.stream()
|
||||
.collect(Collectors.groupingBy(x -> x.topic(), Collectors.counting())).entrySet().stream()
|
||||
.collect(Collectors.toMap(x -> formatRemoteTopic(x.getKey()), x -> x.getValue()));
|
||||
.collect(Collectors.groupingBy(TopicPartition::topic, Collectors.counting())).entrySet().stream()
|
||||
.collect(Collectors.toMap(x -> formatRemoteTopic(x.getKey()), Entry::getValue));
|
||||
Set<String> knownTargetTopics = toTopics(knownTargetTopicPartitions);
|
||||
List<NewTopic> newTopics = partitionCounts.entrySet().stream()
|
||||
.filter(x -> !knownTargetTopics.contains(x.getKey()))
|
||||
|
@ -318,7 +319,7 @@ public class MirrorSourceConnector extends SourceConnector {
|
|||
.collect(Collectors.toList());
|
||||
Map<String, NewPartitions> newPartitions = partitionCounts.entrySet().stream()
|
||||
.filter(x -> knownTargetTopics.contains(x.getKey()))
|
||||
.collect(Collectors.toMap(x -> x.getKey(), x -> NewPartitions.increaseTo(x.getValue().intValue())));
|
||||
.collect(Collectors.toMap(Entry::getKey, x -> NewPartitions.increaseTo(x.getValue().intValue())));
|
||||
createTopicPartitions(partitionCounts, newTopics, newPartitions);
|
||||
}
|
||||
|
||||
|
@ -364,7 +365,7 @@ public class MirrorSourceConnector extends SourceConnector {
|
|||
throws InterruptedException, ExecutionException {
|
||||
Map<ConfigResource, Config> configs = topicConfigs.entrySet().stream()
|
||||
.collect(Collectors.toMap(x ->
|
||||
new ConfigResource(ConfigResource.Type.TOPIC, x.getKey()), x -> x.getValue()));
|
||||
new ConfigResource(ConfigResource.Type.TOPIC, x.getKey()), Entry::getValue));
|
||||
log.trace("Syncing configs for {} topics.", configs.size());
|
||||
targetAdminClient.alterConfigs(configs).values().forEach((k, v) -> v.whenComplete((x, e) -> {
|
||||
if (e != null) {
|
||||
|
@ -395,7 +396,7 @@ public class MirrorSourceConnector extends SourceConnector {
|
|||
.map(x -> new ConfigResource(ConfigResource.Type.TOPIC, x))
|
||||
.collect(Collectors.toSet());
|
||||
return sourceAdminClient.describeConfigs(resources).all().get().entrySet().stream()
|
||||
.collect(Collectors.toMap(x -> x.getKey().name(), x -> x.getValue()));
|
||||
.collect(Collectors.toMap(x -> x.getKey().name(), Entry::getValue));
|
||||
}
|
||||
|
||||
Config targetConfig(Config sourceConfig) {
|
||||
|
|
|
@ -219,7 +219,7 @@ public class MirrorSourceTask extends SourceTask {
|
|||
}
|
||||
|
||||
private Map<TopicPartition, Long> loadOffsets(Set<TopicPartition> topicPartitions) {
|
||||
return topicPartitions.stream().collect(Collectors.toMap(x -> x, x -> loadOffset(x)));
|
||||
return topicPartitions.stream().collect(Collectors.toMap(x -> x, this::loadOffset));
|
||||
}
|
||||
|
||||
private Long loadOffset(TopicPartition topicPartition) {
|
||||
|
|
|
@ -90,7 +90,7 @@ public class MirrorCheckpointConnectorTest {
|
|||
doReturn(true).when(connector).shouldReplicate(anyString());
|
||||
List<String> groupFound = connector.findConsumerGroups();
|
||||
|
||||
Set<String> expectedGroups = groups.stream().map(g -> g.groupId()).collect(Collectors.toSet());
|
||||
Set<String> expectedGroups = groups.stream().map(ConsumerGroupListing::groupId).collect(Collectors.toSet());
|
||||
assertEquals(expectedGroups, new HashSet<>(groupFound));
|
||||
}
|
||||
|
||||
|
|
|
@ -522,7 +522,7 @@ public abstract class MirrorConnectorsIntegrationBaseTest {
|
|||
Map<TopicPartition, OffsetAndMetadata> consumerGroupOffsets =
|
||||
adminClient.listConsumerGroupOffsets(consumerGroupId).partitionsToOffsetAndMetadata().get();
|
||||
long consumerGroupOffsetTotal = consumerGroupOffsets.values().stream()
|
||||
.mapToLong(metadata -> metadata.offset()).sum();
|
||||
.mapToLong(OffsetAndMetadata::offset).sum();
|
||||
|
||||
Map<TopicPartition, Long> offsets = consumer.endOffsets(tps, CONSUMER_POLL_TIMEOUT_MS);
|
||||
long totalOffsets = offsets.values().stream().mapToLong(l -> l).sum();
|
||||
|
|
|
@ -32,7 +32,6 @@ import org.apache.kafka.connect.runtime.rest.entities.ConnectorInfo;
|
|||
import org.apache.kafka.connect.runtime.standalone.StandaloneConfig;
|
||||
import org.apache.kafka.connect.runtime.standalone.StandaloneHerder;
|
||||
import org.apache.kafka.connect.storage.FileOffsetBackingStore;
|
||||
import org.apache.kafka.connect.util.Callback;
|
||||
import org.apache.kafka.connect.util.ConnectUtils;
|
||||
import org.apache.kafka.connect.util.FutureCallback;
|
||||
import org.slf4j.Logger;
|
||||
|
@ -103,14 +102,11 @@ public class ConnectStandalone {
|
|||
connect.start();
|
||||
for (final String connectorPropsFile : Arrays.copyOfRange(args, 1, args.length)) {
|
||||
Map<String, String> connectorProps = Utils.propsToStringMap(Utils.loadProps(connectorPropsFile));
|
||||
FutureCallback<Herder.Created<ConnectorInfo>> cb = new FutureCallback<>(new Callback<Herder.Created<ConnectorInfo>>() {
|
||||
@Override
|
||||
public void onCompletion(Throwable error, Herder.Created<ConnectorInfo> info) {
|
||||
if (error != null)
|
||||
log.error("Failed to create job for {}", connectorPropsFile);
|
||||
else
|
||||
log.info("Created connector {}", info.result().name());
|
||||
}
|
||||
FutureCallback<Herder.Created<ConnectorInfo>> cb = new FutureCallback<>((error, info) -> {
|
||||
if (error != null)
|
||||
log.error("Failed to create job for {}", connectorPropsFile);
|
||||
else
|
||||
log.info("Created connector {}", info.result().name());
|
||||
});
|
||||
herder.putConnectorConfig(
|
||||
connectorProps.get(ConnectorConfig.NAME_CONFIG),
|
||||
|
|
|
@ -79,12 +79,9 @@ class SourceTaskOffsetCommitter {
|
|||
|
||||
public void schedule(final ConnectorTaskId id, final WorkerSourceTask workerTask) {
|
||||
long commitIntervalMs = config.getLong(WorkerConfig.OFFSET_COMMIT_INTERVAL_MS_CONFIG);
|
||||
ScheduledFuture<?> commitFuture = commitExecutorService.scheduleWithFixedDelay(new Runnable() {
|
||||
@Override
|
||||
public void run() {
|
||||
try (LoggingContext loggingContext = LoggingContext.forOffsets(id)) {
|
||||
commit(workerTask);
|
||||
}
|
||||
ScheduledFuture<?> commitFuture = commitExecutorService.scheduleWithFixedDelay(() -> {
|
||||
try (LoggingContext loggingContext = LoggingContext.forOffsets(id)) {
|
||||
commit(workerTask);
|
||||
}
|
||||
}, commitIntervalMs, commitIntervalMs, TimeUnit.MILLISECONDS);
|
||||
committers.put(id, commitFuture);
|
||||
|
|
|
@ -90,12 +90,9 @@ public class WorkerConfigTransformer implements AutoCloseable {
|
|||
}
|
||||
}
|
||||
log.info("Scheduling a restart of connector {} in {} ms", connectorName, ttl);
|
||||
Callback<Void> cb = new Callback<Void>() {
|
||||
@Override
|
||||
public void onCompletion(Throwable error, Void result) {
|
||||
if (error != null) {
|
||||
log.error("Unexpected error during connector restart: ", error);
|
||||
}
|
||||
Callback<Void> cb = (error, result) -> {
|
||||
if (error != null) {
|
||||
log.error("Unexpected error during connector restart: ", error);
|
||||
}
|
||||
};
|
||||
HerderRequest request = worker.herder().restartConnector(ttl, connectorName, cb);
|
||||
|
|
|
@ -350,12 +350,7 @@ class WorkerSinkTask extends WorkerTask {
|
|||
|
||||
private void doCommitAsync(Map<TopicPartition, OffsetAndMetadata> offsets, final int seqno) {
|
||||
log.debug("{} Committing offsets asynchronously using sequence number {}: {}", this, seqno, offsets);
|
||||
OffsetCommitCallback cb = new OffsetCommitCallback() {
|
||||
@Override
|
||||
public void onComplete(Map<TopicPartition, OffsetAndMetadata> offsets, Exception error) {
|
||||
onCommitCompleted(error, seqno, offsets);
|
||||
}
|
||||
};
|
||||
OffsetCommitCallback cb = (tpOffsets, error) -> onCommitCompleted(error, seqno, tpOffsets);
|
||||
consumer.commitAsync(offsets, cb);
|
||||
}
|
||||
|
||||
|
|
|
@ -18,15 +18,12 @@ package org.apache.kafka.connect.runtime;
|
|||
|
||||
import org.apache.kafka.common.MetricName;
|
||||
import org.apache.kafka.common.MetricNameTemplate;
|
||||
import org.apache.kafka.common.metrics.Measurable;
|
||||
import org.apache.kafka.common.metrics.MetricConfig;
|
||||
import org.apache.kafka.common.metrics.Sensor;
|
||||
import org.apache.kafka.common.metrics.stats.Avg;
|
||||
import org.apache.kafka.common.metrics.stats.Frequencies;
|
||||
import org.apache.kafka.common.metrics.stats.Max;
|
||||
import org.apache.kafka.common.utils.Time;
|
||||
import org.apache.kafka.connect.runtime.AbstractStatus.State;
|
||||
import org.apache.kafka.connect.runtime.ConnectMetrics.LiteralSupplier;
|
||||
import org.apache.kafka.connect.runtime.ConnectMetrics.MetricGroup;
|
||||
import org.apache.kafka.connect.runtime.errors.RetryWithToleranceOperator;
|
||||
import org.apache.kafka.connect.runtime.isolation.Plugins;
|
||||
|
@ -344,12 +341,9 @@ abstract class WorkerTask implements Runnable {
|
|||
// prevent collisions by removing any previously created metrics in this group.
|
||||
metricGroup.close();
|
||||
|
||||
metricGroup.addValueMetric(registry.taskStatus, new LiteralSupplier<String>() {
|
||||
@Override
|
||||
public String metricValue(long now) {
|
||||
return taskStateTimer.currentState().toString().toLowerCase(Locale.getDefault());
|
||||
}
|
||||
});
|
||||
metricGroup.addValueMetric(registry.taskStatus, now ->
|
||||
taskStateTimer.currentState().toString().toLowerCase(Locale.getDefault())
|
||||
);
|
||||
|
||||
addRatioMetric(State.RUNNING, registry.taskRunningRatio);
|
||||
addRatioMetric(State.PAUSED, registry.taskPauseRatio);
|
||||
|
@ -372,12 +366,8 @@ abstract class WorkerTask implements Runnable {
|
|||
private void addRatioMetric(final State matchingState, MetricNameTemplate template) {
|
||||
MetricName metricName = metricGroup.metricName(template);
|
||||
if (metricGroup.metrics().metric(metricName) == null) {
|
||||
metricGroup.metrics().addMetric(metricName, new Measurable() {
|
||||
@Override
|
||||
public double measure(MetricConfig config, long now) {
|
||||
return taskStateTimer.durationRatio(matchingState, now);
|
||||
}
|
||||
});
|
||||
metricGroup.metrics().addMetric(metricName, (config, now) ->
|
||||
taskStateTimer.durationRatio(matchingState, now));
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -36,7 +36,6 @@ import org.apache.kafka.connect.errors.NotFoundException;
|
|||
import org.apache.kafka.connect.runtime.AbstractHerder;
|
||||
import org.apache.kafka.connect.runtime.CloseableConnectorContext;
|
||||
import org.apache.kafka.connect.runtime.ConnectMetrics;
|
||||
import org.apache.kafka.connect.runtime.ConnectMetrics.LiteralSupplier;
|
||||
import org.apache.kafka.connect.runtime.ConnectMetrics.MetricGroup;
|
||||
import org.apache.kafka.connect.runtime.ConnectMetricsRegistry;
|
||||
import org.apache.kafka.connect.runtime.ConnectorConfig;
|
||||
|
@ -708,17 +707,12 @@ public class DistributedHerder extends AbstractHerder implements Runnable {
|
|||
log.trace("Submitting connector listing request");
|
||||
|
||||
addRequest(
|
||||
new Callable<Void>() {
|
||||
@Override
|
||||
public Void call() throws Exception {
|
||||
if (checkRebalanceNeeded(callback))
|
||||
return null;
|
||||
|
||||
callback.onCompletion(null, configState.connectors());
|
||||
return null;
|
||||
}
|
||||
},
|
||||
forwardErrorCallback(callback)
|
||||
() -> {
|
||||
if (!checkRebalanceNeeded(callback))
|
||||
callback.onCompletion(null, configState.connectors());
|
||||
return null;
|
||||
},
|
||||
forwardErrorCallback(callback)
|
||||
);
|
||||
}
|
||||
|
||||
|
@ -727,21 +721,19 @@ public class DistributedHerder extends AbstractHerder implements Runnable {
|
|||
log.trace("Submitting connector info request {}", connName);
|
||||
|
||||
addRequest(
|
||||
new Callable<Void>() {
|
||||
@Override
|
||||
public Void call() throws Exception {
|
||||
if (checkRebalanceNeeded(callback))
|
||||
return null;
|
||||
() -> {
|
||||
if (checkRebalanceNeeded(callback))
|
||||
return null;
|
||||
|
||||
if (!configState.contains(connName)) {
|
||||
callback.onCompletion(new NotFoundException("Connector " + connName + " not found"), null);
|
||||
} else {
|
||||
callback.onCompletion(null, connectorInfo(connName));
|
||||
}
|
||||
return null;
|
||||
}
|
||||
},
|
||||
forwardErrorCallback(callback)
|
||||
if (!configState.contains(connName)) {
|
||||
callback.onCompletion(
|
||||
new NotFoundException("Connector " + connName + " not found"), null);
|
||||
} else {
|
||||
callback.onCompletion(null, connectorInfo(connName));
|
||||
}
|
||||
return null;
|
||||
},
|
||||
forwardErrorCallback(callback)
|
||||
);
|
||||
}
|
||||
|
||||
|
@ -759,26 +751,23 @@ public class DistributedHerder extends AbstractHerder implements Runnable {
|
|||
@Override
|
||||
public void deleteConnectorConfig(final String connName, final Callback<Created<ConnectorInfo>> callback) {
|
||||
addRequest(
|
||||
new Callable<Void>() {
|
||||
@Override
|
||||
public Void call() throws Exception {
|
||||
log.trace("Handling connector config request {}", connName);
|
||||
if (!isLeader()) {
|
||||
callback.onCompletion(new NotLeaderException("Only the leader can delete connector configs.", leaderUrl()), null);
|
||||
return null;
|
||||
}
|
||||
() -> {
|
||||
log.trace("Handling connector config request {}", connName);
|
||||
if (!isLeader()) {
|
||||
callback.onCompletion(new NotLeaderException("Only the leader can delete connector configs.", leaderUrl()), null);
|
||||
return null;
|
||||
}
|
||||
|
||||
if (!configState.contains(connName)) {
|
||||
callback.onCompletion(new NotFoundException("Connector " + connName + " not found"), null);
|
||||
} else {
|
||||
log.trace("Removing connector config {} {}", connName, configState.connectors());
|
||||
configBackingStore.removeConnectorConfig(connName);
|
||||
callback.onCompletion(null, new Created<ConnectorInfo>(false, null));
|
||||
}
|
||||
return null;
|
||||
}
|
||||
},
|
||||
forwardErrorCallback(callback)
|
||||
if (!configState.contains(connName)) {
|
||||
callback.onCompletion(new NotFoundException("Connector " + connName + " not found"), null);
|
||||
} else {
|
||||
log.trace("Removing connector config {} {}", connName, configState.connectors());
|
||||
configBackingStore.removeConnectorConfig(connName);
|
||||
callback.onCompletion(null, new Created<ConnectorInfo>(false, null));
|
||||
}
|
||||
return null;
|
||||
},
|
||||
forwardErrorCallback(callback)
|
||||
);
|
||||
}
|
||||
|
||||
|
@ -804,56 +793,50 @@ public class DistributedHerder extends AbstractHerder implements Runnable {
|
|||
final Callback<Created<ConnectorInfo>> callback) {
|
||||
log.trace("Submitting connector config write request {}", connName);
|
||||
addRequest(
|
||||
new Callable<Void>() {
|
||||
@Override
|
||||
public Void call() throws Exception {
|
||||
validateConnectorConfig(config, (error, configInfos) -> {
|
||||
if (error != null) {
|
||||
callback.onCompletion(error, null);
|
||||
return;
|
||||
() -> {
|
||||
validateConnectorConfig(config, (error, configInfos) -> {
|
||||
if (error != null) {
|
||||
callback.onCompletion(error, null);
|
||||
return;
|
||||
}
|
||||
|
||||
// Complete the connector config write via another herder request in order to
|
||||
// perform the write to the backing store (or forward to the leader) during
|
||||
// the "external request" portion of the tick loop
|
||||
addRequest(
|
||||
() -> {
|
||||
if (maybeAddConfigErrors(configInfos, callback)) {
|
||||
return null;
|
||||
}
|
||||
|
||||
// Complete the connector config write via another herder request in order to
|
||||
// perform the write to the backing store (or forward to the leader) during
|
||||
// the "external request" portion of the tick loop
|
||||
addRequest(
|
||||
new Callable<Void>() {
|
||||
@Override
|
||||
public Void call() {
|
||||
if (maybeAddConfigErrors(configInfos, callback)) {
|
||||
return null;
|
||||
}
|
||||
log.trace("Handling connector config request {}", connName);
|
||||
if (!isLeader()) {
|
||||
callback.onCompletion(new NotLeaderException("Only the leader can set connector configs.", leaderUrl()), null);
|
||||
return null;
|
||||
}
|
||||
boolean exists = configState.contains(connName);
|
||||
if (!allowReplace && exists) {
|
||||
callback.onCompletion(new AlreadyExistsException("Connector " + connName + " already exists"), null);
|
||||
return null;
|
||||
}
|
||||
|
||||
log.trace("Handling connector config request {}", connName);
|
||||
if (!isLeader()) {
|
||||
callback.onCompletion(new NotLeaderException("Only the leader can set connector configs.", leaderUrl()), null);
|
||||
return null;
|
||||
}
|
||||
boolean exists = configState.contains(connName);
|
||||
if (!allowReplace && exists) {
|
||||
callback.onCompletion(new AlreadyExistsException("Connector " + connName + " already exists"), null);
|
||||
return null;
|
||||
}
|
||||
log.trace("Submitting connector config {} {} {}", connName, allowReplace, configState.connectors());
|
||||
configBackingStore.putConnectorConfig(connName, config);
|
||||
|
||||
log.trace("Submitting connector config {} {} {}", connName, allowReplace, configState.connectors());
|
||||
configBackingStore.putConnectorConfig(connName, config);
|
||||
|
||||
// Note that we use the updated connector config despite the fact that we don't have an updated
|
||||
// snapshot yet. The existing task info should still be accurate.
|
||||
ConnectorInfo info = new ConnectorInfo(connName, config, configState.tasks(connName),
|
||||
// validateConnectorConfig have checked the existence of CONNECTOR_CLASS_CONFIG
|
||||
connectorTypeForClass(config.get(ConnectorConfig.CONNECTOR_CLASS_CONFIG)));
|
||||
callback.onCompletion(null, new Created<>(!exists, info));
|
||||
return null;
|
||||
}
|
||||
},
|
||||
forwardErrorCallback(callback)
|
||||
);
|
||||
});
|
||||
return null;
|
||||
}
|
||||
},
|
||||
forwardErrorCallback(callback)
|
||||
// Note that we use the updated connector config despite the fact that we don't have an updated
|
||||
// snapshot yet. The existing task info should still be accurate.
|
||||
ConnectorInfo info = new ConnectorInfo(connName, config, configState.tasks(connName),
|
||||
// validateConnectorConfig have checked the existence of CONNECTOR_CLASS_CONFIG
|
||||
connectorTypeForClass(config.get(ConnectorConfig.CONNECTOR_CLASS_CONFIG)));
|
||||
callback.onCompletion(null, new Created<>(!exists, info));
|
||||
return null;
|
||||
},
|
||||
forwardErrorCallback(callback)
|
||||
);
|
||||
});
|
||||
return null;
|
||||
},
|
||||
forwardErrorCallback(callback)
|
||||
);
|
||||
}
|
||||
|
||||
|
@ -862,12 +845,9 @@ public class DistributedHerder extends AbstractHerder implements Runnable {
|
|||
log.trace("Submitting connector task reconfiguration request {}", connName);
|
||||
|
||||
addRequest(
|
||||
new Callable<Void>() {
|
||||
@Override
|
||||
public Void call() throws Exception {
|
||||
reconfigureConnectorTasksWithRetry(time.milliseconds(), connName);
|
||||
return null;
|
||||
}
|
||||
() -> {
|
||||
reconfigureConnectorTasksWithRetry(time.milliseconds(), connName);
|
||||
return null;
|
||||
},
|
||||
(error, result) -> {
|
||||
if (error != null) {
|
||||
|
@ -883,26 +863,23 @@ public class DistributedHerder extends AbstractHerder implements Runnable {
|
|||
log.trace("Submitting get task configuration request {}", connName);
|
||||
|
||||
addRequest(
|
||||
new Callable<Void>() {
|
||||
@Override
|
||||
public Void call() throws Exception {
|
||||
if (checkRebalanceNeeded(callback))
|
||||
return null;
|
||||
() -> {
|
||||
if (checkRebalanceNeeded(callback))
|
||||
return null;
|
||||
|
||||
if (!configState.contains(connName)) {
|
||||
callback.onCompletion(new NotFoundException("Connector " + connName + " not found"), null);
|
||||
} else {
|
||||
List<TaskInfo> result = new ArrayList<>();
|
||||
for (int i = 0; i < configState.taskCount(connName); i++) {
|
||||
ConnectorTaskId id = new ConnectorTaskId(connName, i);
|
||||
result.add(new TaskInfo(id, configState.rawTaskConfig(id)));
|
||||
}
|
||||
callback.onCompletion(null, result);
|
||||
}
|
||||
return null;
|
||||
if (!configState.contains(connName)) {
|
||||
callback.onCompletion(new NotFoundException("Connector " + connName + " not found"), null);
|
||||
} else {
|
||||
List<TaskInfo> result = new ArrayList<>();
|
||||
for (int i = 0; i < configState.taskCount(connName); i++) {
|
||||
ConnectorTaskId id = new ConnectorTaskId(connName, i);
|
||||
result.add(new TaskInfo(id, configState.rawTaskConfig(id)));
|
||||
}
|
||||
},
|
||||
forwardErrorCallback(callback)
|
||||
callback.onCompletion(null, result);
|
||||
}
|
||||
return null;
|
||||
},
|
||||
forwardErrorCallback(callback)
|
||||
);
|
||||
}
|
||||
|
||||
|
@ -937,21 +914,18 @@ public class DistributedHerder extends AbstractHerder implements Runnable {
|
|||
}
|
||||
|
||||
addRequest(
|
||||
new Callable<Void>() {
|
||||
@Override
|
||||
public Void call() throws Exception {
|
||||
if (!isLeader())
|
||||
callback.onCompletion(new NotLeaderException("Only the leader may write task configurations.", leaderUrl()), null);
|
||||
else if (!configState.contains(connName))
|
||||
callback.onCompletion(new NotFoundException("Connector " + connName + " not found"), null);
|
||||
else {
|
||||
configBackingStore.putTaskConfigs(connName, configs);
|
||||
callback.onCompletion(null, null);
|
||||
}
|
||||
return null;
|
||||
}
|
||||
},
|
||||
forwardErrorCallback(callback)
|
||||
() -> {
|
||||
if (!isLeader())
|
||||
callback.onCompletion(new NotLeaderException("Only the leader may write task configurations.", leaderUrl()), null);
|
||||
else if (!configState.contains(connName))
|
||||
callback.onCompletion(new NotFoundException("Connector " + connName + " not found"), null);
|
||||
else {
|
||||
configBackingStore.putTaskConfigs(connName, configs);
|
||||
callback.onCompletion(null, null);
|
||||
}
|
||||
return null;
|
||||
},
|
||||
forwardErrorCallback(callback)
|
||||
);
|
||||
}
|
||||
|
||||
|
@ -962,9 +936,9 @@ public class DistributedHerder extends AbstractHerder implements Runnable {
|
|||
|
||||
@Override
|
||||
public HerderRequest restartConnector(final long delayMs, final String connName, final Callback<Void> callback) {
|
||||
return addRequest(delayMs, new Callable<Void>() {
|
||||
@Override
|
||||
public Void call() throws Exception {
|
||||
return addRequest(
|
||||
delayMs,
|
||||
() -> {
|
||||
if (checkRebalanceNeeded(callback))
|
||||
return null;
|
||||
|
||||
|
@ -986,15 +960,14 @@ public class DistributedHerder extends AbstractHerder implements Runnable {
|
|||
callback.onCompletion(new NotLeaderException("Cannot restart connector since it is not assigned to this member", leaderUrl()), null);
|
||||
}
|
||||
return null;
|
||||
}
|
||||
}, forwardErrorCallback(callback));
|
||||
},
|
||||
forwardErrorCallback(callback));
|
||||
}
|
||||
|
||||
@Override
|
||||
public void restartTask(final ConnectorTaskId id, final Callback<Void> callback) {
|
||||
addRequest(new Callable<Void>() {
|
||||
@Override
|
||||
public Void call() throws Exception {
|
||||
addRequest(
|
||||
() -> {
|
||||
if (checkRebalanceNeeded(callback))
|
||||
return null;
|
||||
|
||||
|
@ -1024,8 +997,8 @@ public class DistributedHerder extends AbstractHerder implements Runnable {
|
|||
callback.onCompletion(new NotLeaderException("Cannot restart task since it is not assigned to this member", leaderUrl()), null);
|
||||
}
|
||||
return null;
|
||||
}
|
||||
}, forwardErrorCallback(callback));
|
||||
},
|
||||
forwardErrorCallback(callback));
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -1261,28 +1234,22 @@ public class DistributedHerder extends AbstractHerder implements Runnable {
|
|||
}
|
||||
|
||||
private Callable<Void> getTaskStartingCallable(final ConnectorTaskId taskId) {
|
||||
return new Callable<Void>() {
|
||||
@Override
|
||||
public Void call() throws Exception {
|
||||
try {
|
||||
startTask(taskId);
|
||||
} catch (Throwable t) {
|
||||
log.error("Couldn't instantiate task {} because it has an invalid task configuration. This task will not execute until reconfigured.",
|
||||
taskId, t);
|
||||
onFailure(taskId, t);
|
||||
}
|
||||
return null;
|
||||
return () -> {
|
||||
try {
|
||||
startTask(taskId);
|
||||
} catch (Throwable t) {
|
||||
log.error("Couldn't instantiate task {} because it has an invalid task configuration. This task will not execute until reconfigured.",
|
||||
taskId, t);
|
||||
onFailure(taskId, t);
|
||||
}
|
||||
return null;
|
||||
};
|
||||
}
|
||||
|
||||
private Callable<Void> getTaskStoppingCallable(final ConnectorTaskId taskId) {
|
||||
return new Callable<Void>() {
|
||||
@Override
|
||||
public Void call() throws Exception {
|
||||
worker.stopAndAwaitTask(taskId);
|
||||
return null;
|
||||
}
|
||||
return () -> {
|
||||
worker.stopAndAwaitTask(taskId);
|
||||
return null;
|
||||
};
|
||||
}
|
||||
|
||||
|
@ -1303,18 +1270,15 @@ public class DistributedHerder extends AbstractHerder implements Runnable {
|
|||
// Use newState here in case the connector has been paused right after being created
|
||||
if (newState == TargetState.STARTED) {
|
||||
addRequest(
|
||||
new Callable<Void>() {
|
||||
@Override
|
||||
public Void call() {
|
||||
// Request configuration since this could be a brand new connector. However, also only update those
|
||||
// task configs if they are actually different from the existing ones to avoid unnecessary updates when this is
|
||||
// just restoring an existing connector.
|
||||
reconfigureConnectorTasksWithRetry(time.milliseconds(), connectorName);
|
||||
callback.onCompletion(null, null);
|
||||
return null;
|
||||
}
|
||||
},
|
||||
forwardErrorCallback(callback)
|
||||
() -> {
|
||||
// Request configuration since this could be a brand new connector. However, also only update those
|
||||
// task configs if they are actually different from the existing ones to avoid unnecessary updates when this is
|
||||
// just restoring an existing connector.
|
||||
reconfigureConnectorTasksWithRetry(time.milliseconds(), connectorName);
|
||||
callback.onCompletion(null, null);
|
||||
return null;
|
||||
},
|
||||
forwardErrorCallback(callback)
|
||||
);
|
||||
} else {
|
||||
callback.onCompletion(null, null);
|
||||
|
@ -1324,70 +1288,55 @@ public class DistributedHerder extends AbstractHerder implements Runnable {
|
|||
}
|
||||
|
||||
private Callable<Void> getConnectorStartingCallable(final String connectorName) {
|
||||
return new Callable<Void>() {
|
||||
@Override
|
||||
public Void call() throws Exception {
|
||||
try {
|
||||
startConnector(connectorName, (error, result) -> {
|
||||
if (error != null) {
|
||||
log.error("Failed to start connector '" + connectorName + "'", error);
|
||||
}
|
||||
});
|
||||
} catch (Throwable t) {
|
||||
log.error("Unexpected error while trying to start connector " + connectorName, t);
|
||||
onFailure(connectorName, t);
|
||||
}
|
||||
return null;
|
||||
return () -> {
|
||||
try {
|
||||
startConnector(connectorName, (error, result) -> {
|
||||
if (error != null) {
|
||||
log.error("Failed to start connector '" + connectorName + "'", error);
|
||||
}
|
||||
});
|
||||
} catch (Throwable t) {
|
||||
log.error("Unexpected error while trying to start connector " + connectorName, t);
|
||||
onFailure(connectorName, t);
|
||||
}
|
||||
return null;
|
||||
};
|
||||
}
|
||||
|
||||
private Callable<Void> getConnectorStoppingCallable(final String connectorName) {
|
||||
return new Callable<Void>() {
|
||||
@Override
|
||||
public Void call() throws Exception {
|
||||
try {
|
||||
worker.stopAndAwaitConnector(connectorName);
|
||||
} catch (Throwable t) {
|
||||
log.error("Failed to shut down connector " + connectorName, t);
|
||||
}
|
||||
return null;
|
||||
return () -> {
|
||||
try {
|
||||
worker.stopAndAwaitConnector(connectorName);
|
||||
} catch (Throwable t) {
|
||||
log.error("Failed to shut down connector " + connectorName, t);
|
||||
}
|
||||
return null;
|
||||
};
|
||||
}
|
||||
|
||||
private void reconfigureConnectorTasksWithRetry(long initialRequestTime, final String connName) {
|
||||
reconfigureConnector(connName, new Callback<Void>() {
|
||||
@Override
|
||||
public void onCompletion(Throwable error, Void result) {
|
||||
// If we encountered an error, we don't have much choice but to just retry. If we don't, we could get
|
||||
// stuck with a connector that thinks it has generated tasks, but wasn't actually successful and therefore
|
||||
// never makes progress. The retry has to run through a DistributedHerderRequest since this callback could be happening
|
||||
// from the HTTP request forwarding thread.
|
||||
if (error != null) {
|
||||
if (isPossibleExpiredKeyException(initialRequestTime, error)) {
|
||||
log.debug("Failed to reconfigure connector's tasks ({}), possibly due to expired session key. Retrying after backoff", connName);
|
||||
} else {
|
||||
log.error("Failed to reconfigure connector's tasks ({}), retrying after backoff:", connName, error);
|
||||
}
|
||||
addRequest(RECONFIGURE_CONNECTOR_TASKS_BACKOFF_MS,
|
||||
new Callable<Void>() {
|
||||
@Override
|
||||
public Void call() throws Exception {
|
||||
reconfigureConnectorTasksWithRetry(initialRequestTime, connName);
|
||||
return null;
|
||||
}
|
||||
}, new Callback<Void>() {
|
||||
@Override
|
||||
public void onCompletion(Throwable error, Void result) {
|
||||
if (error != null) {
|
||||
log.error("Unexpected error during connector task reconfiguration: ", error);
|
||||
log.error("Task reconfiguration for {} failed unexpectedly, this connector will not be properly reconfigured unless manually triggered.", connName);
|
||||
}
|
||||
}
|
||||
}
|
||||
);
|
||||
reconfigureConnector(connName, (error, result) -> {
|
||||
// If we encountered an error, we don't have much choice but to just retry. If we don't, we could get
|
||||
// stuck with a connector that thinks it has generated tasks, but wasn't actually successful and therefore
|
||||
// never makes progress. The retry has to run through a DistributedHerderRequest since this callback could be happening
|
||||
// from the HTTP request forwarding thread.
|
||||
if (error != null) {
|
||||
if (isPossibleExpiredKeyException(initialRequestTime, error)) {
|
||||
log.debug("Failed to reconfigure connector's tasks ({}), possibly due to expired session key. Retrying after backoff", connName);
|
||||
} else {
|
||||
log.error("Failed to reconfigure connector's tasks ({}), retrying after backoff:", connName, error);
|
||||
}
|
||||
addRequest(RECONFIGURE_CONNECTOR_TASKS_BACKOFF_MS,
|
||||
() -> {
|
||||
reconfigureConnectorTasksWithRetry(initialRequestTime, connName);
|
||||
return null;
|
||||
}, (err, res) -> {
|
||||
if (err != null) {
|
||||
log.error("Unexpected error during connector task reconfiguration: ", err);
|
||||
log.error("Task reconfiguration for {} failed unexpectedly, this connector will not be properly reconfigured unless manually triggered.", connName);
|
||||
}
|
||||
}
|
||||
);
|
||||
}
|
||||
});
|
||||
}
|
||||
|
@ -1807,30 +1756,12 @@ public class DistributedHerder extends AbstractHerder implements Runnable {
|
|||
ConnectMetricsRegistry registry = connectMetrics.registry();
|
||||
metricGroup = connectMetrics.group(registry.workerRebalanceGroupName());
|
||||
|
||||
metricGroup.addValueMetric(registry.connectProtocol, new LiteralSupplier<String>() {
|
||||
@Override
|
||||
public String metricValue(long now) {
|
||||
return ConnectProtocolCompatibility.fromProtocolVersion(member.currentProtocolVersion()).name();
|
||||
}
|
||||
});
|
||||
metricGroup.addValueMetric(registry.leaderName, new LiteralSupplier<String>() {
|
||||
@Override
|
||||
public String metricValue(long now) {
|
||||
return leaderUrl();
|
||||
}
|
||||
});
|
||||
metricGroup.addValueMetric(registry.epoch, new LiteralSupplier<Double>() {
|
||||
@Override
|
||||
public Double metricValue(long now) {
|
||||
return (double) generation;
|
||||
}
|
||||
});
|
||||
metricGroup.addValueMetric(registry.rebalanceMode, new LiteralSupplier<Double>() {
|
||||
@Override
|
||||
public Double metricValue(long now) {
|
||||
return rebalancing ? 1.0d : 0.0d;
|
||||
}
|
||||
});
|
||||
metricGroup.addValueMetric(registry.connectProtocol, now ->
|
||||
ConnectProtocolCompatibility.fromProtocolVersion(member.currentProtocolVersion()).name()
|
||||
);
|
||||
metricGroup.addValueMetric(registry.leaderName, now -> leaderUrl());
|
||||
metricGroup.addValueMetric(registry.epoch, now -> (double) generation);
|
||||
metricGroup.addValueMetric(registry.rebalanceMode, now -> rebalancing ? 1.0d : 0.0d);
|
||||
|
||||
rebalanceCompletedCounts = metricGroup.sensor("completed-rebalance-count");
|
||||
rebalanceCompletedCounts.add(metricGroup.metricName(registry.rebalanceCompletedTotal), new CumulativeSum());
|
||||
|
@ -1839,12 +1770,8 @@ public class DistributedHerder extends AbstractHerder implements Runnable {
|
|||
rebalanceTime.add(metricGroup.metricName(registry.rebalanceTimeMax), new Max());
|
||||
rebalanceTime.add(metricGroup.metricName(registry.rebalanceTimeAvg), new Avg());
|
||||
|
||||
metricGroup.addValueMetric(registry.rebalanceTimeSinceLast, new LiteralSupplier<Double>() {
|
||||
@Override
|
||||
public Double metricValue(long now) {
|
||||
return lastRebalanceCompletedAtMillis == Long.MIN_VALUE ? Double.POSITIVE_INFINITY : (double) (now - lastRebalanceCompletedAtMillis);
|
||||
}
|
||||
});
|
||||
metricGroup.addValueMetric(registry.rebalanceTimeSinceLast, now ->
|
||||
lastRebalanceCompletedAtMillis == Long.MIN_VALUE ? Double.POSITIVE_INFINITY : (double) (now - lastRebalanceCompletedAtMillis));
|
||||
}
|
||||
|
||||
void close() {
|
||||
|
|
|
@ -16,6 +16,7 @@
|
|||
*/
|
||||
package org.apache.kafka.connect.runtime.distributed;
|
||||
|
||||
import java.util.Map.Entry;
|
||||
import org.apache.kafka.common.utils.LogContext;
|
||||
import org.apache.kafka.common.utils.Time;
|
||||
import org.apache.kafka.connect.runtime.distributed.WorkerCoordinator.ConnectorsAndTasks;
|
||||
|
@ -368,7 +369,7 @@ public class IncrementalCooperativeAssignor implements ConnectAssignor {
|
|||
.collect(Collectors.groupingBy(Function.identity(), Collectors.counting()))
|
||||
.entrySet().stream()
|
||||
.filter(entry -> entry.getValue() > 1L)
|
||||
.map(entry -> entry.getKey())
|
||||
.map(Entry::getKey)
|
||||
.collect(Collectors.toSet());
|
||||
|
||||
Set<ConnectorTaskId> tasks = memberConfigs.values().stream()
|
||||
|
@ -376,7 +377,7 @@ public class IncrementalCooperativeAssignor implements ConnectAssignor {
|
|||
.collect(Collectors.groupingBy(Function.identity(), Collectors.counting()))
|
||||
.entrySet().stream()
|
||||
.filter(entry -> entry.getValue() > 1L)
|
||||
.map(entry -> entry.getKey())
|
||||
.map(Entry::getKey)
|
||||
.collect(Collectors.toSet());
|
||||
return new ConnectorsAndTasks.Builder().with(connectors, tasks).build();
|
||||
}
|
||||
|
|
|
@ -20,7 +20,6 @@ import org.apache.kafka.clients.consumer.internals.AbstractCoordinator;
|
|||
import org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient;
|
||||
import org.apache.kafka.clients.GroupRebalanceConfig;
|
||||
import org.apache.kafka.common.metrics.Measurable;
|
||||
import org.apache.kafka.common.metrics.MetricConfig;
|
||||
import org.apache.kafka.common.metrics.Metrics;
|
||||
import org.apache.kafka.common.requests.JoinGroupRequest;
|
||||
import org.apache.kafka.common.utils.LogContext;
|
||||
|
@ -347,26 +346,20 @@ public class WorkerCoordinator extends AbstractCoordinator implements Closeable
|
|||
public WorkerCoordinatorMetrics(Metrics metrics, String metricGrpPrefix) {
|
||||
this.metricGrpName = metricGrpPrefix + "-coordinator-metrics";
|
||||
|
||||
Measurable numConnectors = new Measurable() {
|
||||
@Override
|
||||
public double measure(MetricConfig config, long now) {
|
||||
final ExtendedAssignment localAssignmentSnapshot = assignmentSnapshot;
|
||||
if (localAssignmentSnapshot == null) {
|
||||
return 0.0;
|
||||
}
|
||||
return localAssignmentSnapshot.connectors().size();
|
||||
Measurable numConnectors = (config, now) -> {
|
||||
final ExtendedAssignment localAssignmentSnapshot = assignmentSnapshot;
|
||||
if (localAssignmentSnapshot == null) {
|
||||
return 0.0;
|
||||
}
|
||||
return localAssignmentSnapshot.connectors().size();
|
||||
};
|
||||
|
||||
Measurable numTasks = new Measurable() {
|
||||
@Override
|
||||
public double measure(MetricConfig config, long now) {
|
||||
final ExtendedAssignment localAssignmentSnapshot = assignmentSnapshot;
|
||||
if (localAssignmentSnapshot == null) {
|
||||
return 0.0;
|
||||
}
|
||||
return localAssignmentSnapshot.tasks().size();
|
||||
Measurable numTasks = (config, now) -> {
|
||||
final ExtendedAssignment localAssignmentSnapshot = assignmentSnapshot;
|
||||
if (localAssignmentSnapshot == null) {
|
||||
return 0.0;
|
||||
}
|
||||
return localAssignmentSnapshot.tasks().size();
|
||||
};
|
||||
|
||||
metrics.addMetric(metrics.metricName("assigned-connectors",
|
||||
|
|
|
@ -293,32 +293,29 @@ public class DelegatingClassLoader extends URLClassLoader {
|
|||
// Apply here what java.sql.DriverManager does to discover and register classes
|
||||
// implementing the java.sql.Driver interface.
|
||||
AccessController.doPrivileged(
|
||||
new PrivilegedAction<Void>() {
|
||||
@Override
|
||||
public Void run() {
|
||||
ServiceLoader<Driver> loadedDrivers = ServiceLoader.load(
|
||||
Driver.class,
|
||||
loader
|
||||
(PrivilegedAction<Void>) () -> {
|
||||
ServiceLoader<Driver> loadedDrivers = ServiceLoader.load(
|
||||
Driver.class,
|
||||
loader
|
||||
);
|
||||
Iterator<Driver> driversIterator = loadedDrivers.iterator();
|
||||
try {
|
||||
while (driversIterator.hasNext()) {
|
||||
Driver driver = driversIterator.next();
|
||||
log.debug(
|
||||
"Registered java.sql.Driver: {} to java.sql.DriverManager",
|
||||
driver
|
||||
);
|
||||
Iterator<Driver> driversIterator = loadedDrivers.iterator();
|
||||
try {
|
||||
while (driversIterator.hasNext()) {
|
||||
Driver driver = driversIterator.next();
|
||||
log.debug(
|
||||
"Registered java.sql.Driver: {} to java.sql.DriverManager",
|
||||
driver
|
||||
);
|
||||
}
|
||||
} catch (Throwable t) {
|
||||
log.debug(
|
||||
"Ignoring java.sql.Driver classes listed in resources but not"
|
||||
+ " present in class loader's classpath: ",
|
||||
t
|
||||
);
|
||||
}
|
||||
return null;
|
||||
}
|
||||
} catch (Throwable t) {
|
||||
log.debug(
|
||||
"Ignoring java.sql.Driver classes listed in resources but not"
|
||||
+ " present in class loader's classpath: ",
|
||||
t
|
||||
);
|
||||
}
|
||||
return null;
|
||||
}
|
||||
);
|
||||
}
|
||||
|
||||
|
|
|
@ -142,13 +142,8 @@ public class PluginUtils {
|
|||
+ "|common\\.config\\.provider\\.(?!ConfigProvider$).*"
|
||||
+ ")$");
|
||||
|
||||
private static final DirectoryStream.Filter<Path> PLUGIN_PATH_FILTER = new DirectoryStream
|
||||
.Filter<Path>() {
|
||||
@Override
|
||||
public boolean accept(Path path) {
|
||||
return Files.isDirectory(path) || isArchive(path) || isClassFile(path);
|
||||
}
|
||||
};
|
||||
private static final DirectoryStream.Filter<Path> PLUGIN_PATH_FILTER = path ->
|
||||
Files.isDirectory(path) || isArchive(path) || isClassFile(path);
|
||||
|
||||
/**
|
||||
* Return whether the class with the given name should be loaded in isolation using a plugin
|
||||
|
|
|
@ -392,9 +392,7 @@ public class StandaloneHerder extends AbstractHerder {
|
|||
}
|
||||
|
||||
if (newState == TargetState.STARTED) {
|
||||
requestExecutorService.submit(() -> {
|
||||
updateConnectorTasks(connector);
|
||||
});
|
||||
requestExecutorService.submit(() -> updateConnectorTasks(connector));
|
||||
}
|
||||
});
|
||||
}
|
||||
|
|
|
@ -488,19 +488,16 @@ public class KafkaConfigBackingStore implements ConfigBackingStore {
|
|||
Map<String, Object> consumerProps,
|
||||
Callback<ConsumerRecord<String, byte[]>> consumedCallback,
|
||||
final NewTopic topicDescription, final Map<String, Object> adminProps) {
|
||||
Runnable createTopics = new Runnable() {
|
||||
@Override
|
||||
public void run() {
|
||||
log.debug("Creating admin client to manage Connect internal config topic");
|
||||
try (TopicAdmin admin = new TopicAdmin(adminProps)) {
|
||||
// Create the topic if it doesn't exist
|
||||
Set<String> newTopics = admin.createTopics(topicDescription);
|
||||
if (!newTopics.contains(topic)) {
|
||||
// It already existed, so check that the topic cleanup policy is compact only and not delete
|
||||
log.debug("Using admin client to check cleanup policy of '{}' topic is '{}'", topic, TopicConfig.CLEANUP_POLICY_COMPACT);
|
||||
admin.verifyTopicCleanupPolicyOnlyCompact(topic,
|
||||
DistributedConfig.CONFIG_TOPIC_CONFIG, "connector configurations");
|
||||
}
|
||||
Runnable createTopics = () -> {
|
||||
log.debug("Creating admin client to manage Connect internal config topic");
|
||||
try (TopicAdmin admin = new TopicAdmin(adminProps)) {
|
||||
// Create the topic if it doesn't exist
|
||||
Set<String> newTopics = admin.createTopics(topicDescription);
|
||||
if (!newTopics.contains(topic)) {
|
||||
// It already existed, so check that the topic cleanup policy is compact only and not delete
|
||||
log.debug("Using admin client to check cleanup policy of '{}' topic is '{}'", topic, TopicConfig.CLEANUP_POLICY_COMPACT);
|
||||
admin.verifyTopicCleanupPolicyOnlyCompact(topic,
|
||||
DistributedConfig.CONFIG_TOPIC_CONFIG, "connector configurations");
|
||||
}
|
||||
}
|
||||
};
|
||||
|
|
|
@ -103,19 +103,16 @@ public class KafkaOffsetBackingStore implements OffsetBackingStore {
|
|||
Map<String, Object> consumerProps,
|
||||
Callback<ConsumerRecord<byte[], byte[]>> consumedCallback,
|
||||
final NewTopic topicDescription, final Map<String, Object> adminProps) {
|
||||
Runnable createTopics = new Runnable() {
|
||||
@Override
|
||||
public void run() {
|
||||
log.debug("Creating admin client to manage Connect internal offset topic");
|
||||
try (TopicAdmin admin = new TopicAdmin(adminProps)) {
|
||||
// Create the topic if it doesn't exist
|
||||
Set<String> newTopics = admin.createTopics(topicDescription);
|
||||
if (!newTopics.contains(topic)) {
|
||||
// It already existed, so check that the topic cleanup policy is compact only and not delete
|
||||
log.debug("Using admin client to check cleanup policy for '{}' topic is '{}'", topic, TopicConfig.CLEANUP_POLICY_COMPACT);
|
||||
admin.verifyTopicCleanupPolicyOnlyCompact(topic,
|
||||
DistributedConfig.OFFSET_STORAGE_TOPIC_CONFIG, "source connector offsets");
|
||||
}
|
||||
Runnable createTopics = () -> {
|
||||
log.debug("Creating admin client to manage Connect internal offset topic");
|
||||
try (TopicAdmin admin = new TopicAdmin(adminProps)) {
|
||||
// Create the topic if it doesn't exist
|
||||
Set<String> newTopics = admin.createTopics(topicDescription);
|
||||
if (!newTopics.contains(topic)) {
|
||||
// It already existed, so check that the topic cleanup policy is compact only and not delete
|
||||
log.debug("Using admin client to check cleanup policy for '{}' topic is '{}'", topic, TopicConfig.CLEANUP_POLICY_COMPACT);
|
||||
admin.verifyTopicCleanupPolicyOnlyCompact(topic,
|
||||
DistributedConfig.OFFSET_STORAGE_TOPIC_CONFIG, "source connector offsets");
|
||||
}
|
||||
}
|
||||
};
|
||||
|
|
|
@ -180,12 +180,7 @@ public class KafkaStatusBackingStore implements StatusBackingStore {
|
|||
.replicationFactor(config.getShort(DistributedConfig.STATUS_STORAGE_REPLICATION_FACTOR_CONFIG))
|
||||
.build();
|
||||
|
||||
Callback<ConsumerRecord<String, byte[]>> readCallback = new Callback<ConsumerRecord<String, byte[]>>() {
|
||||
@Override
|
||||
public void onCompletion(Throwable error, ConsumerRecord<String, byte[]> record) {
|
||||
read(record);
|
||||
}
|
||||
};
|
||||
Callback<ConsumerRecord<String, byte[]>> readCallback = (error, record) -> read(record);
|
||||
this.kafkaLog = createKafkaBasedLog(statusTopic, producerProps, consumerProps, readCallback, topicDescription, adminProps);
|
||||
}
|
||||
|
||||
|
@ -193,19 +188,16 @@ public class KafkaStatusBackingStore implements StatusBackingStore {
|
|||
Map<String, Object> consumerProps,
|
||||
Callback<ConsumerRecord<String, byte[]>> consumedCallback,
|
||||
final NewTopic topicDescription, final Map<String, Object> adminProps) {
|
||||
Runnable createTopics = new Runnable() {
|
||||
@Override
|
||||
public void run() {
|
||||
log.debug("Creating admin client to manage Connect internal status topic");
|
||||
try (TopicAdmin admin = new TopicAdmin(adminProps)) {
|
||||
// Create the topic if it doesn't exist
|
||||
Set<String> newTopics = admin.createTopics(topicDescription);
|
||||
if (!newTopics.contains(topic)) {
|
||||
// It already existed, so check that the topic cleanup policy is compact only and not delete
|
||||
log.debug("Using admin client to check cleanup policy of '{}' topic is '{}'", topic, TopicConfig.CLEANUP_POLICY_COMPACT);
|
||||
admin.verifyTopicCleanupPolicyOnlyCompact(topic,
|
||||
DistributedConfig.STATUS_STORAGE_TOPIC_CONFIG, "connector and task statuses");
|
||||
}
|
||||
Runnable createTopics = () -> {
|
||||
log.debug("Creating admin client to manage Connect internal status topic");
|
||||
try (TopicAdmin admin = new TopicAdmin(adminProps)) {
|
||||
// Create the topic if it doesn't exist
|
||||
Set<String> newTopics = admin.createTopics(topicDescription);
|
||||
if (!newTopics.contains(topic)) {
|
||||
// It already existed, so check that the topic cleanup policy is compact only and not delete
|
||||
log.debug("Using admin client to check cleanup policy of '{}' topic is '{}'", topic, TopicConfig.CLEANUP_POLICY_COMPACT);
|
||||
admin.verifyTopicCleanupPolicyOnlyCompact(topic,
|
||||
DistributedConfig.STATUS_STORAGE_TOPIC_CONFIG, "connector and task statuses");
|
||||
}
|
||||
}
|
||||
};
|
||||
|
|
|
@ -27,7 +27,6 @@ import java.nio.ByteBuffer;
|
|||
import java.util.Collection;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.Callable;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
import java.util.concurrent.Executors;
|
||||
import java.util.concurrent.Future;
|
||||
|
@ -78,33 +77,26 @@ public class MemoryOffsetBackingStore implements OffsetBackingStore {
|
|||
|
||||
@Override
|
||||
public Future<Map<ByteBuffer, ByteBuffer>> get(final Collection<ByteBuffer> keys) {
|
||||
return executor.submit(new Callable<Map<ByteBuffer, ByteBuffer>>() {
|
||||
@Override
|
||||
public Map<ByteBuffer, ByteBuffer> call() throws Exception {
|
||||
Map<ByteBuffer, ByteBuffer> result = new HashMap<>();
|
||||
for (ByteBuffer key : keys) {
|
||||
result.put(key, data.get(key));
|
||||
}
|
||||
return result;
|
||||
return executor.submit(() -> {
|
||||
Map<ByteBuffer, ByteBuffer> result = new HashMap<>();
|
||||
for (ByteBuffer key : keys) {
|
||||
result.put(key, data.get(key));
|
||||
}
|
||||
return result;
|
||||
});
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public Future<Void> set(final Map<ByteBuffer, ByteBuffer> values,
|
||||
final Callback<Void> callback) {
|
||||
return executor.submit(new Callable<Void>() {
|
||||
@Override
|
||||
public Void call() throws Exception {
|
||||
for (Map.Entry<ByteBuffer, ByteBuffer> entry : values.entrySet()) {
|
||||
data.put(entry.getKey(), entry.getValue());
|
||||
}
|
||||
save();
|
||||
if (callback != null)
|
||||
callback.onCompletion(null, null);
|
||||
return null;
|
||||
return executor.submit(() -> {
|
||||
for (Map.Entry<ByteBuffer, ByteBuffer> entry : values.entrySet()) {
|
||||
data.put(entry.getKey(), entry.getValue());
|
||||
}
|
||||
save();
|
||||
if (callback != null)
|
||||
callback.onCompletion(null, null);
|
||||
return null;
|
||||
});
|
||||
}
|
||||
|
||||
|
|
|
@ -166,13 +166,10 @@ public class OffsetStorageWriter {
|
|||
log.debug("Submitting {} entries to backing store. The offsets are: {}", offsetsSerialized.size(), toFlush);
|
||||
}
|
||||
|
||||
return backingStore.set(offsetsSerialized, new Callback<Void>() {
|
||||
@Override
|
||||
public void onCompletion(Throwable error, Void result) {
|
||||
boolean isCurrent = handleFinishWrite(flushId, error, result);
|
||||
if (isCurrent && callback != null) {
|
||||
callback.onCompletion(error, result);
|
||||
}
|
||||
return backingStore.set(offsetsSerialized, (error, result) -> {
|
||||
boolean isCurrent = handleFinishWrite(flushId, error, result);
|
||||
if (isCurrent && callback != null) {
|
||||
callback.onCompletion(error, result);
|
||||
}
|
||||
});
|
||||
}
|
||||
|
|
|
@ -75,12 +75,9 @@ public class MockConnector extends Connector {
|
|||
|
||||
log.debug("Started MockConnector with failure delay of {} ms", delayMs);
|
||||
executor = Executors.newSingleThreadScheduledExecutor();
|
||||
executor.schedule(new Runnable() {
|
||||
@Override
|
||||
public void run() {
|
||||
log.debug("Triggering connector failure");
|
||||
context.raiseError(new RuntimeException());
|
||||
}
|
||||
executor.schedule(() -> {
|
||||
log.debug("Triggering connector failure");
|
||||
context.raiseError(new RuntimeException());
|
||||
}, delayMs, TimeUnit.MILLISECONDS);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -117,11 +117,7 @@ public class KafkaBasedLog<K, V> {
|
|||
this.stopRequested = false;
|
||||
this.readLogEndOffsetCallbacks = new ArrayDeque<>();
|
||||
this.time = time;
|
||||
this.initializer = initializer != null ? initializer : new Runnable() {
|
||||
@Override
|
||||
public void run() {
|
||||
}
|
||||
};
|
||||
this.initializer = initializer != null ? initializer : () -> { };
|
||||
}
|
||||
|
||||
public void start() {
|
||||
|
|
|
@ -48,7 +48,6 @@ import org.apache.kafka.connect.transforms.predicates.Predicate;
|
|||
import org.apache.kafka.connect.util.ConnectorTaskId;
|
||||
import org.easymock.Capture;
|
||||
import org.easymock.EasyMock;
|
||||
import org.easymock.IAnswer;
|
||||
import org.junit.Test;
|
||||
import org.junit.runner.RunWith;
|
||||
import org.powermock.api.easymock.PowerMock;
|
||||
|
@ -253,12 +252,7 @@ public class AbstractHerderTest {
|
|||
statusStore.putSafe(EasyMock.capture(statusCapture));
|
||||
EasyMock.expectLastCall();
|
||||
|
||||
EasyMock.expect(statusStore.get(taskId)).andAnswer(new IAnswer<TaskStatus>() {
|
||||
@Override
|
||||
public TaskStatus answer() throws Throwable {
|
||||
return statusCapture.getValue();
|
||||
}
|
||||
});
|
||||
EasyMock.expect(statusStore.get(taskId)).andAnswer(statusCapture::getValue);
|
||||
|
||||
replayAll();
|
||||
|
||||
|
|
|
@ -55,7 +55,6 @@ import org.apache.kafka.connect.util.ConnectorTaskId;
|
|||
import org.easymock.Capture;
|
||||
import org.easymock.CaptureType;
|
||||
import org.easymock.EasyMock;
|
||||
import org.easymock.IAnswer;
|
||||
import org.easymock.IExpectationSetters;
|
||||
import org.junit.After;
|
||||
import org.junit.Before;
|
||||
|
@ -340,14 +339,11 @@ public class WorkerSinkTaskTest {
|
|||
|
||||
// WorkerSinkTask::close
|
||||
consumer.close();
|
||||
PowerMock.expectLastCall().andAnswer(new IAnswer<Object>() {
|
||||
@Override
|
||||
public Object answer() throws Throwable {
|
||||
rebalanceListener.getValue().onPartitionsRevoked(
|
||||
asList(TOPIC_PARTITION, TOPIC_PARTITION2)
|
||||
);
|
||||
return null;
|
||||
}
|
||||
PowerMock.expectLastCall().andAnswer(() -> {
|
||||
rebalanceListener.getValue().onPartitionsRevoked(
|
||||
asList(TOPIC_PARTITION, TOPIC_PARTITION2)
|
||||
);
|
||||
return null;
|
||||
});
|
||||
transformationChain.close();
|
||||
PowerMock.expectLastCall();
|
||||
|
@ -531,14 +527,11 @@ public class WorkerSinkTaskTest {
|
|||
EasyMock.expectLastCall();
|
||||
|
||||
EasyMock.expect(consumer.poll(Duration.ofMillis(EasyMock.anyLong()))).andAnswer(
|
||||
new IAnswer<ConsumerRecords<byte[], byte[]>>() {
|
||||
@Override
|
||||
public ConsumerRecords<byte[], byte[]> answer() throws Throwable {
|
||||
rebalanceListener.getValue().onPartitionsRevoked(partitions);
|
||||
rebalanceListener.getValue().onPartitionsAssigned(partitions);
|
||||
return ConsumerRecords.empty();
|
||||
}
|
||||
});
|
||||
() -> {
|
||||
rebalanceListener.getValue().onPartitionsRevoked(partitions);
|
||||
rebalanceListener.getValue().onPartitionsAssigned(partitions);
|
||||
return ConsumerRecords.empty();
|
||||
});
|
||||
|
||||
EasyMock.expect(consumer.assignment()).andReturn(new HashSet<>(partitions));
|
||||
|
||||
|
@ -599,13 +592,10 @@ public class WorkerSinkTaskTest {
|
|||
sinkTask.put(EasyMock.<Collection<SinkRecord>>anyObject());
|
||||
EasyMock.expectLastCall();
|
||||
|
||||
EasyMock.expect(consumer.poll(Duration.ofMillis(EasyMock.anyLong()))).andAnswer(new IAnswer<ConsumerRecords<byte[], byte[]>>() {
|
||||
@Override
|
||||
public ConsumerRecords<byte[], byte[]> answer() throws Throwable {
|
||||
// stop the task during its second iteration
|
||||
workerTask.stop();
|
||||
return new ConsumerRecords<>(Collections.emptyMap());
|
||||
}
|
||||
EasyMock.expect(consumer.poll(Duration.ofMillis(EasyMock.anyLong()))).andAnswer(() -> {
|
||||
// stop the task during its second iteration
|
||||
workerTask.stop();
|
||||
return new ConsumerRecords<>(Collections.emptyMap());
|
||||
});
|
||||
consumer.wakeup();
|
||||
EasyMock.expectLastCall();
|
||||
|
@ -661,12 +651,9 @@ public class WorkerSinkTaskTest {
|
|||
|
||||
final Capture<OffsetCommitCallback> callback = EasyMock.newCapture();
|
||||
consumer.commitAsync(EasyMock.eq(offsets), EasyMock.capture(callback));
|
||||
EasyMock.expectLastCall().andAnswer(new IAnswer<Void>() {
|
||||
@Override
|
||||
public Void answer() throws Throwable {
|
||||
callback.getValue().onComplete(offsets, null);
|
||||
return null;
|
||||
}
|
||||
EasyMock.expectLastCall().andAnswer(() -> {
|
||||
callback.getValue().onComplete(offsets, null);
|
||||
return null;
|
||||
});
|
||||
|
||||
expectConsumerPoll(0);
|
||||
|
@ -785,12 +772,9 @@ public class WorkerSinkTaskTest {
|
|||
EasyMock.expect(consumer.assignment()).andReturn(workerCurrentOffsets.keySet());
|
||||
final Capture<OffsetCommitCallback> callback = EasyMock.newCapture();
|
||||
consumer.commitAsync(EasyMock.eq(committableOffsets), EasyMock.capture(callback));
|
||||
EasyMock.expectLastCall().andAnswer(new IAnswer<Void>() {
|
||||
@Override
|
||||
public Void answer() throws Throwable {
|
||||
callback.getValue().onComplete(committableOffsets, null);
|
||||
return null;
|
||||
}
|
||||
EasyMock.expectLastCall().andAnswer(() -> {
|
||||
callback.getValue().onComplete(committableOffsets, null);
|
||||
return null;
|
||||
});
|
||||
expectConsumerPoll(0);
|
||||
sinkTask.put(EasyMock.<Collection<SinkRecord>>anyObject());
|
||||
|
@ -901,30 +885,24 @@ public class WorkerSinkTaskTest {
|
|||
final CountDownLatch latch = new CountDownLatch(1);
|
||||
|
||||
consumer.commitAsync(EasyMock.eq(workerCurrentOffsets), EasyMock.<OffsetCommitCallback>anyObject());
|
||||
EasyMock.expectLastCall().andAnswer(new IAnswer<Void>() {
|
||||
EasyMock.expectLastCall().andAnswer(() -> {
|
||||
// Grab the arguments passed to the consumer.commitAsync method
|
||||
final Object[] args = EasyMock.getCurrentArguments();
|
||||
@SuppressWarnings("unchecked")
|
||||
@Override
|
||||
public Void answer() throws Throwable {
|
||||
// Grab the arguments passed to the consumer.commitAsync method
|
||||
final Object[] args = EasyMock.getCurrentArguments();
|
||||
final Map<TopicPartition, OffsetAndMetadata> offsets = (Map<TopicPartition, OffsetAndMetadata>) args[0];
|
||||
final OffsetCommitCallback callback = (OffsetCommitCallback) args[1];
|
||||
final Map<TopicPartition, OffsetAndMetadata> offsets = (Map<TopicPartition, OffsetAndMetadata>) args[0];
|
||||
final OffsetCommitCallback callback = (OffsetCommitCallback) args[1];
|
||||
|
||||
executor.execute(new Runnable() {
|
||||
@Override
|
||||
public void run() {
|
||||
try {
|
||||
latch.await();
|
||||
} catch (InterruptedException e) {
|
||||
Thread.currentThread().interrupt();
|
||||
}
|
||||
executor.execute(() -> {
|
||||
try {
|
||||
latch.await();
|
||||
} catch (InterruptedException e) {
|
||||
Thread.currentThread().interrupt();
|
||||
}
|
||||
|
||||
callback.onComplete(offsets, null);
|
||||
}
|
||||
});
|
||||
callback.onComplete(offsets, null);
|
||||
});
|
||||
|
||||
return null;
|
||||
}
|
||||
return null;
|
||||
});
|
||||
|
||||
// no actual consumer.commit() triggered
|
||||
|
@ -1097,57 +1075,48 @@ public class WorkerSinkTaskTest {
|
|||
final AtomicBoolean asyncCallbackRan = new AtomicBoolean();
|
||||
|
||||
consumer.commitAsync(EasyMock.eq(workerCurrentOffsets), EasyMock.<OffsetCommitCallback>anyObject());
|
||||
EasyMock.expectLastCall().andAnswer(new IAnswer<Void>() {
|
||||
EasyMock.expectLastCall().andAnswer(() -> {
|
||||
// Grab the arguments passed to the consumer.commitAsync method
|
||||
final Object[] args = EasyMock.getCurrentArguments();
|
||||
@SuppressWarnings("unchecked")
|
||||
@Override
|
||||
public Void answer() throws Throwable {
|
||||
// Grab the arguments passed to the consumer.commitAsync method
|
||||
final Object[] args = EasyMock.getCurrentArguments();
|
||||
final Map<TopicPartition, OffsetAndMetadata> offsets = (Map<TopicPartition, OffsetAndMetadata>) args[0];
|
||||
final OffsetCommitCallback callback = (OffsetCommitCallback) args[1];
|
||||
asyncCallbackRunner.set(new Runnable() {
|
||||
@Override
|
||||
public void run() {
|
||||
callback.onComplete(offsets, null);
|
||||
asyncCallbackRan.set(true);
|
||||
}
|
||||
});
|
||||
return null;
|
||||
}
|
||||
final Map<TopicPartition, OffsetAndMetadata> offsets = (Map<TopicPartition, OffsetAndMetadata>) args[0];
|
||||
final OffsetCommitCallback callback = (OffsetCommitCallback) args[1];
|
||||
asyncCallbackRunner.set(() -> {
|
||||
callback.onComplete(offsets, null);
|
||||
asyncCallbackRan.set(true);
|
||||
});
|
||||
return null;
|
||||
});
|
||||
|
||||
// Expect the next poll to discover and perform the rebalance, THEN complete the previous callback handler,
|
||||
// and then return one record for TP1 and one for TP3.
|
||||
final AtomicBoolean rebalanced = new AtomicBoolean();
|
||||
EasyMock.expect(consumer.poll(Duration.ofMillis(EasyMock.anyLong()))).andAnswer(
|
||||
new IAnswer<ConsumerRecords<byte[], byte[]>>() {
|
||||
@Override
|
||||
public ConsumerRecords<byte[], byte[]> answer() throws Throwable {
|
||||
// Rebalance always begins with revoking current partitions ...
|
||||
rebalanceListener.getValue().onPartitionsRevoked(originalPartitions);
|
||||
// Respond to the rebalance
|
||||
Map<TopicPartition, Long> offsets = new HashMap<>();
|
||||
offsets.put(TOPIC_PARTITION, rebalanceOffsets.get(TOPIC_PARTITION).offset());
|
||||
offsets.put(TOPIC_PARTITION2, rebalanceOffsets.get(TOPIC_PARTITION2).offset());
|
||||
offsets.put(TOPIC_PARTITION3, rebalanceOffsets.get(TOPIC_PARTITION3).offset());
|
||||
sinkTaskContext.getValue().offset(offsets);
|
||||
rebalanceListener.getValue().onPartitionsAssigned(rebalancedPartitions);
|
||||
rebalanced.set(true);
|
||||
() -> {
|
||||
// Rebalance always begins with revoking current partitions ...
|
||||
rebalanceListener.getValue().onPartitionsRevoked(originalPartitions);
|
||||
// Respond to the rebalance
|
||||
Map<TopicPartition, Long> offsets = new HashMap<>();
|
||||
offsets.put(TOPIC_PARTITION, rebalanceOffsets.get(TOPIC_PARTITION).offset());
|
||||
offsets.put(TOPIC_PARTITION2, rebalanceOffsets.get(TOPIC_PARTITION2).offset());
|
||||
offsets.put(TOPIC_PARTITION3, rebalanceOffsets.get(TOPIC_PARTITION3).offset());
|
||||
sinkTaskContext.getValue().offset(offsets);
|
||||
rebalanceListener.getValue().onPartitionsAssigned(rebalancedPartitions);
|
||||
rebalanced.set(true);
|
||||
|
||||
// Run the previous async commit handler
|
||||
asyncCallbackRunner.get().run();
|
||||
// Run the previous async commit handler
|
||||
asyncCallbackRunner.get().run();
|
||||
|
||||
// And prep the two records to return
|
||||
long timestamp = RecordBatch.NO_TIMESTAMP;
|
||||
TimestampType timestampType = TimestampType.NO_TIMESTAMP_TYPE;
|
||||
List<ConsumerRecord<byte[], byte[]>> records = new ArrayList<>();
|
||||
records.add(new ConsumerRecord<>(TOPIC, PARTITION, FIRST_OFFSET + recordsReturnedTp1 + 1, timestamp, timestampType, 0L, 0, 0, RAW_KEY, RAW_VALUE));
|
||||
records.add(new ConsumerRecord<>(TOPIC, PARTITION3, FIRST_OFFSET + recordsReturnedTp3 + 1, timestamp, timestampType, 0L, 0, 0, RAW_KEY, RAW_VALUE));
|
||||
recordsReturnedTp1 += 1;
|
||||
recordsReturnedTp3 += 1;
|
||||
return new ConsumerRecords<>(Collections.singletonMap(new TopicPartition(TOPIC, PARTITION), records));
|
||||
}
|
||||
});
|
||||
// And prep the two records to return
|
||||
long timestamp = RecordBatch.NO_TIMESTAMP;
|
||||
TimestampType timestampType = TimestampType.NO_TIMESTAMP_TYPE;
|
||||
List<ConsumerRecord<byte[], byte[]>> records = new ArrayList<>();
|
||||
records.add(new ConsumerRecord<>(TOPIC, PARTITION, FIRST_OFFSET + recordsReturnedTp1 + 1, timestamp, timestampType, 0L, 0, 0, RAW_KEY, RAW_VALUE));
|
||||
records.add(new ConsumerRecord<>(TOPIC, PARTITION3, FIRST_OFFSET + recordsReturnedTp3 + 1, timestamp, timestampType, 0L, 0, 0, RAW_KEY, RAW_VALUE));
|
||||
recordsReturnedTp1 += 1;
|
||||
recordsReturnedTp3 += 1;
|
||||
return new ConsumerRecords<>(Collections.singletonMap(new TopicPartition(TOPIC, PARTITION), records));
|
||||
});
|
||||
|
||||
// onPartitionsRevoked
|
||||
sinkTask.preCommit(workerCurrentOffsets);
|
||||
|
@ -1185,12 +1154,9 @@ public class WorkerSinkTaskTest {
|
|||
|
||||
final Capture<OffsetCommitCallback> callback = EasyMock.newCapture();
|
||||
consumer.commitAsync(EasyMock.eq(postRebalanceCurrentOffsets), EasyMock.capture(callback));
|
||||
EasyMock.expectLastCall().andAnswer(new IAnswer<Void>() {
|
||||
@Override
|
||||
public Void answer() throws Throwable {
|
||||
callback.getValue().onComplete(postRebalanceCurrentOffsets, null);
|
||||
return null;
|
||||
}
|
||||
EasyMock.expectLastCall().andAnswer(() -> {
|
||||
callback.getValue().onComplete(postRebalanceCurrentOffsets, null);
|
||||
return null;
|
||||
});
|
||||
|
||||
// no actual consumer.commit() triggered
|
||||
|
@ -1292,12 +1258,9 @@ public class WorkerSinkTaskTest {
|
|||
|
||||
final Capture<OffsetCommitCallback> callback = EasyMock.newCapture();
|
||||
consumer.commitAsync(EasyMock.eq(offsets), EasyMock.capture(callback));
|
||||
EasyMock.expectLastCall().andAnswer(new IAnswer<Void>() {
|
||||
@Override
|
||||
public Void answer() throws Throwable {
|
||||
callback.getValue().onComplete(offsets, null);
|
||||
return null;
|
||||
}
|
||||
EasyMock.expectLastCall().andAnswer(() -> {
|
||||
callback.getValue().onComplete(offsets, null);
|
||||
return null;
|
||||
});
|
||||
|
||||
expectConsumerPoll(0);
|
||||
|
@ -1582,13 +1545,10 @@ public class WorkerSinkTaskTest {
|
|||
EasyMock.expectLastCall().andReturn(Collections.emptyMap());
|
||||
|
||||
EasyMock.expect(consumer.poll(Duration.ofMillis(EasyMock.anyLong()))).andAnswer(
|
||||
new IAnswer<ConsumerRecords<byte[], byte[]>>() {
|
||||
@Override
|
||||
public ConsumerRecords<byte[], byte[]> answer() throws Throwable {
|
||||
rebalanceListener.getValue().onPartitionsRevoked(partitions);
|
||||
return ConsumerRecords.empty();
|
||||
}
|
||||
});
|
||||
() -> {
|
||||
rebalanceListener.getValue().onPartitionsRevoked(partitions);
|
||||
return ConsumerRecords.empty();
|
||||
});
|
||||
}
|
||||
|
||||
private void expectRebalanceAssignmentError(RuntimeException e) {
|
||||
|
@ -1607,14 +1567,11 @@ public class WorkerSinkTaskTest {
|
|||
EasyMock.expectLastCall().andThrow(e);
|
||||
|
||||
EasyMock.expect(consumer.poll(Duration.ofMillis(EasyMock.anyLong()))).andAnswer(
|
||||
new IAnswer<ConsumerRecords<byte[], byte[]>>() {
|
||||
@Override
|
||||
public ConsumerRecords<byte[], byte[]> answer() throws Throwable {
|
||||
rebalanceListener.getValue().onPartitionsRevoked(partitions);
|
||||
rebalanceListener.getValue().onPartitionsAssigned(partitions);
|
||||
return ConsumerRecords.empty();
|
||||
}
|
||||
});
|
||||
() -> {
|
||||
rebalanceListener.getValue().onPartitionsRevoked(partitions);
|
||||
rebalanceListener.getValue().onPartitionsAssigned(partitions);
|
||||
return ConsumerRecords.empty();
|
||||
});
|
||||
}
|
||||
|
||||
private void expectPollInitialAssignment() {
|
||||
|
@ -1623,12 +1580,9 @@ public class WorkerSinkTaskTest {
|
|||
sinkTask.open(partitions);
|
||||
EasyMock.expectLastCall();
|
||||
|
||||
EasyMock.expect(consumer.poll(Duration.ofMillis(EasyMock.anyLong()))).andAnswer(new IAnswer<ConsumerRecords<byte[], byte[]>>() {
|
||||
@Override
|
||||
public ConsumerRecords<byte[], byte[]> answer() throws Throwable {
|
||||
rebalanceListener.getValue().onPartitionsAssigned(partitions);
|
||||
return ConsumerRecords.empty();
|
||||
}
|
||||
EasyMock.expect(consumer.poll(Duration.ofMillis(EasyMock.anyLong()))).andAnswer(() -> {
|
||||
rebalanceListener.getValue().onPartitionsAssigned(partitions);
|
||||
return ConsumerRecords.empty();
|
||||
});
|
||||
EasyMock.expect(consumer.position(TOPIC_PARTITION)).andReturn(FIRST_OFFSET);
|
||||
EasyMock.expect(consumer.position(TOPIC_PARTITION2)).andReturn(FIRST_OFFSET);
|
||||
|
@ -1657,34 +1611,26 @@ public class WorkerSinkTaskTest {
|
|||
|
||||
private void expectConsumerPoll(final int numMessages, final long timestamp, final TimestampType timestampType, Headers headers) {
|
||||
EasyMock.expect(consumer.poll(Duration.ofMillis(EasyMock.anyLong()))).andAnswer(
|
||||
new IAnswer<ConsumerRecords<byte[], byte[]>>() {
|
||||
@Override
|
||||
public ConsumerRecords<byte[], byte[]> answer() throws Throwable {
|
||||
List<ConsumerRecord<byte[], byte[]>> records = new ArrayList<>();
|
||||
for (int i = 0; i < numMessages; i++)
|
||||
records.add(new ConsumerRecord<>(TOPIC, PARTITION, FIRST_OFFSET + recordsReturnedTp1 + i, timestamp, timestampType, 0L, 0, 0, RAW_KEY, RAW_VALUE, headers));
|
||||
recordsReturnedTp1 += numMessages;
|
||||
return new ConsumerRecords<>(
|
||||
numMessages > 0 ?
|
||||
Collections.singletonMap(new TopicPartition(TOPIC, PARTITION), records) :
|
||||
Collections.<TopicPartition, List<ConsumerRecord<byte[], byte[]>>>emptyMap()
|
||||
);
|
||||
}
|
||||
});
|
||||
() -> {
|
||||
List<ConsumerRecord<byte[], byte[]>> records = new ArrayList<>();
|
||||
for (int i = 0; i < numMessages; i++)
|
||||
records.add(new ConsumerRecord<>(TOPIC, PARTITION, FIRST_OFFSET + recordsReturnedTp1 + i, timestamp, timestampType, 0L, 0, 0, RAW_KEY, RAW_VALUE, headers));
|
||||
recordsReturnedTp1 += numMessages;
|
||||
return new ConsumerRecords<>(
|
||||
numMessages > 0 ?
|
||||
Collections.singletonMap(new TopicPartition(TOPIC, PARTITION), records) :
|
||||
Collections.<TopicPartition, List<ConsumerRecord<byte[], byte[]>>>emptyMap()
|
||||
);
|
||||
});
|
||||
}
|
||||
|
||||
private void expectConsumerPoll(List<ConsumerRecord<byte[], byte[]>> records) {
|
||||
EasyMock.expect(consumer.poll(Duration.ofMillis(EasyMock.anyLong()))).andAnswer(
|
||||
new IAnswer<ConsumerRecords<byte[], byte[]>>() {
|
||||
@Override
|
||||
public ConsumerRecords<byte[], byte[]> answer() throws Throwable {
|
||||
return new ConsumerRecords<>(
|
||||
records.isEmpty() ?
|
||||
Collections.<TopicPartition, List<ConsumerRecord<byte[], byte[]>>>emptyMap() :
|
||||
Collections.singletonMap(new TopicPartition(TOPIC, PARTITION), records)
|
||||
);
|
||||
}
|
||||
});
|
||||
() -> new ConsumerRecords<>(
|
||||
records.isEmpty() ?
|
||||
Collections.<TopicPartition, List<ConsumerRecord<byte[], byte[]>>>emptyMap() :
|
||||
Collections.singletonMap(new TopicPartition(TOPIC, PARTITION), records)
|
||||
));
|
||||
}
|
||||
|
||||
private void expectConversionAndTransformation(final int numMessages) {
|
||||
|
@ -1709,23 +1655,20 @@ public class WorkerSinkTaskTest {
|
|||
private void expectTransformation(final int numMessages, final String topicPrefix) {
|
||||
final Capture<SinkRecord> recordCapture = EasyMock.newCapture();
|
||||
EasyMock.expect(transformationChain.apply(EasyMock.capture(recordCapture)))
|
||||
.andAnswer(new IAnswer<SinkRecord>() {
|
||||
@Override
|
||||
public SinkRecord answer() {
|
||||
SinkRecord origRecord = recordCapture.getValue();
|
||||
return topicPrefix != null && !topicPrefix.isEmpty()
|
||||
? origRecord.newRecord(
|
||||
topicPrefix + origRecord.topic(),
|
||||
origRecord.kafkaPartition(),
|
||||
origRecord.keySchema(),
|
||||
origRecord.key(),
|
||||
origRecord.valueSchema(),
|
||||
origRecord.value(),
|
||||
origRecord.timestamp(),
|
||||
origRecord.headers()
|
||||
)
|
||||
: origRecord;
|
||||
}
|
||||
.andAnswer(() -> {
|
||||
SinkRecord origRecord = recordCapture.getValue();
|
||||
return topicPrefix != null && !topicPrefix.isEmpty()
|
||||
? origRecord.newRecord(
|
||||
topicPrefix + origRecord.topic(),
|
||||
origRecord.kafkaPartition(),
|
||||
origRecord.keySchema(),
|
||||
origRecord.key(),
|
||||
origRecord.valueSchema(),
|
||||
origRecord.value(),
|
||||
origRecord.timestamp(),
|
||||
origRecord.headers()
|
||||
)
|
||||
: origRecord;
|
||||
}).times(numMessages);
|
||||
}
|
||||
|
||||
|
|
|
@ -45,7 +45,6 @@ import org.apache.kafka.connect.util.ThreadedTest;
|
|||
import org.easymock.Capture;
|
||||
import org.easymock.CaptureType;
|
||||
import org.easymock.EasyMock;
|
||||
import org.easymock.IAnswer;
|
||||
import org.easymock.IExpectationSetters;
|
||||
import org.junit.After;
|
||||
import org.junit.Test;
|
||||
|
@ -386,47 +385,38 @@ public class WorkerSinkTaskThreadedTest extends ThreadedTest {
|
|||
expectTaskGetTopic(true);
|
||||
|
||||
expectPollInitialAssignment();
|
||||
expectOnePoll().andAnswer(new IAnswer<Object>() {
|
||||
@Override
|
||||
public Object answer() throws Throwable {
|
||||
assertEquals(new HashSet<>(Arrays.asList(TOPIC_PARTITION, TOPIC_PARTITION2, TOPIC_PARTITION3)),
|
||||
sinkTaskContext.getValue().assignment());
|
||||
return null;
|
||||
}
|
||||
expectOnePoll().andAnswer(() -> {
|
||||
assertEquals(new HashSet<>(Arrays.asList(TOPIC_PARTITION, TOPIC_PARTITION2, TOPIC_PARTITION3)),
|
||||
sinkTaskContext.getValue().assignment());
|
||||
return null;
|
||||
});
|
||||
EasyMock.expect(consumer.assignment()).andReturn(new HashSet<>(Arrays.asList(TOPIC_PARTITION, TOPIC_PARTITION2, TOPIC_PARTITION3)));
|
||||
|
||||
expectOnePoll().andAnswer(new IAnswer<Object>() {
|
||||
@Override
|
||||
public Object answer() throws Throwable {
|
||||
try {
|
||||
sinkTaskContext.getValue().pause(UNASSIGNED_TOPIC_PARTITION);
|
||||
fail("Trying to pause unassigned partition should have thrown an Connect exception");
|
||||
} catch (ConnectException e) {
|
||||
// expected
|
||||
}
|
||||
sinkTaskContext.getValue().pause(TOPIC_PARTITION, TOPIC_PARTITION2);
|
||||
return null;
|
||||
expectOnePoll().andAnswer(() -> {
|
||||
try {
|
||||
sinkTaskContext.getValue().pause(UNASSIGNED_TOPIC_PARTITION);
|
||||
fail("Trying to pause unassigned partition should have thrown an Connect exception");
|
||||
} catch (ConnectException e) {
|
||||
// expected
|
||||
}
|
||||
sinkTaskContext.getValue().pause(TOPIC_PARTITION, TOPIC_PARTITION2);
|
||||
return null;
|
||||
});
|
||||
consumer.pause(Arrays.asList(UNASSIGNED_TOPIC_PARTITION));
|
||||
PowerMock.expectLastCall().andThrow(new IllegalStateException("unassigned topic partition"));
|
||||
consumer.pause(Arrays.asList(TOPIC_PARTITION, TOPIC_PARTITION2));
|
||||
PowerMock.expectLastCall();
|
||||
|
||||
expectOnePoll().andAnswer(new IAnswer<Object>() {
|
||||
@Override
|
||||
public Object answer() throws Throwable {
|
||||
try {
|
||||
sinkTaskContext.getValue().resume(UNASSIGNED_TOPIC_PARTITION);
|
||||
fail("Trying to resume unassigned partition should have thrown an Connect exception");
|
||||
} catch (ConnectException e) {
|
||||
// expected
|
||||
}
|
||||
|
||||
sinkTaskContext.getValue().resume(TOPIC_PARTITION, TOPIC_PARTITION2);
|
||||
return null;
|
||||
expectOnePoll().andAnswer(() -> {
|
||||
try {
|
||||
sinkTaskContext.getValue().resume(UNASSIGNED_TOPIC_PARTITION);
|
||||
fail("Trying to resume unassigned partition should have thrown an Connect exception");
|
||||
} catch (ConnectException e) {
|
||||
// expected
|
||||
}
|
||||
|
||||
sinkTaskContext.getValue().resume(TOPIC_PARTITION, TOPIC_PARTITION2);
|
||||
return null;
|
||||
});
|
||||
consumer.resume(Arrays.asList(UNASSIGNED_TOPIC_PARTITION));
|
||||
PowerMock.expectLastCall().andThrow(new IllegalStateException("unassigned topic partition"));
|
||||
|
@ -458,25 +448,19 @@ public class WorkerSinkTaskThreadedTest extends ThreadedTest {
|
|||
final long startOffset = 40L;
|
||||
final Map<TopicPartition, Long> offsets = new HashMap<>();
|
||||
|
||||
expectOnePoll().andAnswer(new IAnswer<Object>() {
|
||||
@Override
|
||||
public Object answer() throws Throwable {
|
||||
offsets.put(TOPIC_PARTITION, startOffset);
|
||||
sinkTaskContext.getValue().offset(offsets);
|
||||
return null;
|
||||
}
|
||||
expectOnePoll().andAnswer(() -> {
|
||||
offsets.put(TOPIC_PARTITION, startOffset);
|
||||
sinkTaskContext.getValue().offset(offsets);
|
||||
return null;
|
||||
});
|
||||
|
||||
consumer.seek(TOPIC_PARTITION, startOffset);
|
||||
EasyMock.expectLastCall();
|
||||
|
||||
expectOnePoll().andAnswer(new IAnswer<Object>() {
|
||||
@Override
|
||||
public Object answer() throws Throwable {
|
||||
Map<TopicPartition, Long> offsets = sinkTaskContext.getValue().offsets();
|
||||
assertEquals(0, offsets.size());
|
||||
return null;
|
||||
}
|
||||
expectOnePoll().andAnswer(() -> {
|
||||
Map<TopicPartition, Long> offsets1 = sinkTaskContext.getValue().offsets();
|
||||
assertEquals(0, offsets1.size());
|
||||
return null;
|
||||
});
|
||||
|
||||
expectStopTask();
|
||||
|
@ -499,13 +483,10 @@ public class WorkerSinkTaskThreadedTest extends ThreadedTest {
|
|||
expectTaskGetTopic(true);
|
||||
expectPollInitialAssignment();
|
||||
|
||||
expectRebalanceDuringPoll().andAnswer(new IAnswer<Object>() {
|
||||
@Override
|
||||
public Object answer() throws Throwable {
|
||||
Map<TopicPartition, Long> offsets = sinkTaskContext.getValue().offsets();
|
||||
assertEquals(0, offsets.size());
|
||||
return null;
|
||||
}
|
||||
expectRebalanceDuringPoll().andAnswer(() -> {
|
||||
Map<TopicPartition, Long> offsets = sinkTaskContext.getValue().offsets();
|
||||
assertEquals(0, offsets.size());
|
||||
return null;
|
||||
});
|
||||
|
||||
expectStopTask();
|
||||
|
@ -538,12 +519,9 @@ public class WorkerSinkTaskThreadedTest extends ThreadedTest {
|
|||
sinkTask.open(partitions);
|
||||
EasyMock.expectLastCall();
|
||||
|
||||
EasyMock.expect(consumer.poll(Duration.ofMillis(EasyMock.anyLong()))).andAnswer(new IAnswer<ConsumerRecords<byte[], byte[]>>() {
|
||||
@Override
|
||||
public ConsumerRecords<byte[], byte[]> answer() throws Throwable {
|
||||
rebalanceListener.getValue().onPartitionsAssigned(partitions);
|
||||
return ConsumerRecords.empty();
|
||||
}
|
||||
EasyMock.expect(consumer.poll(Duration.ofMillis(EasyMock.anyLong()))).andAnswer(() -> {
|
||||
rebalanceListener.getValue().onPartitionsAssigned(partitions);
|
||||
return ConsumerRecords.empty();
|
||||
});
|
||||
EasyMock.expect(consumer.position(TOPIC_PARTITION)).andReturn(FIRST_OFFSET);
|
||||
EasyMock.expect(consumer.position(TOPIC_PARTITION2)).andReturn(FIRST_OFFSET);
|
||||
|
@ -571,27 +549,24 @@ public class WorkerSinkTaskThreadedTest extends ThreadedTest {
|
|||
// Stub out all the consumer stream/iterator responses, which we just want to verify occur,
|
||||
// but don't care about the exact details here.
|
||||
EasyMock.expect(consumer.poll(Duration.ofMillis(EasyMock.anyLong()))).andStubAnswer(
|
||||
new IAnswer<ConsumerRecords<byte[], byte[]>>() {
|
||||
@Override
|
||||
public ConsumerRecords<byte[], byte[]> answer() throws Throwable {
|
||||
// "Sleep" so time will progress
|
||||
time.sleep(pollDelayMs);
|
||||
ConsumerRecords<byte[], byte[]> records = new ConsumerRecords<>(
|
||||
Collections.singletonMap(
|
||||
new TopicPartition(TOPIC, PARTITION),
|
||||
Arrays.asList(
|
||||
new ConsumerRecord<>(TOPIC, PARTITION, FIRST_OFFSET + recordsReturned, TIMESTAMP, TIMESTAMP_TYPE, 0L, 0, 0, RAW_KEY, RAW_VALUE)
|
||||
)));
|
||||
recordsReturned++;
|
||||
return records;
|
||||
}
|
||||
});
|
||||
() -> {
|
||||
// "Sleep" so time will progress
|
||||
time.sleep(pollDelayMs);
|
||||
ConsumerRecords<byte[], byte[]> records = new ConsumerRecords<>(
|
||||
Collections.singletonMap(
|
||||
new TopicPartition(TOPIC, PARTITION),
|
||||
Arrays.asList(
|
||||
new ConsumerRecord<>(TOPIC, PARTITION, FIRST_OFFSET + recordsReturned, TIMESTAMP, TIMESTAMP_TYPE, 0L, 0, 0, RAW_KEY, RAW_VALUE)
|
||||
)));
|
||||
recordsReturned++;
|
||||
return records;
|
||||
});
|
||||
EasyMock.expect(keyConverter.toConnectData(TOPIC, emptyHeaders(), RAW_KEY)).andReturn(new SchemaAndValue(KEY_SCHEMA, KEY)).anyTimes();
|
||||
EasyMock.expect(valueConverter.toConnectData(TOPIC, emptyHeaders(), RAW_VALUE)).andReturn(new SchemaAndValue(VALUE_SCHEMA, VALUE)).anyTimes();
|
||||
|
||||
final Capture<SinkRecord> recordCapture = EasyMock.newCapture();
|
||||
EasyMock.expect(transformationChain.apply(EasyMock.capture(recordCapture))).andAnswer(
|
||||
(IAnswer<SinkRecord>) () -> recordCapture.getValue()).anyTimes();
|
||||
recordCapture::getValue).anyTimes();
|
||||
|
||||
Capture<Collection<SinkRecord>> capturedRecords = EasyMock.newCapture(CaptureType.ALL);
|
||||
sinkTask.put(EasyMock.capture(capturedRecords));
|
||||
|
@ -605,21 +580,18 @@ public class WorkerSinkTaskThreadedTest extends ThreadedTest {
|
|||
// returning empty data, we return one record. The expectation is that the data will be ignored by the
|
||||
// response behavior specified using the return value of this method.
|
||||
EasyMock.expect(consumer.poll(Duration.ofMillis(EasyMock.anyLong()))).andAnswer(
|
||||
new IAnswer<ConsumerRecords<byte[], byte[]>>() {
|
||||
@Override
|
||||
public ConsumerRecords<byte[], byte[]> answer() throws Throwable {
|
||||
// "Sleep" so time will progress
|
||||
time.sleep(1L);
|
||||
ConsumerRecords<byte[], byte[]> records = new ConsumerRecords<>(
|
||||
Collections.singletonMap(
|
||||
new TopicPartition(TOPIC, PARTITION),
|
||||
Arrays.asList(
|
||||
new ConsumerRecord<>(TOPIC, PARTITION, FIRST_OFFSET + recordsReturned, TIMESTAMP, TIMESTAMP_TYPE, 0L, 0, 0, RAW_KEY, RAW_VALUE)
|
||||
)));
|
||||
recordsReturned++;
|
||||
return records;
|
||||
}
|
||||
});
|
||||
() -> {
|
||||
// "Sleep" so time will progress
|
||||
time.sleep(1L);
|
||||
ConsumerRecords<byte[], byte[]> records = new ConsumerRecords<>(
|
||||
Collections.singletonMap(
|
||||
new TopicPartition(TOPIC, PARTITION),
|
||||
Arrays.asList(
|
||||
new ConsumerRecord<>(TOPIC, PARTITION, FIRST_OFFSET + recordsReturned, TIMESTAMP, TIMESTAMP_TYPE, 0L, 0, 0, RAW_KEY, RAW_VALUE)
|
||||
)));
|
||||
recordsReturned++;
|
||||
return records;
|
||||
});
|
||||
EasyMock.expect(keyConverter.toConnectData(TOPIC, emptyHeaders(), RAW_KEY)).andReturn(new SchemaAndValue(KEY_SCHEMA, KEY));
|
||||
EasyMock.expect(valueConverter.toConnectData(TOPIC, emptyHeaders(), RAW_VALUE)).andReturn(new SchemaAndValue(VALUE_SCHEMA, VALUE));
|
||||
sinkTask.put(EasyMock.anyObject(Collection.class));
|
||||
|
@ -635,25 +607,22 @@ public class WorkerSinkTaskThreadedTest extends ThreadedTest {
|
|||
offsets.put(TOPIC_PARTITION, startOffset);
|
||||
|
||||
EasyMock.expect(consumer.poll(Duration.ofMillis(EasyMock.anyLong()))).andAnswer(
|
||||
new IAnswer<ConsumerRecords<byte[], byte[]>>() {
|
||||
@Override
|
||||
public ConsumerRecords<byte[], byte[]> answer() throws Throwable {
|
||||
// "Sleep" so time will progress
|
||||
time.sleep(1L);
|
||||
() -> {
|
||||
// "Sleep" so time will progress
|
||||
time.sleep(1L);
|
||||
|
||||
sinkTaskContext.getValue().offset(offsets);
|
||||
rebalanceListener.getValue().onPartitionsAssigned(partitions);
|
||||
sinkTaskContext.getValue().offset(offsets);
|
||||
rebalanceListener.getValue().onPartitionsAssigned(partitions);
|
||||
|
||||
ConsumerRecords<byte[], byte[]> records = new ConsumerRecords<>(
|
||||
Collections.singletonMap(
|
||||
new TopicPartition(TOPIC, PARTITION),
|
||||
Arrays.asList(
|
||||
new ConsumerRecord<>(TOPIC, PARTITION, FIRST_OFFSET + recordsReturned, TIMESTAMP, TIMESTAMP_TYPE, 0L, 0, 0, RAW_KEY, RAW_VALUE)
|
||||
)));
|
||||
recordsReturned++;
|
||||
return records;
|
||||
}
|
||||
});
|
||||
ConsumerRecords<byte[], byte[]> records = new ConsumerRecords<>(
|
||||
Collections.singletonMap(
|
||||
new TopicPartition(TOPIC, PARTITION),
|
||||
Arrays.asList(
|
||||
new ConsumerRecord<>(TOPIC, PARTITION, FIRST_OFFSET + recordsReturned, TIMESTAMP, TIMESTAMP_TYPE, 0L, 0, 0, RAW_KEY, RAW_VALUE)
|
||||
)));
|
||||
recordsReturned++;
|
||||
return records;
|
||||
});
|
||||
|
||||
EasyMock.expect(consumer.position(TOPIC_PARTITION)).andReturn(FIRST_OFFSET);
|
||||
EasyMock.expect(consumer.position(TOPIC_PARTITION2)).andReturn(FIRST_OFFSET);
|
||||
|
@ -696,14 +665,11 @@ public class WorkerSinkTaskThreadedTest extends ThreadedTest {
|
|||
final Capture<OffsetCommitCallback> capturedCallback = EasyMock.newCapture();
|
||||
consumer.commitAsync(EasyMock.eq(offsetsToCommit),
|
||||
EasyMock.capture(capturedCallback));
|
||||
PowerMock.expectLastCall().andAnswer(new IAnswer<Object>() {
|
||||
@Override
|
||||
public Object answer() throws Throwable {
|
||||
time.sleep(consumerCommitDelayMs);
|
||||
if (invokeCallback)
|
||||
capturedCallback.getValue().onComplete(offsetsToCommit, consumerCommitError);
|
||||
return null;
|
||||
}
|
||||
PowerMock.expectLastCall().andAnswer(() -> {
|
||||
time.sleep(consumerCommitDelayMs);
|
||||
if (invokeCallback)
|
||||
capturedCallback.getValue().onComplete(offsetsToCommit, consumerCommitError);
|
||||
return null;
|
||||
});
|
||||
return capturedCallback;
|
||||
}
|
||||
|
|
|
@ -207,12 +207,9 @@ public class WorkerSourceTaskTest extends ThreadedTest {
|
|||
createWorkerTask(TargetState.PAUSED);
|
||||
|
||||
statusListener.onPause(taskId);
|
||||
EasyMock.expectLastCall().andAnswer(new IAnswer<Void>() {
|
||||
@Override
|
||||
public Void answer() throws Throwable {
|
||||
pauseLatch.countDown();
|
||||
return null;
|
||||
}
|
||||
EasyMock.expectLastCall().andAnswer(() -> {
|
||||
pauseLatch.countDown();
|
||||
return null;
|
||||
});
|
||||
|
||||
expectClose();
|
||||
|
@ -338,12 +335,9 @@ public class WorkerSourceTaskTest extends ThreadedTest {
|
|||
|
||||
final CountDownLatch pollLatch = new CountDownLatch(1);
|
||||
final RuntimeException exception = new RuntimeException();
|
||||
EasyMock.expect(sourceTask.poll()).andAnswer(new IAnswer<List<SourceRecord>>() {
|
||||
@Override
|
||||
public List<SourceRecord> answer() throws Throwable {
|
||||
pollLatch.countDown();
|
||||
throw exception;
|
||||
}
|
||||
EasyMock.expect(sourceTask.poll()).andAnswer(() -> {
|
||||
pollLatch.countDown();
|
||||
throw exception;
|
||||
});
|
||||
|
||||
statusListener.onFailure(taskId, exception);
|
||||
|
@ -673,13 +667,10 @@ public class WorkerSourceTaskTest extends ThreadedTest {
|
|||
sourceTask.initialize(EasyMock.anyObject(SourceTaskContext.class));
|
||||
EasyMock.expectLastCall();
|
||||
sourceTask.start(TASK_PROPS);
|
||||
EasyMock.expectLastCall().andAnswer(new IAnswer<Object>() {
|
||||
@Override
|
||||
public Object answer() throws Throwable {
|
||||
startupLatch.countDown();
|
||||
assertTrue(awaitLatch(finishStartupLatch));
|
||||
return null;
|
||||
}
|
||||
EasyMock.expectLastCall().andAnswer(() -> {
|
||||
startupLatch.countDown();
|
||||
assertTrue(awaitLatch(finishStartupLatch));
|
||||
return null;
|
||||
});
|
||||
|
||||
statusListener.onStartup(taskId);
|
||||
|
@ -856,14 +847,11 @@ public class WorkerSourceTaskTest extends ThreadedTest {
|
|||
// run. The count passed in + latch returned just makes sure we get *at least* that number of
|
||||
// calls
|
||||
EasyMock.expect(sourceTask.poll())
|
||||
.andStubAnswer(new IAnswer<List<SourceRecord>>() {
|
||||
@Override
|
||||
public List<SourceRecord> answer() throws Throwable {
|
||||
count.incrementAndGet();
|
||||
latch.countDown();
|
||||
Thread.sleep(10);
|
||||
return Collections.emptyList();
|
||||
}
|
||||
.andStubAnswer(() -> {
|
||||
count.incrementAndGet();
|
||||
latch.countDown();
|
||||
Thread.sleep(10);
|
||||
return Collections.emptyList();
|
||||
});
|
||||
return latch;
|
||||
}
|
||||
|
@ -874,14 +862,11 @@ public class WorkerSourceTaskTest extends ThreadedTest {
|
|||
// run. The count passed in + latch returned just makes sure we get *at least* that number of
|
||||
// calls
|
||||
EasyMock.expect(sourceTask.poll())
|
||||
.andStubAnswer(new IAnswer<List<SourceRecord>>() {
|
||||
@Override
|
||||
public List<SourceRecord> answer() throws Throwable {
|
||||
count.incrementAndGet();
|
||||
latch.countDown();
|
||||
Thread.sleep(10);
|
||||
return RECORDS;
|
||||
}
|
||||
.andStubAnswer(() -> {
|
||||
count.incrementAndGet();
|
||||
latch.countDown();
|
||||
Thread.sleep(10);
|
||||
return RECORDS;
|
||||
});
|
||||
// Fallout of the poll() call
|
||||
expectSendRecordAnyTimes();
|
||||
|
@ -959,22 +944,19 @@ public class WorkerSourceTaskTest extends ThreadedTest {
|
|||
IExpectationSetters<Future<RecordMetadata>> expect = EasyMock.expect(
|
||||
producer.send(EasyMock.capture(sent),
|
||||
EasyMock.capture(producerCallbacks)));
|
||||
IAnswer<Future<RecordMetadata>> expectResponse = new IAnswer<Future<RecordMetadata>>() {
|
||||
@Override
|
||||
public Future<RecordMetadata> answer() throws Throwable {
|
||||
synchronized (producerCallbacks) {
|
||||
for (org.apache.kafka.clients.producer.Callback cb : producerCallbacks.getValues()) {
|
||||
if (sendSuccess) {
|
||||
cb.onCompletion(new RecordMetadata(new TopicPartition("foo", 0), 0, 0,
|
||||
0L, 0L, 0, 0), null);
|
||||
} else {
|
||||
cb.onCompletion(null, new TopicAuthorizationException("foo"));
|
||||
}
|
||||
IAnswer<Future<RecordMetadata>> expectResponse = () -> {
|
||||
synchronized (producerCallbacks) {
|
||||
for (org.apache.kafka.clients.producer.Callback cb : producerCallbacks.getValues()) {
|
||||
if (sendSuccess) {
|
||||
cb.onCompletion(new RecordMetadata(new TopicPartition("foo", 0), 0, 0,
|
||||
0L, 0L, 0, 0), null);
|
||||
} else {
|
||||
cb.onCompletion(null, new TopicAuthorizationException("foo"));
|
||||
}
|
||||
producerCallbacks.reset();
|
||||
}
|
||||
return sendFuture;
|
||||
producerCallbacks.reset();
|
||||
}
|
||||
return sendFuture;
|
||||
};
|
||||
if (anyTimes)
|
||||
expect.andStubAnswer(expectResponse);
|
||||
|
@ -1018,19 +1000,9 @@ public class WorkerSourceTaskTest extends ThreadedTest {
|
|||
final Capture<SourceRecord> recordCapture = EasyMock.newCapture();
|
||||
IExpectationSetters<SourceRecord> convertKeyExpect = EasyMock.expect(transformationChain.apply(EasyMock.capture(recordCapture)));
|
||||
if (anyTimes)
|
||||
convertKeyExpect.andStubAnswer(new IAnswer<SourceRecord>() {
|
||||
@Override
|
||||
public SourceRecord answer() {
|
||||
return recordCapture.getValue();
|
||||
}
|
||||
});
|
||||
convertKeyExpect.andStubAnswer(recordCapture::getValue);
|
||||
else
|
||||
convertKeyExpect.andAnswer(new IAnswer<SourceRecord>() {
|
||||
@Override
|
||||
public SourceRecord answer() {
|
||||
return recordCapture.getValue();
|
||||
}
|
||||
});
|
||||
convertKeyExpect.andAnswer(recordCapture::getValue);
|
||||
}
|
||||
|
||||
private void expectTaskCommitRecordWithOffset(boolean anyTimes, boolean succeed) throws InterruptedException {
|
||||
|
|
|
@ -232,12 +232,9 @@ public class WorkerSourceTaskWithTopicCreationTest extends ThreadedTest {
|
|||
createWorkerTask(TargetState.PAUSED);
|
||||
|
||||
statusListener.onPause(taskId);
|
||||
EasyMock.expectLastCall().andAnswer(new IAnswer<Void>() {
|
||||
@Override
|
||||
public Void answer() throws Throwable {
|
||||
pauseLatch.countDown();
|
||||
return null;
|
||||
}
|
||||
EasyMock.expectLastCall().andAnswer(() -> {
|
||||
pauseLatch.countDown();
|
||||
return null;
|
||||
});
|
||||
|
||||
expectClose();
|
||||
|
@ -363,12 +360,9 @@ public class WorkerSourceTaskWithTopicCreationTest extends ThreadedTest {
|
|||
|
||||
final CountDownLatch pollLatch = new CountDownLatch(1);
|
||||
final RuntimeException exception = new RuntimeException();
|
||||
EasyMock.expect(sourceTask.poll()).andAnswer(new IAnswer<List<SourceRecord>>() {
|
||||
@Override
|
||||
public List<SourceRecord> answer() throws Throwable {
|
||||
pollLatch.countDown();
|
||||
throw exception;
|
||||
}
|
||||
EasyMock.expect(sourceTask.poll()).andAnswer(() -> {
|
||||
pollLatch.countDown();
|
||||
throw exception;
|
||||
});
|
||||
|
||||
statusListener.onFailure(taskId, exception);
|
||||
|
@ -717,13 +711,10 @@ public class WorkerSourceTaskWithTopicCreationTest extends ThreadedTest {
|
|||
sourceTask.initialize(EasyMock.anyObject(SourceTaskContext.class));
|
||||
EasyMock.expectLastCall();
|
||||
sourceTask.start(TASK_PROPS);
|
||||
EasyMock.expectLastCall().andAnswer(new IAnswer<Object>() {
|
||||
@Override
|
||||
public Object answer() throws Throwable {
|
||||
startupLatch.countDown();
|
||||
assertTrue(awaitLatch(finishStartupLatch));
|
||||
return null;
|
||||
}
|
||||
EasyMock.expectLastCall().andAnswer(() -> {
|
||||
startupLatch.countDown();
|
||||
assertTrue(awaitLatch(finishStartupLatch));
|
||||
return null;
|
||||
});
|
||||
|
||||
statusListener.onStartup(taskId);
|
||||
|
@ -1189,14 +1180,11 @@ public class WorkerSourceTaskWithTopicCreationTest extends ThreadedTest {
|
|||
// run. The count passed in + latch returned just makes sure we get *at least* that number of
|
||||
// calls
|
||||
EasyMock.expect(sourceTask.poll())
|
||||
.andStubAnswer(new IAnswer<List<SourceRecord>>() {
|
||||
@Override
|
||||
public List<SourceRecord> answer() throws Throwable {
|
||||
count.incrementAndGet();
|
||||
latch.countDown();
|
||||
Thread.sleep(10);
|
||||
return Collections.emptyList();
|
||||
}
|
||||
.andStubAnswer(() -> {
|
||||
count.incrementAndGet();
|
||||
latch.countDown();
|
||||
Thread.sleep(10);
|
||||
return Collections.emptyList();
|
||||
});
|
||||
return latch;
|
||||
}
|
||||
|
@ -1207,14 +1195,11 @@ public class WorkerSourceTaskWithTopicCreationTest extends ThreadedTest {
|
|||
// run. The count passed in + latch returned just makes sure we get *at least* that number of
|
||||
// calls
|
||||
EasyMock.expect(sourceTask.poll())
|
||||
.andStubAnswer(new IAnswer<List<SourceRecord>>() {
|
||||
@Override
|
||||
public List<SourceRecord> answer() throws Throwable {
|
||||
count.incrementAndGet();
|
||||
latch.countDown();
|
||||
Thread.sleep(10);
|
||||
return RECORDS;
|
||||
}
|
||||
.andStubAnswer(() -> {
|
||||
count.incrementAndGet();
|
||||
latch.countDown();
|
||||
Thread.sleep(10);
|
||||
return RECORDS;
|
||||
});
|
||||
// Fallout of the poll() call
|
||||
expectSendRecordAnyTimes();
|
||||
|
@ -1293,22 +1278,19 @@ public class WorkerSourceTaskWithTopicCreationTest extends ThreadedTest {
|
|||
IExpectationSetters<Future<RecordMetadata>> expect = EasyMock.expect(
|
||||
producer.send(EasyMock.capture(sent),
|
||||
EasyMock.capture(producerCallbacks)));
|
||||
IAnswer<Future<RecordMetadata>> expectResponse = new IAnswer<Future<RecordMetadata>>() {
|
||||
@Override
|
||||
public Future<RecordMetadata> answer() throws Throwable {
|
||||
synchronized (producerCallbacks) {
|
||||
for (org.apache.kafka.clients.producer.Callback cb : producerCallbacks.getValues()) {
|
||||
if (sendSuccess) {
|
||||
cb.onCompletion(new RecordMetadata(new TopicPartition("foo", 0), 0, 0,
|
||||
0L, 0L, 0, 0), null);
|
||||
} else {
|
||||
cb.onCompletion(null, new TopicAuthorizationException("foo"));
|
||||
}
|
||||
IAnswer<Future<RecordMetadata>> expectResponse = () -> {
|
||||
synchronized (producerCallbacks) {
|
||||
for (org.apache.kafka.clients.producer.Callback cb : producerCallbacks.getValues()) {
|
||||
if (sendSuccess) {
|
||||
cb.onCompletion(new RecordMetadata(new TopicPartition("foo", 0), 0, 0,
|
||||
0L, 0L, 0, 0), null);
|
||||
} else {
|
||||
cb.onCompletion(null, new TopicAuthorizationException("foo"));
|
||||
}
|
||||
producerCallbacks.reset();
|
||||
}
|
||||
return sendFuture;
|
||||
producerCallbacks.reset();
|
||||
}
|
||||
return sendFuture;
|
||||
};
|
||||
if (anyTimes)
|
||||
expect.andStubAnswer(expectResponse);
|
||||
|
@ -1352,19 +1334,9 @@ public class WorkerSourceTaskWithTopicCreationTest extends ThreadedTest {
|
|||
final Capture<SourceRecord> recordCapture = EasyMock.newCapture();
|
||||
IExpectationSetters<SourceRecord> convertKeyExpect = EasyMock.expect(transformationChain.apply(EasyMock.capture(recordCapture)));
|
||||
if (anyTimes)
|
||||
convertKeyExpect.andStubAnswer(new IAnswer<SourceRecord>() {
|
||||
@Override
|
||||
public SourceRecord answer() {
|
||||
return recordCapture.getValue();
|
||||
}
|
||||
});
|
||||
convertKeyExpect.andStubAnswer(recordCapture::getValue);
|
||||
else
|
||||
convertKeyExpect.andAnswer(new IAnswer<SourceRecord>() {
|
||||
@Override
|
||||
public SourceRecord answer() {
|
||||
return recordCapture.getValue();
|
||||
}
|
||||
});
|
||||
convertKeyExpect.andAnswer(recordCapture::getValue);
|
||||
}
|
||||
|
||||
private void expectTaskCommitRecordWithOffset(boolean anyTimes, boolean succeed) throws InterruptedException {
|
||||
|
|
|
@ -26,7 +26,6 @@ import org.apache.kafka.connect.storage.StatusBackingStore;
|
|||
import org.apache.kafka.connect.util.ConnectorTaskId;
|
||||
import org.apache.kafka.common.utils.MockTime;
|
||||
import org.easymock.EasyMock;
|
||||
import org.easymock.IAnswer;
|
||||
import org.easymock.Mock;
|
||||
import org.junit.After;
|
||||
import org.junit.Before;
|
||||
|
@ -185,26 +184,20 @@ public class WorkerTaskTest {
|
|||
.createStrictMock();
|
||||
|
||||
final CountDownLatch stopped = new CountDownLatch(1);
|
||||
final Thread thread = new Thread() {
|
||||
@Override
|
||||
public void run() {
|
||||
try {
|
||||
stopped.await();
|
||||
} catch (Exception e) {
|
||||
}
|
||||
final Thread thread = new Thread(() -> {
|
||||
try {
|
||||
stopped.await();
|
||||
} catch (Exception e) {
|
||||
}
|
||||
};
|
||||
});
|
||||
|
||||
workerTask.initialize(TASK_CONFIG);
|
||||
EasyMock.expectLastCall();
|
||||
|
||||
workerTask.execute();
|
||||
expectLastCall().andAnswer(new IAnswer<Void>() {
|
||||
@Override
|
||||
public Void answer() throws Throwable {
|
||||
thread.start();
|
||||
return null;
|
||||
}
|
||||
expectLastCall().andAnswer(() -> {
|
||||
thread.start();
|
||||
return null;
|
||||
});
|
||||
|
||||
statusListener.onStartup(taskId);
|
||||
|
|
|
@ -214,9 +214,7 @@ public class DistributedConfigTest {
|
|||
topicSettings.put("bar", "bar value");
|
||||
topicSettings.put("baz.bim", "100");
|
||||
Map<String, String> settings = configs();
|
||||
topicSettings.entrySet().forEach(e -> {
|
||||
settings.put(DistributedConfig.CONFIG_STORAGE_PREFIX + e.getKey(), e.getValue());
|
||||
});
|
||||
topicSettings.forEach((k, v) -> settings.put(DistributedConfig.CONFIG_STORAGE_PREFIX + k, v));
|
||||
DistributedConfig config = new DistributedConfig(settings);
|
||||
assertEquals(topicSettings, config.configStorageTopicSettings());
|
||||
}
|
||||
|
@ -228,9 +226,7 @@ public class DistributedConfigTest {
|
|||
topicSettings.put("bar", "bar value");
|
||||
topicSettings.put("baz.bim", "100");
|
||||
Map<String, String> settings = configs();
|
||||
topicSettings.entrySet().forEach(e -> {
|
||||
settings.put(DistributedConfig.OFFSET_STORAGE_PREFIX + e.getKey(), e.getValue());
|
||||
});
|
||||
topicSettings.forEach((k, v) -> settings.put(DistributedConfig.OFFSET_STORAGE_PREFIX + k, v));
|
||||
DistributedConfig config = new DistributedConfig(settings);
|
||||
assertEquals(topicSettings, config.offsetStorageTopicSettings());
|
||||
}
|
||||
|
@ -242,9 +238,7 @@ public class DistributedConfigTest {
|
|||
topicSettings.put("bar", "bar value");
|
||||
topicSettings.put("baz.bim", "100");
|
||||
Map<String, String> settings = configs();
|
||||
topicSettings.entrySet().forEach(e -> {
|
||||
settings.put(DistributedConfig.STATUS_STORAGE_PREFIX + e.getKey(), e.getValue());
|
||||
});
|
||||
topicSettings.forEach((k, v) -> settings.put(DistributedConfig.STATUS_STORAGE_PREFIX + k, v));
|
||||
DistributedConfig config = new DistributedConfig(settings);
|
||||
assertEquals(topicSettings, config.statusStorageTopicSettings());
|
||||
}
|
||||
|
@ -260,9 +254,7 @@ public class DistributedConfigTest {
|
|||
topicSettings.put("partitions", "3");
|
||||
|
||||
Map<String, String> settings = configs();
|
||||
topicSettings.forEach((k, v) -> {
|
||||
settings.put(DistributedConfig.CONFIG_STORAGE_PREFIX + k, v);
|
||||
});
|
||||
topicSettings.forEach((k, v) -> settings.put(DistributedConfig.CONFIG_STORAGE_PREFIX + k, v));
|
||||
DistributedConfig config = new DistributedConfig(settings);
|
||||
Map<String, Object> actual = config.configStorageTopicSettings();
|
||||
assertEquals(expectedTopicSettings, actual);
|
||||
|
@ -279,9 +271,7 @@ public class DistributedConfigTest {
|
|||
topicSettings.put("cleanup.policy", "something-else");
|
||||
|
||||
Map<String, String> settings = configs();
|
||||
topicSettings.forEach((k, v) -> {
|
||||
settings.put(DistributedConfig.OFFSET_STORAGE_PREFIX + k, v);
|
||||
});
|
||||
topicSettings.forEach((k, v) -> settings.put(DistributedConfig.OFFSET_STORAGE_PREFIX + k, v));
|
||||
DistributedConfig config = new DistributedConfig(settings);
|
||||
Map<String, Object> actual = config.offsetStorageTopicSettings();
|
||||
assertEquals(expectedTopicSettings, actual);
|
||||
|
@ -298,9 +288,7 @@ public class DistributedConfigTest {
|
|||
topicSettings.put("cleanup.policy", "something-else");
|
||||
|
||||
Map<String, String> settings = configs();
|
||||
topicSettings.forEach((k, v) -> {
|
||||
settings.put(DistributedConfig.STATUS_STORAGE_PREFIX + k, v);
|
||||
});
|
||||
topicSettings.forEach((k, v) -> settings.put(DistributedConfig.STATUS_STORAGE_PREFIX + k, v));
|
||||
DistributedConfig config = new DistributedConfig(settings);
|
||||
Map<String, Object> actual = config.statusStorageTopicSettings();
|
||||
assertEquals(expectedTopicSettings, actual);
|
||||
|
|
|
@ -57,7 +57,6 @@ import org.apache.kafka.connect.util.ConnectorTaskId;
|
|||
import org.apache.kafka.connect.util.FutureCallback;
|
||||
import org.easymock.Capture;
|
||||
import org.easymock.EasyMock;
|
||||
import org.easymock.IAnswer;
|
||||
import org.junit.After;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
|
@ -246,12 +245,9 @@ public class DistributedHerderTest {
|
|||
Capture<Callback<TargetState>> onStart = newCapture();
|
||||
worker.startConnector(EasyMock.eq(CONN1), EasyMock.<Map<String, String>>anyObject(), EasyMock.<CloseableConnectorContext>anyObject(),
|
||||
EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED), capture(onStart));
|
||||
PowerMock.expectLastCall().andAnswer(new IAnswer<Boolean>() {
|
||||
@Override
|
||||
public Boolean answer() throws Throwable {
|
||||
onStart.getValue().onCompletion(null, TargetState.STARTED);
|
||||
return true;
|
||||
}
|
||||
PowerMock.expectLastCall().andAnswer(() -> {
|
||||
onStart.getValue().onCompletion(null, TargetState.STARTED);
|
||||
return true;
|
||||
});
|
||||
member.wakeup();
|
||||
PowerMock.expectLastCall();
|
||||
|
@ -284,12 +280,9 @@ public class DistributedHerderTest {
|
|||
Capture<Callback<TargetState>> onFirstStart = newCapture();
|
||||
worker.startConnector(EasyMock.eq(CONN1), EasyMock.<Map<String, String>>anyObject(), EasyMock.<CloseableConnectorContext>anyObject(),
|
||||
EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED), capture(onFirstStart));
|
||||
PowerMock.expectLastCall().andAnswer(new IAnswer<Boolean>() {
|
||||
@Override
|
||||
public Boolean answer() throws Throwable {
|
||||
onFirstStart.getValue().onCompletion(null, TargetState.STARTED);
|
||||
return true;
|
||||
}
|
||||
PowerMock.expectLastCall().andAnswer(() -> {
|
||||
onFirstStart.getValue().onCompletion(null, TargetState.STARTED);
|
||||
return true;
|
||||
});
|
||||
member.wakeup();
|
||||
PowerMock.expectLastCall();
|
||||
|
@ -309,12 +302,9 @@ public class DistributedHerderTest {
|
|||
Capture<Callback<TargetState>> onSecondStart = newCapture();
|
||||
worker.startConnector(EasyMock.eq(CONN1), EasyMock.<Map<String, String>>anyObject(), EasyMock.<CloseableConnectorContext>anyObject(),
|
||||
EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED), capture(onSecondStart));
|
||||
PowerMock.expectLastCall().andAnswer(new IAnswer<Boolean>() {
|
||||
@Override
|
||||
public Boolean answer() throws Throwable {
|
||||
onSecondStart.getValue().onCompletion(null, TargetState.STARTED);
|
||||
return true;
|
||||
}
|
||||
PowerMock.expectLastCall().andAnswer(() -> {
|
||||
onSecondStart.getValue().onCompletion(null, TargetState.STARTED);
|
||||
return true;
|
||||
});
|
||||
member.wakeup();
|
||||
PowerMock.expectLastCall();
|
||||
|
@ -362,12 +352,9 @@ public class DistributedHerderTest {
|
|||
Capture<Callback<TargetState>> onStart = newCapture();
|
||||
worker.startConnector(EasyMock.eq(CONN1), EasyMock.anyObject(), EasyMock.anyObject(),
|
||||
EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED), capture(onStart));
|
||||
PowerMock.expectLastCall().andAnswer(new IAnswer<Boolean>() {
|
||||
@Override
|
||||
public Boolean answer() throws Throwable {
|
||||
onStart.getValue().onCompletion(null, TargetState.STARTED);
|
||||
return true;
|
||||
}
|
||||
PowerMock.expectLastCall().andAnswer(() -> {
|
||||
onStart.getValue().onCompletion(null, TargetState.STARTED);
|
||||
return true;
|
||||
});
|
||||
member.wakeup();
|
||||
PowerMock.expectLastCall();
|
||||
|
@ -470,12 +457,9 @@ public class DistributedHerderTest {
|
|||
Capture<Callback<TargetState>> onStart = newCapture();
|
||||
worker.startConnector(EasyMock.eq(CONN1), EasyMock.anyObject(), EasyMock.anyObject(),
|
||||
EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED), capture(onStart));
|
||||
PowerMock.expectLastCall().andAnswer(new IAnswer<Boolean>() {
|
||||
@Override
|
||||
public Boolean answer() throws Throwable {
|
||||
onStart.getValue().onCompletion(null, TargetState.STARTED);
|
||||
return true;
|
||||
}
|
||||
PowerMock.expectLastCall().andAnswer(() -> {
|
||||
onStart.getValue().onCompletion(null, TargetState.STARTED);
|
||||
return true;
|
||||
});
|
||||
member.wakeup();
|
||||
PowerMock.expectLastCall();
|
||||
|
@ -513,12 +497,9 @@ public class DistributedHerderTest {
|
|||
Capture<Callback<TargetState>> onFirstStart = newCapture();
|
||||
worker.startConnector(EasyMock.eq(CONN1), EasyMock.<Map<String, String>>anyObject(), EasyMock.<CloseableConnectorContext>anyObject(),
|
||||
EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED), capture(onFirstStart));
|
||||
PowerMock.expectLastCall().andAnswer(new IAnswer<Boolean>() {
|
||||
@Override
|
||||
public Boolean answer() throws Throwable {
|
||||
onFirstStart.getValue().onCompletion(null, TargetState.STARTED);
|
||||
return true;
|
||||
}
|
||||
PowerMock.expectLastCall().andAnswer(() -> {
|
||||
onFirstStart.getValue().onCompletion(null, TargetState.STARTED);
|
||||
return true;
|
||||
});
|
||||
member.wakeup();
|
||||
PowerMock.expectLastCall();
|
||||
|
@ -537,12 +518,9 @@ public class DistributedHerderTest {
|
|||
Capture<Callback<TargetState>> onSecondStart = newCapture();
|
||||
worker.startConnector(EasyMock.eq(CONN1), EasyMock.<Map<String, String>>anyObject(), EasyMock.<CloseableConnectorContext>anyObject(),
|
||||
EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED), capture(onSecondStart));
|
||||
PowerMock.expectLastCall().andAnswer(new IAnswer<Boolean>() {
|
||||
@Override
|
||||
public Boolean answer() throws Throwable {
|
||||
onSecondStart.getValue().onCompletion(null, TargetState.STARTED);
|
||||
return true;
|
||||
}
|
||||
PowerMock.expectLastCall().andAnswer(() -> {
|
||||
onSecondStart.getValue().onCompletion(null, TargetState.STARTED);
|
||||
return true;
|
||||
});
|
||||
member.wakeup();
|
||||
PowerMock.expectLastCall();
|
||||
|
@ -584,17 +562,15 @@ public class DistributedHerderTest {
|
|||
EasyMock.expect(member.memberId()).andStubReturn("member");
|
||||
EasyMock.expect(member.currentProtocolVersion()).andStubReturn(connectProtocolVersion);
|
||||
EasyMock.expect(worker.getPlugins()).andReturn(plugins);
|
||||
expectRebalance(configOffset, Arrays.asList(CONN1), Arrays.asList(TASK1));
|
||||
// The lists need to be mutable because assignments might be removed
|
||||
expectRebalance(configOffset, new ArrayList<>(singletonList(CONN1)), new ArrayList<>(singletonList(TASK1)));
|
||||
expectPostRebalanceCatchup(SNAPSHOT);
|
||||
Capture<Callback<TargetState>> onFirstStart = newCapture();
|
||||
worker.startConnector(EasyMock.eq(CONN1), EasyMock.<Map<String, String>>anyObject(), EasyMock.<CloseableConnectorContext>anyObject(),
|
||||
EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED), capture(onFirstStart));
|
||||
PowerMock.expectLastCall().andAnswer(new IAnswer<Boolean>() {
|
||||
@Override
|
||||
public Boolean answer() throws Throwable {
|
||||
onFirstStart.getValue().onCompletion(null, TargetState.STARTED);
|
||||
return true;
|
||||
}
|
||||
PowerMock.expectLastCall().andAnswer(() -> {
|
||||
onFirstStart.getValue().onCompletion(null, TargetState.STARTED);
|
||||
return true;
|
||||
});
|
||||
member.wakeup();
|
||||
PowerMock.expectLastCall();
|
||||
|
@ -645,12 +621,9 @@ public class DistributedHerderTest {
|
|||
worker.startConnector(EasyMock.eq(CONN1), EasyMock.<Map<String, String>>anyObject(),
|
||||
EasyMock.<CloseableConnectorContext>anyObject(),
|
||||
EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED), capture(onSecondStart));
|
||||
PowerMock.expectLastCall().andAnswer(new IAnswer<Boolean>() {
|
||||
@Override
|
||||
public Boolean answer() throws Throwable {
|
||||
onSecondStart.getValue().onCompletion(null, TargetState.STARTED);
|
||||
return true;
|
||||
}
|
||||
PowerMock.expectLastCall().andAnswer(() -> {
|
||||
onSecondStart.getValue().onCompletion(null, TargetState.STARTED);
|
||||
return true;
|
||||
});
|
||||
member.wakeup();
|
||||
PowerMock.expectLastCall();
|
||||
|
@ -710,12 +683,9 @@ public class DistributedHerderTest {
|
|||
// be covered sufficiently by the unit tests for the AbstractHerder class
|
||||
Capture<Callback<ConfigInfos>> validateCallback = newCapture();
|
||||
herder.validateConnectorConfig(EasyMock.eq(CONN2_CONFIG), capture(validateCallback));
|
||||
PowerMock.expectLastCall().andAnswer(new IAnswer<Void>() {
|
||||
@Override
|
||||
public Void answer() throws Throwable {
|
||||
validateCallback.getValue().onCompletion(null, CONN2_CONFIG_INFOS);
|
||||
return null;
|
||||
}
|
||||
PowerMock.expectLastCall().andAnswer(() -> {
|
||||
validateCallback.getValue().onCompletion(null, CONN2_CONFIG_INFOS);
|
||||
return null;
|
||||
});
|
||||
|
||||
// CONN2 is new, should succeed
|
||||
|
@ -771,13 +741,10 @@ public class DistributedHerderTest {
|
|||
// be covered sufficiently by the unit tests for the AbstractHerder class
|
||||
Capture<Callback<ConfigInfos>> validateCallback = newCapture();
|
||||
herder.validateConnectorConfig(EasyMock.eq(config), capture(validateCallback));
|
||||
PowerMock.expectLastCall().andAnswer(new IAnswer<Void>() {
|
||||
@Override
|
||||
public Void answer() throws Throwable {
|
||||
// CONN2 creation should fail
|
||||
validateCallback.getValue().onCompletion(null, CONN2_INVALID_CONFIG_INFOS);
|
||||
return null;
|
||||
}
|
||||
PowerMock.expectLastCall().andAnswer(() -> {
|
||||
// CONN2 creation should fail
|
||||
validateCallback.getValue().onCompletion(null, CONN2_INVALID_CONFIG_INFOS);
|
||||
return null;
|
||||
});
|
||||
|
||||
Capture<Throwable> error = newCapture();
|
||||
|
@ -840,12 +807,9 @@ public class DistributedHerderTest {
|
|||
// be covered sufficiently by the unit tests for the AbstractHerder class
|
||||
Capture<Callback<ConfigInfos>> validateCallback = newCapture();
|
||||
herder.validateConnectorConfig(EasyMock.eq(CONN1_CONFIG), capture(validateCallback));
|
||||
PowerMock.expectLastCall().andAnswer(new IAnswer<Void>() {
|
||||
@Override
|
||||
public Void answer() throws Throwable {
|
||||
validateCallback.getValue().onCompletion(null, CONN1_CONFIG_INFOS);
|
||||
return null;
|
||||
}
|
||||
PowerMock.expectLastCall().andAnswer(() -> {
|
||||
validateCallback.getValue().onCompletion(null, CONN1_CONFIG_INFOS);
|
||||
return null;
|
||||
});
|
||||
|
||||
expectRebalance(1, Collections.<String>emptyList(), Collections.<ConnectorTaskId>emptyList());
|
||||
|
@ -892,12 +856,9 @@ public class DistributedHerderTest {
|
|||
Capture<Callback<TargetState>> onStart = newCapture();
|
||||
worker.startConnector(EasyMock.eq(CONN1), EasyMock.<Map<String, String>>anyObject(), EasyMock.<CloseableConnectorContext>anyObject(),
|
||||
EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED), capture(onStart));
|
||||
PowerMock.expectLastCall().andAnswer(new IAnswer<Boolean>() {
|
||||
@Override
|
||||
public Boolean answer() throws Throwable {
|
||||
onStart.getValue().onCompletion(null, TargetState.STARTED);
|
||||
return true;
|
||||
}
|
||||
PowerMock.expectLastCall().andAnswer(() -> {
|
||||
onStart.getValue().onCompletion(null, TargetState.STARTED);
|
||||
return true;
|
||||
});
|
||||
EasyMock.expect(worker.isRunning(CONN1)).andReturn(true);
|
||||
EasyMock.expect(worker.connectorTaskConfigs(CONN1, conn1SinkConfig)).andReturn(TASK_CONFIGS);
|
||||
|
@ -961,12 +922,9 @@ public class DistributedHerderTest {
|
|||
Capture<Callback<TargetState>> onFirstStart = newCapture();
|
||||
worker.startConnector(EasyMock.eq(CONN1), EasyMock.<Map<String, String>>anyObject(), EasyMock.<CloseableConnectorContext>anyObject(),
|
||||
EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED), capture(onFirstStart));
|
||||
PowerMock.expectLastCall().andAnswer(new IAnswer<Boolean>() {
|
||||
@Override
|
||||
public Boolean answer() throws Throwable {
|
||||
onFirstStart.getValue().onCompletion(null, TargetState.STARTED);
|
||||
return true;
|
||||
}
|
||||
PowerMock.expectLastCall().andAnswer(() -> {
|
||||
onFirstStart.getValue().onCompletion(null, TargetState.STARTED);
|
||||
return true;
|
||||
});
|
||||
member.wakeup();
|
||||
PowerMock.expectLastCall();
|
||||
|
@ -986,12 +944,9 @@ public class DistributedHerderTest {
|
|||
Capture<Callback<TargetState>> onSecondStart = newCapture();
|
||||
worker.startConnector(EasyMock.eq(CONN1), EasyMock.<Map<String, String>>anyObject(), EasyMock.<CloseableConnectorContext>anyObject(),
|
||||
EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED), capture(onSecondStart));
|
||||
PowerMock.expectLastCall().andAnswer(new IAnswer<Boolean>() {
|
||||
@Override
|
||||
public Boolean answer() throws Throwable {
|
||||
onSecondStart.getValue().onCompletion(null, TargetState.STARTED);
|
||||
return true;
|
||||
}
|
||||
PowerMock.expectLastCall().andAnswer(() -> {
|
||||
onSecondStart.getValue().onCompletion(null, TargetState.STARTED);
|
||||
return true;
|
||||
});
|
||||
member.wakeup();
|
||||
PowerMock.expectLastCall();
|
||||
|
@ -1311,12 +1266,9 @@ public class DistributedHerderTest {
|
|||
Capture<Callback<TargetState>> onStart = newCapture();
|
||||
worker.startConnector(EasyMock.eq(CONN1), EasyMock.<Map<String, String>>anyObject(), EasyMock.<CloseableConnectorContext>anyObject(),
|
||||
EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED), capture(onStart));
|
||||
PowerMock.expectLastCall().andAnswer(new IAnswer<Boolean>() {
|
||||
@Override
|
||||
public Boolean answer() throws Throwable {
|
||||
onStart.getValue().onCompletion(null, TargetState.STARTED);
|
||||
return true;
|
||||
}
|
||||
PowerMock.expectLastCall().andAnswer(() -> {
|
||||
onStart.getValue().onCompletion(null, TargetState.STARTED);
|
||||
return true;
|
||||
});
|
||||
member.wakeup();
|
||||
PowerMock.expectLastCall();
|
||||
|
@ -1351,12 +1303,9 @@ public class DistributedHerderTest {
|
|||
Capture<Callback<TargetState>> onFirstStart = newCapture();
|
||||
worker.startConnector(EasyMock.eq(CONN1), EasyMock.<Map<String, String>>anyObject(), EasyMock.<CloseableConnectorContext>anyObject(),
|
||||
EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED), capture(onFirstStart));
|
||||
PowerMock.expectLastCall().andAnswer(new IAnswer<Boolean>() {
|
||||
@Override
|
||||
public Boolean answer() throws Throwable {
|
||||
onFirstStart.getValue().onCompletion(null, TargetState.STARTED);
|
||||
return true;
|
||||
}
|
||||
PowerMock.expectLastCall().andAnswer(() -> {
|
||||
onFirstStart.getValue().onCompletion(null, TargetState.STARTED);
|
||||
return true;
|
||||
});
|
||||
member.wakeup();
|
||||
PowerMock.expectLastCall();
|
||||
|
@ -1378,12 +1327,9 @@ public class DistributedHerderTest {
|
|||
Capture<Callback<TargetState>> onSecondStart = newCapture();
|
||||
worker.startConnector(EasyMock.eq(CONN1), EasyMock.<Map<String, String>>anyObject(), EasyMock.<CloseableConnectorContext>anyObject(),
|
||||
EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED), capture(onSecondStart));
|
||||
PowerMock.expectLastCall().andAnswer(new IAnswer<Boolean>() {
|
||||
@Override
|
||||
public Boolean answer() throws Throwable {
|
||||
onSecondStart.getValue().onCompletion(null, TargetState.STARTED);
|
||||
return true;
|
||||
}
|
||||
PowerMock.expectLastCall().andAnswer(() -> {
|
||||
onSecondStart.getValue().onCompletion(null, TargetState.STARTED);
|
||||
return true;
|
||||
});
|
||||
member.wakeup();
|
||||
PowerMock.expectLastCall();
|
||||
|
@ -1424,12 +1370,9 @@ public class DistributedHerderTest {
|
|||
Capture<Callback<TargetState>> onStart = newCapture();
|
||||
worker.startConnector(EasyMock.eq(CONN1), EasyMock.<Map<String, String>>anyObject(), EasyMock.<CloseableConnectorContext>anyObject(),
|
||||
EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED), capture(onStart));
|
||||
PowerMock.expectLastCall().andAnswer(new IAnswer<Boolean>() {
|
||||
@Override
|
||||
public Boolean answer() throws Throwable {
|
||||
onStart.getValue().onCompletion(null, TargetState.STARTED);
|
||||
return true;
|
||||
}
|
||||
PowerMock.expectLastCall().andAnswer(() -> {
|
||||
onStart.getValue().onCompletion(null, TargetState.STARTED);
|
||||
return true;
|
||||
});
|
||||
member.wakeup();
|
||||
PowerMock.expectLastCall();
|
||||
|
@ -1450,12 +1393,9 @@ public class DistributedHerderTest {
|
|||
|
||||
Capture<Callback<TargetState>> onPause = newCapture();
|
||||
worker.setTargetState(EasyMock.eq(CONN1), EasyMock.eq(TargetState.PAUSED), capture(onPause));
|
||||
PowerMock.expectLastCall().andAnswer(new IAnswer<Void>() {
|
||||
@Override
|
||||
public Void answer() throws Throwable {
|
||||
onStart.getValue().onCompletion(null, TargetState.PAUSED);
|
||||
return null;
|
||||
}
|
||||
PowerMock.expectLastCall().andAnswer(() -> {
|
||||
onStart.getValue().onCompletion(null, TargetState.PAUSED);
|
||||
return null;
|
||||
});
|
||||
|
||||
member.poll(EasyMock.anyInt());
|
||||
|
@ -1490,12 +1430,9 @@ public class DistributedHerderTest {
|
|||
Capture<Callback<TargetState>> onStart = newCapture();
|
||||
worker.startConnector(EasyMock.eq(CONN1), EasyMock.<Map<String, String>>anyObject(), EasyMock.<CloseableConnectorContext>anyObject(),
|
||||
EasyMock.eq(herder), EasyMock.eq(TargetState.PAUSED), capture(onStart));
|
||||
PowerMock.expectLastCall().andAnswer(new IAnswer<Boolean>() {
|
||||
@Override
|
||||
public Boolean answer() throws Throwable {
|
||||
onStart.getValue().onCompletion(null, TargetState.PAUSED);
|
||||
return true;
|
||||
}
|
||||
PowerMock.expectLastCall().andAnswer(() -> {
|
||||
onStart.getValue().onCompletion(null, TargetState.PAUSED);
|
||||
return true;
|
||||
});
|
||||
EasyMock.expect(worker.getPlugins()).andReturn(plugins);
|
||||
|
||||
|
@ -1517,12 +1454,9 @@ public class DistributedHerderTest {
|
|||
|
||||
Capture<Callback<TargetState>> onResume = newCapture();
|
||||
worker.setTargetState(EasyMock.eq(CONN1), EasyMock.eq(TargetState.STARTED), capture(onResume));
|
||||
PowerMock.expectLastCall().andAnswer(new IAnswer<Void>() {
|
||||
@Override
|
||||
public Void answer() throws Throwable {
|
||||
onResume.getValue().onCompletion(null, TargetState.STARTED);
|
||||
return null;
|
||||
}
|
||||
PowerMock.expectLastCall().andAnswer(() -> {
|
||||
onResume.getValue().onCompletion(null, TargetState.STARTED);
|
||||
return null;
|
||||
});
|
||||
member.wakeup();
|
||||
PowerMock.expectLastCall();
|
||||
|
@ -1611,12 +1545,9 @@ public class DistributedHerderTest {
|
|||
|
||||
Capture<Callback<TargetState>> onPause = newCapture();
|
||||
worker.setTargetState(EasyMock.eq(CONN1), EasyMock.eq(TargetState.PAUSED), capture(onPause));
|
||||
PowerMock.expectLastCall().andAnswer(new IAnswer<Void>() {
|
||||
@Override
|
||||
public Void answer() throws Throwable {
|
||||
onPause.getValue().onCompletion(null, TargetState.STARTED);
|
||||
return null;
|
||||
}
|
||||
PowerMock.expectLastCall().andAnswer(() -> {
|
||||
onPause.getValue().onCompletion(null, TargetState.STARTED);
|
||||
return null;
|
||||
});
|
||||
member.wakeup();
|
||||
PowerMock.expectLastCall();
|
||||
|
@ -1662,12 +1593,9 @@ public class DistributedHerderTest {
|
|||
|
||||
Capture<Callback<TargetState>> onStart = newCapture();
|
||||
worker.setTargetState(EasyMock.eq(CONN1), EasyMock.eq(TargetState.STARTED), capture(onStart));
|
||||
PowerMock.expectLastCall().andAnswer(new IAnswer<Void>() {
|
||||
@Override
|
||||
public Void answer() throws Throwable {
|
||||
onStart.getValue().onCompletion(null, TargetState.STARTED);
|
||||
return null;
|
||||
}
|
||||
PowerMock.expectLastCall().andAnswer(() -> {
|
||||
onStart.getValue().onCompletion(null, TargetState.STARTED);
|
||||
return null;
|
||||
});
|
||||
member.wakeup();
|
||||
PowerMock.expectLastCall();
|
||||
|
@ -1756,12 +1684,9 @@ public class DistributedHerderTest {
|
|||
Capture<Callback<TargetState>> onStart = newCapture();
|
||||
worker.startConnector(EasyMock.eq(CONN1), EasyMock.<Map<String, String>>anyObject(), EasyMock.<CloseableConnectorContext>anyObject(),
|
||||
EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED), capture(onStart));
|
||||
PowerMock.expectLastCall().andAnswer(new IAnswer<Boolean>() {
|
||||
@Override
|
||||
public Boolean answer() throws Throwable {
|
||||
onStart.getValue().onCompletion(null, TargetState.STARTED);
|
||||
return true;
|
||||
}
|
||||
PowerMock.expectLastCall().andAnswer(() -> {
|
||||
onStart.getValue().onCompletion(null, TargetState.STARTED);
|
||||
return true;
|
||||
});
|
||||
member.wakeup();
|
||||
PowerMock.expectLastCall();
|
||||
|
@ -1825,12 +1750,9 @@ public class DistributedHerderTest {
|
|||
Capture<Callback<TargetState>> onStart = newCapture();
|
||||
worker.startConnector(EasyMock.eq(CONN1), EasyMock.anyObject(), EasyMock.anyObject(),
|
||||
EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED), capture(onStart));
|
||||
PowerMock.expectLastCall().andAnswer(new IAnswer<Boolean>() {
|
||||
@Override
|
||||
public Boolean answer() throws Throwable {
|
||||
onStart.getValue().onCompletion(null, TargetState.STARTED);
|
||||
return true;
|
||||
}
|
||||
PowerMock.expectLastCall().andAnswer(() -> {
|
||||
onStart.getValue().onCompletion(null, TargetState.STARTED);
|
||||
return true;
|
||||
});
|
||||
member.wakeup();
|
||||
PowerMock.expectLastCall();
|
||||
|
@ -1920,12 +1842,9 @@ public class DistributedHerderTest {
|
|||
Capture<Callback<TargetState>> onStart = newCapture();
|
||||
worker.startConnector(EasyMock.eq(CONN1), EasyMock.anyObject(), EasyMock.anyObject(),
|
||||
EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED), capture(onStart));
|
||||
PowerMock.expectLastCall().andAnswer(new IAnswer<Boolean>() {
|
||||
@Override
|
||||
public Boolean answer() throws Throwable {
|
||||
onStart.getValue().onCompletion(null, TargetState.STARTED);
|
||||
return true;
|
||||
}
|
||||
PowerMock.expectLastCall().andAnswer(() -> {
|
||||
onStart.getValue().onCompletion(null, TargetState.STARTED);
|
||||
return true;
|
||||
});
|
||||
member.wakeup();
|
||||
PowerMock.expectLastCall();
|
||||
|
@ -2064,12 +1983,9 @@ public class DistributedHerderTest {
|
|||
Capture<Callback<TargetState>> onFirstStart = newCapture();
|
||||
worker.startConnector(EasyMock.eq(CONN1), EasyMock.<Map<String, String>>anyObject(), EasyMock.<CloseableConnectorContext>anyObject(),
|
||||
EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED), capture(onFirstStart));
|
||||
PowerMock.expectLastCall().andAnswer(new IAnswer<Boolean>() {
|
||||
@Override
|
||||
public Boolean answer() throws Throwable {
|
||||
onFirstStart.getValue().onCompletion(null, TargetState.STARTED);
|
||||
return true;
|
||||
}
|
||||
PowerMock.expectLastCall().andAnswer(() -> {
|
||||
onFirstStart.getValue().onCompletion(null, TargetState.STARTED);
|
||||
return true;
|
||||
});
|
||||
EasyMock.expect(worker.isRunning(CONN1)).andReturn(true);
|
||||
EasyMock.expect(worker.connectorTaskConfigs(CONN1, conn1SinkConfig)).andReturn(TASK_CONFIGS);
|
||||
|
@ -2089,21 +2005,15 @@ public class DistributedHerderTest {
|
|||
|
||||
Capture<Callback<ConfigInfos>> validateCallback = newCapture();
|
||||
herder.validateConnectorConfig(EasyMock.eq(CONN1_CONFIG_UPDATED), capture(validateCallback));
|
||||
PowerMock.expectLastCall().andAnswer(new IAnswer<Void>() {
|
||||
@Override
|
||||
public Void answer() throws Throwable {
|
||||
validateCallback.getValue().onCompletion(null, CONN1_CONFIG_INFOS);
|
||||
return null;
|
||||
}
|
||||
PowerMock.expectLastCall().andAnswer(() -> {
|
||||
validateCallback.getValue().onCompletion(null, CONN1_CONFIG_INFOS);
|
||||
return null;
|
||||
});
|
||||
configBackingStore.putConnectorConfig(CONN1, CONN1_CONFIG_UPDATED);
|
||||
PowerMock.expectLastCall().andAnswer(new IAnswer<Object>() {
|
||||
@Override
|
||||
public Object answer() throws Throwable {
|
||||
// Simulate response to writing config + waiting until end of log to be read
|
||||
configUpdateListener.onConnectorConfigUpdate(CONN1);
|
||||
return null;
|
||||
}
|
||||
PowerMock.expectLastCall().andAnswer(() -> {
|
||||
// Simulate response to writing config + waiting until end of log to be read
|
||||
configUpdateListener.onConnectorConfigUpdate(CONN1);
|
||||
return null;
|
||||
});
|
||||
// As a result of reconfig, should need to update snapshot. With only connector updates, we'll just restart
|
||||
// connector without rebalance
|
||||
|
@ -2114,12 +2024,9 @@ public class DistributedHerderTest {
|
|||
Capture<Callback<TargetState>> onSecondStart = newCapture();
|
||||
worker.startConnector(EasyMock.eq(CONN1), EasyMock.<Map<String, String>>anyObject(), EasyMock.<CloseableConnectorContext>anyObject(),
|
||||
EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED), capture(onSecondStart));
|
||||
PowerMock.expectLastCall().andAnswer(new IAnswer<Boolean>() {
|
||||
@Override
|
||||
public Boolean answer() throws Throwable {
|
||||
onSecondStart.getValue().onCompletion(null, TargetState.STARTED);
|
||||
return true;
|
||||
}
|
||||
PowerMock.expectLastCall().andAnswer(() -> {
|
||||
onSecondStart.getValue().onCompletion(null, TargetState.STARTED);
|
||||
return true;
|
||||
});
|
||||
EasyMock.expect(worker.isRunning(CONN1)).andReturn(true);
|
||||
EasyMock.expect(worker.connectorTaskConfigs(CONN1, conn1SinkConfigUpdated)).andReturn(TASK_CONFIGS);
|
||||
|
@ -2331,29 +2238,26 @@ public class DistributedHerderTest {
|
|||
final List<ConnectorTaskId> assignedTasks,
|
||||
int delay) {
|
||||
member.ensureActive();
|
||||
PowerMock.expectLastCall().andAnswer(new IAnswer<Object>() {
|
||||
@Override
|
||||
public Object answer() throws Throwable {
|
||||
ExtendedAssignment assignment;
|
||||
if (!revokedConnectors.isEmpty() || !revokedTasks.isEmpty()) {
|
||||
rebalanceListener.onRevoked("leader", revokedConnectors, revokedTasks);
|
||||
}
|
||||
|
||||
if (connectProtocolVersion == CONNECT_PROTOCOL_V0) {
|
||||
assignment = new ExtendedAssignment(
|
||||
connectProtocolVersion, error, "leader", "leaderUrl", offset,
|
||||
assignedConnectors, assignedTasks,
|
||||
Collections.emptyList(), Collections.emptyList(), 0);
|
||||
} else {
|
||||
assignment = new ExtendedAssignment(
|
||||
connectProtocolVersion, error, "leader", "leaderUrl", offset,
|
||||
new ArrayList<>(assignedConnectors), new ArrayList<>(assignedTasks),
|
||||
new ArrayList<>(revokedConnectors), new ArrayList<>(revokedTasks), delay);
|
||||
}
|
||||
rebalanceListener.onAssigned(assignment, 3);
|
||||
time.sleep(100L);
|
||||
return null;
|
||||
PowerMock.expectLastCall().andAnswer(() -> {
|
||||
ExtendedAssignment assignment;
|
||||
if (!revokedConnectors.isEmpty() || !revokedTasks.isEmpty()) {
|
||||
rebalanceListener.onRevoked("leader", revokedConnectors, revokedTasks);
|
||||
}
|
||||
|
||||
if (connectProtocolVersion == CONNECT_PROTOCOL_V0) {
|
||||
assignment = new ExtendedAssignment(
|
||||
connectProtocolVersion, error, "leader", "leaderUrl", offset,
|
||||
assignedConnectors, assignedTasks,
|
||||
Collections.emptyList(), Collections.emptyList(), 0);
|
||||
} else {
|
||||
assignment = new ExtendedAssignment(
|
||||
connectProtocolVersion, error, "leader", "leaderUrl", offset,
|
||||
assignedConnectors, assignedTasks,
|
||||
new ArrayList<>(revokedConnectors), new ArrayList<>(revokedTasks), delay);
|
||||
}
|
||||
rebalanceListener.onAssigned(assignment, 3);
|
||||
time.sleep(100L);
|
||||
return null;
|
||||
});
|
||||
|
||||
if (!revokedConnectors.isEmpty()) {
|
||||
|
|
|
@ -27,7 +27,6 @@ import org.apache.kafka.common.message.JoinGroupResponseData;
|
|||
import org.apache.kafka.common.message.SyncGroupResponseData;
|
||||
import org.apache.kafka.common.metrics.Metrics;
|
||||
import org.apache.kafka.common.protocol.Errors;
|
||||
import org.apache.kafka.common.requests.AbstractRequest;
|
||||
import org.apache.kafka.common.requests.FindCoordinatorResponse;
|
||||
import org.apache.kafka.common.requests.JoinGroupResponse;
|
||||
import org.apache.kafka.common.requests.RequestTestUtils;
|
||||
|
@ -253,14 +252,11 @@ public class WorkerCoordinatorTest {
|
|||
memberConfigOffsets.put("leader", 1L);
|
||||
memberConfigOffsets.put("member", 1L);
|
||||
client.prepareResponse(joinGroupLeaderResponse(1, consumerId, memberConfigOffsets, Errors.NONE));
|
||||
client.prepareResponse(new MockClient.RequestMatcher() {
|
||||
@Override
|
||||
public boolean matches(AbstractRequest body) {
|
||||
SyncGroupRequest sync = (SyncGroupRequest) body;
|
||||
return sync.data().memberId().equals(consumerId) &&
|
||||
sync.data().generationId() == 1 &&
|
||||
sync.groupAssignments().containsKey(consumerId);
|
||||
}
|
||||
client.prepareResponse(body -> {
|
||||
SyncGroupRequest sync = (SyncGroupRequest) body;
|
||||
return sync.data().memberId().equals(consumerId) &&
|
||||
sync.data().generationId() == 1 &&
|
||||
sync.groupAssignments().containsKey(consumerId);
|
||||
}, syncGroupResponse(ConnectProtocol.Assignment.NO_ERROR, "leader", 1L, Collections.singletonList(connectorId1),
|
||||
Collections.<ConnectorTaskId>emptyList(), Errors.NONE));
|
||||
coordinator.ensureActiveGroup();
|
||||
|
@ -290,14 +286,11 @@ public class WorkerCoordinatorTest {
|
|||
|
||||
// normal join group
|
||||
client.prepareResponse(joinGroupFollowerResponse(1, memberId, "leader", Errors.NONE));
|
||||
client.prepareResponse(new MockClient.RequestMatcher() {
|
||||
@Override
|
||||
public boolean matches(AbstractRequest body) {
|
||||
SyncGroupRequest sync = (SyncGroupRequest) body;
|
||||
return sync.data().memberId().equals(memberId) &&
|
||||
sync.data().generationId() == 1 &&
|
||||
sync.data().assignments().isEmpty();
|
||||
}
|
||||
client.prepareResponse(body -> {
|
||||
SyncGroupRequest sync = (SyncGroupRequest) body;
|
||||
return sync.data().memberId().equals(memberId) &&
|
||||
sync.data().generationId() == 1 &&
|
||||
sync.data().assignments().isEmpty();
|
||||
}, syncGroupResponse(ConnectProtocol.Assignment.NO_ERROR, "leader", 1L, Collections.<String>emptyList(),
|
||||
Collections.singletonList(taskId1x0), Errors.NONE));
|
||||
coordinator.ensureActiveGroup();
|
||||
|
@ -331,14 +324,11 @@ public class WorkerCoordinatorTest {
|
|||
|
||||
// config mismatch results in assignment error
|
||||
client.prepareResponse(joinGroupFollowerResponse(1, memberId, "leader", Errors.NONE));
|
||||
MockClient.RequestMatcher matcher = new MockClient.RequestMatcher() {
|
||||
@Override
|
||||
public boolean matches(AbstractRequest body) {
|
||||
SyncGroupRequest sync = (SyncGroupRequest) body;
|
||||
return sync.data().memberId().equals(memberId) &&
|
||||
sync.data().generationId() == 1 &&
|
||||
sync.data().assignments().isEmpty();
|
||||
}
|
||||
MockClient.RequestMatcher matcher = body -> {
|
||||
SyncGroupRequest sync = (SyncGroupRequest) body;
|
||||
return sync.data().memberId().equals(memberId) &&
|
||||
sync.data().generationId() == 1 &&
|
||||
sync.data().assignments().isEmpty();
|
||||
};
|
||||
client.prepareResponse(matcher, syncGroupResponse(ConnectProtocol.Assignment.CONFIG_MISMATCH, "leader", 10L,
|
||||
Collections.<String>emptyList(), Collections.<ConnectorTaskId>emptyList(), Errors.NONE));
|
||||
|
|
|
@ -21,7 +21,6 @@ import org.apache.kafka.connect.runtime.Herder;
|
|||
import org.apache.kafka.connect.util.Callback;
|
||||
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;
|
||||
|
@ -63,12 +62,9 @@ public class ConnectClusterStateImplTest {
|
|||
public void connectors() {
|
||||
Capture<Callback<Collection<String>>> callback = EasyMock.newCapture();
|
||||
herder.connectors(EasyMock.capture(callback));
|
||||
EasyMock.expectLastCall().andAnswer(new IAnswer<Void>() {
|
||||
@Override
|
||||
public Void answer() {
|
||||
callback.getValue().onCompletion(null, expectedConnectors);
|
||||
return null;
|
||||
}
|
||||
EasyMock.expectLastCall().andAnswer(() -> {
|
||||
callback.getValue().onCompletion(null, expectedConnectors);
|
||||
return null;
|
||||
});
|
||||
EasyMock.replay(herder);
|
||||
assertEquals(expectedConnectors, connectClusterState.connectors());
|
||||
|
@ -80,12 +76,9 @@ public class ConnectClusterStateImplTest {
|
|||
final Map<String, String> expectedConfig = Collections.singletonMap("key", "value");
|
||||
Capture<Callback<Map<String, String>>> callback = EasyMock.newCapture();
|
||||
herder.connectorConfig(EasyMock.eq(connName), EasyMock.capture(callback));
|
||||
EasyMock.expectLastCall().andAnswer(new IAnswer<Void>() {
|
||||
@Override
|
||||
public Void answer() {
|
||||
callback.getValue().onCompletion(null, expectedConfig);
|
||||
return null;
|
||||
}
|
||||
EasyMock.expectLastCall().andAnswer(() -> {
|
||||
callback.getValue().onCompletion(null, expectedConfig);
|
||||
return null;
|
||||
});
|
||||
EasyMock.replay(herder);
|
||||
Map<String, String> actualConfig = connectClusterState.connectorConfig(connName);
|
||||
|
@ -106,13 +99,10 @@ public class ConnectClusterStateImplTest {
|
|||
public void connectorsFailure() {
|
||||
Capture<Callback<Collection<String>>> callback = EasyMock.newCapture();
|
||||
herder.connectors(EasyMock.capture(callback));
|
||||
EasyMock.expectLastCall().andAnswer(new IAnswer<Void>() {
|
||||
@Override
|
||||
public Void answer() {
|
||||
Throwable timeout = new TimeoutException();
|
||||
callback.getValue().onCompletion(timeout, null);
|
||||
return null;
|
||||
}
|
||||
EasyMock.expectLastCall().andAnswer(() -> {
|
||||
Throwable timeout = new TimeoutException();
|
||||
callback.getValue().onCompletion(timeout, null);
|
||||
return null;
|
||||
});
|
||||
EasyMock.replay(herder);
|
||||
assertThrows(ConnectException.class, connectClusterState::connectors);
|
||||
|
|
|
@ -916,9 +916,8 @@ public class ConnectorsResourceTest {
|
|||
|
||||
PowerMock.replayAll();
|
||||
|
||||
ConnectRestException e = assertThrows(ConnectRestException.class, () -> {
|
||||
connectorsResource.destroyConnector(CONNECTOR_NAME, NULL_HEADERS, FORWARD);
|
||||
});
|
||||
ConnectRestException e = assertThrows(ConnectRestException.class, () ->
|
||||
connectorsResource.destroyConnector(CONNECTOR_NAME, NULL_HEADERS, FORWARD));
|
||||
assertTrue(e.getMessage().contains("no known leader URL"));
|
||||
PowerMock.verifyAll();
|
||||
}
|
||||
|
@ -928,12 +927,9 @@ public class ConnectorsResourceTest {
|
|||
}
|
||||
|
||||
private <T> void expectAndCallbackResult(final Capture<Callback<T>> cb, final T value) {
|
||||
PowerMock.expectLastCall().andAnswer(new IAnswer<Void>() {
|
||||
@Override
|
||||
public Void answer() throws Throwable {
|
||||
cb.getValue().onCompletion(null, value);
|
||||
return null;
|
||||
}
|
||||
PowerMock.expectLastCall().andAnswer(() -> {
|
||||
cb.getValue().onCompletion(null, value);
|
||||
return null;
|
||||
});
|
||||
}
|
||||
|
||||
|
|
|
@ -59,7 +59,6 @@ import org.apache.kafka.connect.util.ConnectorTaskId;
|
|||
import org.apache.kafka.connect.util.FutureCallback;
|
||||
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;
|
||||
|
@ -288,12 +287,9 @@ public class StandaloneHerderTest {
|
|||
Capture<Callback<TargetState>> onStart = EasyMock.newCapture();
|
||||
worker.startConnector(EasyMock.eq(CONNECTOR_NAME), EasyMock.eq(config), EasyMock.anyObject(HerderConnectorContext.class),
|
||||
EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED), EasyMock.capture(onStart));
|
||||
EasyMock.expectLastCall().andAnswer(new IAnswer<Boolean>() {
|
||||
@Override
|
||||
public Boolean answer() throws Throwable {
|
||||
onStart.getValue().onCompletion(null, TargetState.STARTED);
|
||||
return true;
|
||||
}
|
||||
EasyMock.expectLastCall().andAnswer(() -> {
|
||||
onStart.getValue().onCompletion(null, TargetState.STARTED);
|
||||
return true;
|
||||
});
|
||||
|
||||
PowerMock.replayAll();
|
||||
|
@ -324,12 +320,9 @@ public class StandaloneHerderTest {
|
|||
worker.startConnector(EasyMock.eq(CONNECTOR_NAME), EasyMock.eq(config), EasyMock.anyObject(HerderConnectorContext.class),
|
||||
EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED), EasyMock.capture(onStart));
|
||||
Exception exception = new ConnectException("Failed to start connector");
|
||||
EasyMock.expectLastCall().andAnswer(new IAnswer<Boolean>() {
|
||||
@Override
|
||||
public Boolean answer() throws Throwable {
|
||||
onStart.getValue().onCompletion(exception, null);
|
||||
return true;
|
||||
}
|
||||
EasyMock.expectLastCall().andAnswer(() -> {
|
||||
onStart.getValue().onCompletion(exception, null);
|
||||
return true;
|
||||
});
|
||||
|
||||
PowerMock.replayAll();
|
||||
|
@ -551,12 +544,9 @@ public class StandaloneHerderTest {
|
|||
Capture<Callback<TargetState>> onStart = EasyMock.newCapture();
|
||||
worker.startConnector(EasyMock.eq(CONNECTOR_NAME), EasyMock.capture(capturedConfig), EasyMock.<CloseableConnectorContext>anyObject(),
|
||||
EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED), EasyMock.capture(onStart));
|
||||
EasyMock.expectLastCall().andAnswer(new IAnswer<Boolean>() {
|
||||
@Override
|
||||
public Boolean answer() throws Throwable {
|
||||
onStart.getValue().onCompletion(null, TargetState.STARTED);
|
||||
return true;
|
||||
}
|
||||
EasyMock.expectLastCall().andAnswer(() -> {
|
||||
onStart.getValue().onCompletion(null, TargetState.STARTED);
|
||||
return true;
|
||||
});
|
||||
EasyMock.expect(worker.isRunning(CONNECTOR_NAME)).andReturn(true);
|
||||
EasyMock.expect(worker.isTopicCreationEnabled()).andReturn(true);
|
||||
|
@ -661,12 +651,9 @@ public class StandaloneHerderTest {
|
|||
worker.startConnector(EasyMock.eq(CONNECTOR_NAME), EasyMock.eq(connectorProps), EasyMock.anyObject(HerderConnectorContext.class),
|
||||
EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED), EasyMock.capture(onStart));
|
||||
// EasyMock.expectLastCall().andReturn(true);
|
||||
EasyMock.expectLastCall().andAnswer(new IAnswer<Boolean>() {
|
||||
@Override
|
||||
public Boolean answer() throws Throwable {
|
||||
onStart.getValue().onCompletion(null, TargetState.STARTED);
|
||||
return true;
|
||||
}
|
||||
EasyMock.expectLastCall().andAnswer(() -> {
|
||||
onStart.getValue().onCompletion(null, TargetState.STARTED);
|
||||
return true;
|
||||
});
|
||||
EasyMock.expect(worker.isRunning(CONNECTOR_NAME)).andReturn(true);
|
||||
if (sourceSink == SourceSink.SOURCE) {
|
||||
|
|
|
@ -37,7 +37,6 @@ import org.apache.kafka.connect.util.KafkaBasedLog;
|
|||
import org.apache.kafka.connect.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;
|
||||
|
@ -55,7 +54,6 @@ import java.util.HashMap;
|
|||
import java.util.LinkedHashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.Future;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
|
@ -908,13 +906,10 @@ public class KafkaConfigBackingStoreTest {
|
|||
private void expectStart(final List<ConsumerRecord<String, byte[]>> preexistingRecords,
|
||||
final Map<byte[], Struct> deserializations) {
|
||||
storeLog.start();
|
||||
PowerMock.expectLastCall().andAnswer(new IAnswer<Object>() {
|
||||
@Override
|
||||
public Object answer() {
|
||||
for (ConsumerRecord<String, byte[]> rec : preexistingRecords)
|
||||
capturedConsumedCallback.getValue().onCompletion(null, rec);
|
||||
return null;
|
||||
}
|
||||
PowerMock.expectLastCall().andAnswer(() -> {
|
||||
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
|
||||
|
@ -956,29 +951,23 @@ public class KafkaConfigBackingStoreTest {
|
|||
storeLog.send(EasyMock.eq(configKey), EasyMock.aryEq(serialized));
|
||||
PowerMock.expectLastCall();
|
||||
EasyMock.expect(converter.toConnectData(EasyMock.eq(TOPIC), EasyMock.aryEq(serialized)))
|
||||
.andAnswer(new IAnswer<SchemaAndValue>() {
|
||||
@Override
|
||||
public SchemaAndValue answer() throws Throwable {
|
||||
if (dataFieldName != null)
|
||||
assertEquals(dataFieldValue, capturedRecord.getValue().get(dataFieldName));
|
||||
// Note null schema because default settings for internal serialization are schema-less
|
||||
return new SchemaAndValue(null, serialized == null ? null : structToMap(capturedRecord.getValue()));
|
||||
}
|
||||
.andAnswer(() -> {
|
||||
if (dataFieldName != null)
|
||||
assertEquals(dataFieldValue, capturedRecord.getValue().get(dataFieldName));
|
||||
// Note null schema because default settings for internal serialization are schema-less
|
||||
return new SchemaAndValue(null, serialized == null ? 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() {
|
||||
TestFuture<Void> future = new TestFuture<Void>();
|
||||
for (Map.Entry<String, byte[]> entry : serializedConfigs.entrySet())
|
||||
capturedConsumedCallback.getValue().onCompletion(null, new ConsumerRecord<>(TOPIC, 0, logOffset++, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, entry.getKey(), entry.getValue()));
|
||||
future.resolveOnGet((Void) null);
|
||||
return future;
|
||||
}
|
||||
.andAnswer(() -> {
|
||||
TestFuture<Void> future = new TestFuture<Void>();
|
||||
for (Map.Entry<String, byte[]> entry : serializedConfigs.entrySet())
|
||||
capturedConsumedCallback.getValue().onCompletion(null, new ConsumerRecord<>(TOPIC, 0, logOffset++, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, entry.getKey(), entry.getValue()));
|
||||
future.resolveOnGet((Void) null);
|
||||
return future;
|
||||
});
|
||||
}
|
||||
|
||||
|
|
|
@ -29,7 +29,6 @@ import org.apache.kafka.connect.util.ConnectUtils;
|
|||
import org.apache.kafka.connect.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;
|
||||
|
@ -194,27 +193,21 @@ public class KafkaOffsetBackingStoreTest {
|
|||
// 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, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, TP0_KEY.array(), TP0_VALUE.array()));
|
||||
capturedConsumedCallback.getValue().onCompletion(null, new ConsumerRecord<>(TOPIC, 1, 0, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, TP1_KEY.array(), TP1_VALUE.array()));
|
||||
secondGetReadToEndCallback.getValue().onCompletion(null, null);
|
||||
return null;
|
||||
}
|
||||
PowerMock.expectLastCall().andAnswer(() -> {
|
||||
capturedConsumedCallback.getValue().onCompletion(null, new ConsumerRecord<>(TOPIC, 0, 0, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, TP0_KEY.array(), TP0_VALUE.array()));
|
||||
capturedConsumedCallback.getValue().onCompletion(null, new ConsumerRecord<>(TOPIC, 1, 0, 0L, TimestampType.CREATE_TIME, 0L, 0, 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, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, TP0_KEY.array(), TP0_VALUE_NEW.array()));
|
||||
capturedConsumedCallback.getValue().onCompletion(null, new ConsumerRecord<>(TOPIC, 1, 1, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, TP1_KEY.array(), TP1_VALUE_NEW.array()));
|
||||
thirdGetReadToEndCallback.getValue().onCompletion(null, null);
|
||||
return null;
|
||||
}
|
||||
PowerMock.expectLastCall().andAnswer(() -> {
|
||||
capturedConsumedCallback.getValue().onCompletion(null, new ConsumerRecord<>(TOPIC, 0, 1, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, TP0_KEY.array(), TP0_VALUE_NEW.array()));
|
||||
capturedConsumedCallback.getValue().onCompletion(null, new ConsumerRecord<>(TOPIC, 1, 1, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, TP1_KEY.array(), TP1_VALUE_NEW.array()));
|
||||
thirdGetReadToEndCallback.getValue().onCompletion(null, null);
|
||||
return null;
|
||||
});
|
||||
|
||||
expectClusterId();
|
||||
|
@ -234,12 +227,7 @@ public class KafkaOffsetBackingStoreTest {
|
|||
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);
|
||||
}
|
||||
});
|
||||
Future<Void> setFuture = store.set(toSet, (error, result) -> invoked.set(true));
|
||||
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
|
||||
|
@ -300,12 +288,7 @@ public class KafkaOffsetBackingStoreTest {
|
|||
toSet.put(null, TP0_VALUE);
|
||||
toSet.put(TP1_KEY, null);
|
||||
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);
|
||||
}
|
||||
});
|
||||
Future<Void> setFuture = store.set(toSet, (error, result) -> invoked.set(true));
|
||||
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
|
||||
|
@ -356,13 +339,10 @@ public class KafkaOffsetBackingStoreTest {
|
|||
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) {
|
||||
invoked.set(true);
|
||||
if (error != null)
|
||||
invokedFailure.set(true);
|
||||
}
|
||||
Future<Void> setFuture = store.set(toSet, (error, result) -> {
|
||||
invoked.set(true);
|
||||
if (error != null)
|
||||
invokedFailure.set(true);
|
||||
});
|
||||
assertFalse(setFuture.isDone());
|
||||
// Out of order callbacks shouldn't matter, should still require all to be invoked before invoking the callback
|
||||
|
@ -396,13 +376,10 @@ public class KafkaOffsetBackingStoreTest {
|
|||
|
||||
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;
|
||||
}
|
||||
PowerMock.expectLastCall().andAnswer(() -> {
|
||||
for (ConsumerRecord<byte[], byte[]> rec : preexistingRecords)
|
||||
capturedConsumedCallback.getValue().onCompletion(null, rec);
|
||||
return null;
|
||||
});
|
||||
}
|
||||
|
||||
|
|
|
@ -35,7 +35,6 @@ import org.apache.kafka.connect.util.KafkaBasedLog;
|
|||
import org.easymock.Capture;
|
||||
import org.easymock.EasyMock;
|
||||
import org.easymock.EasyMockSupport;
|
||||
import org.easymock.IAnswer;
|
||||
import org.easymock.Mock;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
|
@ -102,12 +101,9 @@ public class KafkaStatusBackingStoreTest extends EasyMockSupport {
|
|||
final Capture<Callback> callbackCapture = newCapture();
|
||||
kafkaBasedLog.send(eq("status-connector-conn"), eq(value), capture(callbackCapture));
|
||||
expectLastCall()
|
||||
.andAnswer(new IAnswer<Void>() {
|
||||
@Override
|
||||
public Void answer() throws Throwable {
|
||||
callbackCapture.getValue().onCompletion(null, null);
|
||||
return null;
|
||||
}
|
||||
.andAnswer(() -> {
|
||||
callbackCapture.getValue().onCompletion(null, null);
|
||||
return null;
|
||||
});
|
||||
replayAll();
|
||||
|
||||
|
@ -129,19 +125,13 @@ public class KafkaStatusBackingStoreTest extends EasyMockSupport {
|
|||
final Capture<Callback> callbackCapture = newCapture();
|
||||
kafkaBasedLog.send(eq("status-connector-conn"), eq(value), capture(callbackCapture));
|
||||
expectLastCall()
|
||||
.andAnswer(new IAnswer<Void>() {
|
||||
@Override
|
||||
public Void answer() throws Throwable {
|
||||
callbackCapture.getValue().onCompletion(null, new TimeoutException());
|
||||
return null;
|
||||
}
|
||||
.andAnswer(() -> {
|
||||
callbackCapture.getValue().onCompletion(null, new TimeoutException());
|
||||
return null;
|
||||
})
|
||||
.andAnswer(new IAnswer<Void>() {
|
||||
@Override
|
||||
public Void answer() throws Throwable {
|
||||
callbackCapture.getValue().onCompletion(null, null);
|
||||
return null;
|
||||
}
|
||||
.andAnswer(() -> {
|
||||
callbackCapture.getValue().onCompletion(null, null);
|
||||
return null;
|
||||
});
|
||||
replayAll();
|
||||
|
||||
|
@ -163,12 +153,9 @@ public class KafkaStatusBackingStoreTest extends EasyMockSupport {
|
|||
final Capture<Callback> callbackCapture = newCapture();
|
||||
kafkaBasedLog.send(eq("status-connector-conn"), eq(value), capture(callbackCapture));
|
||||
expectLastCall()
|
||||
.andAnswer(new IAnswer<Void>() {
|
||||
@Override
|
||||
public Void answer() throws Throwable {
|
||||
callbackCapture.getValue().onCompletion(null, new UnknownServerException());
|
||||
return null;
|
||||
}
|
||||
.andAnswer(() -> {
|
||||
callbackCapture.getValue().onCompletion(null, new UnknownServerException());
|
||||
return null;
|
||||
});
|
||||
replayAll();
|
||||
|
||||
|
@ -257,13 +244,10 @@ public class KafkaStatusBackingStoreTest extends EasyMockSupport {
|
|||
final Capture<Callback> callbackCapture = newCapture();
|
||||
kafkaBasedLog.send(eq("status-connector-conn"), eq(value), capture(callbackCapture));
|
||||
expectLastCall()
|
||||
.andAnswer(new IAnswer<Void>() {
|
||||
@Override
|
||||
public Void answer() throws Throwable {
|
||||
callbackCapture.getValue().onCompletion(null, null);
|
||||
store.read(consumerRecord(1, "status-connector-conn", value));
|
||||
return null;
|
||||
}
|
||||
.andAnswer(() -> {
|
||||
callbackCapture.getValue().onCompletion(null, null);
|
||||
store.read(consumerRecord(1, "status-connector-conn", value));
|
||||
return null;
|
||||
});
|
||||
|
||||
replayAll();
|
||||
|
@ -303,13 +287,10 @@ public class KafkaStatusBackingStoreTest extends EasyMockSupport {
|
|||
final Capture<Callback> callbackCapture = newCapture();
|
||||
kafkaBasedLog.send(eq("status-connector-conn"), eq(value), capture(callbackCapture));
|
||||
expectLastCall()
|
||||
.andAnswer(new IAnswer<Void>() {
|
||||
@Override
|
||||
public Void answer() throws Throwable {
|
||||
callbackCapture.getValue().onCompletion(null, null);
|
||||
store.read(consumerRecord(1, "status-connector-conn", value));
|
||||
return null;
|
||||
}
|
||||
.andAnswer(() -> {
|
||||
callbackCapture.getValue().onCompletion(null, null);
|
||||
store.read(consumerRecord(1, "status-connector-conn", value));
|
||||
return null;
|
||||
});
|
||||
replayAll();
|
||||
|
||||
|
@ -353,12 +334,9 @@ public class KafkaStatusBackingStoreTest extends EasyMockSupport {
|
|||
final Capture<Callback> callbackCapture = newCapture();
|
||||
kafkaBasedLog.send(eq("status-task-conn-0"), eq(value), capture(callbackCapture));
|
||||
expectLastCall()
|
||||
.andAnswer(new IAnswer<Void>() {
|
||||
@Override
|
||||
public Void answer() throws Throwable {
|
||||
callbackCapture.getValue().onCompletion(null, null);
|
||||
return null;
|
||||
}
|
||||
.andAnswer(() -> {
|
||||
callbackCapture.getValue().onCompletion(null, null);
|
||||
return null;
|
||||
});
|
||||
replayAll();
|
||||
|
||||
|
|
|
@ -20,7 +20,6 @@ import org.apache.kafka.connect.errors.ConnectException;
|
|||
import org.apache.kafka.connect.util.Callback;
|
||||
import org.easymock.Capture;
|
||||
import org.easymock.EasyMock;
|
||||
import org.easymock.IAnswer;
|
||||
import org.junit.After;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
|
@ -39,7 +38,6 @@ import java.util.concurrent.ExecutorService;
|
|||
import java.util.concurrent.Executors;
|
||||
import java.util.concurrent.Future;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.Callable;
|
||||
|
||||
import static org.junit.Assert.assertFalse;
|
||||
import static org.junit.Assert.assertThrows;
|
||||
|
@ -245,25 +243,19 @@ public class OffsetStorageWriterTest {
|
|||
keySerialized == null ? null : ByteBuffer.wrap(keySerialized),
|
||||
valueSerialized == null ? null : ByteBuffer.wrap(valueSerialized));
|
||||
EasyMock.expect(store.set(EasyMock.eq(offsetsSerialized), EasyMock.capture(storeCallback)))
|
||||
.andAnswer(new IAnswer<Future<Void>>() {
|
||||
@Override
|
||||
public Future<Void> answer() throws Throwable {
|
||||
return service.submit(new Callable<Void>() {
|
||||
@Override
|
||||
public Void call() throws Exception {
|
||||
if (waitForCompletion != null)
|
||||
assertTrue(waitForCompletion.await(10000, TimeUnit.MILLISECONDS));
|
||||
.andAnswer(() ->
|
||||
service.submit(() -> {
|
||||
if (waitForCompletion != null)
|
||||
assertTrue(waitForCompletion.await(10000, TimeUnit.MILLISECONDS));
|
||||
|
||||
if (fail) {
|
||||
storeCallback.getValue().onCompletion(exception, null);
|
||||
} else {
|
||||
storeCallback.getValue().onCompletion(null, null);
|
||||
}
|
||||
return null;
|
||||
}
|
||||
});
|
||||
if (fail) {
|
||||
storeCallback.getValue().onCompletion(exception, null);
|
||||
} else {
|
||||
storeCallback.getValue().onCompletion(null, null);
|
||||
}
|
||||
});
|
||||
return null;
|
||||
})
|
||||
);
|
||||
if (callback != null) {
|
||||
if (fail) {
|
||||
callback.onCompletion(EasyMock.eq(exception), EasyMock.eq((Void) null));
|
||||
|
|
|
@ -119,17 +119,14 @@ public class KafkaBasedLogTest {
|
|||
private MockConsumer<String, String> consumer;
|
||||
|
||||
private Map<TopicPartition, List<ConsumerRecord<String, String>>> consumedRecords = new HashMap<>();
|
||||
private Callback<ConsumerRecord<String, String>> consumedCallback = new Callback<ConsumerRecord<String, String>>() {
|
||||
@Override
|
||||
public void onCompletion(Throwable error, ConsumerRecord<String, String> record) {
|
||||
TopicPartition partition = new TopicPartition(record.topic(), record.partition());
|
||||
List<ConsumerRecord<String, String>> records = consumedRecords.get(partition);
|
||||
if (records == null) {
|
||||
records = new ArrayList<>();
|
||||
consumedRecords.put(partition, records);
|
||||
}
|
||||
records.add(record);
|
||||
private Callback<ConsumerRecord<String, String>> consumedCallback = (error, record) -> {
|
||||
TopicPartition partition = new TopicPartition(record.topic(), record.partition());
|
||||
List<ConsumerRecord<String, String>> records = consumedRecords.get(partition);
|
||||
if (records == null) {
|
||||
records = new ArrayList<>();
|
||||
consumedRecords.put(partition, records);
|
||||
}
|
||||
records.add(record);
|
||||
};
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
|
@ -178,34 +175,21 @@ public class KafkaBasedLogTest {
|
|||
endOffsets.put(TP1, 1L);
|
||||
consumer.updateEndOffsets(endOffsets);
|
||||
final CountDownLatch finishedLatch = new CountDownLatch(1);
|
||||
consumer.schedulePollTask(new Runnable() { // Use first poll task to setup sequence of remaining responses to polls
|
||||
@Override
|
||||
public void run() {
|
||||
// Should keep polling until it reaches current log end offset for all partitions. Should handle
|
||||
// as many empty polls as needed
|
||||
consumer.scheduleNopPollTask();
|
||||
consumer.scheduleNopPollTask();
|
||||
consumer.schedulePollTask(new Runnable() {
|
||||
@Override
|
||||
public void run() {
|
||||
consumer.addRecord(new ConsumerRecord<>(TOPIC, 0, 0, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, TP0_KEY, TP0_VALUE));
|
||||
}
|
||||
});
|
||||
consumer.scheduleNopPollTask();
|
||||
consumer.scheduleNopPollTask();
|
||||
consumer.schedulePollTask(new Runnable() {
|
||||
@Override
|
||||
public void run() {
|
||||
consumer.addRecord(new ConsumerRecord<>(TOPIC, 1, 0, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, TP1_KEY, TP1_VALUE));
|
||||
}
|
||||
});
|
||||
consumer.schedulePollTask(new Runnable() {
|
||||
@Override
|
||||
public void run() {
|
||||
finishedLatch.countDown();
|
||||
}
|
||||
});
|
||||
}
|
||||
consumer.schedulePollTask(() -> {
|
||||
// Use first poll task to setup sequence of remaining responses to polls
|
||||
// Should keep polling until it reaches current log end offset for all partitions. Should handle
|
||||
// as many empty polls as needed
|
||||
consumer.scheduleNopPollTask();
|
||||
consumer.scheduleNopPollTask();
|
||||
consumer.schedulePollTask(() ->
|
||||
consumer.addRecord(new ConsumerRecord<>(TOPIC, 0, 0, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, TP0_KEY, TP0_VALUE))
|
||||
);
|
||||
consumer.scheduleNopPollTask();
|
||||
consumer.scheduleNopPollTask();
|
||||
consumer.schedulePollTask(() ->
|
||||
consumer.addRecord(new ConsumerRecord<>(TOPIC, 1, 0, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, TP1_KEY, TP1_VALUE))
|
||||
);
|
||||
consumer.schedulePollTask(finishedLatch::countDown);
|
||||
});
|
||||
store.start();
|
||||
assertTrue(finishedLatch.await(10000, TimeUnit.MILLISECONDS));
|
||||
|
@ -237,14 +221,11 @@ public class KafkaBasedLogTest {
|
|||
// Better test with an advanced offset other than just 0L
|
||||
consumer.updateBeginningOffsets(endOffsets);
|
||||
|
||||
consumer.schedulePollTask(new Runnable() {
|
||||
@Override
|
||||
public void run() {
|
||||
// Throw an exception that will not be ignored or handled by Connect framework. In
|
||||
// reality a misplaced call to poll blocks indefinitely and connect aborts due to
|
||||
// time outs (for instance via ConnectRestException)
|
||||
throw new WakeupException();
|
||||
}
|
||||
consumer.schedulePollTask(() -> {
|
||||
// Throw an exception that will not be ignored or handled by Connect framework. In
|
||||
// reality a misplaced call to poll blocks indefinitely and connect aborts due to
|
||||
// time outs (for instance via ConnectRestException)
|
||||
throw new WakeupException();
|
||||
});
|
||||
|
||||
store.start();
|
||||
|
@ -291,12 +272,7 @@ public class KafkaBasedLogTest {
|
|||
|
||||
// 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();
|
||||
}
|
||||
};
|
||||
org.apache.kafka.clients.producer.Callback producerCallback = (metadata, exception) -> invoked.incrementAndGet();
|
||||
store.send(TP0_KEY, TP0_VALUE, producerCallback);
|
||||
store.send(TP1_KEY, TP1_VALUE, producerCallback);
|
||||
assertEquals(0, invoked.get());
|
||||
|
@ -309,46 +285,31 @@ public class KafkaBasedLogTest {
|
|||
|
||||
// Now we should have to wait for the records to be read back when we call readToEnd()
|
||||
final AtomicBoolean getInvoked = new AtomicBoolean(false);
|
||||
final FutureCallback<Void> readEndFutureCallback = new FutureCallback<>(new Callback<Void>() {
|
||||
@Override
|
||||
public void onCompletion(Throwable error, Void result) {
|
||||
getInvoked.set(true);
|
||||
}
|
||||
});
|
||||
consumer.schedulePollTask(new Runnable() {
|
||||
@Override
|
||||
public void run() {
|
||||
// Once we're synchronized in a poll, start the read to end and schedule the exact set of poll events
|
||||
// that should follow. This readToEnd call will immediately wakeup this consumer.poll() call without
|
||||
// returning any data.
|
||||
Map<TopicPartition, Long> newEndOffsets = new HashMap<>();
|
||||
newEndOffsets.put(TP0, 2L);
|
||||
newEndOffsets.put(TP1, 2L);
|
||||
consumer.updateEndOffsets(newEndOffsets);
|
||||
store.readToEnd(readEndFutureCallback);
|
||||
final FutureCallback<Void> readEndFutureCallback = new FutureCallback<>((error, result) -> getInvoked.set(true));
|
||||
consumer.schedulePollTask(() -> {
|
||||
// Once we're synchronized in a poll, start the read to end and schedule the exact set of poll events
|
||||
// that should follow. This readToEnd call will immediately wakeup this consumer.poll() call without
|
||||
// returning any data.
|
||||
Map<TopicPartition, Long> newEndOffsets = new HashMap<>();
|
||||
newEndOffsets.put(TP0, 2L);
|
||||
newEndOffsets.put(TP1, 2L);
|
||||
consumer.updateEndOffsets(newEndOffsets);
|
||||
store.readToEnd(readEndFutureCallback);
|
||||
|
||||
// Should keep polling until it reaches current log end offset for all partitions
|
||||
consumer.scheduleNopPollTask();
|
||||
consumer.scheduleNopPollTask();
|
||||
consumer.scheduleNopPollTask();
|
||||
consumer.schedulePollTask(new Runnable() {
|
||||
@Override
|
||||
public void run() {
|
||||
consumer.addRecord(new ConsumerRecord<>(TOPIC, 0, 0, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, TP0_KEY, TP0_VALUE));
|
||||
consumer.addRecord(new ConsumerRecord<>(TOPIC, 0, 1, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, TP0_KEY, TP0_VALUE_NEW));
|
||||
consumer.addRecord(new ConsumerRecord<>(TOPIC, 1, 0, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, TP1_KEY, TP1_VALUE));
|
||||
}
|
||||
});
|
||||
// Should keep polling until it reaches current log end offset for all partitions
|
||||
consumer.scheduleNopPollTask();
|
||||
consumer.scheduleNopPollTask();
|
||||
consumer.scheduleNopPollTask();
|
||||
consumer.schedulePollTask(() -> {
|
||||
consumer.addRecord(new ConsumerRecord<>(TOPIC, 0, 0, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, TP0_KEY, TP0_VALUE));
|
||||
consumer.addRecord(new ConsumerRecord<>(TOPIC, 0, 1, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, TP0_KEY, TP0_VALUE_NEW));
|
||||
consumer.addRecord(new ConsumerRecord<>(TOPIC, 1, 0, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, TP1_KEY, TP1_VALUE));
|
||||
});
|
||||
|
||||
consumer.schedulePollTask(new Runnable() {
|
||||
@Override
|
||||
public void run() {
|
||||
consumer.addRecord(new ConsumerRecord<>(TOPIC, 1, 1, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, TP1_KEY, TP1_VALUE_NEW));
|
||||
}
|
||||
});
|
||||
consumer.schedulePollTask(() ->
|
||||
consumer.addRecord(new ConsumerRecord<>(TOPIC, 1, 1, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, TP1_KEY, TP1_VALUE_NEW)));
|
||||
|
||||
// Already have FutureCallback that should be invoked/awaited, so no need for follow up finishedLatch
|
||||
}
|
||||
// Already have FutureCallback that should be invoked/awaited, so no need for follow up finishedLatch
|
||||
});
|
||||
readEndFutureCallback.get(10000, TimeUnit.MILLISECONDS);
|
||||
assertTrue(getInvoked.get());
|
||||
|
@ -382,35 +343,20 @@ public class KafkaBasedLogTest {
|
|||
endOffsets.put(TP0, 1L);
|
||||
endOffsets.put(TP1, 1L);
|
||||
consumer.updateEndOffsets(endOffsets);
|
||||
consumer.schedulePollTask(new Runnable() {
|
||||
@Override
|
||||
public void run() {
|
||||
// Trigger exception
|
||||
consumer.schedulePollTask(new Runnable() {
|
||||
@Override
|
||||
public void run() {
|
||||
consumer.setPollException(Errors.COORDINATOR_NOT_AVAILABLE.exception());
|
||||
}
|
||||
});
|
||||
consumer.schedulePollTask(() -> {
|
||||
// Trigger exception
|
||||
consumer.schedulePollTask(() ->
|
||||
consumer.setPollException(Errors.COORDINATOR_NOT_AVAILABLE.exception()));
|
||||
|
||||
// Should keep polling until it reaches current log end offset for all partitions
|
||||
consumer.scheduleNopPollTask();
|
||||
consumer.scheduleNopPollTask();
|
||||
consumer.schedulePollTask(new Runnable() {
|
||||
@Override
|
||||
public void run() {
|
||||
consumer.addRecord(new ConsumerRecord<>(TOPIC, 0, 0, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, TP0_KEY, TP0_VALUE_NEW));
|
||||
consumer.addRecord(new ConsumerRecord<>(TOPIC, 1, 0, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, TP0_KEY, TP0_VALUE_NEW));
|
||||
}
|
||||
});
|
||||
// Should keep polling until it reaches current log end offset for all partitions
|
||||
consumer.scheduleNopPollTask();
|
||||
consumer.scheduleNopPollTask();
|
||||
consumer.schedulePollTask(() -> {
|
||||
consumer.addRecord(new ConsumerRecord<>(TOPIC, 0, 0, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, TP0_KEY, TP0_VALUE_NEW));
|
||||
consumer.addRecord(new ConsumerRecord<>(TOPIC, 1, 0, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, TP0_KEY, TP0_VALUE_NEW));
|
||||
});
|
||||
|
||||
consumer.schedulePollTask(new Runnable() {
|
||||
@Override
|
||||
public void run() {
|
||||
finishedLatch.countDown();
|
||||
}
|
||||
});
|
||||
}
|
||||
consumer.schedulePollTask(finishedLatch::countDown);
|
||||
});
|
||||
store.start();
|
||||
assertTrue(finishedLatch.await(10000, TimeUnit.MILLISECONDS));
|
||||
|
@ -442,45 +388,29 @@ public class KafkaBasedLogTest {
|
|||
consumer.updateEndOffsets(endOffsets);
|
||||
store.start();
|
||||
final AtomicBoolean getInvoked = new AtomicBoolean(false);
|
||||
final FutureCallback<Void> readEndFutureCallback = new FutureCallback<>(new Callback<Void>() {
|
||||
@Override
|
||||
public void onCompletion(Throwable error, Void result) {
|
||||
getInvoked.set(true);
|
||||
}
|
||||
});
|
||||
consumer.schedulePollTask(new Runnable() {
|
||||
@Override
|
||||
public void run() {
|
||||
// Once we're synchronized in a poll, start the read to end and schedule the exact set of poll events
|
||||
// that should follow. This readToEnd call will immediately wakeup this consumer.poll() call without
|
||||
// returning any data.
|
||||
Map<TopicPartition, Long> newEndOffsets = new HashMap<>();
|
||||
newEndOffsets.put(TP0, 1L);
|
||||
newEndOffsets.put(TP1, 1L);
|
||||
consumer.updateEndOffsets(newEndOffsets);
|
||||
// Set exception to occur when getting offsets to read log to end. It'll be caught in the work thread,
|
||||
// which will retry and eventually get the correct offsets and read log to end.
|
||||
consumer.setOffsetsException(new TimeoutException("Failed to get offsets by times"));
|
||||
store.readToEnd(readEndFutureCallback);
|
||||
final FutureCallback<Void> readEndFutureCallback = new FutureCallback<>((error, result) -> getInvoked.set(true));
|
||||
consumer.schedulePollTask(() -> {
|
||||
// Once we're synchronized in a poll, start the read to end and schedule the exact set of poll events
|
||||
// that should follow. This readToEnd call will immediately wakeup this consumer.poll() call without
|
||||
// returning any data.
|
||||
Map<TopicPartition, Long> newEndOffsets = new HashMap<>();
|
||||
newEndOffsets.put(TP0, 1L);
|
||||
newEndOffsets.put(TP1, 1L);
|
||||
consumer.updateEndOffsets(newEndOffsets);
|
||||
// Set exception to occur when getting offsets to read log to end. It'll be caught in the work thread,
|
||||
// which will retry and eventually get the correct offsets and read log to end.
|
||||
consumer.setOffsetsException(new TimeoutException("Failed to get offsets by times"));
|
||||
store.readToEnd(readEndFutureCallback);
|
||||
|
||||
// Should keep polling until it reaches current log end offset for all partitions
|
||||
consumer.scheduleNopPollTask();
|
||||
consumer.scheduleNopPollTask();
|
||||
consumer.schedulePollTask(new Runnable() {
|
||||
@Override
|
||||
public void run() {
|
||||
consumer.addRecord(new ConsumerRecord<>(TOPIC, 0, 0, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, TP0_KEY, TP0_VALUE));
|
||||
consumer.addRecord(new ConsumerRecord<>(TOPIC, 1, 0, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, TP0_KEY, TP0_VALUE_NEW));
|
||||
}
|
||||
});
|
||||
// Should keep polling until it reaches current log end offset for all partitions
|
||||
consumer.scheduleNopPollTask();
|
||||
consumer.scheduleNopPollTask();
|
||||
consumer.schedulePollTask(() -> {
|
||||
consumer.addRecord(new ConsumerRecord<>(TOPIC, 0, 0, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, TP0_KEY, TP0_VALUE));
|
||||
consumer.addRecord(new ConsumerRecord<>(TOPIC, 1, 0, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, TP0_KEY, TP0_VALUE_NEW));
|
||||
});
|
||||
|
||||
consumer.schedulePollTask(new Runnable() {
|
||||
@Override
|
||||
public void run() {
|
||||
finishedLatch.countDown();
|
||||
}
|
||||
});
|
||||
}
|
||||
consumer.schedulePollTask(finishedLatch::countDown);
|
||||
});
|
||||
readEndFutureCallback.get(10000, TimeUnit.MILLISECONDS);
|
||||
assertTrue(getInvoked.get());
|
||||
|
@ -517,12 +447,9 @@ public class KafkaBasedLogTest {
|
|||
assertEquals(0L, consumer.position(TP1));
|
||||
|
||||
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);
|
||||
}
|
||||
store.send(TP0_KEY, TP0_VALUE, (metadata, exception) -> {
|
||||
assertNull(setException.get()); // Should only be invoked once
|
||||
setException.set(exception);
|
||||
});
|
||||
KafkaException exc = new LeaderNotAvailableException("Error");
|
||||
tp0Future.resolve(exc);
|
||||
|
|
|
@ -360,9 +360,7 @@ public class TopicAdminTest {
|
|||
assertEquals(1, result.size());
|
||||
Config config = result.get("myTopic");
|
||||
assertNotNull(config);
|
||||
config.entries().forEach(entry -> {
|
||||
assertEquals(newTopic.configs().get(entry.name()), entry.value());
|
||||
});
|
||||
config.entries().forEach(entry -> assertEquals(newTopic.configs().get(entry.name()), entry.value()));
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -425,9 +423,7 @@ public class TopicAdminTest {
|
|||
TopicPartitionInfo topicPartitionInfo = new TopicPartitionInfo(0, cluster.nodeById(0), cluster.nodes(), Collections.<Node>emptyList());
|
||||
mockAdminClient.addTopic(false, topicName, Collections.singletonList(topicPartitionInfo), topicConfigs);
|
||||
TopicAdmin admin = new TopicAdmin(null, mockAdminClient);
|
||||
ConfigException e = assertThrows(ConfigException.class, () -> {
|
||||
admin.verifyTopicCleanupPolicyOnlyCompact("myTopic", "worker.topic", "purpose");
|
||||
});
|
||||
ConfigException e = assertThrows(ConfigException.class, () -> admin.verifyTopicCleanupPolicyOnlyCompact("myTopic", "worker.topic", "purpose"));
|
||||
assertTrue(e.getMessage().contains("to guarantee consistency and durability"));
|
||||
}
|
||||
}
|
||||
|
@ -441,9 +437,7 @@ public class TopicAdminTest {
|
|||
TopicPartitionInfo topicPartitionInfo = new TopicPartitionInfo(0, cluster.nodeById(0), cluster.nodes(), Collections.<Node>emptyList());
|
||||
mockAdminClient.addTopic(false, topicName, Collections.singletonList(topicPartitionInfo), topicConfigs);
|
||||
TopicAdmin admin = new TopicAdmin(null, mockAdminClient);
|
||||
ConfigException e = assertThrows(ConfigException.class, () -> {
|
||||
admin.verifyTopicCleanupPolicyOnlyCompact("myTopic", "worker.topic", "purpose");
|
||||
});
|
||||
ConfigException e = assertThrows(ConfigException.class, () -> admin.verifyTopicCleanupPolicyOnlyCompact("myTopic", "worker.topic", "purpose"));
|
||||
assertTrue(e.getMessage().contains("to guarantee consistency and durability"));
|
||||
}
|
||||
}
|
||||
|
|
|
@ -61,14 +61,11 @@ public class TimestampRouter<R extends ConnectRecord<R>> implements Transformati
|
|||
topicFormat = config.getString(ConfigName.TOPIC_FORMAT);
|
||||
|
||||
final String timestampFormatStr = config.getString(ConfigName.TIMESTAMP_FORMAT);
|
||||
timestampFormat = new ThreadLocal<SimpleDateFormat>() {
|
||||
@Override
|
||||
protected SimpleDateFormat initialValue() {
|
||||
final SimpleDateFormat fmt = new SimpleDateFormat(timestampFormatStr);
|
||||
fmt.setTimeZone(TimeZone.getTimeZone("UTC"));
|
||||
return fmt;
|
||||
}
|
||||
};
|
||||
timestampFormat = ThreadLocal.withInitial(() -> {
|
||||
final SimpleDateFormat fmt = new SimpleDateFormat(timestampFormatStr);
|
||||
fmt.setTimeZone(TimeZone.getTimeZone("UTC"));
|
||||
return fmt;
|
||||
});
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -84,7 +84,7 @@ public class HasHeaderKeyTest {
|
|||
|
||||
private SourceRecord recordWithHeaders(String... headers) {
|
||||
return new SourceRecord(null, null, null, null, null, null, null, null, null,
|
||||
Arrays.stream(headers).map(header -> new TestHeader(header)).collect(Collectors.toList()));
|
||||
Arrays.stream(headers).map(TestHeader::new).collect(Collectors.toList()));
|
||||
}
|
||||
|
||||
private static class TestHeader implements Header {
|
||||
|
|
Loading…
Reference in New Issue