KAFKA-10835: Replace Runnable and Callable overrides with lambdas in Connect (#9867)

Reviewers: Konstantine Karantasis <k.karantasis@gmail.com>
This commit is contained in:
Lev Zemlyanov 2021-02-04 12:15:49 -05:00 committed by GitHub
parent 131d4753cf
commit c19a35d1b7
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
50 changed files with 967 additions and 1621 deletions

View File

@ -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

View File

@ -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;
};
}

View File

@ -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;
});
}

View File

@ -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());
}

View File

@ -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) {

View File

@ -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");
}

View File

@ -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));
}
}

View File

@ -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));
}

View File

@ -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) {

View File

@ -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) {

View File

@ -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));
}

View File

@ -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();

View File

@ -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),

View File

@ -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);

View File

@ -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);

View File

@ -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);
}

View File

@ -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));
}
}

View File

@ -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() {

View File

@ -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();
}

View File

@ -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",

View File

@ -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;
}
);
}

View File

@ -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

View File

@ -392,9 +392,7 @@ public class StandaloneHerder extends AbstractHerder {
}
if (newState == TargetState.STARTED) {
requestExecutorService.submit(() -> {
updateConnectorTasks(connector);
});
requestExecutorService.submit(() -> updateConnectorTasks(connector));
}
});
}

View File

@ -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");
}
}
};

View File

@ -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");
}
}
};

View File

@ -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");
}
}
};

View File

@ -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;
});
}

View File

@ -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);
}
});
}

View File

@ -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);
}
}

View File

@ -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() {

View File

@ -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();

View File

@ -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);
}

View File

@ -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;
}

View File

@ -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 {

View File

@ -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 {

View File

@ -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);

View File

@ -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);

View File

@ -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()) {

View File

@ -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));

View File

@ -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);

View File

@ -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;
});
}

View File

@ -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) {

View File

@ -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;
});
}

View File

@ -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;
});
}

View File

@ -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();

View File

@ -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));

View File

@ -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);

View File

@ -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"));
}
}

View File

@ -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

View File

@ -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 {