diff --git a/build.gradle b/build.gradle index ef7c959dda2b..f65cf2ff3788 100644 --- a/build.gradle +++ b/build.gradle @@ -438,7 +438,8 @@ subprojects { "**/ErrorHandlingTaskTest.*", "**/KafkaConfigBackingStoreTest.*", "**/KafkaOffsetBackingStoreTest.*", "**/KafkaBasedLogTest.*", "**/OffsetStorageWriterTest.*", "**/StandaloneHerderTest.*", "**/SourceTaskOffsetCommitterTest.*", "**/WorkerConfigTransformerTest.*", "**/WorkerGroupMemberTest.*", - "**/WorkerSinkTaskTest.*", "**/WorkerSinkTaskThreadedTest.*", "**/WorkerSourceTaskTest.*", + "**/WorkerTest.*", "**/WorkerSinkTaskTest.*", "**/WorkerSinkTaskThreadedTest.*", + "**/WorkerSourceTaskTest.*", "**/AbstractWorkerSourceTaskTest.*", "**/ExactlyOnceWorkerSourceTaskTest.*", "**/WorkerTaskTest.*", // streams tests "**/KafkaStreamsTest.*" diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java b/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java index cc913bfda0e7..613b0c3a9f38 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java @@ -2716,7 +2716,11 @@ private static Map logDirDescriptions(DescribeLogDirs new ReplicaInfo(p.partitionSize(), p.offsetLag(), p.isFutureKey())); } } - result.put(logDirResult.logDir(), new LogDirDescription(Errors.forCode(logDirResult.errorCode()).exception(), replicaInfoMap)); + result.put(logDirResult.logDir(), new LogDirDescription( + Errors.forCode(logDirResult.errorCode()).exception(), + replicaInfoMap, + logDirResult.totalBytes(), + logDirResult.usableBytes())); } return result; } diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/LogDirDescription.java b/clients/src/main/java/org/apache/kafka/clients/admin/LogDirDescription.java index 1c326ec43b92..665c86649ba3 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/LogDirDescription.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/LogDirDescription.java @@ -20,8 +20,10 @@ import org.apache.kafka.common.errors.ApiException; import java.util.Map; +import java.util.OptionalLong; import static java.util.Collections.unmodifiableMap; +import static org.apache.kafka.common.requests.DescribeLogDirsResponse.UNKNOWN_VOLUME_BYTES; /** * A description of a log directory on a particular broker. @@ -29,10 +31,18 @@ public class LogDirDescription { private final Map replicaInfos; private final ApiException error; + private final OptionalLong totalBytes; + private final OptionalLong usableBytes; public LogDirDescription(ApiException error, Map replicaInfos) { + this(error, replicaInfos, UNKNOWN_VOLUME_BYTES, UNKNOWN_VOLUME_BYTES); + } + + public LogDirDescription(ApiException error, Map replicaInfos, long totalBytes, long usableBytes) { this.error = error; this.replicaInfos = replicaInfos; + this.totalBytes = (totalBytes == UNKNOWN_VOLUME_BYTES) ? OptionalLong.empty() : OptionalLong.of(totalBytes); + this.usableBytes = (usableBytes == UNKNOWN_VOLUME_BYTES) ? OptionalLong.empty() : OptionalLong.of(usableBytes); } /** @@ -54,11 +64,29 @@ public Map replicaInfos() { return unmodifiableMap(replicaInfos); } + /** + * The total size of the volume this log directory is on or empty if the broker did not return a value. + * For volumes larger than Long.MAX_VALUE, Long.MAX_VALUE is returned. + */ + public OptionalLong totalBytes() { + return totalBytes; + } + + /** + * The usable size on the volume this log directory is on or empty if the broker did not return a value. + * For usable sizes larger than Long.MAX_VALUE, Long.MAX_VALUE is returned. + */ + public OptionalLong usableBytes() { + return usableBytes; + } + @Override public String toString() { return "LogDirDescription(" + "replicaInfos=" + replicaInfos + ", error=" + error + + ", totalBytes=" + totalBytes + + ", usableBytes=" + usableBytes + ')'; } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java index 1b7f2374b0ba..f9363832cc6c 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java @@ -807,6 +807,9 @@ public void handle(SyncGroupResponse syncResponse, } else if (error == Errors.REBALANCE_IN_PROGRESS) { log.info("SyncGroup failed: The group began another rebalance. Need to re-join the group. " + "Sent generation was {}", sentGeneration); + // consumer didn't get assignment in this generation, so we need to reset generation + // to avoid joinGroup with out-of-data ownedPartitions in cooperative rebalance + resetStateOnResponseError(ApiKeys.SYNC_GROUP, error, false); future.raise(error); } else if (error == Errors.FENCED_INSTANCE_ID) { // for sync-group request, even if the generation has changed we would not expect the instance id diff --git a/clients/src/main/java/org/apache/kafka/common/errors/IneligibleReplicaException.java b/clients/src/main/java/org/apache/kafka/common/errors/IneligibleReplicaException.java new file mode 100644 index 000000000000..6c79add033a4 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/errors/IneligibleReplicaException.java @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.common.errors; + +public class IneligibleReplicaException extends ApiException { + public IneligibleReplicaException(String message) { + super(message); + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/errors/NewLeaderElectedException.java b/clients/src/main/java/org/apache/kafka/common/errors/NewLeaderElectedException.java new file mode 100644 index 000000000000..20fd869df9f0 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/errors/NewLeaderElectedException.java @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.common.errors; + +public class NewLeaderElectedException extends ApiException { + public NewLeaderElectedException(String message) { + super(message); + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/metrics/Metrics.java b/clients/src/main/java/org/apache/kafka/common/metrics/Metrics.java index 52b7794a4c10..398819016cb5 100644 --- a/clients/src/main/java/org/apache/kafka/common/metrics/Metrics.java +++ b/clients/src/main/java/org/apache/kafka/common/metrics/Metrics.java @@ -509,7 +509,10 @@ public void addMetric(MetricName metricName, MetricConfig config, MetricValuePro Objects.requireNonNull(metricValueProvider), config == null ? this.config : config, time); - registerMetric(m); + KafkaMetric existingMetric = registerMetric(m); + if (existingMetric != null) { + throw new IllegalArgumentException("A metric named '" + metricName + "' already exists, can't register another one."); + } } /** @@ -524,6 +527,26 @@ public void addMetric(MetricName metricName, MetricValueProvider metricValueP addMetric(metricName, null, metricValueProvider); } + /** + * Create or get an existing metric to monitor an object that implements MetricValueProvider. + * This metric won't be associated with any sensor. This is a way to expose existing values as metrics. + * This method takes care of synchronisation while updating/accessing metrics by concurrent threads. + * + * @param metricName The name of the metric + * @param metricValueProvider The metric value provider associated with this metric + * @return Existing KafkaMetric if already registered or else a newly created one + */ + public KafkaMetric addMetricIfAbsent(MetricName metricName, MetricConfig config, MetricValueProvider metricValueProvider) { + KafkaMetric metric = new KafkaMetric(new Object(), + Objects.requireNonNull(metricName), + Objects.requireNonNull(metricValueProvider), + config == null ? this.config : config, + time); + + KafkaMetric existingMetric = registerMetric(metric); + return existingMetric == null ? metric : existingMetric; + } + /** * Remove a metric if it exists and return it. Return null otherwise. If a metric is removed, `metricRemoval` * will be invoked for each reporter. @@ -563,10 +586,18 @@ public synchronized void removeReporter(MetricsReporter reporter) { } } - synchronized void registerMetric(KafkaMetric metric) { + /** + * Register a metric if not present or return an already existing metric otherwise. + * When a metric is newly registered, this method returns null + * + * @param metric The KafkaMetric to register + * @return KafkaMetric if the metric already exists, null otherwise + */ + synchronized KafkaMetric registerMetric(KafkaMetric metric) { MetricName metricName = metric.metricName(); - if (this.metrics.containsKey(metricName)) - throw new IllegalArgumentException("A metric named '" + metricName + "' already exists, can't register another one."); + if (this.metrics.containsKey(metricName)) { + return this.metrics.get(metricName); + } this.metrics.put(metricName, metric); for (MetricsReporter reporter : reporters) { try { @@ -576,6 +607,7 @@ synchronized void registerMetric(KafkaMetric metric) { } } log.trace("Registered metric named {}", metricName); + return null; } /** diff --git a/clients/src/main/java/org/apache/kafka/common/metrics/Sensor.java b/clients/src/main/java/org/apache/kafka/common/metrics/Sensor.java index 5ae3b8d997a6..25f3c21a3136 100644 --- a/clients/src/main/java/org/apache/kafka/common/metrics/Sensor.java +++ b/clients/src/main/java/org/apache/kafka/common/metrics/Sensor.java @@ -297,7 +297,10 @@ public synchronized boolean add(CompoundStat stat, MetricConfig config) { for (NamedMeasurable m : stat.stats()) { final KafkaMetric metric = new KafkaMetric(lock, m.name(), m.stat(), statConfig, time); if (!metrics.containsKey(metric.metricName())) { - registry.registerMetric(metric); + KafkaMetric existingMetric = registry.registerMetric(metric); + if (existingMetric != null) { + throw new IllegalArgumentException("A metric named '" + metric.metricName() + "' already exists, can't register another one."); + } metrics.put(metric.metricName(), metric); } } @@ -336,7 +339,10 @@ public synchronized boolean add(final MetricName metricName, final MeasurableSta statConfig, time ); - registry.registerMetric(metric); + KafkaMetric existingMetric = registry.registerMetric(metric); + if (existingMetric != null) { + throw new IllegalArgumentException("A metric named '" + metricName + "' already exists, can't register another one."); + } metrics.put(metric.metricName(), metric); stats.add(new StatAndConfig(Objects.requireNonNull(stat), metric::config)); return true; diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java b/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java index 5db97179064b..2ca42bafcfb8 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java @@ -52,6 +52,7 @@ import org.apache.kafka.common.errors.InconsistentTopicIdException; import org.apache.kafka.common.errors.InconsistentVoterSetException; import org.apache.kafka.common.errors.InconsistentClusterIdException; +import org.apache.kafka.common.errors.IneligibleReplicaException; import org.apache.kafka.common.errors.InvalidCommitOffsetSizeException; import org.apache.kafka.common.errors.InvalidConfigurationException; import org.apache.kafka.common.errors.InvalidFetchSessionEpochException; @@ -77,6 +78,7 @@ import org.apache.kafka.common.errors.LogDirNotFoundException; import org.apache.kafka.common.errors.MemberIdRequiredException; import org.apache.kafka.common.errors.NetworkException; +import org.apache.kafka.common.errors.NewLeaderElectedException; import org.apache.kafka.common.errors.NoReassignmentInProgressException; import org.apache.kafka.common.errors.NotControllerException; import org.apache.kafka.common.errors.NotCoordinatorException; @@ -364,7 +366,9 @@ public enum Errors { INCONSISTENT_TOPIC_ID(103, "The log's topic ID did not match the topic ID in the request", InconsistentTopicIdException::new), INCONSISTENT_CLUSTER_ID(104, "The clusterId in the request does not match that found on the server", InconsistentClusterIdException::new), TRANSACTIONAL_ID_NOT_FOUND(105, "The transactionalId could not be found", TransactionalIdNotFoundException::new), - FETCH_SESSION_TOPIC_ID_ERROR(106, "The fetch session encountered inconsistent topic ID usage", FetchSessionTopicIdException::new); + FETCH_SESSION_TOPIC_ID_ERROR(106, "The fetch session encountered inconsistent topic ID usage", FetchSessionTopicIdException::new), + INELIGIBLE_REPLICA(107, "The new ISR contains at least one ineligible replica.", IneligibleReplicaException::new), + NEW_LEADER_ELECTED(108, "The AlterPartition request successfully updated the partition state but the leader has changed.", NewLeaderElectedException::new); private static final Logger log = LoggerFactory.getLogger(Errors.class); diff --git a/clients/src/main/java/org/apache/kafka/common/requests/AlterPartitionRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/AlterPartitionRequest.java index 091bed617355..2d246f21041f 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/AlterPartitionRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/AlterPartitionRequest.java @@ -45,8 +45,8 @@ public AlterPartitionRequestData data() { @Override public AbstractResponse getErrorResponse(int throttleTimeMs, Throwable e) { return new AlterPartitionResponse(new AlterPartitionResponseData() - .setThrottleTimeMs(throttleTimeMs) - .setErrorCode(Errors.forException(e).code())); + .setThrottleTimeMs(throttleTimeMs) + .setErrorCode(Errors.forException(e).code())); } public static AlterPartitionRequest parse(ByteBuffer buffer, short version) { @@ -57,8 +57,21 @@ public static class Builder extends AbstractRequest.Builder prepareDescribeLogDirsTopics( long partitionSize, long offsetLag, String topic, int partition, boolean isFuture) { return singletonList(new DescribeLogDirsTopic() @@ -1610,6 +1615,19 @@ private static DescribeLogDirsResponse prepareDescribeLogDirsResponse(Errors err ))); } + private static DescribeLogDirsResponse prepareDescribeLogDirsResponse(Errors error, String logDir, + List topics, + long totalBytes, long usableBytes) { + return new DescribeLogDirsResponse( + new DescribeLogDirsResponseData().setResults(singletonList(new DescribeLogDirsResponseData.DescribeLogDirsResult() + .setErrorCode(error.code()) + .setLogDir(logDir) + .setTopics(topics) + .setTotalBytes(totalBytes) + .setUsableBytes(usableBytes) + ))); + } + private static DescribeLogDirsResponse prepareEmptyDescribeLogDirsResponse(Optional error) { DescribeLogDirsResponseData data = new DescribeLogDirsResponseData(); if (error.isPresent()) data.setErrorCode(error.get().code()); @@ -1661,6 +1679,11 @@ public void testDescribeLogDirs() throws ExecutionException, InterruptedExceptio private static void assertDescriptionContains(Map descriptionsMap, String logDir, TopicPartition tp, long partitionSize, long offsetLag) { + assertDescriptionContains(descriptionsMap, logDir, tp, partitionSize, offsetLag, OptionalLong.empty(), OptionalLong.empty()); + } + + private static void assertDescriptionContains(Map descriptionsMap, String logDir, + TopicPartition tp, long partitionSize, long offsetLag, OptionalLong totalBytes, OptionalLong usableBytes) { assertNotNull(descriptionsMap); assertEquals(singleton(logDir), descriptionsMap.keySet()); assertNull(descriptionsMap.get(logDir).error()); @@ -1669,6 +1692,53 @@ private static void assertDescriptionContains(Map des assertEquals(partitionSize, descriptionsReplicaInfos.get(tp).size()); assertEquals(offsetLag, descriptionsReplicaInfos.get(tp).offsetLag()); assertFalse(descriptionsReplicaInfos.get(tp).isFuture()); + assertEquals(totalBytes, descriptionsMap.get(logDir).totalBytes()); + assertEquals(usableBytes, descriptionsMap.get(logDir).usableBytes()); + } + + @Test + public void testDescribeLogDirsWithVolumeBytes() throws ExecutionException, InterruptedException { + Set brokers = singleton(0); + String logDir = "/var/data/kafka"; + TopicPartition tp = new TopicPartition("topic", 12); + long partitionSize = 1234567890; + long offsetLag = 24; + long totalBytes = 123L; + long usableBytes = 456L; + + try (AdminClientUnitTestEnv env = mockClientEnv()) { + env.kafkaClient().setNodeApiVersions(NodeApiVersions.create()); + env.kafkaClient().prepareResponseFrom( + prepareDescribeLogDirsResponse(Errors.NONE, logDir, tp, partitionSize, offsetLag, totalBytes, usableBytes), + env.cluster().nodeById(0)); + + DescribeLogDirsResult result = env.adminClient().describeLogDirs(brokers); + + Map>> descriptions = result.descriptions(); + assertEquals(brokers, descriptions.keySet()); + assertNotNull(descriptions.get(0)); + assertDescriptionContains(descriptions.get(0).get(), logDir, tp, partitionSize, offsetLag, OptionalLong.of(totalBytes), OptionalLong.of(usableBytes)); + + Map> allDescriptions = result.allDescriptions().get(); + assertEquals(brokers, allDescriptions.keySet()); + assertDescriptionContains(allDescriptions.get(0), logDir, tp, partitionSize, offsetLag, OptionalLong.of(totalBytes), OptionalLong.of(usableBytes)); + + // Empty results when not authorized with version < 3 + env.kafkaClient().prepareResponseFrom( + prepareEmptyDescribeLogDirsResponse(Optional.empty()), + env.cluster().nodeById(0)); + final DescribeLogDirsResult errorResult = env.adminClient().describeLogDirs(brokers); + ExecutionException exception = assertThrows(ExecutionException.class, () -> errorResult.allDescriptions().get()); + assertTrue(exception.getCause() instanceof ClusterAuthorizationException); + + // Empty results with an error with version >= 3 + env.kafkaClient().prepareResponseFrom( + prepareEmptyDescribeLogDirsResponse(Optional.of(Errors.UNKNOWN_SERVER_ERROR)), + env.cluster().nodeById(0)); + final DescribeLogDirsResult errorResult2 = env.adminClient().describeLogDirs(brokers); + exception = assertThrows(ExecutionException.class, () -> errorResult2.allDescriptions().get()); + assertTrue(exception.getCause() instanceof UnknownServerException); + } } @SuppressWarnings("deprecation") diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinatorTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinatorTest.java index 48ed136ebc4c..45094ee03035 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinatorTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinatorTest.java @@ -67,6 +67,7 @@ import java.util.Map; import java.util.Optional; import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.Future; @@ -488,6 +489,54 @@ public void testRetainMemberIdAfterSyncGroupDisconnect() { ensureActiveGroup(rejoinedGeneration, memberId); } + @Test + public void testResetGenerationIdAfterSyncGroupFailedWithRebalanceInProgress() throws InterruptedException, ExecutionException { + setupCoordinator(); + + String memberId = "memberId"; + int generation = 5; + + // Rebalance once to initialize the generation and memberId + mockClient.prepareResponse(groupCoordinatorResponse(node, Errors.NONE)); + expectJoinGroup("", generation, memberId); + expectSyncGroup(generation, memberId); + ensureActiveGroup(generation, memberId); + + // Force a rebalance + coordinator.requestRejoin("Manual test trigger"); + assertTrue(coordinator.rejoinNeededOrPending()); + + ExecutorService executor = Executors.newFixedThreadPool(1); + try { + // Return RebalanceInProgress in syncGroup + int rejoinedGeneration = 10; + expectJoinGroup(memberId, rejoinedGeneration, memberId); + expectRebalanceInProgressForSyncGroup(rejoinedGeneration, memberId); + Future secondJoin = executor.submit(() -> + coordinator.ensureActiveGroup(mockTime.timer(Integer.MAX_VALUE))); + + TestUtils.waitForCondition(() -> { + AbstractCoordinator.Generation currentGeneration = coordinator.generation(); + return currentGeneration.generationId == AbstractCoordinator.Generation.NO_GENERATION.generationId && + currentGeneration.memberId.equals(memberId); + }, 2000, "Generation should be reset"); + + rejoinedGeneration = 20; + expectSyncGroup(rejoinedGeneration, memberId); + mockClient.respond(joinGroupFollowerResponse( + rejoinedGeneration, + memberId, + "leaderId", + Errors.NONE, + PROTOCOL_TYPE + )); + assertTrue(secondJoin.get()); + } finally { + executor.shutdownNow(); + executor.awaitTermination(1000, TimeUnit.MILLISECONDS); + } + } + @Test public void testRejoinReason() { setupCoordinator(); @@ -566,6 +615,22 @@ private void expectDisconnectInSyncGroup( }, null, true); } + private void expectRebalanceInProgressForSyncGroup( + int expectedGeneration, + String expectedMemberId + ) { + mockClient.prepareResponse(body -> { + if (!(body instanceof SyncGroupRequest)) { + return false; + } + SyncGroupRequestData syncGroupRequest = ((SyncGroupRequest) body).data(); + return syncGroupRequest.generationId() == expectedGeneration + && syncGroupRequest.memberId().equals(expectedMemberId) + && syncGroupRequest.protocolType().equals(PROTOCOL_TYPE) + && syncGroupRequest.protocolName().equals(PROTOCOL_NAME); + }, syncGroupResponse(Errors.REBALANCE_IN_PROGRESS, PROTOCOL_TYPE, PROTOCOL_NAME)); + } + private void expectDisconnectInJoinGroup( String expectedMemberId ) { diff --git a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java index 138a9c03337e..13f47a977157 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java @@ -1321,9 +1321,10 @@ private AlterPartitionRequest createAlterPartitionRequest(short version) { .setBrokerEpoch(123L) .setBrokerId(1) .setTopics(singletonList(new AlterPartitionRequestData.TopicData() - .setName("topic1") + .setTopicName("topic1") + .setTopicId(Uuid.randomUuid()) .setPartitions(singletonList(partitionData)))); - return new AlterPartitionRequest.Builder(data).build(version); + return new AlterPartitionRequest.Builder(data, version >= 1).build(version); } private AlterPartitionResponse createAlterPartitionResponse(int version) { @@ -1343,8 +1344,9 @@ private AlterPartitionResponse createAlterPartitionResponse(int version) { .setErrorCode(Errors.NONE.code()) .setThrottleTimeMs(123) .setTopics(singletonList(new AlterPartitionResponseData.TopicData() - .setName("topic1") - .setPartitions(singletonList(partitionData)))); + .setTopicName("topic1") + .setTopicId(Uuid.randomUuid()) + .setPartitions(singletonList(partitionData)))); return new AlterPartitionResponse(data); } diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractWorkerSourceTask.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractWorkerSourceTask.java new file mode 100644 index 000000000000..3a805bdd7d5c --- /dev/null +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractWorkerSourceTask.java @@ -0,0 +1,642 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.connect.runtime; + +import org.apache.kafka.clients.admin.NewTopic; +import org.apache.kafka.clients.admin.TopicDescription; +import org.apache.kafka.clients.producer.Callback; +import org.apache.kafka.clients.producer.Producer; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.clients.producer.RecordMetadata; +import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.header.internals.RecordHeaders; +import org.apache.kafka.common.metrics.Sensor; +import org.apache.kafka.common.metrics.stats.Avg; +import org.apache.kafka.common.metrics.stats.CumulativeSum; +import org.apache.kafka.common.metrics.stats.Max; +import org.apache.kafka.common.metrics.stats.Rate; +import org.apache.kafka.common.metrics.stats.Value; +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.connect.errors.ConnectException; +import org.apache.kafka.connect.errors.RetriableException; +import org.apache.kafka.connect.header.Header; +import org.apache.kafka.connect.header.Headers; +import org.apache.kafka.connect.runtime.errors.RetryWithToleranceOperator; +import org.apache.kafka.connect.runtime.errors.Stage; +import org.apache.kafka.connect.source.SourceRecord; +import org.apache.kafka.connect.source.SourceTask; +import org.apache.kafka.connect.source.SourceTaskContext; +import org.apache.kafka.connect.storage.CloseableOffsetStorageReader; +import org.apache.kafka.connect.storage.ConnectorOffsetBackingStore; +import org.apache.kafka.connect.storage.Converter; +import org.apache.kafka.connect.storage.HeaderConverter; +import org.apache.kafka.connect.storage.OffsetStorageWriter; +import org.apache.kafka.connect.storage.StatusBackingStore; +import org.apache.kafka.connect.util.ConnectUtils; +import org.apache.kafka.connect.util.ConnectorTaskId; +import org.apache.kafka.connect.util.TopicAdmin; +import org.apache.kafka.connect.util.TopicCreation; +import org.apache.kafka.connect.util.TopicCreationGroup; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.time.Duration; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.Executor; +import java.util.concurrent.TimeUnit; + +import static org.apache.kafka.connect.runtime.WorkerConfig.TOPIC_TRACKING_ENABLE_CONFIG; + +/** + * WorkerTask that contains shared logic for running source tasks with either standard or exactly-once delivery guarantees. + */ +public abstract class AbstractWorkerSourceTask extends WorkerTask { + private static final Logger log = LoggerFactory.getLogger(AbstractWorkerSourceTask.class); + + private static final long SEND_FAILED_BACKOFF_MS = 100; + + /** + * Hook to define custom startup behavior before the calling {@link SourceTask#initialize(SourceTaskContext)} + * and {@link SourceTask#start(Map)}. + */ + protected abstract void prepareToInitializeTask(); + + /** + * Hook to define custom initialization behavior when preparing to begin the poll-convert-send loop for the first time, + * or when re-entering the loop after being paused. + */ + protected abstract void prepareToEnterSendLoop(); + + /** + * Hook to define custom periodic behavior to be performed at the top of every iteration of the poll-convert-send loop. + */ + protected abstract void beginSendIteration(); + + /** + * Hook to define custom periodic checks for health, metrics, etc. Called whenever {@link SourceTask#poll()} is about to be invoked. + */ + protected abstract void prepareToPollTask(); + + /** + * Invoked when a record provided by the task has been filtered out by a transform or the converter, + * or will be discarded due to failures during transformation or conversion. + * @param record the pre-transform record that has been dropped; never null. + */ + protected abstract void recordDropped(SourceRecord record); + + /** + * Invoked when a record is about to be dispatched to the producer. May be invoked multiple times for the same + * record if retriable errors are encountered. + * @param sourceRecord the pre-transform {@link SourceRecord} provided by the source task; never null. + * @param producerRecord the {@link ProducerRecord} produced by transforming and converting the + * {@code sourceRecord}; never null; + * @return a {@link SubmittedRecords.SubmittedRecord} to be {@link SubmittedRecords.SubmittedRecord#ack() acknowledged} + * if the corresponding producer record is ack'd by Kafka or {@link SubmittedRecords.SubmittedRecord#drop() dropped} + * if synchronously rejected by the producer. Can also be {@link Optional#empty()} if it is not necessary to track the acknowledgment + * of individual producer records + */ + protected abstract Optional prepareToSendRecord( + SourceRecord sourceRecord, + ProducerRecord producerRecord + ); + + /** + * Invoked when a record has been transformed, converted, and dispatched to the producer successfully via + * {@link Producer#send}. Does not guarantee that the record has been sent to Kafka or ack'd by the required number + * of brokers, but does guarantee that it will never be re-processed. + * @param record the pre-transform {@link SourceRecord} that was successfully dispatched to the producer; never null. + */ + protected abstract void recordDispatched(SourceRecord record); + + /** + * Invoked when an entire batch of records returned from {@link SourceTask#poll} has been transformed, converted, + * and either discarded due to transform/conversion errors, filtered by a transform, or dispatched to the producer + * successfully via {@link Producer#send}. Does not guarantee that the records have been sent to Kafka or ack'd by the + * required number of brokers, but does guarantee that none of the records in the batch will ever be re-processed during + * the lifetime of this task. At most one record batch is polled from the task in between calls to this method. + */ + protected abstract void batchDispatched(); + + /** + * Invoked when a record has been sent and ack'd by the Kafka cluster. Note that this method may be invoked + * concurrently and should therefore be made thread-safe. + * @param sourceRecord the pre-transform {@link SourceRecord} that was successfully sent to Kafka; never null. + * @param producerRecord the {@link ProducerRecord} produced by transforming and converting the + * {@code sourceRecord}; never null; + * @param recordMetadata the {@link RecordMetadata} for the corresponding producer record; never null. + */ + protected abstract void recordSent( + SourceRecord sourceRecord, + ProducerRecord producerRecord, + RecordMetadata recordMetadata + ); + + /** + * Invoked when a record given to {@link Producer#send(ProducerRecord, Callback)} has failed with a non-retriable error. + * @param synchronous whether the error occurred during the invocation of {@link Producer#send(ProducerRecord, Callback)}. + * If {@code false}, indicates that the error was reported asynchronously by the producer by a {@link Callback} + * @param producerRecord the {@link ProducerRecord} that the producer failed to send; never null + * @param preTransformRecord the pre-transform {@link SourceRecord} that the producer record was derived from; never null + * @param e the exception that was either thrown from {@link Producer#send(ProducerRecord, Callback)}, or reported by the producer + * via {@link Callback} after the call to {@link Producer#send(ProducerRecord, Callback)} completed + */ + protected abstract void producerSendFailed( + boolean synchronous, + ProducerRecord producerRecord, + SourceRecord preTransformRecord, + Exception e + ); + + /** + * Invoked when no more records will be polled from the task or dispatched to the producer. Should attempt to + * commit the offsets for any outstanding records when possible. + * @param failed whether the task is undergoing a healthy or an unhealthy shutdown + */ + protected abstract void finalOffsetCommit(boolean failed); + + + protected final WorkerConfig workerConfig; + protected final WorkerSourceTaskContext sourceTaskContext; + protected final OffsetStorageWriter offsetWriter; + protected final Producer producer; + + private final SourceTask task; + private final Converter keyConverter; + private final Converter valueConverter; + private final HeaderConverter headerConverter; + private final TransformationChain transformationChain; + private final TopicAdmin admin; + private final CloseableOffsetStorageReader offsetReader; + private final ConnectorOffsetBackingStore offsetStore; + private final SourceTaskMetricsGroup sourceTaskMetricsGroup; + private final CountDownLatch stopRequestedLatch; + private final boolean topicTrackingEnabled; + private final TopicCreation topicCreation; + private final Executor closeExecutor; + + // Visible for testing + List toSend; + protected Map taskConfig; + protected boolean started = false; + + protected AbstractWorkerSourceTask(ConnectorTaskId id, + SourceTask task, + TaskStatus.Listener statusListener, + TargetState initialState, + Converter keyConverter, + Converter valueConverter, + HeaderConverter headerConverter, + TransformationChain transformationChain, + WorkerSourceTaskContext sourceTaskContext, + Producer producer, + TopicAdmin admin, + Map topicGroups, + CloseableOffsetStorageReader offsetReader, + OffsetStorageWriter offsetWriter, + ConnectorOffsetBackingStore offsetStore, + WorkerConfig workerConfig, + ConnectMetrics connectMetrics, + ClassLoader loader, + Time time, + RetryWithToleranceOperator retryWithToleranceOperator, + StatusBackingStore statusBackingStore, + Executor closeExecutor) { + + super(id, statusListener, initialState, loader, connectMetrics, + retryWithToleranceOperator, time, statusBackingStore); + + this.workerConfig = workerConfig; + this.task = task; + this.keyConverter = keyConverter; + this.valueConverter = valueConverter; + this.headerConverter = headerConverter; + this.transformationChain = transformationChain; + this.producer = producer; + this.admin = admin; + this.offsetReader = offsetReader; + this.offsetWriter = offsetWriter; + this.offsetStore = offsetStore; + this.closeExecutor = closeExecutor; + this.sourceTaskContext = sourceTaskContext; + + this.stopRequestedLatch = new CountDownLatch(1); + this.sourceTaskMetricsGroup = new SourceTaskMetricsGroup(id, connectMetrics); + this.topicTrackingEnabled = workerConfig.getBoolean(TOPIC_TRACKING_ENABLE_CONFIG); + this.topicCreation = TopicCreation.newTopicCreation(workerConfig, topicGroups); + } + + @Override + public void initialize(TaskConfig taskConfig) { + try { + this.taskConfig = taskConfig.originalsStrings(); + } catch (Throwable t) { + log.error("{} Task failed initialization and will not be started.", this, t); + onFailure(t); + } + } + + @Override + protected void initializeAndStart() { + prepareToInitializeTask(); + // If we try to start the task at all by invoking initialize, then count this as + // "started" and expect a subsequent call to the task's stop() method + // to properly clean up any resources allocated by its initialize() or + // start() methods. If the task throws an exception during stop(), + // the worst thing that happens is another exception gets logged for an already- + // failed task + started = true; + task.initialize(sourceTaskContext); + task.start(taskConfig); + log.info("{} Source task finished initialization and start", this); + } + + @Override + public void cancel() { + super.cancel(); + // Preemptively close the offset reader in case the task is blocked on an offset read. + offsetReader.close(); + // We proactively close the producer here as the main work thread for the task may + // be blocked indefinitely in a call to Producer::send if automatic topic creation is + // not enabled on either the connector or the Kafka cluster. Closing the producer should + // unblock it in that case and allow shutdown to proceed normally. + // With a duration of 0, the producer's own shutdown logic should be fairly quick, + // but closing user-pluggable classes like interceptors may lag indefinitely. So, we + // call close on a separate thread in order to avoid blocking the herder's tick thread. + closeExecutor.execute(() -> closeProducer(Duration.ZERO)); + } + + @Override + public void stop() { + super.stop(); + stopRequestedLatch.countDown(); + } + + @Override + public void removeMetrics() { + Utils.closeQuietly(sourceTaskMetricsGroup::close, "source task metrics tracker"); + super.removeMetrics(); + } + + @Override + protected void close() { + if (started) { + Utils.closeQuietly(task::stop, "source task"); + } + + closeProducer(Duration.ofSeconds(30)); + + if (admin != null) { + Utils.closeQuietly(() -> admin.close(Duration.ofSeconds(30)), "source task admin"); + } + Utils.closeQuietly(transformationChain, "transformation chain"); + Utils.closeQuietly(retryWithToleranceOperator, "retry operator"); + Utils.closeQuietly(offsetReader, "offset reader"); + Utils.closeQuietly(offsetStore::stop, "offset backing store"); + } + + private void closeProducer(Duration duration) { + if (producer != null) { + Utils.closeQuietly(() -> producer.close(duration), "source task producer"); + } + } + + @Override + public void execute() { + try { + prepareToEnterSendLoop(); + while (!isStopping()) { + beginSendIteration(); + + if (shouldPause()) { + onPause(); + if (awaitUnpause()) { + onResume(); + prepareToEnterSendLoop(); + } + continue; + } + + if (toSend == null) { + prepareToPollTask(); + + log.trace("{} Nothing to send to Kafka. Polling source for additional records", this); + long start = time.milliseconds(); + toSend = poll(); + if (toSend != null) { + recordPollReturned(toSend.size(), time.milliseconds() - start); + } + } + if (toSend == null) + continue; + log.trace("{} About to send {} records to Kafka", this, toSend.size()); + if (sendRecords()) { + batchDispatched(); + } else { + stopRequestedLatch.await(SEND_FAILED_BACKOFF_MS, TimeUnit.MILLISECONDS); + } + } + } catch (InterruptedException e) { + // Ignore and allow to exit. + } catch (RuntimeException e) { + try { + finalOffsetCommit(true); + } catch (Exception offsetException) { + log.error("Failed to commit offsets for already-failing task", offsetException); + } + throw e; + } + finalOffsetCommit(false); + } + + /** + * Try to send a batch of records. If a send fails and is retriable, this saves the remainder of the batch so it can + * be retried after backing off. If a send fails and is not retriable, this will throw a ConnectException. + * @return true if all messages were sent, false if some need to be retried + */ + // Visible for testing + boolean sendRecords() { + int processed = 0; + recordBatch(toSend.size()); + final SourceRecordWriteCounter counter = + toSend.size() > 0 ? new SourceRecordWriteCounter(toSend.size(), sourceTaskMetricsGroup) : null; + for (final SourceRecord preTransformRecord : toSend) { + retryWithToleranceOperator.sourceRecord(preTransformRecord); + final SourceRecord record = transformationChain.apply(preTransformRecord); + final ProducerRecord producerRecord = convertTransformedRecord(record); + if (producerRecord == null || retryWithToleranceOperator.failed()) { + counter.skipRecord(); + recordDropped(preTransformRecord); + continue; + } + + log.trace("{} Appending record to the topic {} with key {}, value {}", this, record.topic(), record.key(), record.value()); + Optional submittedRecord = prepareToSendRecord(preTransformRecord, producerRecord); + try { + final String topic = producerRecord.topic(); + maybeCreateTopic(topic); + producer.send( + producerRecord, + (recordMetadata, e) -> { + if (e != null) { + log.debug("{} failed to send record to {}: ", AbstractWorkerSourceTask.this, topic, e); + log.trace("{} Failed record: {}", AbstractWorkerSourceTask.this, preTransformRecord); + producerSendFailed(false, producerRecord, preTransformRecord, e); + } else { + counter.completeRecord(); + log.trace("{} Wrote record successfully: topic {} partition {} offset {}", + AbstractWorkerSourceTask.this, + recordMetadata.topic(), recordMetadata.partition(), + recordMetadata.offset()); + recordSent(preTransformRecord, producerRecord, recordMetadata); + submittedRecord.ifPresent(SubmittedRecords.SubmittedRecord::ack); + if (topicTrackingEnabled) { + recordActiveTopic(producerRecord.topic()); + } + } + }); + // Note that this will cause retries to take place within a transaction + } catch (RetriableException | org.apache.kafka.common.errors.RetriableException e) { + log.warn("{} Failed to send record to topic '{}' and partition '{}'. Backing off before retrying: ", + this, producerRecord.topic(), producerRecord.partition(), e); + toSend = toSend.subList(processed, toSend.size()); + submittedRecord.ifPresent(SubmittedRecords.SubmittedRecord::drop); + counter.retryRemaining(); + return false; + } catch (ConnectException e) { + log.warn("{} Failed to send record to topic '{}' and partition '{}' due to an unrecoverable exception: ", + this, producerRecord.topic(), producerRecord.partition(), e); + log.trace("{} Failed to send {} with unrecoverable exception: ", this, producerRecord, e); + throw e; + } catch (KafkaException e) { + producerSendFailed(true, producerRecord, preTransformRecord, e); + } + processed++; + recordDispatched(preTransformRecord); + } + toSend = null; + return true; + } + + protected List poll() throws InterruptedException { + try { + return task.poll(); + } catch (RetriableException | org.apache.kafka.common.errors.RetriableException e) { + log.warn("{} failed to poll records from SourceTask. Will retry operation.", this, e); + // Do nothing. Let the framework poll whenever it's ready. + return null; + } + } + + /** + * Convert the source record into a producer record. + * + * @param record the transformed record + * @return the producer record which can sent over to Kafka. A null is returned if the input is null or + * if an error was encountered during any of the converter stages. + */ + protected ProducerRecord convertTransformedRecord(SourceRecord record) { + if (record == null) { + return null; + } + + RecordHeaders headers = retryWithToleranceOperator.execute(() -> convertHeaderFor(record), Stage.HEADER_CONVERTER, headerConverter.getClass()); + + byte[] key = retryWithToleranceOperator.execute(() -> keyConverter.fromConnectData(record.topic(), headers, record.keySchema(), record.key()), + Stage.KEY_CONVERTER, keyConverter.getClass()); + + byte[] value = retryWithToleranceOperator.execute(() -> valueConverter.fromConnectData(record.topic(), headers, record.valueSchema(), record.value()), + Stage.VALUE_CONVERTER, valueConverter.getClass()); + + if (retryWithToleranceOperator.failed()) { + return null; + } + + return new ProducerRecord<>(record.topic(), record.kafkaPartition(), + ConnectUtils.checkAndConvertTimestamp(record.timestamp()), key, value, headers); + } + + // Due to transformations that may change the destination topic of a record (such as + // RegexRouter) topic creation can not be batched for multiple topics + private void maybeCreateTopic(String topic) { + if (!topicCreation.isTopicCreationRequired(topic)) { + log.trace("Topic creation by the connector is disabled or the topic {} was previously created." + + "If auto.create.topics.enable is enabled on the broker, " + + "the topic will be created with default settings", topic); + return; + } + log.info("The task will send records to topic '{}' for the first time. Checking " + + "whether topic exists", topic); + Map existing = admin.describeTopics(topic); + if (!existing.isEmpty()) { + log.info("Topic '{}' already exists.", topic); + topicCreation.addTopic(topic); + return; + } + + log.info("Creating topic '{}'", topic); + TopicCreationGroup topicGroup = topicCreation.findFirstGroup(topic); + log.debug("Topic '{}' matched topic creation group: {}", topic, topicGroup); + NewTopic newTopic = topicGroup.newTopic(topic); + + TopicAdmin.TopicCreationResponse response = admin.createOrFindTopics(newTopic); + if (response.isCreated(newTopic.name())) { + topicCreation.addTopic(topic); + log.info("Created topic '{}' using creation group {}", newTopic, topicGroup); + } else if (response.isExisting(newTopic.name())) { + topicCreation.addTopic(topic); + log.info("Found existing topic '{}'", newTopic); + } else { + // The topic still does not exist and could not be created, so treat it as a task failure + log.warn("Request to create new topic '{}' failed", topic); + throw new ConnectException("Task failed to create new topic " + newTopic + ". Ensure " + + "that the task is authorized to create topics or that the topic exists and " + + "restart the task"); + } + } + + protected RecordHeaders convertHeaderFor(SourceRecord record) { + Headers headers = record.headers(); + RecordHeaders result = new RecordHeaders(); + if (headers != null) { + String topic = record.topic(); + for (Header header : headers) { + String key = header.key(); + byte[] rawHeader = headerConverter.fromConnectHeader(topic, key, header.schema(), header.value()); + result.add(key, rawHeader); + } + } + return result; + } + + protected void commitTaskRecord(SourceRecord record, RecordMetadata metadata) { + try { + task.commitRecord(record, metadata); + } catch (Throwable t) { + log.error("{} Exception thrown while calling task.commitRecord()", this, t); + } + } + + protected void commitSourceTask() { + try { + this.task.commit(); + } catch (Throwable t) { + log.error("{} Exception thrown while calling task.commit()", this, t); + } + } + + protected void recordPollReturned(int numRecordsInBatch, long duration) { + sourceTaskMetricsGroup.recordPoll(numRecordsInBatch, duration); + } + + SourceTaskMetricsGroup sourceTaskMetricsGroup() { + return sourceTaskMetricsGroup; + } + + static class SourceRecordWriteCounter { + private final SourceTaskMetricsGroup metricsGroup; + private final int batchSize; + private boolean completed = false; + private int counter; + public SourceRecordWriteCounter(int batchSize, SourceTaskMetricsGroup metricsGroup) { + assert batchSize > 0; + assert metricsGroup != null; + this.batchSize = batchSize; + counter = batchSize; + this.metricsGroup = metricsGroup; + } + public void skipRecord() { + if (counter > 0 && --counter == 0) { + finishedAllWrites(); + } + } + public void completeRecord() { + if (counter > 0 && --counter == 0) { + finishedAllWrites(); + } + } + public void retryRemaining() { + finishedAllWrites(); + } + private void finishedAllWrites() { + if (!completed) { + metricsGroup.recordWrite(batchSize - counter); + completed = true; + } + } + } + + static class SourceTaskMetricsGroup { + private final ConnectMetrics.MetricGroup metricGroup; + private final Sensor sourceRecordPoll; + private final Sensor sourceRecordWrite; + private final Sensor sourceRecordActiveCount; + private final Sensor pollTime; + private int activeRecordCount; + + public SourceTaskMetricsGroup(ConnectorTaskId id, ConnectMetrics connectMetrics) { + ConnectMetricsRegistry registry = connectMetrics.registry(); + metricGroup = connectMetrics.group(registry.sourceTaskGroupName(), + registry.connectorTagName(), id.connector(), + registry.taskTagName(), Integer.toString(id.task())); + // remove any previously created metrics in this group to prevent collisions. + metricGroup.close(); + + sourceRecordPoll = metricGroup.sensor("source-record-poll"); + sourceRecordPoll.add(metricGroup.metricName(registry.sourceRecordPollRate), new Rate()); + sourceRecordPoll.add(metricGroup.metricName(registry.sourceRecordPollTotal), new CumulativeSum()); + + sourceRecordWrite = metricGroup.sensor("source-record-write"); + sourceRecordWrite.add(metricGroup.metricName(registry.sourceRecordWriteRate), new Rate()); + sourceRecordWrite.add(metricGroup.metricName(registry.sourceRecordWriteTotal), new CumulativeSum()); + + pollTime = metricGroup.sensor("poll-batch-time"); + pollTime.add(metricGroup.metricName(registry.sourceRecordPollBatchTimeMax), new Max()); + pollTime.add(metricGroup.metricName(registry.sourceRecordPollBatchTimeAvg), new Avg()); + + sourceRecordActiveCount = metricGroup.sensor("source-record-active-count"); + sourceRecordActiveCount.add(metricGroup.metricName(registry.sourceRecordActiveCount), new Value()); + sourceRecordActiveCount.add(metricGroup.metricName(registry.sourceRecordActiveCountMax), new Max()); + sourceRecordActiveCount.add(metricGroup.metricName(registry.sourceRecordActiveCountAvg), new Avg()); + } + + void close() { + metricGroup.close(); + } + + void recordPoll(int batchSize, long duration) { + sourceRecordPoll.record(batchSize); + pollTime.record(duration); + activeRecordCount += batchSize; + sourceRecordActiveCount.record(activeRecordCount); + } + + void recordWrite(int recordCount) { + sourceRecordWrite.record(recordCount); + activeRecordCount -= recordCount; + activeRecordCount = Math.max(0, activeRecordCount); + sourceRecordActiveCount.record(activeRecordCount); + } + + protected ConnectMetrics.MetricGroup metricGroup() { + return metricGroup; + } + } +} diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ConnectMetrics.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ConnectMetrics.java index 7dad6aec0af1..ed81be657a02 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ConnectMetrics.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ConnectMetrics.java @@ -319,9 +319,7 @@ Map tags() { */ public void addValueMetric(MetricNameTemplate nameTemplate, final LiteralSupplier supplier) { MetricName metricName = metricName(nameTemplate); - if (metrics().metric(metricName) == null) { - metrics().addMetric(metricName, (Gauge) (config, now) -> supplier.metricValue(now)); - } + metrics().addMetricIfAbsent(metricName, null, (Gauge) (config, now) -> supplier.metricValue(now)); } /** @@ -333,9 +331,7 @@ public void addValueMetric(MetricNameTemplate nameTemplate, final LiteralSup */ public void addImmutableValueMetric(MetricNameTemplate nameTemplate, final T value) { MetricName metricName = metricName(nameTemplate); - if (metrics().metric(metricName) == null) { - metrics().addMetric(metricName, (Gauge) (config, now) -> value); - } + metrics().addMetricIfAbsent(metricName, null, (Gauge) (config, now) -> value); } /** diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ExactlyOnceWorkerSourceTask.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ExactlyOnceWorkerSourceTask.java new file mode 100644 index 000000000000..faf0e8cbd867 --- /dev/null +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ExactlyOnceWorkerSourceTask.java @@ -0,0 +1,522 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.connect.runtime; + +import org.apache.kafka.clients.producer.Producer; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.clients.producer.RecordMetadata; +import org.apache.kafka.common.errors.InvalidProducerEpochException; +import org.apache.kafka.common.metrics.Sensor; +import org.apache.kafka.common.metrics.stats.Avg; +import org.apache.kafka.common.metrics.stats.Max; +import org.apache.kafka.common.metrics.stats.Min; +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.connect.errors.ConnectException; +import org.apache.kafka.connect.runtime.errors.RetryWithToleranceOperator; +import org.apache.kafka.connect.source.SourceRecord; +import org.apache.kafka.connect.source.SourceTask; +import org.apache.kafka.connect.source.SourceTask.TransactionBoundary; +import org.apache.kafka.connect.storage.CloseableOffsetStorageReader; +import org.apache.kafka.connect.storage.ClusterConfigState; +import org.apache.kafka.connect.storage.ConnectorOffsetBackingStore; +import org.apache.kafka.connect.storage.Converter; +import org.apache.kafka.connect.storage.HeaderConverter; +import org.apache.kafka.connect.storage.OffsetStorageWriter; +import org.apache.kafka.connect.storage.StatusBackingStore; +import org.apache.kafka.connect.util.ConnectorTaskId; +import org.apache.kafka.connect.util.LoggingContext; +import org.apache.kafka.connect.util.TopicAdmin; +import org.apache.kafka.connect.util.TopicCreationGroup; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.LinkedHashMap; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; +import java.util.concurrent.Executor; +import java.util.concurrent.atomic.AtomicReference; + + +/** + * WorkerTask that uses a SourceTask to ingest data into Kafka, with support for exactly-once delivery guarantees. + */ +class ExactlyOnceWorkerSourceTask extends AbstractWorkerSourceTask { + private static final Logger log = LoggerFactory.getLogger(ExactlyOnceWorkerSourceTask.class); + + private boolean transactionOpen; + private final LinkedHashMap commitableRecords; + + private final TransactionBoundaryManager transactionBoundaryManager; + private final TransactionMetricsGroup transactionMetrics; + + private final ConnectorOffsetBackingStore offsetBackingStore; + private final Runnable preProducerCheck; + private final Runnable postProducerCheck; + + public ExactlyOnceWorkerSourceTask(ConnectorTaskId id, + SourceTask task, + TaskStatus.Listener statusListener, + TargetState initialState, + Converter keyConverter, + Converter valueConverter, + HeaderConverter headerConverter, + TransformationChain transformationChain, + Producer producer, + TopicAdmin admin, + Map topicGroups, + CloseableOffsetStorageReader offsetReader, + OffsetStorageWriter offsetWriter, + ConnectorOffsetBackingStore offsetBackingStore, + WorkerConfig workerConfig, + ClusterConfigState configState, + ConnectMetrics connectMetrics, + ClassLoader loader, + Time time, + RetryWithToleranceOperator retryWithToleranceOperator, + StatusBackingStore statusBackingStore, + SourceConnectorConfig sourceConfig, + Executor closeExecutor, + Runnable preProducerCheck, + Runnable postProducerCheck) { + super(id, task, statusListener, initialState, keyConverter, valueConverter, headerConverter, transformationChain, + new WorkerSourceTaskContext(offsetReader, id, configState, buildTransactionContext(sourceConfig)), + producer, admin, topicGroups, offsetReader, offsetWriter, offsetBackingStore, workerConfig, connectMetrics, + loader, time, retryWithToleranceOperator, statusBackingStore, closeExecutor); + + this.transactionOpen = false; + this.commitableRecords = new LinkedHashMap<>(); + this.offsetBackingStore = offsetBackingStore; + + this.preProducerCheck = preProducerCheck; + this.postProducerCheck = postProducerCheck; + + this.transactionBoundaryManager = buildTransactionManager(workerConfig, sourceConfig, sourceTaskContext.transactionContext()); + this.transactionMetrics = new TransactionMetricsGroup(id, connectMetrics); + } + + private static WorkerTransactionContext buildTransactionContext(SourceConnectorConfig sourceConfig) { + return TransactionBoundary.CONNECTOR.equals(sourceConfig.transactionBoundary()) + ? new WorkerTransactionContext() + : null; + } + + @Override + protected void prepareToInitializeTask() { + preProducerCheck.run(); + + // Try not to start up the offset store (which has its own producer and consumer) if we've already been shut down at this point + if (isStopping()) + return; + offsetBackingStore.start(); + + // Try not to initialize the transactional producer (which may accidentally fence out other, later task generations) if we've already + // been shut down at this point + if (isStopping()) + return; + producer.initTransactions(); + + postProducerCheck.run(); + } + + @Override + protected void prepareToEnterSendLoop() { + transactionBoundaryManager.initialize(); + } + + @Override + protected void beginSendIteration() { + // No-op + } + + @Override + protected void prepareToPollTask() { + // No-op + } + + @Override + protected void recordDropped(SourceRecord record) { + synchronized (commitableRecords) { + commitableRecords.put(record, null); + } + transactionBoundaryManager.maybeCommitTransactionForRecord(record); + } + + @Override + protected Optional prepareToSendRecord( + SourceRecord sourceRecord, + ProducerRecord producerRecord + ) { + if (offsetBackingStore.primaryOffsetsTopic().equals(producerRecord.topic())) { + // This is to prevent deadlock that occurs when: + // 1. A task provides a record whose topic is the task's offsets topic + // 2. That record is dispatched to the task's producer in a transaction that remains open + // at least until the worker polls the task again + // 3. In the subsequent call to SourceTask::poll, the task requests offsets from the worker + // (which requires a read to the end of the offsets topic, and will block until any open + // transactions on the topic are either committed or aborted) + throw new ConnectException("Source tasks may not produce to their own offsets topics when exactly-once support is enabled"); + } + maybeBeginTransaction(); + return Optional.empty(); + } + + @Override + protected void recordDispatched(SourceRecord record) { + // Offsets are converted & serialized in the OffsetWriter + // Important: we only save offsets for the record after it has been accepted by the producer; this way, + // we commit those offsets if and only if the record is sent successfully. + offsetWriter.offset(record.sourcePartition(), record.sourceOffset()); + transactionMetrics.addRecord(); + transactionBoundaryManager.maybeCommitTransactionForRecord(record); + } + + @Override + protected void batchDispatched() { + transactionBoundaryManager.maybeCommitTransactionForBatch(); + } + + @Override + protected void recordSent( + SourceRecord sourceRecord, + ProducerRecord producerRecord, + RecordMetadata recordMetadata + ) { + synchronized (commitableRecords) { + commitableRecords.put(sourceRecord, recordMetadata); + } + } + + @Override + protected void producerSendFailed( + boolean synchronous, + ProducerRecord producerRecord, + SourceRecord preTransformRecord, + Exception e + ) { + if (synchronous) { + throw maybeWrapProducerSendException( + "Unrecoverable exception trying to send", + e + ); + } else { + // No-op; all asynchronously-reported producer exceptions should be bubbled up again by Producer::commitTransaction + } + } + + @Override + protected void finalOffsetCommit(boolean failed) { + if (failed) { + log.debug("Skipping final offset commit as task has failed"); + return; + } + + // It should be safe to commit here even if we were in the middle of retrying on RetriableExceptions in the + // send loop since we only track source offsets for records that have been successfully dispatched to the + // producer. + // Any records that we were retrying on (and any records after them in the batch) won't be included in the + // transaction and their offsets won't be committed, but (unless the user has requested connector-defined + // transaction boundaries), it's better to commit some data than none. + transactionBoundaryManager.maybeCommitFinalTransaction(); + } + + @Override + protected void onPause() { + super.onPause(); + // Commit the transaction now so that we don't end up with a hanging transaction, or worse, get fenced out + // and fail the task once unpaused + transactionBoundaryManager.maybeCommitFinalTransaction(); + } + + private void maybeBeginTransaction() { + if (!transactionOpen) { + producer.beginTransaction(); + transactionOpen = true; + } + } + + private void commitTransaction() { + log.debug("{} Committing offsets", this); + + long started = time.milliseconds(); + + // We might have just aborted a transaction, in which case we'll have to begin a new one + // in order to commit offsets + maybeBeginTransaction(); + + AtomicReference flushError = new AtomicReference<>(); + if (offsetWriter.beginFlush()) { + // Now we can actually write the offsets to the internal topic. + // No need to track the flush future here since it's guaranteed to complete by the time + // Producer::commitTransaction completes + // We do have to track failures for that callback though, since they may originate from outside + // the producer (i.e., the offset writer or the backing offset store), and would not cause + // Producer::commitTransaction to fail + offsetWriter.doFlush((error, result) -> { + if (error != null) { + log.error("{} Failed to flush offsets to storage: ", ExactlyOnceWorkerSourceTask.this, error); + flushError.compareAndSet(null, error); + } else { + log.trace("{} Finished flushing offsets to storage", ExactlyOnceWorkerSourceTask.this); + } + }); + } + + // Commit the transaction + // Blocks until all outstanding records have been sent and ack'd + try { + producer.commitTransaction(); + } catch (Throwable t) { + log.error("{} Failed to commit producer transaction", ExactlyOnceWorkerSourceTask.this, t); + flushError.compareAndSet(null, t); + } + + transactionOpen = false; + + Throwable error = flushError.get(); + if (error != null) { + recordCommitFailure(time.milliseconds() - started, null); + offsetWriter.cancelFlush(); + throw maybeWrapProducerSendException( + "Failed to flush offsets and/or records for task " + id, + error + ); + } + + transactionMetrics.commitTransaction(); + + long durationMillis = time.milliseconds() - started; + recordCommitSuccess(durationMillis); + log.debug("{} Finished commitOffsets successfully in {} ms", this, durationMillis); + + // Synchronize in order to guarantee that writes on other threads are picked up by this one + synchronized (commitableRecords) { + commitableRecords.forEach(this::commitTaskRecord); + commitableRecords.clear(); + } + commitSourceTask(); + } + + private RuntimeException maybeWrapProducerSendException(String message, Throwable error) { + if (isPossibleTransactionTimeoutError(error)) { + return wrapTransactionTimeoutError(error); + } else { + return new ConnectException(message, error); + } + } + + private static boolean isPossibleTransactionTimeoutError(Throwable error) { + return error instanceof InvalidProducerEpochException + || error.getCause() instanceof InvalidProducerEpochException; + } + + private ConnectException wrapTransactionTimeoutError(Throwable error) { + return new ConnectException( + "The task " + id + " was unable to finish writing records to Kafka before its producer transaction expired. " + + "It may be necessary to reconfigure this connector in order for it to run healthily with exactly-once support. " + + "Options for this include: tune the connector's producer configuration for higher throughput, " + + "increase the transaction timeout for the connector's producers, " + + "decrease the offset commit interval (if using interval-based transaction boundaries), " + + "or use the 'poll' transaction boundary (if the connector is not already configured to use it).", + error + ); + } + + @Override + public String toString() { + return "ExactlyOnceWorkerSourceTask{" + + "id=" + id + + '}'; + } + + private abstract class TransactionBoundaryManager { + protected boolean shouldCommitTransactionForRecord(SourceRecord record) { + return false; + } + + protected boolean shouldCommitTransactionForBatch(long currentTimeMs) { + return false; + } + + protected boolean shouldCommitFinalTransaction() { + return false; + } + + /** + * Hook to signal that a new transaction cycle has been started. May be invoked + * multiple times if the task is paused and then resumed. It can be assumed that + * a new transaction is created at least every time an existing transaction is + * committed; this is just a hook to notify that a new transaction may have been + * created outside of that flow as well. + */ + protected void initialize() { + } + + public void maybeCommitTransactionForRecord(SourceRecord record) { + maybeCommitTransaction(shouldCommitTransactionForRecord(record)); + } + + public void maybeCommitTransactionForBatch() { + maybeCommitTransaction(shouldCommitTransactionForBatch(time.milliseconds())); + } + + public void maybeCommitFinalTransaction() { + maybeCommitTransaction(shouldCommitFinalTransaction()); + } + + private void maybeCommitTransaction(boolean shouldCommit) { + if (shouldCommit && (transactionOpen || offsetWriter.willFlush())) { + try (LoggingContext loggingContext = LoggingContext.forOffsets(id)) { + commitTransaction(); + } + } + } + } + + private TransactionBoundaryManager buildTransactionManager( + WorkerConfig workerConfig, + SourceConnectorConfig sourceConfig, + WorkerTransactionContext transactionContext) { + TransactionBoundary boundary = sourceConfig.transactionBoundary(); + switch (boundary) { + case POLL: + return new TransactionBoundaryManager() { + @Override + protected boolean shouldCommitTransactionForBatch(long currentTimeMs) { + return true; + } + + @Override + protected boolean shouldCommitFinalTransaction() { + return true; + } + }; + + case INTERVAL: + long transactionBoundaryInterval = Optional.ofNullable(sourceConfig.transactionBoundaryInterval()) + .orElse(workerConfig.offsetCommitInterval()); + return new TransactionBoundaryManager() { + private final long commitInterval = transactionBoundaryInterval; + private long lastCommit; + + @Override + public void initialize() { + this.lastCommit = time.milliseconds(); + } + + @Override + protected boolean shouldCommitTransactionForBatch(long currentTimeMs) { + if (time.milliseconds() >= lastCommit + commitInterval) { + lastCommit = time.milliseconds(); + return true; + } else { + return false; + } + } + + @Override + protected boolean shouldCommitFinalTransaction() { + return true; + } + }; + + case CONNECTOR: + Objects.requireNonNull(transactionContext, "Transaction context must be provided when using connector-defined transaction boundaries"); + return new TransactionBoundaryManager() { + @Override + protected boolean shouldCommitFinalTransaction() { + return shouldCommitTransactionForBatch(time.milliseconds()); + } + + @Override + protected boolean shouldCommitTransactionForBatch(long currentTimeMs) { + if (transactionContext.shouldAbortBatch()) { + log.info("Aborting transaction for batch as requested by connector"); + abortTransaction(); + // We abort the transaction, which causes all the records up to this point to be dropped, but we still want to + // commit offsets so that the task doesn't see the same records all over again + return true; + } + return transactionContext.shouldCommitBatch(); + } + + @Override + protected boolean shouldCommitTransactionForRecord(SourceRecord record) { + if (transactionContext.shouldAbortOn(record)) { + log.info("Aborting transaction for record on topic {} as requested by connector", record.topic()); + log.trace("Last record in aborted transaction: {}", record); + abortTransaction(); + // We abort the transaction, which causes all the records up to this point to be dropped, but we still want to + // commit offsets so that the task doesn't see the same records all over again + return true; + } + return transactionContext.shouldCommitOn(record); + } + + private void abortTransaction() { + producer.abortTransaction(); + transactionMetrics.abortTransaction(); + transactionOpen = false; + } + }; + default: + throw new IllegalArgumentException("Unrecognized transaction boundary: " + boundary); + } + } + + TransactionMetricsGroup transactionMetricsGroup() { + return transactionMetrics; + } + + + static class TransactionMetricsGroup { + private final Sensor transactionSize; + private int size; + private final ConnectMetrics.MetricGroup metricGroup; + + public TransactionMetricsGroup(ConnectorTaskId id, ConnectMetrics connectMetrics) { + ConnectMetricsRegistry registry = connectMetrics.registry(); + metricGroup = connectMetrics.group(registry.sourceTaskGroupName(), + registry.connectorTagName(), id.connector(), + registry.taskTagName(), Integer.toString(id.task())); + + transactionSize = metricGroup.sensor("transaction-size"); + transactionSize.add(metricGroup.metricName(registry.transactionSizeAvg), new Avg()); + transactionSize.add(metricGroup.metricName(registry.transactionSizeMin), new Min()); + transactionSize.add(metricGroup.metricName(registry.transactionSizeMax), new Max()); + } + + void addRecord() { + size++; + } + + void abortTransaction() { + size = 0; + } + + void commitTransaction() { + transactionSize.record(size); + size = 0; + } + + protected ConnectMetrics.MetricGroup metricGroup() { + return metricGroup; + } + + } + +} diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/SubmittedRecords.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/SubmittedRecords.java index 6cdd2c1842b7..b77a6fa9841d 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/SubmittedRecords.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/SubmittedRecords.java @@ -35,7 +35,7 @@ * source offsets. Records are tracked in the order in which they are submitted, which should match the order they were * returned from {@link SourceTask#poll()}. The latest-eligible offsets for each source partition can be retrieved via * {@link #committableOffsets()}, where every record up to and including the record for each returned offset has been - * either {@link SubmittedRecord#ack() acknowledged} or {@link #removeLastOccurrence(SubmittedRecord) removed}. + * either {@link SubmittedRecord#ack() acknowledged} or {@link SubmittedRecord#drop dropped}. * Note that this class is not thread-safe, though a {@link SubmittedRecord} can be * {@link SubmittedRecord#ack() acknowledged} from a different thread. */ @@ -54,13 +54,13 @@ public SubmittedRecords() { /** * Enqueue a new source record before dispatching it to a producer. * The returned {@link SubmittedRecord} should either be {@link SubmittedRecord#ack() acknowledged} in the - * producer callback, or {@link #removeLastOccurrence(SubmittedRecord) removed} if the record could not be successfully + * producer callback, or {@link SubmittedRecord#drop() dropped} if the record could not be successfully * sent to the producer. - * + * * @param record the record about to be dispatched; may not be null but may have a null * {@link SourceRecord#sourcePartition()} and/or {@link SourceRecord#sourceOffset()} * @return a {@link SubmittedRecord} that can be either {@link SubmittedRecord#ack() acknowledged} once ack'd by - * the producer, or {@link #removeLastOccurrence removed} if synchronously rejected by the producer + * the producer, or {@link SubmittedRecord#drop() dropped} if synchronously rejected by the producer */ @SuppressWarnings("unchecked") public SubmittedRecord submit(SourceRecord record) { @@ -78,32 +78,6 @@ SubmittedRecord submit(Map partition, Map offset return result; } - /** - * Remove a source record and do not take it into account any longer when tracking offsets. - * Useful if the record has been synchronously rejected by the producer. - * If multiple instances of the same {@link SubmittedRecord} have been submitted already, only the first one found - * (traversing from the end of the deque backward) will be removed. - * @param record the {@link #submit previously-submitted} record to stop tracking; may not be null - * @return whether an instance of the record was removed - */ - public boolean removeLastOccurrence(SubmittedRecord record) { - Deque deque = records.get(record.partition()); - if (deque == null) { - log.warn("Attempted to remove record from submitted queue for partition {}, but no records with that partition appear to have been submitted", record.partition()); - return false; - } - boolean result = deque.removeLastOccurrence(record); - if (deque.isEmpty()) { - records.remove(record.partition()); - } - if (result) { - messageAcked(); - } else { - log.warn("Attempted to remove record from submitted queue for partition {}, but the record has not been submitted or has already been removed", record.partition()); - } - return result; - } - /** * Clear out any acknowledged records at the head of the deques and return a {@link CommittableOffsets snapshot} of the offsets and offset metadata * accrued between the last time this method was invoked and now. This snapshot can be {@link CommittableOffsets#updatedWith(CommittableOffsets) combined} @@ -187,7 +161,7 @@ private synchronized void messageAcked() { } } - class SubmittedRecord { + public class SubmittedRecord { private final Map partition; private final Map offset; private final AtomicBoolean acked; @@ -208,6 +182,34 @@ public void ack() { } } + /** + * Remove this record and do not take it into account any longer when tracking offsets. + * Useful if the record has been synchronously rejected by the producer. + * If multiple instances of this record have been submitted already, only the first one found + * (traversing from the end of the deque backward) will be removed. + *

+ * This is not safe to be called from a different thread + * than what called {@link SubmittedRecords#submit(SourceRecord)}. + * @return whether this instance was dropped + */ + public boolean drop() { + Deque deque = records.get(partition); + if (deque == null) { + log.warn("Attempted to remove record from submitted queue for partition {}, but no records with that partition appear to have been submitted", partition); + return false; + } + boolean result = deque.removeLastOccurrence(this); + if (deque.isEmpty()) { + records.remove(partition); + } + if (result) { + messageAcked(); + } else { + log.warn("Attempted to remove record from submitted queue for partition {}, but the record has not been submitted or has already been removed", partition); + } + return result; + } + private boolean acked() { return acked.get(); } diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java index 34258f82fb1c..d06a06a73a1f 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java @@ -24,6 +24,7 @@ import org.apache.kafka.clients.producer.KafkaProducer; import org.apache.kafka.clients.producer.ProducerConfig; import org.apache.kafka.common.KafkaFuture; +import org.apache.kafka.common.IsolationLevel; import org.apache.kafka.common.MetricNameTemplate; import org.apache.kafka.common.config.ConfigValue; import org.apache.kafka.common.config.provider.ConfigProvider; @@ -41,6 +42,7 @@ import org.apache.kafka.connect.runtime.isolation.LoaderSwap; import org.apache.kafka.connect.runtime.rest.resources.ConnectorsResource; import org.apache.kafka.connect.storage.ClusterConfigState; +import org.apache.kafka.connect.runtime.distributed.DistributedConfig; import org.apache.kafka.connect.runtime.errors.DeadLetterQueueReporter; import org.apache.kafka.connect.runtime.errors.ErrorHandlingMetrics; import org.apache.kafka.connect.runtime.errors.ErrorReporter; @@ -54,10 +56,10 @@ import org.apache.kafka.connect.source.SourceRecord; import org.apache.kafka.connect.source.SourceTask; import org.apache.kafka.connect.storage.CloseableOffsetStorageReader; +import org.apache.kafka.connect.storage.ConnectorOffsetBackingStore; import org.apache.kafka.connect.storage.Converter; import org.apache.kafka.connect.storage.HeaderConverter; import org.apache.kafka.connect.storage.OffsetBackingStore; -import org.apache.kafka.connect.storage.OffsetStorageReader; import org.apache.kafka.connect.storage.OffsetStorageReaderImpl; import org.apache.kafka.connect.storage.OffsetStorageWriter; import org.apache.kafka.connect.util.Callback; @@ -75,7 +77,10 @@ import java.util.Collections; import java.util.HashMap; import java.util.List; +import java.util.Locale; import java.util.Map; +import java.util.Objects; +import java.util.Optional; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; @@ -115,11 +120,11 @@ public class Worker { private final WorkerConfig config; private final Converter internalKeyConverter; private final Converter internalValueConverter; - private final OffsetBackingStore offsetBackingStore; + private final OffsetBackingStore globalOffsetBackingStore; private final ConcurrentMap connectors = new ConcurrentHashMap<>(); private final ConcurrentMap tasks = new ConcurrentHashMap<>(); - private SourceTaskOffsetCommitter sourceTaskOffsetCommitter; + private Optional sourceTaskOffsetCommitter; private final WorkerConfigTransformer workerConfigTransformer; private final ConnectorClientConfigOverridePolicy connectorClientConfigOverridePolicy; @@ -128,9 +133,9 @@ public Worker( Time time, Plugins plugins, WorkerConfig config, - OffsetBackingStore offsetBackingStore, + OffsetBackingStore globalOffsetBackingStore, ConnectorClientConfigOverridePolicy connectorClientConfigOverridePolicy) { - this(workerId, time, plugins, config, offsetBackingStore, Executors.newCachedThreadPool(), connectorClientConfigOverridePolicy); + this(workerId, time, plugins, config, globalOffsetBackingStore, Executors.newCachedThreadPool(), connectorClientConfigOverridePolicy); } Worker( @@ -138,7 +143,7 @@ public Worker( Time time, Plugins plugins, WorkerConfig config, - OffsetBackingStore offsetBackingStore, + OffsetBackingStore globalOffsetBackingStore, ExecutorService executorService, ConnectorClientConfigOverridePolicy connectorClientConfigOverridePolicy ) { @@ -156,8 +161,8 @@ public Worker( this.internalKeyConverter = plugins.newInternalConverter(true, JsonConverter.class.getName(), internalConverterConfig); this.internalValueConverter = plugins.newInternalConverter(false, JsonConverter.class.getName(), internalConverterConfig); - this.offsetBackingStore = offsetBackingStore; - this.offsetBackingStore.configure(config); + this.globalOffsetBackingStore = globalOffsetBackingStore; + this.globalOffsetBackingStore.configure(config); this.workerConfigTransformer = initConfigTransformer(); @@ -191,8 +196,11 @@ protected Herder herder() { public void start() { log.info("Worker starting"); - offsetBackingStore.start(); - sourceTaskOffsetCommitter = new SourceTaskOffsetCommitter(config); + globalOffsetBackingStore.start(); + + sourceTaskOffsetCommitter = config.exactlyOnceSourceEnabled() + ? Optional.empty() + : Optional.of(new SourceTaskOffsetCommitter(config)); connectorStatusMetricsGroup = new ConnectorStatusMetricsGroup(metrics, tasks, herder); @@ -219,9 +227,9 @@ public void stop() { } long timeoutMs = limit - time.milliseconds(); - sourceTaskOffsetCommitter.close(timeoutMs); + sourceTaskOffsetCommitter.ifPresent(committer -> committer.close(timeoutMs)); - offsetBackingStore.stop(); + globalOffsetBackingStore.stop(); metrics.stop(); log.info("Worker stopped"); @@ -285,14 +293,25 @@ public void startConnector( log.info("Creating connector {} of type {}", connName, connClass); final Connector connector = plugins.newConnector(connClass); - final ConnectorConfig connConfig = ConnectUtils.isSinkConnector(connector) - ? new SinkConnectorConfig(plugins, connProps) - : new SourceConnectorConfig(plugins, connProps, config.topicCreationEnable()); - - final OffsetStorageReader offsetReader = new OffsetStorageReaderImpl( - offsetBackingStore, connName, internalKeyConverter, internalValueConverter); + final ConnectorConfig connConfig; + final CloseableOffsetStorageReader offsetReader; + final ConnectorOffsetBackingStore offsetStore; + if (ConnectUtils.isSinkConnector(connector)) { + connConfig = new SinkConnectorConfig(plugins, connProps); + offsetReader = null; + offsetStore = null; + } else { + SourceConnectorConfig sourceConfig = new SourceConnectorConfig(plugins, connProps, config.topicCreationEnable()); + connConfig = sourceConfig; + + // Set up the offset backing store for this connector instance + // (This logic is implemented in a follow-up pull request: https://github.com/apache/kafka/pull/11781) + offsetStore = new ConnectorOffsetBackingStore(globalOffsetBackingStore, "TODO"); + offsetStore.configure(config); + offsetReader = new OffsetStorageReaderImpl(offsetStore, connName, internalKeyConverter, internalValueConverter); + } workerConnector = new WorkerConnector( - connName, connector, connConfig, ctx, metrics, connectorStatusListener, offsetReader, connectorLoader); + connName, connector, connConfig, ctx, metrics, connectorStatusListener, offsetReader, offsetStore, connectorLoader); log.info("Instantiated connector {} with version {} of type {}", connName, connector.version(), connector.getClass()); workerConnector.transitionTo(initialState, onConnectorStateChange); Plugins.compareAndSwapLoaders(savedLoader); @@ -497,22 +516,95 @@ public boolean isRunning(String connName) { } /** - * Start a task managed by this worker. + * Start a sink task managed by this worker. + * + * @param id the task ID. + * @param configState the most recent {@link ClusterConfigState} known to the worker + * @param connProps the connector properties. + * @param taskProps the tasks properties. + * @param statusListener a listener for the runtime status transitions of the task. + * @param initialState the initial state of the connector. + * @return true if the task started successfully. + */ + public boolean startSinkTask( + ConnectorTaskId id, + ClusterConfigState configState, + Map connProps, + Map taskProps, + TaskStatus.Listener statusListener, + TargetState initialState + ) { + return startTask(id, connProps, taskProps, statusListener, + new SinkTaskBuilder(id, configState, statusListener, initialState)); + } + + /** + * Start a source task managed by this worker using older behavior that does not provide exactly-once support. * * @param id the task ID. + * @param configState the most recent {@link ClusterConfigState} known to the worker * @param connProps the connector properties. * @param taskProps the tasks properties. * @param statusListener a listener for the runtime status transitions of the task. * @param initialState the initial state of the connector. * @return true if the task started successfully. */ - public boolean startTask( + public boolean startSourceTask( ConnectorTaskId id, ClusterConfigState configState, Map connProps, Map taskProps, TaskStatus.Listener statusListener, TargetState initialState + ) { + return startTask(id, connProps, taskProps, statusListener, + new SourceTaskBuilder(id, configState, statusListener, initialState)); + } + + /** + * Start a source task with exactly-once support managed by this worker. + * + * @param id the task ID. + * @param configState the most recent {@link ClusterConfigState} known to the worker + * @param connProps the connector properties. + * @param taskProps the tasks properties. + * @param statusListener a listener for the runtime status transitions of the task. + * @param initialState the initial state of the connector. + * @param preProducerCheck a preflight check that should be performed before the task initializes its transactional producer. + * @param postProducerCheck a preflight check that should be performed after the task initializes its transactional producer, + * but before producing any source records or offsets. + * @return true if the task started successfully. + */ + public boolean startExactlyOnceSourceTask( + ConnectorTaskId id, + ClusterConfigState configState, + Map connProps, + Map taskProps, + TaskStatus.Listener statusListener, + TargetState initialState, + Runnable preProducerCheck, + Runnable postProducerCheck + ) { + return startTask(id, connProps, taskProps, statusListener, + new ExactlyOnceSourceTaskBuilder(id, configState, statusListener, initialState, preProducerCheck, postProducerCheck)); + } + + /** + * Start a task managed by this worker. + * + * @param id the task ID. + * @param connProps the connector properties. + * @param taskProps the tasks properties. + * @param statusListener a listener for the runtime status transitions of the task. + * @param taskBuilder the {@link TaskBuilder} used to create the {@link WorkerTask} that manages the lifecycle of the task. + * @return true if the task started successfully. + */ + private boolean startTask( + ConnectorTaskId id, + Map connProps, + Map taskProps, + TaskStatus.Listener statusListener, + TaskBuilder taskBuilder ) { final WorkerTask workerTask; final TaskStatus.Listener taskStatusListener = workerMetricsGroup.wrapStatusListener(statusListener); @@ -563,8 +655,15 @@ public boolean startTask( log.info("Set up the header converter {} for task {} using the connector config", headerConverter.getClass(), id); } - workerTask = buildWorkerTask(configState, connConfig, id, task, taskStatusListener, - initialState, keyConverter, valueConverter, headerConverter, connectorLoader); + workerTask = taskBuilder + .withTask(task) + .withConnectorConfig(connConfig) + .withKeyConverter(keyConverter) + .withValueConverter(valueConverter) + .withHeaderConverter(headerConverter) + .withClassloader(connectorLoader) + .build(); + workerTask.initialize(taskConfig); Plugins.compareAndSwapLoaders(savedLoader); } catch (Throwable t) { @@ -583,7 +682,7 @@ public boolean startTask( executor.submit(workerTask); if (workerTask instanceof WorkerSourceTask) { - sourceTaskOffsetCommitter.schedule(id, (WorkerSourceTask) workerTask); + sourceTaskOffsetCommitter.ifPresent(committer -> committer.schedule(id, (WorkerSourceTask) workerTask)); } return true; } @@ -643,74 +742,28 @@ KafkaFuture fenceZombies(String connName, int numTasks, Map connectorClass = plugins.connectorClass( - connConfig.getString(ConnectorConfig.CONNECTOR_CLASS_CONFIG)); - RetryWithToleranceOperator retryWithToleranceOperator = new RetryWithToleranceOperator(connConfig.errorRetryTimeout(), - connConfig.errorMaxDelayInMillis(), connConfig.errorToleranceType(), Time.SYSTEM); - retryWithToleranceOperator.metrics(errorHandlingMetrics); - - // Decide which type of worker task we need based on the type of task. - if (task instanceof SourceTask) { - SourceConnectorConfig sourceConfig = new SourceConnectorConfig(plugins, - connConfig.originalsStrings(), config.topicCreationEnable()); - retryWithToleranceOperator.reporters(sourceTaskReporters(id, sourceConfig, errorHandlingMetrics)); - TransformationChain transformationChain = new TransformationChain<>(sourceConfig.transformations(), retryWithToleranceOperator); - log.info("Initializing: {}", transformationChain); - CloseableOffsetStorageReader offsetReader = new OffsetStorageReaderImpl(offsetBackingStore, id.connector(), - internalKeyConverter, internalValueConverter); - OffsetStorageWriter offsetWriter = new OffsetStorageWriter(offsetBackingStore, id.connector(), - internalKeyConverter, internalValueConverter); - Map producerProps = producerConfigs(id.connector(), "connector-producer-" + id, config, sourceConfig, connectorClass, - connectorClientConfigOverridePolicy, kafkaClusterId); - KafkaProducer producer = new KafkaProducer<>(producerProps); - TopicAdmin admin; - Map topicCreationGroups; - if (config.topicCreationEnable() && sourceConfig.usesTopicCreation()) { - Map adminProps = adminConfigs(id.connector(), "connector-adminclient-" + id, config, - sourceConfig, connectorClass, connectorClientConfigOverridePolicy, kafkaClusterId, ConnectorType.SOURCE); - admin = new TopicAdmin(adminProps); - topicCreationGroups = TopicCreationGroup.configuredGroups(sourceConfig); - } else { - admin = null; - topicCreationGroups = null; - } - - // Note we pass the configState as it performs dynamic transformations under the covers - return new WorkerSourceTask(id, (SourceTask) task, statusListener, initialState, keyConverter, valueConverter, - headerConverter, transformationChain, producer, admin, topicCreationGroups, - offsetReader, offsetWriter, config, configState, metrics, loader, time, retryWithToleranceOperator, herder.statusBackingStore(), executor); - } else if (task instanceof SinkTask) { - TransformationChain transformationChain = new TransformationChain<>(connConfig.transformations(), retryWithToleranceOperator); - log.info("Initializing: {}", transformationChain); - SinkConnectorConfig sinkConfig = new SinkConnectorConfig(plugins, connConfig.originalsStrings()); - retryWithToleranceOperator.reporters(sinkTaskReporters(id, sinkConfig, errorHandlingMetrics, connectorClass)); - WorkerErrantRecordReporter workerErrantRecordReporter = createWorkerErrantRecordReporter(sinkConfig, retryWithToleranceOperator, - keyConverter, valueConverter, headerConverter); - - Map consumerProps = consumerConfigs(id, config, connConfig, connectorClass, connectorClientConfigOverridePolicy, kafkaClusterId); - KafkaConsumer consumer = new KafkaConsumer<>(consumerProps); - - return new WorkerSinkTask(id, (SinkTask) task, statusListener, initialState, config, configState, metrics, keyConverter, - valueConverter, headerConverter, transformationChain, consumer, loader, time, - retryWithToleranceOperator, workerErrantRecordReporter, herder.statusBackingStore()); - } else { - log.error("Tasks must be a subclass of either SourceTask or SinkTask and current is {}", task); - throw new ConnectException("Tasks must be a subclass of either SourceTask or SinkTask"); - } + static Map exactlyOnceSourceTaskProducerConfigs(ConnectorTaskId id, + WorkerConfig config, + ConnectorConfig connConfig, + Class connectorClass, + ConnectorClientConfigOverridePolicy connectorClientConfigOverridePolicy, + String clusterId) { + Map result = baseProducerConfigs(id.connector(), "connector-producer-" + id, config, connConfig, connectorClass, connectorClientConfigOverridePolicy, clusterId); + ConnectUtils.ensureProperty( + result, ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG, "true", + "for connectors when exactly-once source support is enabled", + false + ); + String transactionalId = taskTransactionalId(config.groupId(), id.connector(), id.task()); + ConnectUtils.ensureProperty( + result, ProducerConfig.TRANSACTIONAL_ID_CONFIG, transactionalId, + "for connectors when exactly-once source support is enabled", + true + ); + return result; } - static Map producerConfigs(String connName, + static Map baseProducerConfigs(String connName, String defaultClientId, WorkerConfig config, ConnectorConfig connConfig, @@ -749,20 +802,56 @@ static Map producerConfigs(String connName, return producerProps; } - static Map consumerConfigs(ConnectorTaskId id, + static Map exactlyOnceSourceOffsetsConsumerConfigs(String connName, + String defaultClientId, + WorkerConfig config, + ConnectorConfig connConfig, + Class connectorClass, + ConnectorClientConfigOverridePolicy connectorClientConfigOverridePolicy, + String clusterId) { + Map result = baseConsumerConfigs( + connName, defaultClientId, config, connConfig, connectorClass, + connectorClientConfigOverridePolicy, clusterId, ConnectorType.SOURCE); + ConnectUtils.ensureProperty( + result, ConsumerConfig.ISOLATION_LEVEL_CONFIG, IsolationLevel.READ_COMMITTED.name().toLowerCase(Locale.ROOT), + "for source connectors' offset consumers when exactly-once source support is enabled", + false + ); + return result; + } + + static Map regularSourceOffsetsConsumerConfigs(String connName, + String defaultClientId, + WorkerConfig config, + ConnectorConfig connConfig, + Class connectorClass, + ConnectorClientConfigOverridePolicy connectorClientConfigOverridePolicy, + String clusterId) { + Map result = baseConsumerConfigs( + connName, defaultClientId, config, connConfig, connectorClass, + connectorClientConfigOverridePolicy, clusterId, ConnectorType.SOURCE); + // Users can disable this if they want to; it won't affect delivery guarantees since the task isn't exactly-once anyways + result.putIfAbsent( + ConsumerConfig.ISOLATION_LEVEL_CONFIG, + IsolationLevel.READ_COMMITTED.toString().toLowerCase(Locale.ROOT)); + return result; + } + + static Map baseConsumerConfigs(String connName, + String defaultClientId, WorkerConfig config, ConnectorConfig connConfig, Class connectorClass, ConnectorClientConfigOverridePolicy connectorClientConfigOverridePolicy, - String clusterId) { + String clusterId, + ConnectorType connectorType) { // Include any unknown worker configs so consumer configs can be set globally on the worker // and through to the task Map consumerProps = new HashMap<>(); - consumerProps.put(ConsumerConfig.GROUP_ID_CONFIG, SinkUtils.consumerGroupId(id.connector())); - consumerProps.put(ConsumerConfig.CLIENT_ID_CONFIG, "connector-consumer-" + id); - consumerProps.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, - Utils.join(config.getList(WorkerConfig.BOOTSTRAP_SERVERS_CONFIG), ",")); + consumerProps.put(ConsumerConfig.GROUP_ID_CONFIG, SinkUtils.consumerGroupId(connName)); + consumerProps.put(ConsumerConfig.CLIENT_ID_CONFIG, defaultClientId); + consumerProps.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, config.bootstrapServers()); consumerProps.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false"); consumerProps.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); consumerProps.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArrayDeserializer"); @@ -773,8 +862,8 @@ static Map consumerConfigs(ConnectorTaskId id, ConnectUtils.addMetricsContextProperties(consumerProps, config, clusterId); // Connector-specified overrides Map consumerOverrides = - connectorClientConfigOverrides(id.connector(), connConfig, connectorClass, ConnectorConfig.CONNECTOR_CLIENT_CONSUMER_OVERRIDES_PREFIX, - ConnectorType.SINK, ConnectorClientConfigRequest.ClientType.CONSUMER, + connectorClientConfigOverrides(connName, connConfig, connectorClass, ConnectorConfig.CONNECTOR_CLIENT_CONSUMER_OVERRIDES_PREFIX, + connectorType, ConnectorClientConfigRequest.ClientType.CONSUMER, connectorClientConfigOverridePolicy); consumerProps.putAll(consumerOverrides); @@ -849,6 +938,11 @@ private String taskTransactionalId(ConnectorTaskId id) { return taskTransactionalId(config.groupId(), id.connector(), id.task()); } + /** + * @return the {@link ProducerConfig#TRANSACTIONAL_ID_CONFIG transactional ID} to use for a task that writes + * records and/or offsets in a transaction. Not to be confused with {@link DistributedConfig#transactionalProducerId()}, + * which is not used by tasks at all, but instead, by the worker itself. + */ public static String taskTransactionalId(String groupId, String connector, int taskId) { return String.format("%s-%s-%d", groupId, connector, taskId); } @@ -867,8 +961,8 @@ private List sinkTaskReporters(ConnectorTaskId id, SinkConnectorC // check if topic for dead letter queue exists String topic = connConfig.dlqTopicName(); if (topic != null && !topic.isEmpty()) { - Map producerProps = producerConfigs(id.connector(), "connector-dlq-producer-" + id, config, connConfig, connectorClass, - connectorClientConfigOverridePolicy, kafkaClusterId); + Map producerProps = baseProducerConfigs(id.connector(), "connector-dlq-producer-" + id, config, connConfig, connectorClass, + connectorClientConfigOverridePolicy, kafkaClusterId); Map adminProps = adminConfigs(id.connector(), "connector-dlq-adminclient-", config, connConfig, connectorClass, connectorClientConfigOverridePolicy, kafkaClusterId, ConnectorType.SINK); DeadLetterQueueReporter reporter = DeadLetterQueueReporter.createAndSetup(adminProps, id, connConfig, producerProps, errorHandlingMetrics); @@ -911,7 +1005,7 @@ private void stopTask(ConnectorTaskId taskId) { log.info("Stopping task {}", task.id()); if (task instanceof WorkerSourceTask) - sourceTaskOffsetCommitter.remove(task.id()); + sourceTaskOffsetCommitter.ifPresent(committer -> committer.remove(task.id())); ClassLoader savedLoader = plugins.currentThreadLoader(); try { @@ -1069,6 +1163,268 @@ WorkerMetricsGroup workerMetricsGroup() { return workerMetricsGroup; } + abstract class TaskBuilder { + + private final ConnectorTaskId id; + private final ClusterConfigState configState; + private final TaskStatus.Listener statusListener; + private final TargetState initialState; + + private Task task = null; + private ConnectorConfig connectorConfig = null; + private Converter keyConverter = null; + private Converter valueConverter = null; + private HeaderConverter headerConverter = null; + private ClassLoader classLoader = null; + + public TaskBuilder(ConnectorTaskId id, + ClusterConfigState configState, + TaskStatus.Listener statusListener, + TargetState initialState) { + this.id = id; + this.configState = configState; + this.statusListener = statusListener; + this.initialState = initialState; + } + + public TaskBuilder withTask(Task task) { + this.task = task; + return this; + } + + public TaskBuilder withConnectorConfig(ConnectorConfig connectorConfig) { + this.connectorConfig = connectorConfig; + return this; + } + + public TaskBuilder withKeyConverter(Converter keyConverter) { + this.keyConverter = keyConverter; + return this; + } + + public TaskBuilder withValueConverter(Converter valueConverter) { + this.valueConverter = valueConverter; + return this; + } + + public TaskBuilder withHeaderConverter(HeaderConverter headerConverter) { + this.headerConverter = headerConverter; + return this; + } + + public TaskBuilder withClassloader(ClassLoader classLoader) { + this.classLoader = classLoader; + return this; + } + + public WorkerTask build() { + Objects.requireNonNull(task, "Task cannot be null"); + Objects.requireNonNull(connectorConfig, "Connector config used by task cannot be null"); + Objects.requireNonNull(keyConverter, "Key converter used by task cannot be null"); + Objects.requireNonNull(valueConverter, "Value converter used by task cannot be null"); + Objects.requireNonNull(headerConverter, "Header converter used by task cannot be null"); + Objects.requireNonNull(classLoader, "Classloader used by task cannot be null"); + + ErrorHandlingMetrics errorHandlingMetrics = errorHandlingMetrics(id); + final Class connectorClass = plugins.connectorClass( + connectorConfig.getString(ConnectorConfig.CONNECTOR_CLASS_CONFIG)); + RetryWithToleranceOperator retryWithToleranceOperator = new RetryWithToleranceOperator(connectorConfig.errorRetryTimeout(), + connectorConfig.errorMaxDelayInMillis(), connectorConfig.errorToleranceType(), Time.SYSTEM); + retryWithToleranceOperator.metrics(errorHandlingMetrics); + + return doBuild(task, id, configState, statusListener, initialState, + connectorConfig, keyConverter, valueConverter, headerConverter, classLoader, + errorHandlingMetrics, connectorClass, retryWithToleranceOperator); + } + + abstract WorkerTask doBuild(Task task, + ConnectorTaskId id, + ClusterConfigState configState, + TaskStatus.Listener statusListener, + TargetState initialState, + ConnectorConfig connectorConfig, + Converter keyConverter, + Converter valueConverter, + HeaderConverter headerConverter, + ClassLoader classLoader, + ErrorHandlingMetrics errorHandlingMetrics, + Class connectorClass, + RetryWithToleranceOperator retryWithToleranceOperator); + + } + + class SinkTaskBuilder extends TaskBuilder { + public SinkTaskBuilder(ConnectorTaskId id, + ClusterConfigState configState, + TaskStatus.Listener statusListener, + TargetState initialState) { + super(id, configState, statusListener, initialState); + } + + @Override + public WorkerTask doBuild(Task task, + ConnectorTaskId id, + ClusterConfigState configState, + TaskStatus.Listener statusListener, + TargetState initialState, + ConnectorConfig connectorConfig, + Converter keyConverter, + Converter valueConverter, + HeaderConverter headerConverter, + ClassLoader classLoader, + ErrorHandlingMetrics errorHandlingMetrics, + Class connectorClass, + RetryWithToleranceOperator retryWithToleranceOperator) { + + TransformationChain transformationChain = new TransformationChain<>(connectorConfig.transformations(), retryWithToleranceOperator); + log.info("Initializing: {}", transformationChain); + SinkConnectorConfig sinkConfig = new SinkConnectorConfig(plugins, connectorConfig.originalsStrings()); + retryWithToleranceOperator.reporters(sinkTaskReporters(id, sinkConfig, errorHandlingMetrics, connectorClass)); + WorkerErrantRecordReporter workerErrantRecordReporter = createWorkerErrantRecordReporter(sinkConfig, retryWithToleranceOperator, + keyConverter, valueConverter, headerConverter); + + Map consumerProps = baseConsumerConfigs( + id.connector(), "connector-consumer-" + id, config, connectorConfig, connectorClass, + connectorClientConfigOverridePolicy, kafkaClusterId, ConnectorType.SINK); + KafkaConsumer consumer = new KafkaConsumer<>(consumerProps); + + return new WorkerSinkTask(id, (SinkTask) task, statusListener, initialState, config, configState, metrics, keyConverter, + valueConverter, headerConverter, transformationChain, consumer, classLoader, time, + retryWithToleranceOperator, workerErrantRecordReporter, herder.statusBackingStore()); + } + } + + class SourceTaskBuilder extends TaskBuilder { + public SourceTaskBuilder(ConnectorTaskId id, + ClusterConfigState configState, + TaskStatus.Listener statusListener, + TargetState initialState) { + super(id, configState, statusListener, initialState); + } + + @Override + public WorkerTask doBuild(Task task, + ConnectorTaskId id, + ClusterConfigState configState, + TaskStatus.Listener statusListener, + TargetState initialState, + ConnectorConfig connectorConfig, + Converter keyConverter, + Converter valueConverter, + HeaderConverter headerConverter, + ClassLoader classLoader, + ErrorHandlingMetrics errorHandlingMetrics, + Class connectorClass, + RetryWithToleranceOperator retryWithToleranceOperator) { + + SourceConnectorConfig sourceConfig = new SourceConnectorConfig(plugins, + connectorConfig.originalsStrings(), config.topicCreationEnable()); + retryWithToleranceOperator.reporters(sourceTaskReporters(id, sourceConfig, errorHandlingMetrics)); + TransformationChain transformationChain = new TransformationChain<>(sourceConfig.transformations(), retryWithToleranceOperator); + log.info("Initializing: {}", transformationChain); + + Map producerProps = baseProducerConfigs(id.connector(), "connector-producer-" + id, config, sourceConfig, connectorClass, + connectorClientConfigOverridePolicy, kafkaClusterId); + KafkaProducer producer = new KafkaProducer<>(producerProps); + + TopicAdmin topicAdmin; + Map topicCreationGroups; + if (config.topicCreationEnable() && sourceConfig.usesTopicCreation()) { + topicCreationGroups = TopicCreationGroup.configuredGroups(sourceConfig); + // Create a topic admin that the task can use for topic creation + Map adminOverrides = adminConfigs(id.connector(), "connector-adminclient-" + id, config, + sourceConfig, connectorClass, connectorClientConfigOverridePolicy, kafkaClusterId, ConnectorType.SOURCE); + topicAdmin = new TopicAdmin(adminOverrides); + } else { + topicAdmin = null; + topicCreationGroups = null; + } + + // Set up the offset backing store for this task instance + // (This logic is implemented in a follow-up pull request: https://github.com/apache/kafka/pull/11781) + ConnectorOffsetBackingStore offsetStore = new ConnectorOffsetBackingStore(globalOffsetBackingStore, "TODO"); + offsetStore.configure(config); + + CloseableOffsetStorageReader offsetReader = new OffsetStorageReaderImpl(offsetStore, id.connector(), internalKeyConverter, internalValueConverter); + OffsetStorageWriter offsetWriter = new OffsetStorageWriter(offsetStore, id.connector(), internalKeyConverter, internalValueConverter); + + // Note we pass the configState as it performs dynamic transformations under the covers + return new WorkerSourceTask(id, (SourceTask) task, statusListener, initialState, keyConverter, valueConverter, + headerConverter, transformationChain, producer, topicAdmin, topicCreationGroups, + offsetReader, offsetWriter, offsetStore, config, configState, metrics, classLoader, time, + retryWithToleranceOperator, herder.statusBackingStore(), executor); + } + } + + class ExactlyOnceSourceTaskBuilder extends TaskBuilder { + private final Runnable preProducerCheck; + private final Runnable postProducerCheck; + + public ExactlyOnceSourceTaskBuilder(ConnectorTaskId id, + ClusterConfigState configState, + TaskStatus.Listener statusListener, + TargetState initialState, + Runnable preProducerCheck, + Runnable postProducerCheck) { + super(id, configState, statusListener, initialState); + this.preProducerCheck = preProducerCheck; + this.postProducerCheck = postProducerCheck; + } + + @Override + public WorkerTask doBuild(Task task, + ConnectorTaskId id, + ClusterConfigState configState, + TaskStatus.Listener statusListener, + TargetState initialState, + ConnectorConfig connectorConfig, + Converter keyConverter, + Converter valueConverter, + HeaderConverter headerConverter, + ClassLoader classLoader, + ErrorHandlingMetrics errorHandlingMetrics, + Class connectorClass, + RetryWithToleranceOperator retryWithToleranceOperator) { + + SourceConnectorConfig sourceConfig = new SourceConnectorConfig(plugins, + connectorConfig.originalsStrings(), config.topicCreationEnable()); + retryWithToleranceOperator.reporters(sourceTaskReporters(id, sourceConfig, errorHandlingMetrics)); + TransformationChain transformationChain = new TransformationChain<>(sourceConfig.transformations(), retryWithToleranceOperator); + log.info("Initializing: {}", transformationChain); + + Map producerProps = exactlyOnceSourceTaskProducerConfigs( + id, config, sourceConfig, connectorClass, + connectorClientConfigOverridePolicy, kafkaClusterId); + KafkaProducer producer = new KafkaProducer<>(producerProps); + + // Create a topic admin that the task will use for its offsets topic and, potentially, automatic topic creation + Map adminOverrides = adminConfigs(id.connector(), "connector-adminclient-" + id, config, + sourceConfig, connectorClass, connectorClientConfigOverridePolicy, kafkaClusterId, ConnectorType.SOURCE); + TopicAdmin topicAdmin = new TopicAdmin(adminOverrides); + + Map topicCreationGroups; + if (config.topicCreationEnable() && sourceConfig.usesTopicCreation()) { + topicCreationGroups = TopicCreationGroup.configuredGroups(sourceConfig); + } else { + topicCreationGroups = null; + } + + // Set up the offset backing store for this task instance + // (This logic is implemented in a follow-up pull request: https://github.com/apache/kafka/pull/11781) + ConnectorOffsetBackingStore offsetStore = new ConnectorOffsetBackingStore(globalOffsetBackingStore, "TODO"); + offsetStore.configure(config); + + CloseableOffsetStorageReader offsetReader = new OffsetStorageReaderImpl(offsetStore, id.connector(), internalKeyConverter, internalValueConverter); + OffsetStorageWriter offsetWriter = new OffsetStorageWriter(offsetStore, id.connector(), internalKeyConverter, internalValueConverter); + + // Note we pass the configState as it performs dynamic transformations under the covers + return new ExactlyOnceWorkerSourceTask(id, (SourceTask) task, statusListener, initialState, keyConverter, valueConverter, + headerConverter, transformationChain, producer, topicAdmin, topicCreationGroups, + offsetReader, offsetWriter, offsetStore, config, configState, metrics, classLoader, time, retryWithToleranceOperator, + herder.statusBackingStore(), sourceConfig, executor, preProducerCheck, postProducerCheck); + } + } + static class ConnectorStatusMetricsGroup { private final ConnectMetrics connectMetrics; private final ConnectMetricsRegistry registry; diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerConnector.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerConnector.java index 09b57fd42a85..dfa13103fa3e 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerConnector.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerConnector.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.connect.runtime; +import org.apache.kafka.common.utils.Utils; import org.apache.kafka.connect.connector.Connector; import org.apache.kafka.connect.connector.ConnectorContext; import org.apache.kafka.connect.errors.ConnectException; @@ -23,6 +24,8 @@ import org.apache.kafka.connect.runtime.isolation.Plugins; import org.apache.kafka.connect.sink.SinkConnectorContext; import org.apache.kafka.connect.source.SourceConnectorContext; +import org.apache.kafka.connect.storage.CloseableOffsetStorageReader; +import org.apache.kafka.connect.storage.ConnectorOffsetBackingStore; import org.apache.kafka.connect.storage.OffsetStorageReader; import org.apache.kafka.connect.util.Callback; import org.apache.kafka.connect.util.ConnectUtils; @@ -74,7 +77,8 @@ private enum State { private volatile boolean cancelled; // indicates whether the Worker has cancelled the connector (e.g. because of slow shutdown) private State state; - private final OffsetStorageReader offsetStorageReader; + private final CloseableOffsetStorageReader offsetStorageReader; + private final ConnectorOffsetBackingStore offsetStore; public WorkerConnector(String connName, Connector connector, @@ -82,7 +86,8 @@ public WorkerConnector(String connName, CloseableConnectorContext ctx, ConnectMetrics metrics, ConnectorStatus.Listener statusListener, - OffsetStorageReader offsetStorageReader, + CloseableOffsetStorageReader offsetStorageReader, + ConnectorOffsetBackingStore offsetStore, ClassLoader loader) { this.connName = connName; this.config = connectorConfig.originalsStrings(); @@ -93,6 +98,7 @@ public WorkerConnector(String connName, this.metrics = new ConnectorMetricsGroup(metrics, AbstractStatus.State.UNASSIGNED, statusListener); this.statusListener = this.metrics; this.offsetStorageReader = offsetStorageReader; + this.offsetStore = offsetStore; this.pendingTargetStateChange = new AtomicReference<>(); this.pendingStateChangeCallback = new AtomicReference<>(); this.shutdownLatch = new CountDownLatch(1); @@ -165,6 +171,7 @@ void initialize() { SinkConnectorConfig.validate(config); connector.initialize(new WorkerSinkConnectorContext()); } else { + Objects.requireNonNull(offsetStorageReader, "Offset reader cannot be null for source connectors"); connector.initialize(new WorkerSourceConnectorContext(offsetStorageReader)); } } catch (Throwable t) { @@ -271,8 +278,10 @@ void doShutdown() { state = State.FAILED; statusListener.onFailure(connName, t); } finally { - ctx.close(); - metrics.close(); + Utils.closeQuietly(ctx, "connector context for " + connName); + Utils.closeQuietly(metrics, "connector metrics for " + connName); + Utils.closeQuietly(offsetStorageReader, "offset reader for " + connName); + Utils.closeQuietly(offsetStore::stop, "offset backing store for " + connName); } } @@ -281,7 +290,9 @@ public synchronized void cancel() { // instance is being abandoned and we won't update the status on its behalf any more // after this since a new instance may be started soon statusListener.onShutdown(connName); - ctx.close(); + Utils.closeQuietly(ctx, "connector context for " + connName); + // Preemptively close the offset reader in case the connector is blocked on an offset read. + Utils.closeQuietly(offsetStorageReader, "offset reader for " + connName); cancelled = true; } diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSourceTask.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSourceTask.java index 43bf64c6af4b..0b8c7fa52ed5 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSourceTask.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSourceTask.java @@ -16,27 +16,11 @@ */ package org.apache.kafka.connect.runtime; -import org.apache.kafka.clients.admin.NewTopic; -import org.apache.kafka.clients.admin.TopicDescription; -import org.apache.kafka.clients.producer.KafkaProducer; +import org.apache.kafka.clients.producer.Producer; import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.clients.producer.RecordMetadata; -import org.apache.kafka.common.KafkaException; -import org.apache.kafka.common.header.internals.RecordHeaders; -import org.apache.kafka.common.metrics.Sensor; -import org.apache.kafka.common.metrics.stats.Avg; -import org.apache.kafka.common.metrics.stats.CumulativeSum; -import org.apache.kafka.common.metrics.stats.Max; -import org.apache.kafka.common.metrics.stats.Rate; -import org.apache.kafka.common.metrics.stats.Value; import org.apache.kafka.common.utils.Time; -import org.apache.kafka.common.utils.Utils; import org.apache.kafka.connect.errors.ConnectException; -import org.apache.kafka.connect.errors.RetriableException; -import org.apache.kafka.connect.header.Header; -import org.apache.kafka.connect.header.Headers; -import org.apache.kafka.connect.runtime.ConnectMetrics.MetricGroup; -import org.apache.kafka.connect.runtime.SubmittedRecords.SubmittedRecord; import org.apache.kafka.connect.storage.ClusterConfigState; import org.apache.kafka.connect.runtime.errors.RetryWithToleranceOperator; import org.apache.kafka.connect.runtime.errors.Stage; @@ -44,22 +28,19 @@ import org.apache.kafka.connect.source.SourceRecord; import org.apache.kafka.connect.source.SourceTask; import org.apache.kafka.connect.storage.CloseableOffsetStorageReader; +import org.apache.kafka.connect.storage.ConnectorOffsetBackingStore; import org.apache.kafka.connect.storage.Converter; import org.apache.kafka.connect.storage.HeaderConverter; import org.apache.kafka.connect.storage.OffsetStorageWriter; import org.apache.kafka.connect.storage.StatusBackingStore; -import org.apache.kafka.connect.util.ConnectUtils; import org.apache.kafka.connect.util.ConnectorTaskId; import org.apache.kafka.connect.util.TopicAdmin; -import org.apache.kafka.connect.util.TopicCreation; import org.apache.kafka.connect.util.TopicCreationGroup; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.time.Duration; -import java.util.List; import java.util.Map; -import java.util.concurrent.CountDownLatch; +import java.util.Optional; import java.util.concurrent.ExecutionException; import java.util.concurrent.Executor; import java.util.concurrent.Future; @@ -68,40 +49,16 @@ import java.util.concurrent.atomic.AtomicReference; import static org.apache.kafka.connect.runtime.SubmittedRecords.CommittableOffsets; -import static org.apache.kafka.connect.runtime.WorkerConfig.TOPIC_TRACKING_ENABLE_CONFIG; /** * WorkerTask that uses a SourceTask to ingest data into Kafka. */ -class WorkerSourceTask extends WorkerTask { +class WorkerSourceTask extends AbstractWorkerSourceTask { private static final Logger log = LoggerFactory.getLogger(WorkerSourceTask.class); - private static final long SEND_FAILED_BACKOFF_MS = 100; - - private final WorkerConfig workerConfig; - private final SourceTask task; - private final ClusterConfigState configState; - private final Converter keyConverter; - private final Converter valueConverter; - private final HeaderConverter headerConverter; - private final TransformationChain transformationChain; - private final KafkaProducer producer; - private final TopicAdmin admin; - private final CloseableOffsetStorageReader offsetReader; - private final OffsetStorageWriter offsetWriter; - private final Executor closeExecutor; - private final SourceTaskMetricsGroup sourceTaskMetricsGroup; - private final AtomicReference producerSendException; - private final boolean isTopicTrackingEnabled; - private final TopicCreation topicCreation; - - private List toSend; private volatile CommittableOffsets committableOffsets; private final SubmittedRecords submittedRecords; - private final CountDownLatch stopRequestedLatch; - - private Map taskConfig; - private boolean started = false; + private final AtomicReference producerSendException; public WorkerSourceTask(ConnectorTaskId id, SourceTask task, @@ -111,11 +68,12 @@ public WorkerSourceTask(ConnectorTaskId id, Converter valueConverter, HeaderConverter headerConverter, TransformationChain transformationChain, - KafkaProducer producer, + Producer producer, TopicAdmin admin, Map topicGroups, CloseableOffsetStorageReader offsetReader, OffsetStorageWriter offsetWriter, + ConnectorOffsetBackingStore offsetBackingStore, WorkerConfig workerConfig, ClusterConfigState configState, ConnectMetrics connectMetrics, @@ -125,355 +83,115 @@ public WorkerSourceTask(ConnectorTaskId id, StatusBackingStore statusBackingStore, Executor closeExecutor) { - super(id, statusListener, initialState, loader, connectMetrics, - retryWithToleranceOperator, time, statusBackingStore); - - this.workerConfig = workerConfig; - this.task = task; - this.configState = configState; - this.keyConverter = keyConverter; - this.valueConverter = valueConverter; - this.headerConverter = headerConverter; - this.transformationChain = transformationChain; - this.producer = producer; - this.admin = admin; - this.offsetReader = offsetReader; - this.offsetWriter = offsetWriter; - this.closeExecutor = closeExecutor; - - this.toSend = null; + super(id, task, statusListener, initialState, keyConverter, valueConverter, headerConverter, transformationChain, + new WorkerSourceTaskContext(offsetReader, id, configState, null), producer, + admin, topicGroups, offsetReader, offsetWriter, offsetBackingStore, workerConfig, connectMetrics, loader, + time, retryWithToleranceOperator, statusBackingStore, closeExecutor); + this.committableOffsets = CommittableOffsets.EMPTY; this.submittedRecords = new SubmittedRecords(); - this.stopRequestedLatch = new CountDownLatch(1); - this.sourceTaskMetricsGroup = new SourceTaskMetricsGroup(id, connectMetrics); this.producerSendException = new AtomicReference<>(); - this.isTopicTrackingEnabled = workerConfig.getBoolean(TOPIC_TRACKING_ENABLE_CONFIG); - this.topicCreation = TopicCreation.newTopicCreation(workerConfig, topicGroups); } @Override - public void initialize(TaskConfig taskConfig) { - try { - this.taskConfig = taskConfig.originalsStrings(); - } catch (Throwable t) { - log.error("{} Task failed initialization and will not be started.", this, t); - onFailure(t); - } + protected void prepareToInitializeTask() { + // No-op } @Override - protected void close() { - if (started) { - try { - task.stop(); - } catch (Throwable t) { - log.warn("Could not stop task", t); - } - } - - closeProducer(Duration.ofSeconds(30)); - - if (admin != null) { - try { - admin.close(Duration.ofSeconds(30)); - } catch (Throwable t) { - log.warn("Failed to close admin client on time", t); - } - } - Utils.closeQuietly(transformationChain, "transformation chain"); - Utils.closeQuietly(retryWithToleranceOperator, "retry operator"); + protected void prepareToEnterSendLoop() { + // No-op } @Override - public void removeMetrics() { - try { - sourceTaskMetricsGroup.close(); - } finally { - super.removeMetrics(); - } + protected void beginSendIteration() { + updateCommittableOffsets(); } @Override - public void cancel() { - super.cancel(); - offsetReader.close(); - // We proactively close the producer here as the main work thread for the task may - // be blocked indefinitely in a call to Producer::send if automatic topic creation is - // not enabled on either the connector or the Kafka cluster. Closing the producer should - // unblock it in that case and allow shutdown to proceed normally. - // With a duration of 0, the producer's own shutdown logic should be fairly quick, - // but closing user-pluggable classes like interceptors may lag indefinitely. So, we - // call close on a separate thread in order to avoid blocking the herder's tick thread. - closeExecutor.execute(() -> closeProducer(Duration.ZERO)); + protected void prepareToPollTask() { + maybeThrowProducerSendException(); } @Override - public void stop() { - super.stop(); - stopRequestedLatch.countDown(); + protected void recordDropped(SourceRecord record) { + commitTaskRecord(record, null); } @Override - protected void initializeAndStart() { - // If we try to start the task at all by invoking initialize, then count this as - // "started" and expect a subsequent call to the task's stop() method - // to properly clean up any resources allocated by its initialize() or - // start() methods. If the task throws an exception during stop(), - // the worst thing that happens is another exception gets logged for an already- - // failed task - started = true; - task.initialize(new WorkerSourceTaskContext(offsetReader, this, configState)); - task.start(taskConfig); - log.info("{} Source task finished initialization and start", this); + protected Optional prepareToSendRecord( + SourceRecord sourceRecord, + ProducerRecord producerRecord + ) { + maybeThrowProducerSendException(); + return Optional.of(submittedRecords.submit(sourceRecord)); } @Override - public void execute() { - try { - log.info("{} Executing source task", this); - while (!isStopping()) { - updateCommittableOffsets(); - - if (shouldPause()) { - onPause(); - if (awaitUnpause()) { - onResume(); - } - continue; - } - - maybeThrowProducerSendException(); - if (toSend == null) { - log.trace("{} Nothing to send to Kafka. Polling source for additional records", this); - long start = time.milliseconds(); - toSend = poll(); - if (toSend != null) { - recordPollReturned(toSend.size(), time.milliseconds() - start); - } - } - - if (toSend == null) - continue; - log.trace("{} About to send {} records to Kafka", this, toSend.size()); - if (!sendRecords()) - stopRequestedLatch.await(SEND_FAILED_BACKOFF_MS, TimeUnit.MILLISECONDS); - } - } catch (InterruptedException e) { - // Ignore and allow to exit. - } finally { - submittedRecords.awaitAllMessages( - workerConfig.getLong(WorkerConfig.OFFSET_COMMIT_TIMEOUT_MS_CONFIG), - TimeUnit.MILLISECONDS - ); - // It should still be safe to commit offsets since any exception would have - // simply resulted in not getting more records but all the existing records should be ok to flush - // and commit offsets. Worst case, task.flush() will also throw an exception causing the offset commit - // to fail. - updateCommittableOffsets(); - commitOffsets(); - } + protected void recordDispatched(SourceRecord record) { + // No-op } - private void closeProducer(Duration duration) { - if (producer != null) { - try { - producer.close(duration); - } catch (Throwable t) { - log.warn("Could not close producer for {}", id, t); - } - } - } - - private void maybeThrowProducerSendException() { - if (producerSendException.get() != null) { - throw new ConnectException( - "Unrecoverable exception from producer send callback", - producerSendException.get() - ); - } - } - - private void updateCommittableOffsets() { - CommittableOffsets newOffsets = submittedRecords.committableOffsets(); - synchronized (this) { - this.committableOffsets = this.committableOffsets.updatedWith(newOffsets); - } - } - - protected List poll() throws InterruptedException { - try { - return task.poll(); - } catch (RetriableException | org.apache.kafka.common.errors.RetriableException e) { - log.warn("{} failed to poll records from SourceTask. Will retry operation.", this, e); - // Do nothing. Let the framework poll whenever it's ready. - return null; - } - } - - /** - * Convert the source record into a producer record. - * - * @param record the transformed record - * @return the producer record which can sent over to Kafka. A null is returned if the input is null or - * if an error was encountered during any of the converter stages. - */ - private ProducerRecord convertTransformedRecord(SourceRecord record) { - if (record == null) { - return null; - } - - RecordHeaders headers = retryWithToleranceOperator.execute(() -> convertHeaderFor(record), Stage.HEADER_CONVERTER, headerConverter.getClass()); - - byte[] key = retryWithToleranceOperator.execute(() -> keyConverter.fromConnectData(record.topic(), headers, record.keySchema(), record.key()), - Stage.KEY_CONVERTER, keyConverter.getClass()); - - byte[] value = retryWithToleranceOperator.execute(() -> valueConverter.fromConnectData(record.topic(), headers, record.valueSchema(), record.value()), - Stage.VALUE_CONVERTER, valueConverter.getClass()); - - if (retryWithToleranceOperator.failed()) { - return null; - } - - return new ProducerRecord<>(record.topic(), record.kafkaPartition(), - ConnectUtils.checkAndConvertTimestamp(record.timestamp()), key, value, headers); + @Override + protected void batchDispatched() { + // No-op } - /** - * Try to send a batch of records. If a send fails and is retriable, this saves the remainder of the batch so it can - * be retried after backing off. If a send fails and is not retriable, this will throw a ConnectException. - * @return true if all messages were sent, false if some need to be retried - */ - private boolean sendRecords() { - int processed = 0; - recordBatch(toSend.size()); - final SourceRecordWriteCounter counter = - toSend.size() > 0 ? new SourceRecordWriteCounter(toSend.size(), sourceTaskMetricsGroup) : null; - for (final SourceRecord preTransformRecord : toSend) { - maybeThrowProducerSendException(); - - retryWithToleranceOperator.sourceRecord(preTransformRecord); - final SourceRecord record = transformationChain.apply(preTransformRecord); - final ProducerRecord producerRecord = convertTransformedRecord(record); - if (producerRecord == null || retryWithToleranceOperator.failed()) { - counter.skipRecord(); - commitTaskRecord(preTransformRecord, null); - continue; - } - - log.trace("{} Appending record to the topic {} with key {}, value {}", this, record.topic(), record.key(), record.value()); - SubmittedRecord submittedRecord = submittedRecords.submit(record); - try { - maybeCreateTopic(record.topic()); - final String topic = producerRecord.topic(); - producer.send( - producerRecord, - (recordMetadata, e) -> { - if (e != null) { - if (retryWithToleranceOperator.getErrorToleranceType() == ToleranceType.ALL) { - log.trace("Ignoring failed record send: {} failed to send record to {}: ", - WorkerSourceTask.this, topic, e); - // executeFailed here allows the use of existing logging infrastructure/configuration - retryWithToleranceOperator.executeFailed(Stage.KAFKA_PRODUCE, WorkerSourceTask.class, - preTransformRecord, e); - commitTaskRecord(preTransformRecord, null); - } else { - log.error("{} failed to send record to {}: ", WorkerSourceTask.this, topic, e); - log.trace("{} Failed record: {}", WorkerSourceTask.this, preTransformRecord); - producerSendException.compareAndSet(null, e); - } - } else { - submittedRecord.ack(); - counter.completeRecord(); - log.trace("{} Wrote record successfully: topic {} partition {} offset {}", - WorkerSourceTask.this, - recordMetadata.topic(), recordMetadata.partition(), - recordMetadata.offset()); - commitTaskRecord(preTransformRecord, recordMetadata); - if (isTopicTrackingEnabled) { - recordActiveTopic(producerRecord.topic()); - } - } - }); - } catch (RetriableException | org.apache.kafka.common.errors.RetriableException e) { - log.warn("{} Failed to send record to topic '{}' and partition '{}'. Backing off before retrying: ", - this, producerRecord.topic(), producerRecord.partition(), e); - toSend = toSend.subList(processed, toSend.size()); - submittedRecords.removeLastOccurrence(submittedRecord); - counter.retryRemaining(); - return false; - } catch (ConnectException e) { - log.warn("{} Failed to send record to topic '{}' and partition '{}' due to an unrecoverable exception: ", - this, producerRecord.topic(), producerRecord.partition(), e); - log.trace("{} Failed to send {} with unrecoverable exception: ", this, producerRecord, e); - throw e; - } catch (KafkaException e) { - throw new ConnectException("Unrecoverable exception trying to send", e); - } - processed++; - } - toSend = null; - return true; + @Override + protected void recordSent( + SourceRecord sourceRecord, + ProducerRecord producerRecord, + RecordMetadata recordMetadata + ) { + commitTaskRecord(sourceRecord, recordMetadata); } - // Due to transformations that may change the destination topic of a record (such as - // RegexRouter) topic creation can not be batched for multiple topics - private void maybeCreateTopic(String topic) { - if (!topicCreation.isTopicCreationRequired(topic)) { - log.trace("Topic creation by the connector is disabled or the topic {} was previously created." + - "If auto.create.topics.enable is enabled on the broker, " + - "the topic will be created with default settings", topic); - return; - } - log.info("The task will send records to topic '{}' for the first time. Checking " - + "whether topic exists", topic); - Map existing = admin.describeTopics(topic); - if (!existing.isEmpty()) { - log.info("Topic '{}' already exists.", topic); - topicCreation.addTopic(topic); - return; - } - - log.info("Creating topic '{}'", topic); - TopicCreationGroup topicGroup = topicCreation.findFirstGroup(topic); - log.debug("Topic '{}' matched topic creation group: {}", topic, topicGroup); - NewTopic newTopic = topicGroup.newTopic(topic); - - TopicAdmin.TopicCreationResponse response = admin.createOrFindTopics(newTopic); - if (response.isCreated(newTopic.name())) { - topicCreation.addTopic(topic); - log.info("Created topic '{}' using creation group {}", newTopic, topicGroup); - } else if (response.isExisting(newTopic.name())) { - topicCreation.addTopic(topic); - log.info("Found existing topic '{}'", newTopic); + @Override + protected void producerSendFailed( + boolean synchronous, + ProducerRecord producerRecord, + SourceRecord preTransformRecord, + Exception e + ) { + if (synchronous) { + throw new ConnectException("Unrecoverable exception trying to send", e); + } + + String topic = producerRecord.topic(); + if (retryWithToleranceOperator.getErrorToleranceType() == ToleranceType.ALL) { + log.trace( + "Ignoring failed record send: {} failed to send record to {}: ", + WorkerSourceTask.this, + topic, + e + ); + // executeFailed here allows the use of existing logging infrastructure/configuration + retryWithToleranceOperator.executeFailed( + Stage.KAFKA_PRODUCE, + WorkerSourceTask.class, + preTransformRecord, + e + ); + commitTaskRecord(preTransformRecord, null); } else { - // The topic still does not exist and could not be created, so treat it as a task failure - log.warn("Request to create new topic '{}' failed", topic); - throw new ConnectException("Task failed to create new topic " + newTopic + ". Ensure " - + "that the task is authorized to create topics or that the topic exists and " - + "restart the task"); - } - } - - private RecordHeaders convertHeaderFor(SourceRecord record) { - Headers headers = record.headers(); - RecordHeaders result = new RecordHeaders(); - if (headers != null) { - String topic = record.topic(); - for (Header header : headers) { - String key = header.key(); - byte[] rawHeader = headerConverter.fromConnectHeader(topic, key, header.schema(), header.value()); - result.add(key, rawHeader); - } + log.error("{} failed to send record to {}: ", WorkerSourceTask.this, topic, e); + log.trace("{} Failed record: {}", WorkerSourceTask.this, preTransformRecord); + producerSendException.compareAndSet(null, e); } - return result; } - private void commitTaskRecord(SourceRecord record, RecordMetadata metadata) { - try { - task.commitRecord(record, metadata); - } catch (Throwable t) { - log.error("{} Exception thrown while calling task.commitRecord()", this, t); - } + @Override + protected void finalOffsetCommit(boolean failed) { + // It should still be safe to commit offsets since any exception would have + // simply resulted in not getting more records but all the existing records should be ok to flush + // and commit offsets. Worst case, task.commit() will also throw an exception causing the offset + // commit to fail. + submittedRecords.awaitAllMessages( + workerConfig.getLong(WorkerConfig.OFFSET_COMMIT_TIMEOUT_MS_CONFIG), + TimeUnit.MILLISECONDS + ); + updateCommittableOffsets(); + commitOffsets(); } public boolean commitOffsets() { @@ -510,8 +228,8 @@ public boolean commitOffsets() { committableOffsets.largestDequeSize() ); } else { - log.debug("{} There are currently no pending messages for this offset commit; " - + "all messages dispatched to the task's producer since the last commit have been acknowledged", + log.debug("{} There are currently no pending messages for this offset commit; " + + "all messages dispatched to the task's producer since the last commit have been acknowledged", this ); } @@ -582,11 +300,19 @@ public boolean commitOffsets() { return true; } - private void commitSourceTask() { - try { - this.task.commit(); - } catch (Throwable t) { - log.error("{} Exception thrown while calling task.commit()", this, t); + private void updateCommittableOffsets() { + CommittableOffsets newOffsets = submittedRecords.committableOffsets(); + synchronized (this) { + this.committableOffsets = this.committableOffsets.updatedWith(newOffsets); + } + } + + private void maybeThrowProducerSendException() { + if (producerSendException.get() != null) { + throw new ConnectException( + "Unrecoverable exception from producer send callback", + producerSendException.get() + ); } } @@ -597,101 +323,4 @@ public String toString() { '}'; } - protected void recordPollReturned(int numRecordsInBatch, long duration) { - sourceTaskMetricsGroup.recordPoll(numRecordsInBatch, duration); - } - - SourceTaskMetricsGroup sourceTaskMetricsGroup() { - return sourceTaskMetricsGroup; - } - - static class SourceRecordWriteCounter { - private final SourceTaskMetricsGroup metricsGroup; - private final int batchSize; - private boolean completed = false; - private int counter; - public SourceRecordWriteCounter(int batchSize, SourceTaskMetricsGroup metricsGroup) { - assert batchSize > 0; - assert metricsGroup != null; - this.batchSize = batchSize; - counter = batchSize; - this.metricsGroup = metricsGroup; - } - public void skipRecord() { - if (counter > 0 && --counter == 0) { - finishedAllWrites(); - } - } - public void completeRecord() { - if (counter > 0 && --counter == 0) { - finishedAllWrites(); - } - } - public void retryRemaining() { - finishedAllWrites(); - } - private void finishedAllWrites() { - if (!completed) { - metricsGroup.recordWrite(batchSize - counter); - completed = true; - } - } - } - - static class SourceTaskMetricsGroup { - private final MetricGroup metricGroup; - private final Sensor sourceRecordPoll; - private final Sensor sourceRecordWrite; - private final Sensor sourceRecordActiveCount; - private final Sensor pollTime; - private int activeRecordCount; - - public SourceTaskMetricsGroup(ConnectorTaskId id, ConnectMetrics connectMetrics) { - ConnectMetricsRegistry registry = connectMetrics.registry(); - metricGroup = connectMetrics.group(registry.sourceTaskGroupName(), - registry.connectorTagName(), id.connector(), - registry.taskTagName(), Integer.toString(id.task())); - // remove any previously created metrics in this group to prevent collisions. - metricGroup.close(); - - sourceRecordPoll = metricGroup.sensor("source-record-poll"); - sourceRecordPoll.add(metricGroup.metricName(registry.sourceRecordPollRate), new Rate()); - sourceRecordPoll.add(metricGroup.metricName(registry.sourceRecordPollTotal), new CumulativeSum()); - - sourceRecordWrite = metricGroup.sensor("source-record-write"); - sourceRecordWrite.add(metricGroup.metricName(registry.sourceRecordWriteRate), new Rate()); - sourceRecordWrite.add(metricGroup.metricName(registry.sourceRecordWriteTotal), new CumulativeSum()); - - pollTime = metricGroup.sensor("poll-batch-time"); - pollTime.add(metricGroup.metricName(registry.sourceRecordPollBatchTimeMax), new Max()); - pollTime.add(metricGroup.metricName(registry.sourceRecordPollBatchTimeAvg), new Avg()); - - sourceRecordActiveCount = metricGroup.sensor("source-record-active-count"); - sourceRecordActiveCount.add(metricGroup.metricName(registry.sourceRecordActiveCount), new Value()); - sourceRecordActiveCount.add(metricGroup.metricName(registry.sourceRecordActiveCountMax), new Max()); - sourceRecordActiveCount.add(metricGroup.metricName(registry.sourceRecordActiveCountAvg), new Avg()); - } - - void close() { - metricGroup.close(); - } - - void recordPoll(int batchSize, long duration) { - sourceRecordPoll.record(batchSize); - pollTime.record(duration); - activeRecordCount += batchSize; - sourceRecordActiveCount.record(activeRecordCount); - } - - void recordWrite(int recordCount) { - sourceRecordWrite.record(recordCount); - activeRecordCount -= recordCount; - activeRecordCount = Math.max(0, activeRecordCount); - sourceRecordActiveCount.record(activeRecordCount); - } - - protected MetricGroup metricGroup() { - return metricGroup; - } - } } diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSourceTaskContext.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSourceTaskContext.java index 5c1451fff14a..d58e98e05744 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSourceTaskContext.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSourceTaskContext.java @@ -19,30 +19,39 @@ import org.apache.kafka.connect.storage.ClusterConfigState; import org.apache.kafka.connect.source.SourceTaskContext; import org.apache.kafka.connect.storage.OffsetStorageReader; +import org.apache.kafka.connect.util.ConnectorTaskId; import java.util.Map; public class WorkerSourceTaskContext implements SourceTaskContext { private final OffsetStorageReader reader; - private final WorkerSourceTask task; + private final ConnectorTaskId id; private final ClusterConfigState configState; + private final WorkerTransactionContext transactionContext; public WorkerSourceTaskContext(OffsetStorageReader reader, - WorkerSourceTask task, - ClusterConfigState configState) { + ConnectorTaskId id, + ClusterConfigState configState, + WorkerTransactionContext transactionContext) { this.reader = reader; - this.task = task; + this.id = id; this.configState = configState; + this.transactionContext = transactionContext; } @Override public Map configs() { - return configState.taskConfig(task.id()); + return configState.taskConfig(id); } @Override public OffsetStorageReader offsetStorageReader() { return reader; } + + @Override + public WorkerTransactionContext transactionContext() { + return transactionContext; + } } diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerTask.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerTask.java index 0d893f56ee56..072e4b34a1eb 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerTask.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerTask.java @@ -55,7 +55,7 @@ abstract class WorkerTask implements Runnable { protected final ConnectorTaskId id; private final TaskStatus.Listener statusListener; protected final ClassLoader loader; - protected final StatusBackingStore statusBackingStore; + private final StatusBackingStore statusBackingStore; protected final Time time; private final CountDownLatch shutdownLatch = new CountDownLatch(1); private final TaskMetricsGroup taskMetricsGroup; diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerTransactionContext.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerTransactionContext.java new file mode 100644 index 000000000000..fde3f6944f72 --- /dev/null +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerTransactionContext.java @@ -0,0 +1,109 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.connect.runtime; + +import org.apache.kafka.connect.source.SourceRecord; +import org.apache.kafka.connect.source.TransactionContext; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.HashSet; +import java.util.Objects; +import java.util.Set; + +/** + * A {@link TransactionContext} that can be given to tasks and then queried by the worker to check on + * requests to abort and commit transactions. This class is thread safe and is designed to accommodate + * concurrent use without external synchronization. + */ +public class WorkerTransactionContext implements TransactionContext { + + private static final Logger log = LoggerFactory.getLogger(WorkerTransactionContext.class); + + private final Set commitableRecords = new HashSet<>(); + private final Set abortableRecords = new HashSet<>(); + private boolean batchCommitRequested = false; + private boolean batchAbortRequested = false; + + @Override + public synchronized void commitTransaction() { + batchCommitRequested = true; + } + + @Override + public synchronized void commitTransaction(SourceRecord record) { + Objects.requireNonNull(record, "Source record used to define transaction boundaries may not be null"); + commitableRecords.add(record); + } + + @Override + public synchronized void abortTransaction() { + batchAbortRequested = true; + } + + @Override + public synchronized void abortTransaction(SourceRecord record) { + Objects.requireNonNull(record, "Source record used to define transaction boundaries may not be null"); + abortableRecords.add(record); + } + + public synchronized boolean shouldCommitBatch() { + checkBatchRequestsConsistency(); + boolean result = batchCommitRequested; + batchCommitRequested = false; + return result; + } + + public synchronized boolean shouldAbortBatch() { + checkBatchRequestsConsistency(); + boolean result = batchAbortRequested; + batchAbortRequested = false; + return result; + } + + public synchronized boolean shouldCommitOn(SourceRecord record) { + // We could perform this check in the connector-facing methods (such as commitTransaction(SourceRecord)), + // but the connector might swallow that exception. + // This way, we can fail the task unconditionally, which is warranted since the alternative may lead to data loss. + // Essentially, instead of telling the task that it screwed up and trusting it to do the right thing, we rat on it to the + // worker and let it get punished accordingly. + checkRecordRequestConsistency(record); + return commitableRecords.remove(record); + } + + public synchronized boolean shouldAbortOn(SourceRecord record) { + checkRecordRequestConsistency(record); + return abortableRecords.remove(record); + } + + private void checkBatchRequestsConsistency() { + if (batchCommitRequested && batchAbortRequested) { + throw new IllegalStateException("Connector requested both commit and abort of same transaction"); + } + } + + private void checkRecordRequestConsistency(SourceRecord record) { + if (commitableRecords.contains(record) && abortableRecords.contains(record)) { + log.trace("Connector will fail as it has requested both commit and abort of transaction for same record: {}", record); + throw new IllegalStateException(String.format( + "Connector requested both commit and abort of same record against topic/partition %s/%s", + record.topic(), record.kafkaPartition() + )); + } + } + +} diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java index 8d269a3649eb..ded833da5991 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java @@ -1699,14 +1699,56 @@ private static Collection assignmentDifference(Collection update, Coll private boolean startTask(ConnectorTaskId taskId) { log.info("Starting task {}", taskId); - return worker.startTask( - taskId, - configState, - configState.connectorConfig(taskId.connector()), - configState.taskConfig(taskId), - this, - configState.targetState(taskId.connector()) - ); + Map connProps = configState.connectorConfig(taskId.connector()); + switch (connectorTypeForConfig(connProps)) { + case SINK: + return worker.startSinkTask( + taskId, + configState, + connProps, + configState.taskConfig(taskId), + this, + configState.targetState(taskId.connector()) + ); + case SOURCE: + if (config.exactlyOnceSourceEnabled()) { + int taskGeneration = configState.taskConfigGeneration(taskId.connector()); + return worker.startExactlyOnceSourceTask( + taskId, + configState, + connProps, + configState.taskConfig(taskId), + this, + configState.targetState(taskId.connector()), + () -> { + FutureCallback preflightFencing = new FutureCallback<>(); + fenceZombieSourceTasks(taskId, preflightFencing); + try { + preflightFencing.get(); + } catch (InterruptedException e) { + throw new ConnectException("Interrupted while attempting to perform round of zombie fencing", e); + } catch (ExecutionException e) { + Throwable cause = e.getCause(); + throw ConnectUtils.maybeWrap(cause, "Failed to perform round of zombie fencing"); + } + }, + () -> { + verifyTaskGenerationAndOwnership(taskId, taskGeneration); + } + ); + } else { + return worker.startSourceTask( + taskId, + configState, + connProps, + configState.taskConfig(taskId), + this, + configState.targetState(taskId.connector()) + ); + } + default: + throw new ConnectException("Failed to start task " + taskId + " since it is not a recognizable type (source or sink)"); + } } private Callable getTaskStartingCallable(final ConnectorTaskId taskId) { diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/standalone/StandaloneHerder.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/standalone/StandaloneHerder.java index f9e812610c3a..8afe3c7b3deb 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/standalone/StandaloneHerder.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/standalone/StandaloneHerder.java @@ -280,9 +280,8 @@ public synchronized void restartTask(ConnectorTaskId taskId, Callback cb) cb.onCompletion(new NotFoundException("Task " + taskId + " not found", null), null); Map connConfigProps = configState.connectorConfig(taskId.connector()); - TargetState targetState = configState.targetState(taskId.connector()); worker.stopAndAwaitTask(taskId); - if (worker.startTask(taskId, configState, connConfigProps, taskConfigProps, this, targetState)) + if (startTask(taskId, connConfigProps)) cb.onCompletion(null, null); else cb.onCompletion(new ConnectException("Failed to start task: " + taskId), null); @@ -377,11 +376,34 @@ private void createConnectorTasks(String connName) { } private void createConnectorTasks(String connName, Collection taskIds) { - TargetState initialState = configState.targetState(connName); Map connConfigs = configState.connectorConfig(connName); for (ConnectorTaskId taskId : taskIds) { - Map taskConfigMap = configState.taskConfig(taskId); - worker.startTask(taskId, configState, connConfigs, taskConfigMap, this, initialState); + startTask(taskId, connConfigs); + } + } + + private boolean startTask(ConnectorTaskId taskId, Map connProps) { + switch (connectorTypeForClass(connProps.get(ConnectorConfig.CONNECTOR_CLASS_CONFIG))) { + case SINK: + return worker.startSinkTask( + taskId, + configState, + connProps, + configState.taskConfig(taskId), + this, + configState.targetState(taskId.connector()) + ); + case SOURCE: + return worker.startSourceTask( + taskId, + configState, + connProps, + configState.taskConfig(taskId), + this, + configState.targetState(taskId.connector()) + ); + default: + throw new ConnectException("Failed to start task " + taskId + " since it is not a recognizable type (source or sink)"); } } diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/storage/ConnectorOffsetBackingStore.java b/connect/runtime/src/main/java/org/apache/kafka/connect/storage/ConnectorOffsetBackingStore.java new file mode 100644 index 000000000000..8f0f068bc09d --- /dev/null +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/storage/ConnectorOffsetBackingStore.java @@ -0,0 +1,71 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.connect.storage; + +import org.apache.kafka.connect.runtime.WorkerConfig; +import org.apache.kafka.connect.util.Callback; + +import java.nio.ByteBuffer; +import java.util.Collection; +import java.util.Map; +import java.util.concurrent.Future; + +public class ConnectorOffsetBackingStore implements OffsetBackingStore { + + private final OffsetBackingStore workerStore; + private final String primaryOffsetsTopic; + + public ConnectorOffsetBackingStore( + OffsetBackingStore workerStore, + String primaryOffsetsTopic + ) { + this.workerStore = workerStore; + this.primaryOffsetsTopic = primaryOffsetsTopic; + } + + public String primaryOffsetsTopic() { + return primaryOffsetsTopic; + } + + @Override + public void start() { + // TODO + } + + @Override + public void stop() { + // TODO + } + + @Override + public Future> get(Collection keys) { + // TODO + return workerStore.get(keys); + } + + @Override + public Future set(Map values, Callback callback) { + // TODO + return workerStore.set(values, callback); + } + + @Override + public void configure(WorkerConfig config) { + // TODO + } + +} diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/storage/OffsetStorageReaderImpl.java b/connect/runtime/src/main/java/org/apache/kafka/connect/storage/OffsetStorageReaderImpl.java index a1eea43103a3..49b9d69d8ebc 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/storage/OffsetStorageReaderImpl.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/storage/OffsetStorageReaderImpl.java @@ -141,6 +141,7 @@ public Map, Map> offsets(Collection partition, Map offset) { - data.put(partition, offset); + @SuppressWarnings("unchecked") + public synchronized void offset(Map partition, Map offset) { + data.put((Map) partition, (Map) offset); } private boolean flushing() { @@ -113,12 +114,18 @@ public synchronized boolean beginFlush() { if (data.isEmpty()) return false; - assert !flushing(); toFlush = data; data = new HashMap<>(); return true; } + /** + * @return whether there's anything to flush right now. + */ + public synchronized boolean willFlush() { + return !data.isEmpty(); + } + /** * Flush the current offsets and clear them from this writer. This is non-blocking: it * moves the current set of offsets out of the way, serializes the data, and asynchronously diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/AbstractWorkerSourceTaskTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/AbstractWorkerSourceTaskTest.java new file mode 100644 index 000000000000..d0833dbffc79 --- /dev/null +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/AbstractWorkerSourceTaskTest.java @@ -0,0 +1,842 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.connect.runtime; + +import org.apache.kafka.clients.admin.NewTopic; +import org.apache.kafka.clients.admin.TopicDescription; +import org.apache.kafka.clients.producer.Callback; +import org.apache.kafka.clients.producer.KafkaProducer; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.clients.producer.RecordMetadata; +import org.apache.kafka.common.InvalidRecordException; +import org.apache.kafka.common.MetricName; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.TopicPartitionInfo; +import org.apache.kafka.common.errors.TopicAuthorizationException; +import org.apache.kafka.common.header.Header; +import org.apache.kafka.common.header.Headers; +import org.apache.kafka.common.header.internals.RecordHeaders; +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.data.SchemaAndValue; +import org.apache.kafka.connect.errors.ConnectException; +import org.apache.kafka.connect.errors.RetriableException; +import org.apache.kafka.connect.header.ConnectHeaders; +import org.apache.kafka.connect.integration.MonitorableSourceConnector; +import org.apache.kafka.connect.runtime.errors.RetryWithToleranceOperatorTest; +import org.apache.kafka.connect.runtime.isolation.Plugins; +import org.apache.kafka.connect.runtime.standalone.StandaloneConfig; +import org.apache.kafka.connect.source.SourceRecord; +import org.apache.kafka.connect.source.SourceTask; +import org.apache.kafka.connect.storage.CloseableOffsetStorageReader; +import org.apache.kafka.connect.storage.ConnectorOffsetBackingStore; +import org.apache.kafka.connect.storage.Converter; +import org.apache.kafka.connect.storage.HeaderConverter; +import org.apache.kafka.connect.storage.OffsetStorageWriter; +import org.apache.kafka.connect.storage.StatusBackingStore; +import org.apache.kafka.connect.storage.StringConverter; +import org.apache.kafka.connect.util.ConnectorTaskId; +import org.apache.kafka.connect.util.TopicAdmin; +import org.apache.kafka.connect.util.TopicCreationGroup; +import org.easymock.Capture; +import org.easymock.EasyMock; +import org.easymock.IAnswer; +import org.easymock.IExpectationSetters; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.powermock.api.easymock.PowerMock; +import org.powermock.api.easymock.annotation.Mock; +import org.powermock.api.easymock.annotation.MockStrict; +import org.powermock.core.classloader.annotations.PowerMockIgnore; +import org.powermock.modules.junit4.PowerMockRunner; + +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.Future; +import java.util.concurrent.TimeoutException; + +import static org.apache.kafka.connect.integration.MonitorableSourceConnector.TOPIC_CONFIG; +import static org.apache.kafka.connect.runtime.ConnectorConfig.CONNECTOR_CLASS_CONFIG; +import static org.apache.kafka.connect.runtime.ConnectorConfig.KEY_CONVERTER_CLASS_CONFIG; +import static org.apache.kafka.connect.runtime.ConnectorConfig.TASKS_MAX_CONFIG; +import static org.apache.kafka.connect.runtime.ConnectorConfig.VALUE_CONVERTER_CLASS_CONFIG; +import static org.apache.kafka.connect.runtime.SourceConnectorConfig.TOPIC_CREATION_GROUPS_CONFIG; +import static org.apache.kafka.connect.runtime.TopicCreationConfig.DEFAULT_TOPIC_CREATION_PREFIX; +import static org.apache.kafka.connect.runtime.TopicCreationConfig.EXCLUDE_REGEX_CONFIG; +import static org.apache.kafka.connect.runtime.TopicCreationConfig.INCLUDE_REGEX_CONFIG; +import static org.apache.kafka.connect.runtime.TopicCreationConfig.PARTITIONS_CONFIG; +import static org.apache.kafka.connect.runtime.TopicCreationConfig.REPLICATION_FACTOR_CONFIG; +import static org.apache.kafka.connect.runtime.WorkerConfig.TOPIC_CREATION_ENABLE_CONFIG; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertThrows; +import static org.junit.Assert.assertTrue; + +@PowerMockIgnore({"javax.management.*", + "org.apache.log4j.*"}) +@RunWith(PowerMockRunner.class) +public class AbstractWorkerSourceTaskTest { + + private static final String TOPIC = "topic"; + private static final String OTHER_TOPIC = "other-topic"; + private static final Map PARTITION = Collections.singletonMap("key", "partition".getBytes()); + private static final Map OFFSET = Collections.singletonMap("key", 12); + + // Connect-format data + private static final Schema KEY_SCHEMA = Schema.INT32_SCHEMA; + private static final Integer KEY = -1; + private static final Schema RECORD_SCHEMA = Schema.INT64_SCHEMA; + private static final Long RECORD = 12L; + // Serialized data. The actual format of this data doesn't matter -- we just want to see that the right version + // is used in the right place. + private static final byte[] SERIALIZED_KEY = "converted-key".getBytes(); + private static final byte[] SERIALIZED_RECORD = "converted-record".getBytes(); + + @Mock private SourceTask sourceTask; + @Mock private TopicAdmin admin; + @Mock private KafkaProducer producer; + @Mock private Converter keyConverter; + @Mock private Converter valueConverter; + @Mock private HeaderConverter headerConverter; + @Mock private TransformationChain transformationChain; + @Mock private CloseableOffsetStorageReader offsetReader; + @Mock private OffsetStorageWriter offsetWriter; + @Mock private ConnectorOffsetBackingStore offsetStore; + @Mock private StatusBackingStore statusBackingStore; + @Mock private WorkerSourceTaskContext sourceTaskContext; + @MockStrict private TaskStatus.Listener statusListener; + + private final ConnectorTaskId taskId = new ConnectorTaskId("job", 0); + private final ConnectorTaskId taskId1 = new ConnectorTaskId("job", 1); + + private Plugins plugins; + private WorkerConfig config; + private SourceConnectorConfig sourceConfig; + private MockConnectMetrics metrics = new MockConnectMetrics(); + private Capture producerCallbacks; + + private AbstractWorkerSourceTask workerTask; + + @Before + public void setup() { + Map workerProps = workerProps(); + plugins = new Plugins(workerProps); + config = new StandaloneConfig(workerProps); + sourceConfig = new SourceConnectorConfig(plugins, sourceConnectorPropsWithGroups(TOPIC), true); + producerCallbacks = EasyMock.newCapture(); + metrics = new MockConnectMetrics(); + } + + private Map workerProps() { + Map props = new HashMap<>(); + props.put("key.converter", "org.apache.kafka.connect.json.JsonConverter"); + props.put("value.converter", "org.apache.kafka.connect.json.JsonConverter"); + props.put("offset.storage.file.filename", "/tmp/connect.offsets"); + props.put(TOPIC_CREATION_ENABLE_CONFIG, "true"); + return props; + } + + private Map sourceConnectorPropsWithGroups(String topic) { + // setup up props for the source connector + Map props = new HashMap<>(); + props.put("name", "foo-connector"); + props.put(CONNECTOR_CLASS_CONFIG, MonitorableSourceConnector.class.getSimpleName()); + props.put(TASKS_MAX_CONFIG, String.valueOf(1)); + props.put(TOPIC_CONFIG, topic); + props.put(KEY_CONVERTER_CLASS_CONFIG, StringConverter.class.getName()); + props.put(VALUE_CONVERTER_CLASS_CONFIG, StringConverter.class.getName()); + props.put(TOPIC_CREATION_GROUPS_CONFIG, String.join(",", "foo", "bar")); + props.put(DEFAULT_TOPIC_CREATION_PREFIX + REPLICATION_FACTOR_CONFIG, String.valueOf(1)); + props.put(DEFAULT_TOPIC_CREATION_PREFIX + PARTITIONS_CONFIG, String.valueOf(1)); + props.put(SourceConnectorConfig.TOPIC_CREATION_PREFIX + "foo" + "." + INCLUDE_REGEX_CONFIG, topic); + props.put(SourceConnectorConfig.TOPIC_CREATION_PREFIX + "bar" + "." + INCLUDE_REGEX_CONFIG, ".*"); + props.put(SourceConnectorConfig.TOPIC_CREATION_PREFIX + "bar" + "." + EXCLUDE_REGEX_CONFIG, topic); + return props; + } + + @After + public void tearDown() { + if (metrics != null) metrics.stop(); + } + + @Test + public void testMetricsGroup() { + AbstractWorkerSourceTask.SourceTaskMetricsGroup group = new AbstractWorkerSourceTask.SourceTaskMetricsGroup(taskId, metrics); + AbstractWorkerSourceTask.SourceTaskMetricsGroup group1 = new AbstractWorkerSourceTask.SourceTaskMetricsGroup(taskId1, metrics); + for (int i = 0; i != 10; ++i) { + group.recordPoll(100, 1000 + i * 100); + group.recordWrite(10); + } + for (int i = 0; i != 20; ++i) { + group1.recordPoll(100, 1000 + i * 100); + group1.recordWrite(10); + } + assertEquals(1900.0, metrics.currentMetricValueAsDouble(group.metricGroup(), "poll-batch-max-time-ms"), 0.001d); + assertEquals(1450.0, metrics.currentMetricValueAsDouble(group.metricGroup(), "poll-batch-avg-time-ms"), 0.001d); + assertEquals(33.333, metrics.currentMetricValueAsDouble(group.metricGroup(), "source-record-poll-rate"), 0.001d); + assertEquals(1000, metrics.currentMetricValueAsDouble(group.metricGroup(), "source-record-poll-total"), 0.001d); + assertEquals(3.3333, metrics.currentMetricValueAsDouble(group.metricGroup(), "source-record-write-rate"), 0.001d); + assertEquals(100, metrics.currentMetricValueAsDouble(group.metricGroup(), "source-record-write-total"), 0.001d); + assertEquals(900.0, metrics.currentMetricValueAsDouble(group.metricGroup(), "source-record-active-count"), 0.001d); + + // Close the group + group.close(); + + for (MetricName metricName : group.metricGroup().metrics().metrics().keySet()) { + // Metrics for this group should no longer exist + assertFalse(group.metricGroup().groupId().includes(metricName)); + } + // Sensors for this group should no longer exist + assertNull(group.metricGroup().metrics().getSensor("sink-record-read")); + assertNull(group.metricGroup().metrics().getSensor("sink-record-send")); + assertNull(group.metricGroup().metrics().getSensor("sink-record-active-count")); + assertNull(group.metricGroup().metrics().getSensor("partition-count")); + assertNull(group.metricGroup().metrics().getSensor("offset-seq-number")); + assertNull(group.metricGroup().metrics().getSensor("offset-commit-completion")); + assertNull(group.metricGroup().metrics().getSensor("offset-commit-completion-skip")); + assertNull(group.metricGroup().metrics().getSensor("put-batch-time")); + + assertEquals(2900.0, metrics.currentMetricValueAsDouble(group1.metricGroup(), "poll-batch-max-time-ms"), 0.001d); + assertEquals(1950.0, metrics.currentMetricValueAsDouble(group1.metricGroup(), "poll-batch-avg-time-ms"), 0.001d); + assertEquals(66.667, metrics.currentMetricValueAsDouble(group1.metricGroup(), "source-record-poll-rate"), 0.001d); + assertEquals(2000, metrics.currentMetricValueAsDouble(group1.metricGroup(), "source-record-poll-total"), 0.001d); + assertEquals(6.667, metrics.currentMetricValueAsDouble(group1.metricGroup(), "source-record-write-rate"), 0.001d); + assertEquals(200, metrics.currentMetricValueAsDouble(group1.metricGroup(), "source-record-write-total"), 0.001d); + assertEquals(1800.0, metrics.currentMetricValueAsDouble(group1.metricGroup(), "source-record-active-count"), 0.001d); + } + + @Test + public void testSendRecordsConvertsData() { + createWorkerTask(); + + List records = new ArrayList<>(); + // Can just use the same record for key and value + records.add(new SourceRecord(PARTITION, OFFSET, "topic", null, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD)); + + Capture> sent = expectSendRecordAnyTimes(); + + expectTopicCreation(TOPIC); + + PowerMock.replayAll(); + + workerTask.toSend = records; + workerTask.sendRecords(); + assertEquals(SERIALIZED_KEY, sent.getValue().key()); + assertEquals(SERIALIZED_RECORD, sent.getValue().value()); + + PowerMock.verifyAll(); + } + + @Test + public void testSendRecordsPropagatesTimestamp() { + final Long timestamp = System.currentTimeMillis(); + + createWorkerTask(); + + List records = Collections.singletonList( + new SourceRecord(PARTITION, OFFSET, "topic", null, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD, timestamp) + ); + + Capture> sent = expectSendRecordAnyTimes(); + + expectTopicCreation(TOPIC); + + PowerMock.replayAll(); + + workerTask.toSend = records; + workerTask.sendRecords(); + assertEquals(timestamp, sent.getValue().timestamp()); + + PowerMock.verifyAll(); + } + + @Test + public void testSendRecordsCorruptTimestamp() { + final Long timestamp = -3L; + createWorkerTask(); + + List records = Collections.singletonList( + new SourceRecord(PARTITION, OFFSET, "topic", null, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD, timestamp) + ); + + Capture> sent = expectSendRecordAnyTimes(); + + PowerMock.replayAll(); + + workerTask.toSend = records; + assertThrows(InvalidRecordException.class, workerTask::sendRecords); + assertFalse(sent.hasCaptured()); + + PowerMock.verifyAll(); + } + + @Test + public void testSendRecordsNoTimestamp() { + final Long timestamp = -1L; + createWorkerTask(); + + List records = Collections.singletonList( + new SourceRecord(PARTITION, OFFSET, "topic", null, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD, timestamp) + ); + + Capture> sent = expectSendRecordAnyTimes(); + + expectTopicCreation(TOPIC); + + PowerMock.replayAll(); + + workerTask.toSend = records; + workerTask.sendRecords(); + assertNull(sent.getValue().timestamp()); + + PowerMock.verifyAll(); + } + + @Test + public void testHeaders() { + Headers headers = new RecordHeaders(); + headers.add("header_key", "header_value".getBytes()); + + org.apache.kafka.connect.header.Headers connectHeaders = new ConnectHeaders(); + connectHeaders.add("header_key", new SchemaAndValue(Schema.STRING_SCHEMA, "header_value")); + + createWorkerTask(); + + List records = new ArrayList<>(); + records.add(new SourceRecord(PARTITION, OFFSET, TOPIC, null, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD, null, connectHeaders)); + + expectTopicCreation(TOPIC); + + Capture> sent = expectSendRecord(TOPIC, true, headers); + + PowerMock.replayAll(); + + workerTask.toSend = records; + workerTask.sendRecords(); + assertEquals(SERIALIZED_KEY, sent.getValue().key()); + assertEquals(SERIALIZED_RECORD, sent.getValue().value()); + assertEquals(headers, sent.getValue().headers()); + + PowerMock.verifyAll(); + } + + @Test + public void testHeadersWithCustomConverter() throws Exception { + StringConverter stringConverter = new StringConverter(); + SampleConverterWithHeaders testConverter = new SampleConverterWithHeaders(); + + createWorkerTask(stringConverter, testConverter, stringConverter); + + List records = new ArrayList<>(); + + String stringA = "Árvíztűrő tükörfúrógép"; + org.apache.kafka.connect.header.Headers headersA = new ConnectHeaders(); + String encodingA = "latin2"; + headersA.addString("encoding", encodingA); + + records.add(new SourceRecord(PARTITION, OFFSET, "topic", null, Schema.STRING_SCHEMA, "a", Schema.STRING_SCHEMA, stringA, null, headersA)); + + String stringB = "Тестовое сообщение"; + org.apache.kafka.connect.header.Headers headersB = new ConnectHeaders(); + String encodingB = "koi8_r"; + headersB.addString("encoding", encodingB); + + records.add(new SourceRecord(PARTITION, OFFSET, "topic", null, Schema.STRING_SCHEMA, "b", Schema.STRING_SCHEMA, stringB, null, headersB)); + + expectTopicCreation(TOPIC); + + Capture> sentRecordA = expectSendRecord(TOPIC, false, null); + Capture> sentRecordB = expectSendRecord(TOPIC, false, null); + + PowerMock.replayAll(); + + workerTask.toSend = records; + workerTask.sendRecords(); + + assertEquals(ByteBuffer.wrap("a".getBytes()), ByteBuffer.wrap(sentRecordA.getValue().key())); + assertEquals( + ByteBuffer.wrap(stringA.getBytes(encodingA)), + ByteBuffer.wrap(sentRecordA.getValue().value()) + ); + assertEquals(encodingA, new String(sentRecordA.getValue().headers().lastHeader("encoding").value())); + + assertEquals(ByteBuffer.wrap("b".getBytes()), ByteBuffer.wrap(sentRecordB.getValue().key())); + assertEquals( + ByteBuffer.wrap(stringB.getBytes(encodingB)), + ByteBuffer.wrap(sentRecordB.getValue().value()) + ); + assertEquals(encodingB, new String(sentRecordB.getValue().headers().lastHeader("encoding").value())); + + PowerMock.verifyAll(); + } + + @Test + public void testTopicCreateWhenTopicExists() { + createWorkerTask(); + + SourceRecord record1 = new SourceRecord(PARTITION, OFFSET, TOPIC, 1, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); + SourceRecord record2 = new SourceRecord(PARTITION, OFFSET, TOPIC, 2, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); + + expectPreliminaryCalls(); + TopicPartitionInfo topicPartitionInfo = new TopicPartitionInfo(0, null, Collections.emptyList(), Collections.emptyList()); + TopicDescription topicDesc = new TopicDescription(TOPIC, false, Collections.singletonList(topicPartitionInfo)); + EasyMock.expect(admin.describeTopics(TOPIC)).andReturn(Collections.singletonMap(TOPIC, topicDesc)); + + expectSendRecord(); + expectSendRecord(); + + PowerMock.replayAll(); + + workerTask.toSend = Arrays.asList(record1, record2); + workerTask.sendRecords(); + } + + @Test + public void testSendRecordsTopicDescribeRetries() { + createWorkerTask(); + + SourceRecord record1 = new SourceRecord(PARTITION, OFFSET, TOPIC, 1, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); + SourceRecord record2 = new SourceRecord(PARTITION, OFFSET, TOPIC, 2, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); + + expectPreliminaryCalls(); + // First round - call to describe the topic times out + EasyMock.expect(admin.describeTopics(TOPIC)) + .andThrow(new RetriableException(new TimeoutException("timeout"))); + + // Second round - calls to describe and create succeed + expectTopicCreation(TOPIC); + // Exactly two records are sent + expectSendRecord(); + expectSendRecord(); + + PowerMock.replayAll(); + + workerTask.toSend = Arrays.asList(record1, record2); + workerTask.sendRecords(); + assertEquals(Arrays.asList(record1, record2), workerTask.toSend); + + // Next they all succeed + workerTask.sendRecords(); + assertNull(workerTask.toSend); + } + + @Test + public void testSendRecordsTopicCreateRetries() { + createWorkerTask(); + + SourceRecord record1 = new SourceRecord(PARTITION, OFFSET, TOPIC, 1, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); + SourceRecord record2 = new SourceRecord(PARTITION, OFFSET, TOPIC, 2, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); + + // First call to describe the topic times out + expectPreliminaryCalls(); + EasyMock.expect(admin.describeTopics(TOPIC)).andReturn(Collections.emptyMap()); + Capture newTopicCapture = EasyMock.newCapture(); + EasyMock.expect(admin.createOrFindTopics(EasyMock.capture(newTopicCapture))) + .andThrow(new RetriableException(new TimeoutException("timeout"))); + + // Second round + expectTopicCreation(TOPIC); + expectSendRecord(); + expectSendRecord(); + + PowerMock.replayAll(); + + workerTask.toSend = Arrays.asList(record1, record2); + workerTask.sendRecords(); + assertEquals(Arrays.asList(record1, record2), workerTask.toSend); + + // Next they all succeed + workerTask.sendRecords(); + assertNull(workerTask.toSend); + } + + @Test + public void testSendRecordsTopicDescribeRetriesMidway() { + createWorkerTask(); + + // Differentiate only by Kafka partition so we can reuse conversion expectations + SourceRecord record1 = new SourceRecord(PARTITION, OFFSET, TOPIC, 1, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); + SourceRecord record2 = new SourceRecord(PARTITION, OFFSET, TOPIC, 2, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); + SourceRecord record3 = new SourceRecord(PARTITION, OFFSET, OTHER_TOPIC, 3, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); + + // First round + expectPreliminaryCalls(OTHER_TOPIC); + expectTopicCreation(TOPIC); + expectSendRecord(); + expectSendRecord(); + + // First call to describe the topic times out + EasyMock.expect(admin.describeTopics(OTHER_TOPIC)) + .andThrow(new RetriableException(new TimeoutException("timeout"))); + + // Second round + expectTopicCreation(OTHER_TOPIC); + expectSendRecord(OTHER_TOPIC, false, emptyHeaders()); + + PowerMock.replayAll(); + + // Try to send 3, make first pass, second fail. Should save last two + workerTask.toSend = Arrays.asList(record1, record2, record3); + workerTask.sendRecords(); + assertEquals(Arrays.asList(record3), workerTask.toSend); + + // Next they all succeed + workerTask.sendRecords(); + assertNull(workerTask.toSend); + + PowerMock.verifyAll(); + } + + @Test + public void testSendRecordsTopicCreateRetriesMidway() { + createWorkerTask(); + + // Differentiate only by Kafka partition so we can reuse conversion expectations + SourceRecord record1 = new SourceRecord(PARTITION, OFFSET, TOPIC, 1, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); + SourceRecord record2 = new SourceRecord(PARTITION, OFFSET, TOPIC, 2, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); + SourceRecord record3 = new SourceRecord(PARTITION, OFFSET, OTHER_TOPIC, 3, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); + + // First round + expectPreliminaryCalls(OTHER_TOPIC); + expectTopicCreation(TOPIC); + expectSendRecord(); + expectSendRecord(); + + EasyMock.expect(admin.describeTopics(OTHER_TOPIC)).andReturn(Collections.emptyMap()); + // First call to create the topic times out + Capture newTopicCapture = EasyMock.newCapture(); + EasyMock.expect(admin.createOrFindTopics(EasyMock.capture(newTopicCapture))) + .andThrow(new RetriableException(new TimeoutException("timeout"))); + + // Second round + expectTopicCreation(OTHER_TOPIC); + expectSendRecord(OTHER_TOPIC, false, emptyHeaders()); + + PowerMock.replayAll(); + + // Try to send 3, make first pass, second fail. Should save last two + workerTask.toSend = Arrays.asList(record1, record2, record3); + workerTask.sendRecords(); + assertEquals(Arrays.asList(record3), workerTask.toSend); + + // Next they all succeed + workerTask.sendRecords(); + assertNull(workerTask.toSend); + + PowerMock.verifyAll(); + } + + @Test + public void testTopicDescribeFails() { + createWorkerTask(); + + SourceRecord record1 = new SourceRecord(PARTITION, OFFSET, TOPIC, 1, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); + SourceRecord record2 = new SourceRecord(PARTITION, OFFSET, TOPIC, 2, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); + + expectPreliminaryCalls(); + EasyMock.expect(admin.describeTopics(TOPIC)) + .andThrow(new ConnectException(new TopicAuthorizationException("unauthorized"))); + + PowerMock.replayAll(); + + workerTask.toSend = Arrays.asList(record1, record2); + assertThrows(ConnectException.class, workerTask::sendRecords); + } + + @Test + public void testTopicCreateFails() { + createWorkerTask(); + + SourceRecord record1 = new SourceRecord(PARTITION, OFFSET, TOPIC, 1, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); + SourceRecord record2 = new SourceRecord(PARTITION, OFFSET, TOPIC, 2, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); + + expectPreliminaryCalls(); + EasyMock.expect(admin.describeTopics(TOPIC)).andReturn(Collections.emptyMap()); + + Capture newTopicCapture = EasyMock.newCapture(); + EasyMock.expect(admin.createOrFindTopics(EasyMock.capture(newTopicCapture))) + .andThrow(new ConnectException(new TopicAuthorizationException("unauthorized"))); + + PowerMock.replayAll(); + + workerTask.toSend = Arrays.asList(record1, record2); + assertThrows(ConnectException.class, workerTask::sendRecords); + assertTrue(newTopicCapture.hasCaptured()); + } + + @Test + public void testTopicCreateFailsWithExceptionWhenCreateReturnsTopicNotCreatedOrFound() { + createWorkerTask(); + + SourceRecord record1 = new SourceRecord(PARTITION, OFFSET, TOPIC, 1, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); + SourceRecord record2 = new SourceRecord(PARTITION, OFFSET, TOPIC, 2, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); + + expectPreliminaryCalls(); + EasyMock.expect(admin.describeTopics(TOPIC)).andReturn(Collections.emptyMap()); + + Capture newTopicCapture = EasyMock.newCapture(); + EasyMock.expect(admin.createOrFindTopics(EasyMock.capture(newTopicCapture))).andReturn(TopicAdmin.EMPTY_CREATION); + + PowerMock.replayAll(); + + workerTask.toSend = Arrays.asList(record1, record2); + assertThrows(ConnectException.class, workerTask::sendRecords); + assertTrue(newTopicCapture.hasCaptured()); + } + + @Test + public void testTopicCreateSucceedsWhenCreateReturnsExistingTopicFound() { + createWorkerTask(); + + SourceRecord record1 = new SourceRecord(PARTITION, OFFSET, TOPIC, 1, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); + SourceRecord record2 = new SourceRecord(PARTITION, OFFSET, TOPIC, 2, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); + + expectPreliminaryCalls(); + EasyMock.expect(admin.describeTopics(TOPIC)).andReturn(Collections.emptyMap()); + + Capture newTopicCapture = EasyMock.newCapture(); + EasyMock.expect(admin.createOrFindTopics(EasyMock.capture(newTopicCapture))).andReturn(foundTopic(TOPIC)); + + expectSendRecord(); + expectSendRecord(); + + PowerMock.replayAll(); + + workerTask.toSend = Arrays.asList(record1, record2); + workerTask.sendRecords(); + } + + @Test + public void testTopicCreateSucceedsWhenCreateReturnsNewTopicFound() { + createWorkerTask(); + + SourceRecord record1 = new SourceRecord(PARTITION, OFFSET, TOPIC, 1, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); + SourceRecord record2 = new SourceRecord(PARTITION, OFFSET, TOPIC, 2, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); + + expectPreliminaryCalls(); + EasyMock.expect(admin.describeTopics(TOPIC)).andReturn(Collections.emptyMap()); + + Capture newTopicCapture = EasyMock.newCapture(); + EasyMock.expect(admin.createOrFindTopics(EasyMock.capture(newTopicCapture))).andReturn(createdTopic(TOPIC)); + + expectSendRecord(); + expectSendRecord(); + + PowerMock.replayAll(); + + workerTask.toSend = Arrays.asList(record1, record2); + workerTask.sendRecords(); + } + + private Capture> expectSendRecord( + String topic, + boolean anyTimes, + Headers headers + ) { + if (headers != null) + expectConvertHeadersAndKeyValue(topic, anyTimes, headers); + + expectApplyTransformationChain(anyTimes); + + Capture> sent = EasyMock.newCapture(); + + IExpectationSetters> expect = EasyMock.expect( + producer.send(EasyMock.capture(sent), EasyMock.capture(producerCallbacks))); + + IAnswer> expectResponse = () -> { + synchronized (producerCallbacks) { + for (Callback cb : producerCallbacks.getValues()) { + cb.onCompletion(new RecordMetadata(new TopicPartition("foo", 0), 0, 0, 0L, 0, 0), null); + } + producerCallbacks.reset(); + } + return null; + }; + + if (anyTimes) + expect.andStubAnswer(expectResponse); + else + expect.andAnswer(expectResponse); + + expectTaskGetTopic(anyTimes); + + return sent; + } + + private Capture> expectSendRecordAnyTimes() { + return expectSendRecord(TOPIC, true, emptyHeaders()); + } + + private Capture> expectSendRecord() { + return expectSendRecord(TOPIC, false, emptyHeaders()); + } + + private void expectTaskGetTopic(boolean anyTimes) { + final Capture connectorCapture = EasyMock.newCapture(); + final Capture topicCapture = EasyMock.newCapture(); + IExpectationSetters expect = EasyMock.expect(statusBackingStore.getTopic( + EasyMock.capture(connectorCapture), + EasyMock.capture(topicCapture))); + if (anyTimes) { + expect.andStubAnswer(() -> new TopicStatus( + topicCapture.getValue(), + new ConnectorTaskId(connectorCapture.getValue(), 0), + Time.SYSTEM.milliseconds())); + } else { + expect.andAnswer(() -> new TopicStatus( + topicCapture.getValue(), + new ConnectorTaskId(connectorCapture.getValue(), 0), + Time.SYSTEM.milliseconds())); + } + if (connectorCapture.hasCaptured() && topicCapture.hasCaptured()) { + assertEquals("job", connectorCapture.getValue()); + assertEquals(TOPIC, topicCapture.getValue()); + } + } + + private void expectTopicCreation(String topic) { + if (config.topicCreationEnable()) { + EasyMock.expect(admin.describeTopics(topic)).andReturn(Collections.emptyMap()); + Capture newTopicCapture = EasyMock.newCapture(); + EasyMock.expect(admin.createOrFindTopics(EasyMock.capture(newTopicCapture))).andReturn(createdTopic(topic)); + } + } + + private TopicAdmin.TopicCreationResponse createdTopic(String topic) { + Set created = Collections.singleton(topic); + Set existing = Collections.emptySet(); + return new TopicAdmin.TopicCreationResponse(created, existing); + } + + private TopicAdmin.TopicCreationResponse foundTopic(String topic) { + Set created = Collections.emptySet(); + Set existing = Collections.singleton(topic); + return new TopicAdmin.TopicCreationResponse(created, existing); + } + + private void expectPreliminaryCalls() { + expectPreliminaryCalls(TOPIC); + } + + private void expectPreliminaryCalls(String topic) { + expectConvertHeadersAndKeyValue(topic, true, emptyHeaders()); + expectApplyTransformationChain(false); + PowerMock.expectLastCall(); + } + + private void expectConvertHeadersAndKeyValue(String topic, boolean anyTimes, Headers headers) { + for (Header header : headers) { + IExpectationSetters convertHeaderExpect = EasyMock.expect(headerConverter.fromConnectHeader(topic, header.key(), Schema.STRING_SCHEMA, new String(header.value()))); + if (anyTimes) + convertHeaderExpect.andStubReturn(header.value()); + else + convertHeaderExpect.andReturn(header.value()); + } + IExpectationSetters convertKeyExpect = EasyMock.expect(keyConverter.fromConnectData(topic, headers, KEY_SCHEMA, KEY)); + if (anyTimes) + convertKeyExpect.andStubReturn(SERIALIZED_KEY); + else + convertKeyExpect.andReturn(SERIALIZED_KEY); + IExpectationSetters convertValueExpect = EasyMock.expect(valueConverter.fromConnectData(topic, headers, RECORD_SCHEMA, RECORD)); + if (anyTimes) + convertValueExpect.andStubReturn(SERIALIZED_RECORD); + else + convertValueExpect.andReturn(SERIALIZED_RECORD); + } + + private void expectApplyTransformationChain(boolean anyTimes) { + final Capture recordCapture = EasyMock.newCapture(); + IExpectationSetters convertKeyExpect = EasyMock.expect(transformationChain.apply(EasyMock.capture(recordCapture))); + if (anyTimes) + convertKeyExpect.andStubAnswer(recordCapture::getValue); + else + convertKeyExpect.andAnswer(recordCapture::getValue); + } + + private RecordHeaders emptyHeaders() { + return new RecordHeaders(); + } + + private void createWorkerTask() { + createWorkerTask(keyConverter, valueConverter, headerConverter); + } + + private void createWorkerTask(Converter keyConverter, Converter valueConverter, HeaderConverter headerConverter) { + workerTask = new AbstractWorkerSourceTask( + taskId, sourceTask, statusListener, TargetState.STARTED, keyConverter, valueConverter, headerConverter, transformationChain, + sourceTaskContext, producer, admin, TopicCreationGroup.configuredGroups(sourceConfig), offsetReader, offsetWriter, offsetStore, + config, metrics, plugins.delegatingLoader(), Time.SYSTEM, RetryWithToleranceOperatorTest.NOOP_OPERATOR, + statusBackingStore, Runnable::run) { + @Override + protected void prepareToInitializeTask() { + } + + @Override + protected void prepareToEnterSendLoop() { + } + + @Override + protected void beginSendIteration() { + } + + @Override + protected void prepareToPollTask() { + } + + @Override + protected void recordDropped(SourceRecord record) { + } + + @Override + protected Optional prepareToSendRecord(SourceRecord sourceRecord, ProducerRecord producerRecord) { + return Optional.empty(); + } + + @Override + protected void recordDispatched(SourceRecord record) { + } + + @Override + protected void batchDispatched() { + } + + @Override + protected void recordSent(SourceRecord sourceRecord, ProducerRecord producerRecord, RecordMetadata recordMetadata) { + } + + @Override + protected void producerSendFailed(boolean synchronous, ProducerRecord producerRecord, SourceRecord preTransformRecord, Exception e) { + } + + @Override + protected void finalOffsetCommit(boolean failed) { + } + }; + + } + +} diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/ErrorHandlingTaskTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/ErrorHandlingTaskTest.java index e222894945fb..be26149d2743 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/ErrorHandlingTaskTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/ErrorHandlingTaskTest.java @@ -48,6 +48,7 @@ import org.apache.kafka.connect.sink.SinkTask; import org.apache.kafka.connect.source.SourceRecord; import org.apache.kafka.connect.source.SourceTask; +import org.apache.kafka.connect.storage.ConnectorOffsetBackingStore; import org.apache.kafka.connect.storage.Converter; import org.apache.kafka.connect.storage.HeaderConverter; import org.apache.kafka.connect.storage.OffsetStorageReaderImpl; @@ -160,6 +161,8 @@ public class ErrorHandlingTaskTest { OffsetStorageReaderImpl offsetReader; @Mock OffsetStorageWriter offsetWriter; + @Mock + private ConnectorOffsetBackingStore offsetStore; private Capture rebalanceListener = EasyMock.newCapture(); @SuppressWarnings("unused") @@ -530,6 +533,12 @@ private void expectClose() { admin.close(EasyMock.anyObject(Duration.class)); EasyMock.expectLastCall(); + + offsetReader.close(); + EasyMock.expectLastCall(); + + offsetStore.stop(); + EasyMock.expectLastCall(); } private void expectTopicCreation(String topic) { @@ -590,7 +599,7 @@ private void createSourceTask(TargetState initialState, RetryWithToleranceOperat WorkerSourceTask.class, new String[]{"commitOffsets", "isStopping"}, taskId, sourceTask, statusListener, initialState, converter, converter, headerConverter, sourceTransforms, producer, admin, TopicCreationGroup.configuredGroups(sourceConfig), - offsetReader, offsetWriter, workerConfig, + offsetReader, offsetWriter, offsetStore, workerConfig, ClusterConfigState.EMPTY, metrics, pluginLoader, time, retryWithToleranceOperator, statusBackingStore, (Executor) Runnable::run); } diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/ExactlyOnceWorkerSourceTaskTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/ExactlyOnceWorkerSourceTaskTest.java new file mode 100644 index 000000000000..5cf34cb9113e --- /dev/null +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/ExactlyOnceWorkerSourceTaskTest.java @@ -0,0 +1,1322 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.connect.runtime; + +import org.apache.kafka.clients.admin.NewTopic; +import org.apache.kafka.clients.producer.KafkaProducer; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.clients.producer.RecordMetadata; +import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.errors.InvalidTopicException; +import org.apache.kafka.common.errors.RecordTooLargeException; +import org.apache.kafka.common.errors.TopicAuthorizationException; +import org.apache.kafka.common.header.Header; +import org.apache.kafka.common.header.Headers; +import org.apache.kafka.common.header.internals.RecordHeaders; +import org.apache.kafka.common.utils.MockTime; +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.errors.ConnectException; +import org.apache.kafka.connect.integration.MonitorableSourceConnector; +import org.apache.kafka.connect.runtime.ConnectMetrics.MetricGroup; +import org.apache.kafka.connect.runtime.errors.RetryWithToleranceOperatorTest; +import org.apache.kafka.connect.runtime.isolation.Plugins; +import org.apache.kafka.connect.runtime.standalone.StandaloneConfig; +import org.apache.kafka.connect.source.SourceRecord; +import org.apache.kafka.connect.source.SourceTask; +import org.apache.kafka.connect.source.SourceTaskContext; +import org.apache.kafka.connect.source.TransactionContext; +import org.apache.kafka.connect.storage.CloseableOffsetStorageReader; +import org.apache.kafka.connect.storage.ClusterConfigState; +import org.apache.kafka.connect.storage.ConnectorOffsetBackingStore; +import org.apache.kafka.connect.storage.Converter; +import org.apache.kafka.connect.storage.HeaderConverter; +import org.apache.kafka.connect.storage.OffsetStorageWriter; +import org.apache.kafka.connect.storage.StatusBackingStore; +import org.apache.kafka.connect.storage.StringConverter; +import org.apache.kafka.connect.util.Callback; +import org.apache.kafka.connect.util.ConnectorTaskId; +import org.apache.kafka.connect.util.ParameterizedTest; +import org.apache.kafka.connect.util.ThreadedTest; +import org.apache.kafka.connect.util.TopicAdmin; +import org.apache.kafka.connect.util.TopicCreationGroup; +import org.easymock.Capture; +import org.easymock.EasyMock; +import org.easymock.IAnswer; +import org.easymock.IExpectationSetters; +import org.junit.After; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.powermock.api.easymock.PowerMock; +import org.powermock.api.easymock.annotation.Mock; +import org.powermock.api.easymock.annotation.MockStrict; +import org.powermock.core.classloader.annotations.PowerMockIgnore; +import org.powermock.modules.junit4.PowerMockRunner; +import org.powermock.modules.junit4.PowerMockRunnerDelegate; + +import java.time.Duration; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.stream.Collectors; +import java.util.stream.IntStream; + +import static org.apache.kafka.connect.integration.MonitorableSourceConnector.TOPIC_CONFIG; +import static org.apache.kafka.connect.runtime.ConnectorConfig.CONNECTOR_CLASS_CONFIG; +import static org.apache.kafka.connect.runtime.ConnectorConfig.KEY_CONVERTER_CLASS_CONFIG; +import static org.apache.kafka.connect.runtime.ConnectorConfig.TASKS_MAX_CONFIG; +import static org.apache.kafka.connect.runtime.ConnectorConfig.VALUE_CONVERTER_CLASS_CONFIG; +import static org.apache.kafka.connect.runtime.SourceConnectorConfig.TOPIC_CREATION_GROUPS_CONFIG; +import static org.apache.kafka.connect.runtime.SourceConnectorConfig.TRANSACTION_BOUNDARY_CONFIG; +import static org.apache.kafka.connect.runtime.SourceConnectorConfig.TRANSACTION_BOUNDARY_INTERVAL_CONFIG; +import static org.apache.kafka.connect.runtime.TopicCreationConfig.DEFAULT_TOPIC_CREATION_PREFIX; +import static org.apache.kafka.connect.runtime.TopicCreationConfig.EXCLUDE_REGEX_CONFIG; +import static org.apache.kafka.connect.runtime.TopicCreationConfig.INCLUDE_REGEX_CONFIG; +import static org.apache.kafka.connect.runtime.TopicCreationConfig.PARTITIONS_CONFIG; +import static org.apache.kafka.connect.runtime.TopicCreationConfig.REPLICATION_FACTOR_CONFIG; +import static org.apache.kafka.connect.runtime.WorkerConfig.TOPIC_CREATION_ENABLE_CONFIG; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertThrows; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +@PowerMockIgnore({"javax.management.*", + "org.apache.log4j.*"}) +@RunWith(PowerMockRunner.class) +@PowerMockRunnerDelegate(ParameterizedTest.class) +public class ExactlyOnceWorkerSourceTaskTest extends ThreadedTest { + private static final String TOPIC = "topic"; + private static final Map PARTITION = Collections.singletonMap("key", "partition".getBytes()); + private static final Map OFFSET = Collections.singletonMap("key", 12); + + // Connect-format data + private static final Schema KEY_SCHEMA = Schema.INT32_SCHEMA; + private static final Integer KEY = -1; + private static final Schema RECORD_SCHEMA = Schema.INT64_SCHEMA; + private static final Long RECORD = 12L; + // Serialized data. The actual format of this data doesn't matter -- we just want to see that the right version + // is used in the right place. + private static final byte[] SERIALIZED_KEY = "converted-key".getBytes(); + private static final byte[] SERIALIZED_RECORD = "converted-record".getBytes(); + + private final ExecutorService executor = Executors.newSingleThreadExecutor(); + private final ConnectorTaskId taskId = new ConnectorTaskId("job", 0); + private WorkerConfig config; + private SourceConnectorConfig sourceConfig; + private Plugins plugins; + private MockConnectMetrics metrics; + private Time time; + private CountDownLatch pollLatch; + @Mock private SourceTask sourceTask; + @Mock private Converter keyConverter; + @Mock private Converter valueConverter; + @Mock private HeaderConverter headerConverter; + @Mock private TransformationChain transformationChain; + @Mock private KafkaProducer producer; + @Mock private TopicAdmin admin; + @Mock private CloseableOffsetStorageReader offsetReader; + @Mock private OffsetStorageWriter offsetWriter; + @Mock private ClusterConfigState clusterConfigState; + private ExactlyOnceWorkerSourceTask workerTask; + @Mock private Future sendFuture; + @MockStrict private TaskStatus.Listener statusListener; + @Mock private StatusBackingStore statusBackingStore; + @Mock private ConnectorOffsetBackingStore offsetStore; + @Mock private Runnable preProducerCheck; + @Mock private Runnable postProducerCheck; + + private Capture producerCallbacks; + + private static final Map TASK_PROPS = new HashMap<>(); + static { + TASK_PROPS.put(TaskConfig.TASK_CLASS_CONFIG, TestSourceTask.class.getName()); + } + private static final TaskConfig TASK_CONFIG = new TaskConfig(TASK_PROPS); + + private static final SourceRecord SOURCE_RECORD = + new SourceRecord(PARTITION, OFFSET, "topic", null, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); + + private static final List RECORDS = Collections.singletonList(SOURCE_RECORD); + + private final boolean enableTopicCreation; + + @ParameterizedTest.Parameters + public static Collection parameters() { + return Arrays.asList(false, true); + } + + public ExactlyOnceWorkerSourceTaskTest(boolean enableTopicCreation) { + this.enableTopicCreation = enableTopicCreation; + } + + @Override + public void setup() { + super.setup(); + Map workerProps = workerProps(); + plugins = new Plugins(workerProps); + config = new StandaloneConfig(workerProps); + sourceConfig = new SourceConnectorConfig(plugins, sourceConnectorProps(), true); + producerCallbacks = EasyMock.newCapture(); + metrics = new MockConnectMetrics(); + time = Time.SYSTEM; + EasyMock.expect(offsetStore.primaryOffsetsTopic()).andStubReturn("offsets-topic"); + pollLatch = new CountDownLatch(1); + } + + private Map workerProps() { + Map props = new HashMap<>(); + props.put("key.converter", "org.apache.kafka.connect.json.JsonConverter"); + props.put("value.converter", "org.apache.kafka.connect.json.JsonConverter"); + props.put("internal.key.converter", "org.apache.kafka.connect.json.JsonConverter"); + props.put("internal.value.converter", "org.apache.kafka.connect.json.JsonConverter"); + props.put("internal.key.converter.schemas.enable", "false"); + props.put("internal.value.converter.schemas.enable", "false"); + props.put("offset.storage.file.filename", "/tmp/connect.offsets"); + props.put(TOPIC_CREATION_ENABLE_CONFIG, String.valueOf(enableTopicCreation)); + return props; + } + + private Map sourceConnectorProps() { + return sourceConnectorProps(SourceTask.TransactionBoundary.DEFAULT); + } + + private Map sourceConnectorProps(SourceTask.TransactionBoundary transactionBoundary) { + // setup up props for the source connector + Map props = new HashMap<>(); + props.put("name", "foo-connector"); + props.put(CONNECTOR_CLASS_CONFIG, MonitorableSourceConnector.class.getSimpleName()); + props.put(TASKS_MAX_CONFIG, String.valueOf(1)); + props.put(TOPIC_CONFIG, TOPIC); + props.put(KEY_CONVERTER_CLASS_CONFIG, StringConverter.class.getName()); + props.put(VALUE_CONVERTER_CLASS_CONFIG, StringConverter.class.getName()); + props.put(TOPIC_CREATION_GROUPS_CONFIG, String.join(",", "foo", "bar")); + props.put(DEFAULT_TOPIC_CREATION_PREFIX + REPLICATION_FACTOR_CONFIG, String.valueOf(1)); + props.put(DEFAULT_TOPIC_CREATION_PREFIX + PARTITIONS_CONFIG, String.valueOf(1)); + props.put(TRANSACTION_BOUNDARY_CONFIG, transactionBoundary.toString()); + props.put(SourceConnectorConfig.TOPIC_CREATION_PREFIX + "foo" + "." + INCLUDE_REGEX_CONFIG, TOPIC); + props.put(SourceConnectorConfig.TOPIC_CREATION_PREFIX + "bar" + "." + INCLUDE_REGEX_CONFIG, ".*"); + props.put(SourceConnectorConfig.TOPIC_CREATION_PREFIX + "bar" + "." + EXCLUDE_REGEX_CONFIG, TOPIC); + return props; + } + + @After + public void tearDown() { + if (metrics != null) metrics.stop(); + } + + private void createWorkerTask() { + createWorkerTask(TargetState.STARTED); + } + + private void createWorkerTask(TargetState initialState) { + createWorkerTask(initialState, keyConverter, valueConverter, headerConverter); + } + + private void createWorkerTask(TargetState initialState, Converter keyConverter, Converter valueConverter, HeaderConverter headerConverter) { + workerTask = new ExactlyOnceWorkerSourceTask(taskId, sourceTask, statusListener, initialState, keyConverter, valueConverter, headerConverter, + transformationChain, producer, admin, TopicCreationGroup.configuredGroups(sourceConfig), offsetReader, offsetWriter, offsetStore, + config, clusterConfigState, metrics, plugins.delegatingLoader(), time, RetryWithToleranceOperatorTest.NOOP_OPERATOR, statusBackingStore, + sourceConfig, Runnable::run, preProducerCheck, postProducerCheck); + } + + @Test + public void testStartPaused() throws Exception { + final CountDownLatch pauseLatch = new CountDownLatch(1); + + createWorkerTask(TargetState.PAUSED); + + expectCall(() -> statusListener.onPause(taskId)).andAnswer(() -> { + pauseLatch.countDown(); + return null; + }); + + // The task checks to see if there are offsets to commit before pausing + EasyMock.expect(offsetWriter.willFlush()).andReturn(false); + + expectClose(); + + expectCall(() -> statusListener.onShutdown(taskId)); + + PowerMock.replayAll(); + + workerTask.initialize(TASK_CONFIG); + Future taskFuture = executor.submit(workerTask); + + assertTrue(pauseLatch.await(5, TimeUnit.SECONDS)); + workerTask.stop(); + assertTrue(workerTask.awaitStop(1000)); + + taskFuture.get(); + + PowerMock.verifyAll(); + } + + @Test + public void testPause() throws Exception { + createWorkerTask(); + + expectPreflight(); + expectStartup(); + + AtomicInteger polls = new AtomicInteger(0); + AtomicInteger flushes = new AtomicInteger(0); + pollLatch = new CountDownLatch(10); + expectPolls(polls); + expectAnyFlushes(flushes); + + expectTopicCreation(TOPIC); + + expectCall(() -> statusListener.onPause(taskId)); + + expectCall(sourceTask::stop); + expectCall(() -> statusListener.onShutdown(taskId)); + + expectClose(); + + PowerMock.replayAll(); + + workerTask.initialize(TASK_CONFIG); + Future taskFuture = executor.submit(workerTask); + assertTrue(awaitLatch(pollLatch)); + + workerTask.transitionTo(TargetState.PAUSED); + + int priorCount = polls.get(); + Thread.sleep(100); + + // since the transition is observed asynchronously, the count could be off by one loop iteration + assertTrue(polls.get() - priorCount <= 1); + + workerTask.stop(); + assertTrue(workerTask.awaitStop(1000)); + + taskFuture.get(); + + assertEquals("Task should have flushed offsets for every record poll, once on pause, and once for end-of-life offset commit", + flushes.get(), polls.get() + 2); + + PowerMock.verifyAll(); + } + + @Test + public void testFailureInPreProducerCheck() { + createWorkerTask(); + + Exception exception = new ConnectException("Failed to perform zombie fencing"); + expectCall(preProducerCheck::run).andThrow(exception); + + expectCall(() -> statusListener.onFailure(taskId, exception)); + + // Don't expect task to be stopped since it was never started to begin with + + expectClose(); + + PowerMock.replayAll(); + + workerTask.initialize(TASK_CONFIG); + // No need to execute on a separate thread; preflight checks should all take place before the poll-send loop starts + workerTask.run(); + + PowerMock.verifyAll(); + } + + @Test + public void testFailureInOffsetStoreStart() { + createWorkerTask(); + + expectCall(preProducerCheck::run); + Exception exception = new ConnectException("No soup for you!"); + expectCall(offsetStore::start).andThrow(exception); + + expectCall(() -> statusListener.onFailure(taskId, exception)); + + // Don't expect task to be stopped since it was never started to begin with + + expectClose(); + + PowerMock.replayAll(); + + workerTask.initialize(TASK_CONFIG); + // No need to execute on a separate thread; preflight checks should all take place before the poll-send loop starts + workerTask.run(); + + PowerMock.verifyAll(); + } + + @Test + public void testFailureInProducerInitialization() { + createWorkerTask(); + + expectCall(preProducerCheck::run); + expectCall(offsetStore::start); + expectCall(producer::initTransactions); + Exception exception = new ConnectException("You can't do that!"); + expectCall(postProducerCheck::run).andThrow(exception); + + expectCall(() -> statusListener.onFailure(taskId, exception)); + + // Don't expect task to be stopped since it was never started to begin with + + expectClose(); + + PowerMock.replayAll(); + + workerTask.initialize(TASK_CONFIG); + // No need to execute on a separate thread; preflight checks should all take place before the poll-send loop starts + workerTask.run(); + + PowerMock.verifyAll(); + } + + @Test + public void testFailureInPostProducerCheck() { + createWorkerTask(); + + expectCall(preProducerCheck::run); + expectCall(offsetStore::start); + Exception exception = new ConnectException("New task configs for the connector have already been generated"); + expectCall(producer::initTransactions).andThrow(exception); + + expectCall(() -> statusListener.onFailure(taskId, exception)); + + // Don't expect task to be stopped since it was never started to begin with + + expectClose(); + + PowerMock.replayAll(); + + workerTask.initialize(TASK_CONFIG); + // No need to execute on a separate thread; preflight checks should all take place before the poll-send loop starts + workerTask.run(); + + PowerMock.verifyAll(); + } + + @Test + public void testPollsInBackground() throws Exception { + createWorkerTask(); + + expectPreflight(); + expectStartup(); + + AtomicInteger polls = new AtomicInteger(0); + AtomicInteger flushes = new AtomicInteger(0); + pollLatch = new CountDownLatch(10); + expectPolls(polls); + expectAnyFlushes(flushes); + + expectTopicCreation(TOPIC); + + expectCall(sourceTask::stop); + expectCall(() -> statusListener.onShutdown(taskId)); + + expectClose(); + + PowerMock.replayAll(); + + workerTask.initialize(TASK_CONFIG); + Future taskFuture = executor.submit(workerTask); + + assertTrue(awaitLatch(pollLatch)); + workerTask.stop(); + assertTrue(workerTask.awaitStop(1000)); + + taskFuture.get(); + assertPollMetrics(10); + assertTransactionMetrics(1); + + assertEquals("Task should have flushed offsets for every record poll and for end-of-life offset commit", + flushes.get(), polls.get() + 1); + + PowerMock.verifyAll(); + } + + @Test + public void testFailureInPoll() throws Exception { + createWorkerTask(); + + expectPreflight(); + expectStartup(); + + final CountDownLatch pollLatch = new CountDownLatch(1); + final RuntimeException exception = new RuntimeException(); + EasyMock.expect(sourceTask.poll()).andAnswer(() -> { + pollLatch.countDown(); + throw exception; + }); + + expectCall(() -> statusListener.onFailure(taskId, exception)); + expectCall(sourceTask::stop); + + expectClose(); + + PowerMock.replayAll(); + + workerTask.initialize(TASK_CONFIG); + Future taskFuture = executor.submit(workerTask); + + assertTrue(awaitLatch(pollLatch)); + //Failure in poll should trigger automatic stop of the worker + assertTrue(workerTask.awaitStop(1000)); + + taskFuture.get(); + assertPollMetrics(0); + + PowerMock.verifyAll(); + } + + @Test + public void testFailureInPollAfterCancel() throws Exception { + createWorkerTask(); + + expectPreflight(); + expectStartup(); + + final CountDownLatch pollLatch = new CountDownLatch(1); + final CountDownLatch workerCancelLatch = new CountDownLatch(1); + final RuntimeException exception = new RuntimeException(); + EasyMock.expect(sourceTask.poll()).andAnswer(() -> { + pollLatch.countDown(); + assertTrue(awaitLatch(workerCancelLatch)); + throw exception; + }); + + expectCall(offsetReader::close); + expectCall(() -> producer.close(Duration.ZERO)); + expectCall(sourceTask::stop); + + expectClose(); + + PowerMock.replayAll(); + + workerTask.initialize(TASK_CONFIG); + Future taskFuture = executor.submit(workerTask); + + assertTrue(awaitLatch(pollLatch)); + workerTask.cancel(); + workerCancelLatch.countDown(); + assertTrue(workerTask.awaitStop(1000)); + + taskFuture.get(); + assertPollMetrics(0); + + PowerMock.verifyAll(); + } + + @Test + public void testFailureInPollAfterStop() throws Exception { + createWorkerTask(); + + expectPreflight(); + expectStartup(); + + final CountDownLatch pollLatch = new CountDownLatch(1); + final CountDownLatch workerStopLatch = new CountDownLatch(1); + final RuntimeException exception = new RuntimeException(); + EasyMock.expect(sourceTask.poll()).andAnswer(() -> { + pollLatch.countDown(); + assertTrue(awaitLatch(workerStopLatch)); + throw exception; + }); + + expectCall(() -> statusListener.onShutdown(taskId)); + expectCall(sourceTask::stop); + + expectClose(); + + PowerMock.replayAll(); + + workerTask.initialize(TASK_CONFIG); + Future taskFuture = executor.submit(workerTask); + + assertTrue(awaitLatch(pollLatch)); + workerTask.stop(); + workerStopLatch.countDown(); + assertTrue(workerTask.awaitStop(1000)); + + taskFuture.get(); + assertPollMetrics(0); + + PowerMock.verifyAll(); + } + + @Test + public void testPollReturnsNoRecords() throws Exception { + // Test that the task handles an empty list of records + createWorkerTask(); + + expectPreflight(); + expectStartup(); + + final CountDownLatch pollLatch = expectEmptyPolls(1, new AtomicInteger()); + EasyMock.expect(offsetWriter.willFlush()).andReturn(false).anyTimes(); + + expectCall(sourceTask::stop); + expectCall(() -> statusListener.onShutdown(taskId)); + + expectClose(); + + PowerMock.replayAll(); + + workerTask.initialize(TASK_CONFIG); + Future taskFuture = executor.submit(workerTask); + + assertTrue(awaitLatch(pollLatch)); + workerTask.stop(); + assertTrue(workerTask.awaitStop(1000)); + + taskFuture.get(); + assertPollMetrics(0); + + PowerMock.verifyAll(); + } + + @Test + public void testPollBasedCommit() throws Exception { + Map connectorProps = sourceConnectorProps(SourceTask.TransactionBoundary.POLL); + sourceConfig = new SourceConnectorConfig(plugins, connectorProps, enableTopicCreation); + + createWorkerTask(); + + expectPreflight(); + expectStartup(); + + AtomicInteger polls = new AtomicInteger(); + AtomicInteger flushes = new AtomicInteger(); + expectPolls(polls); + expectAnyFlushes(flushes); + + expectTopicCreation(TOPIC); + + expectCall(sourceTask::stop); + expectCall(() -> statusListener.onShutdown(taskId)); + + expectClose(); + + PowerMock.replayAll(); + + workerTask.initialize(TASK_CONFIG); + Future taskFuture = executor.submit(workerTask); + + assertTrue(awaitLatch(pollLatch)); + workerTask.stop(); + assertTrue(workerTask.awaitStop(1000)); + + taskFuture.get(); + + assertEquals("Task should have flushed offsets for every record poll, and for end-of-life offset commit", + flushes.get(), polls.get() + 1); + + assertPollMetrics(1); + assertTransactionMetrics(1); + + PowerMock.verifyAll(); + } + + @Test + public void testIntervalBasedCommit() throws Exception { + long commitInterval = 618; + Map connectorProps = sourceConnectorProps(SourceTask.TransactionBoundary.INTERVAL); + connectorProps.put(TRANSACTION_BOUNDARY_INTERVAL_CONFIG, Long.toString(commitInterval)); + sourceConfig = new SourceConnectorConfig(plugins, connectorProps, enableTopicCreation); + + time = new MockTime(); + + createWorkerTask(); + + expectPreflight(); + expectStartup(); + + expectPolls(); + final CountDownLatch firstPollLatch = new CountDownLatch(2); + final CountDownLatch secondPollLatch = new CountDownLatch(2); + final CountDownLatch thirdPollLatch = new CountDownLatch(2); + + AtomicInteger flushes = new AtomicInteger(); + expectFlush(FlushOutcome.SUCCEED, flushes); + expectFlush(FlushOutcome.SUCCEED, flushes); + expectFlush(FlushOutcome.SUCCEED, flushes); + + expectTopicCreation(TOPIC); + + expectCall(sourceTask::stop); + expectCall(() -> statusListener.onShutdown(taskId)); + + expectClose(); + + PowerMock.replayAll(); + + workerTask.initialize(TASK_CONFIG); + Future taskFuture = executor.submit(workerTask); + + pollLatch = firstPollLatch; + assertTrue(awaitLatch(pollLatch)); + assertEquals("No flushes should have taken place before offset commit interval has elapsed", 0, flushes.get()); + time.sleep(commitInterval); + + pollLatch = secondPollLatch; + assertTrue(awaitLatch(pollLatch)); + assertEquals("One flush should have taken place after offset commit interval has elapsed", 1, flushes.get()); + time.sleep(commitInterval * 2); + + pollLatch = thirdPollLatch; + assertTrue(awaitLatch(pollLatch)); + assertEquals("Two flushes should have taken place after offset commit interval has elapsed again", 2, flushes.get()); + + workerTask.stop(); + assertTrue(workerTask.awaitStop(1000)); + + taskFuture.get(); + + assertEquals("Task should have flushed offsets twice based on offset commit interval, and performed final end-of-life offset commit", + 3, flushes.get()); + + assertPollMetrics(2); + + PowerMock.verifyAll(); + } + + @Test + public void testConnectorBasedCommit() throws Exception { + Map connectorProps = sourceConnectorProps(SourceTask.TransactionBoundary.CONNECTOR); + sourceConfig = new SourceConnectorConfig(plugins, connectorProps, enableTopicCreation); + createWorkerTask(); + + expectPreflight(); + expectStartup(); + + expectPolls(); + List pollLatches = IntStream.range(0, 7).mapToObj(i -> new CountDownLatch(3)).collect(Collectors.toList()); + + AtomicInteger flushes = new AtomicInteger(); + // First flush: triggered by TransactionContext::commitTransaction (batch) + expectFlush(FlushOutcome.SUCCEED, flushes); + + // Second flush: triggered by TransactionContext::commitTransaction (record) + expectFlush(FlushOutcome.SUCCEED, flushes); + + // Third flush: triggered by TransactionContext::abortTransaction (batch) + expectCall(producer::abortTransaction); + EasyMock.expect(offsetWriter.willFlush()).andReturn(true); + expectFlush(FlushOutcome.SUCCEED, flushes); + + // Third flush: triggered by TransactionContext::abortTransaction (record) + EasyMock.expect(offsetWriter.willFlush()).andReturn(true); + expectCall(producer::abortTransaction); + expectFlush(FlushOutcome.SUCCEED, flushes); + + expectTopicCreation(TOPIC); + + expectCall(sourceTask::stop); + expectCall(() -> statusListener.onShutdown(taskId)); + + expectClose(); + + PowerMock.replayAll(); + + workerTask.initialize(TASK_CONFIG); + Future taskFuture = executor.submit(workerTask); + + TransactionContext transactionContext = workerTask.sourceTaskContext.transactionContext(); + + int poll = -1; + pollLatch = pollLatches.get(++poll); + assertTrue(awaitLatch(pollLatch)); + assertEquals("No flushes should have taken place without connector requesting transaction commit", 0, flushes.get()); + + transactionContext.commitTransaction(); + pollLatch = pollLatches.get(++poll); + assertTrue(awaitLatch(pollLatch)); + assertEquals("One flush should have taken place after connector requested batch commit", 1, flushes.get()); + + transactionContext.commitTransaction(SOURCE_RECORD); + pollLatch = pollLatches.get(++poll); + assertTrue(awaitLatch(pollLatch)); + assertEquals("Two flushes should have taken place after connector requested individual record commit", 2, flushes.get()); + + pollLatch = pollLatches.get(++poll); + assertTrue(awaitLatch(pollLatch)); + assertEquals("Only two flushes should still have taken place without connector re-requesting commit, even on identical records", 2, flushes.get()); + + transactionContext.abortTransaction(); + pollLatch = pollLatches.get(++poll); + assertTrue(awaitLatch(pollLatch)); + assertEquals("Three flushes should have taken place after connector requested batch abort", 3, flushes.get()); + + transactionContext.abortTransaction(SOURCE_RECORD); + pollLatch = pollLatches.get(++poll); + assertTrue(awaitLatch(pollLatch)); + assertEquals("Four flushes should have taken place after connector requested individual record abort", 4, flushes.get()); + + pollLatch = pollLatches.get(++poll); + assertTrue(awaitLatch(pollLatch)); + assertEquals("Only four flushes should still have taken place without connector re-requesting abort, even on identical records", 4, flushes.get()); + + workerTask.stop(); + assertTrue(workerTask.awaitStop(1000)); + + taskFuture.get(); + + assertEquals("Task should have flushed offsets four times based on connector-defined boundaries, and skipped final end-of-life offset commit", + 4, flushes.get()); + + assertPollMetrics(1); + assertTransactionMetrics(2); + + PowerMock.verifyAll(); + } + + @Test + public void testCommitFlushCallbackFailure() throws Exception { + testCommitFailure(FlushOutcome.FAIL_FLUSH_CALLBACK); + } + + @Test + public void testCommitTransactionFailure() throws Exception { + testCommitFailure(FlushOutcome.FAIL_TRANSACTION_COMMIT); + } + + private void testCommitFailure(FlushOutcome causeOfFailure) throws Exception { + createWorkerTask(); + + expectPreflight(); + expectStartup(); + + expectPolls(); + expectFlush(causeOfFailure); + + expectTopicCreation(TOPIC); + + expectCall(sourceTask::stop); + // Unlike the standard WorkerSourceTask class, this one fails permanently when offset commits don't succeed + final CountDownLatch taskFailure = new CountDownLatch(1); + expectCall(() -> statusListener.onFailure(EasyMock.eq(taskId), EasyMock.anyObject())) + .andAnswer(() -> { + taskFailure.countDown(); + return null; + }); + + expectClose(); + + PowerMock.replayAll(); + + workerTask.initialize(TASK_CONFIG); + Future taskFuture = executor.submit(workerTask); + + assertTrue(awaitLatch(taskFailure)); + workerTask.stop(); + assertTrue(workerTask.awaitStop(1000)); + + taskFuture.get(); + assertPollMetrics(1); + + PowerMock.verifyAll(); + } + + @Test + public void testSendRecordsRetries() throws Exception { + createWorkerTask(); + + // Differentiate only by Kafka partition so we can reuse conversion expectations + SourceRecord record1 = new SourceRecord(PARTITION, OFFSET, "topic", 1, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); + SourceRecord record2 = new SourceRecord(PARTITION, OFFSET, "topic", 2, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); + SourceRecord record3 = new SourceRecord(PARTITION, OFFSET, "topic", 3, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); + + expectTopicCreation(TOPIC); + + // First round + expectSendRecordOnce(false); + expectCall(producer::beginTransaction); + // Any Producer retriable exception should work here + expectSendRecordSyncFailure(new org.apache.kafka.common.errors.TimeoutException("retriable sync failure")); + + // Second round + expectSendRecordOnce(true); + expectSendRecordOnce(false); + + PowerMock.replayAll(); + + // Try to send 3, make first pass, second fail. Should save last two + workerTask.toSend = Arrays.asList(record1, record2, record3); + workerTask.sendRecords(); + assertEquals(Arrays.asList(record2, record3), workerTask.toSend); + + // Next they all succeed + workerTask.sendRecords(); + assertNull(workerTask.toSend); + + PowerMock.verifyAll(); + } + + @Test + public void testSendRecordsProducerSendFailsImmediately() { + if (!enableTopicCreation) + // should only test with topic creation enabled + return; + + createWorkerTask(); + + SourceRecord record1 = new SourceRecord(PARTITION, OFFSET, TOPIC, 1, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); + SourceRecord record2 = new SourceRecord(PARTITION, OFFSET, TOPIC, 2, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); + + expectCall(producer::beginTransaction); + expectTopicCreation(TOPIC); + expectConvertHeadersAndKeyValue(TOPIC, true, emptyHeaders()); + expectApplyTransformationChain(false); + + EasyMock.expect(producer.send(EasyMock.anyObject(), EasyMock.anyObject())) + .andThrow(new KafkaException("Producer closed while send in progress", new InvalidTopicException(TOPIC))); + + PowerMock.replayAll(); + + workerTask.toSend = Arrays.asList(record1, record2); + assertThrows(ConnectException.class, workerTask::sendRecords); + + PowerMock.verifyAll(); + } + + @Test + public void testSlowTaskStart() throws Exception { + final CountDownLatch startupLatch = new CountDownLatch(1); + final CountDownLatch finishStartupLatch = new CountDownLatch(1); + + createWorkerTask(); + + expectPreflight(); + + expectCall(() -> sourceTask.initialize(EasyMock.anyObject(SourceTaskContext.class))); + expectCall(() -> sourceTask.start(TASK_PROPS)); + EasyMock.expectLastCall().andAnswer(() -> { + startupLatch.countDown(); + assertTrue(awaitLatch(finishStartupLatch)); + return null; + }); + + expectCall(() -> statusListener.onStartup(taskId)); + + expectCall(sourceTask::stop); + EasyMock.expect(offsetWriter.willFlush()).andReturn(false); + + expectCall(() -> statusListener.onShutdown(taskId)); + + expectClose(); + + PowerMock.replayAll(); + + workerTask.initialize(TASK_CONFIG); + Future workerTaskFuture = executor.submit(workerTask); + + // Stopping immediately while the other thread has work to do should result in no polling, no offset commits, + // exiting the work thread immediately, and the stop() method will be invoked in the background thread since it + // cannot be invoked immediately in the thread trying to stop the task. + assertTrue(awaitLatch(startupLatch)); + workerTask.stop(); + finishStartupLatch.countDown(); + assertTrue(workerTask.awaitStop(1000)); + + workerTaskFuture.get(); + + PowerMock.verifyAll(); + } + + @Test + public void testCancel() { + createWorkerTask(); + + expectCall(offsetReader::close); + expectCall(() -> producer.close(Duration.ZERO)); + + PowerMock.replayAll(); + + // workerTask said something dumb on twitter + workerTask.cancel(); + + PowerMock.verifyAll(); + } + + private TopicAdmin.TopicCreationResponse createdTopic(String topic) { + Set created = Collections.singleton(topic); + Set existing = Collections.emptySet(); + return new TopicAdmin.TopicCreationResponse(created, existing); + } + + private CountDownLatch expectEmptyPolls(int minimum, final AtomicInteger count) throws InterruptedException { + final CountDownLatch latch = new CountDownLatch(minimum); + // Note that we stub these to allow any number of calls because the thread will continue to + // run. The count passed in + latch returned just makes sure we get *at least* that number of + // calls + EasyMock.expect(sourceTask.poll()) + .andStubAnswer(() -> { + count.incrementAndGet(); + latch.countDown(); + Thread.sleep(10); + return Collections.emptyList(); + }); + return latch; + } + + private void expectPolls(final AtomicInteger pollCount) throws Exception { + expectCall(producer::beginTransaction).atLeastOnce(); + // Note that we stub these to allow any number of calls because the thread will continue to + // run. The count passed in + latch returned just makes sure we get *at least* that number of + // calls + EasyMock.expect(sourceTask.poll()) + .andStubAnswer(() -> { + pollCount.incrementAndGet(); + pollLatch.countDown(); + Thread.sleep(10); + return RECORDS; + }); + // Fallout of the poll() call + expectSendRecordAnyTimes(); + } + + private void expectPolls() throws Exception { + expectPolls(new AtomicInteger()); + } + + @SuppressWarnings("unchecked") + private void expectSendRecordSyncFailure(Throwable error) { + expectConvertHeadersAndKeyValue(false); + expectApplyTransformationChain(false); + + offsetWriter.offset(PARTITION, OFFSET); + PowerMock.expectLastCall(); + + EasyMock.expect( + producer.send(EasyMock.anyObject(ProducerRecord.class), + EasyMock.anyObject(org.apache.kafka.clients.producer.Callback.class))) + .andThrow(error); + } + + private Capture> expectSendRecordAnyTimes() { + return expectSendRecordSendSuccess(true, false); + } + + private Capture> expectSendRecordOnce(boolean isRetry) { + return expectSendRecordSendSuccess(false, isRetry); + } + + private Capture> expectSendRecordSendSuccess(boolean anyTimes, boolean isRetry) { + return expectSendRecord(TOPIC, anyTimes, isRetry, true, true, emptyHeaders()); + } + + private Capture> expectSendRecord( + String topic, + boolean anyTimes, + boolean isRetry, + boolean sendSuccess, + boolean isMockedConverters, + Headers headers + ) { + if (isMockedConverters) { + expectConvertHeadersAndKeyValue(topic, anyTimes, headers); + } + + expectApplyTransformationChain(anyTimes); + + Capture> sent = EasyMock.newCapture(); + + // 1. Offset data is passed to the offset storage. + if (!isRetry) { + offsetWriter.offset(PARTITION, OFFSET); + if (anyTimes) + PowerMock.expectLastCall().anyTimes(); + else + PowerMock.expectLastCall(); + } + + // 2. Converted data passed to the producer, which will need callbacks invoked for flush to work + IExpectationSetters> expect = EasyMock.expect( + producer.send(EasyMock.capture(sent), + EasyMock.capture(producerCallbacks))); + IAnswer> 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, 0, 0), null); + } else { + cb.onCompletion(null, new TopicAuthorizationException("foo")); + } + } + producerCallbacks.reset(); + } + return sendFuture; + }; + if (anyTimes) + expect.andStubAnswer(expectResponse); + else + expect.andAnswer(expectResponse); + + if (sendSuccess) { + // 3. As a result of a successful producer send callback, we note the use of the topic + expectTaskGetTopic(anyTimes); + } + + return sent; + } + + private void expectConvertHeadersAndKeyValue(boolean anyTimes) { + expectConvertHeadersAndKeyValue(TOPIC, anyTimes, emptyHeaders()); + } + + private void expectConvertHeadersAndKeyValue(String topic, boolean anyTimes, Headers headers) { + for (Header header : headers) { + IExpectationSetters convertHeaderExpect = EasyMock.expect(headerConverter.fromConnectHeader(topic, header.key(), Schema.STRING_SCHEMA, new String(header.value()))); + if (anyTimes) + convertHeaderExpect.andStubReturn(header.value()); + else + convertHeaderExpect.andReturn(header.value()); + } + IExpectationSetters convertKeyExpect = EasyMock.expect(keyConverter.fromConnectData(topic, headers, KEY_SCHEMA, KEY)); + if (anyTimes) + convertKeyExpect.andStubReturn(SERIALIZED_KEY); + else + convertKeyExpect.andReturn(SERIALIZED_KEY); + IExpectationSetters convertValueExpect = EasyMock.expect(valueConverter.fromConnectData(topic, headers, RECORD_SCHEMA, RECORD)); + if (anyTimes) + convertValueExpect.andStubReturn(SERIALIZED_RECORD); + else + convertValueExpect.andReturn(SERIALIZED_RECORD); + } + + private void expectApplyTransformationChain(boolean anyTimes) { + final Capture recordCapture = EasyMock.newCapture(); + IExpectationSetters convertKeyExpect = EasyMock.expect(transformationChain.apply(EasyMock.capture(recordCapture))); + if (anyTimes) + convertKeyExpect.andStubAnswer(recordCapture::getValue); + else + convertKeyExpect.andAnswer(recordCapture::getValue); + } + + private void expectTaskGetTopic(boolean anyTimes) { + final Capture connectorCapture = EasyMock.newCapture(); + final Capture topicCapture = EasyMock.newCapture(); + IExpectationSetters expect = EasyMock.expect(statusBackingStore.getTopic( + EasyMock.capture(connectorCapture), + EasyMock.capture(topicCapture))); + if (anyTimes) { + expect.andStubAnswer(() -> new TopicStatus( + topicCapture.getValue(), + new ConnectorTaskId(connectorCapture.getValue(), 0), + time.milliseconds())); + } else { + expect.andAnswer(() -> new TopicStatus( + topicCapture.getValue(), + new ConnectorTaskId(connectorCapture.getValue(), 0), + time.milliseconds())); + } + if (connectorCapture.hasCaptured() && topicCapture.hasCaptured()) { + assertEquals("job", connectorCapture.getValue()); + assertEquals(TOPIC, topicCapture.getValue()); + } + } + + private boolean awaitLatch(CountDownLatch latch) { + try { + return latch.await(5000, TimeUnit.MILLISECONDS); + } catch (InterruptedException e) { + // ignore + } + return false; + } + + private enum FlushOutcome { + SUCCEED, + SUCCEED_ANY_TIMES, + FAIL_FLUSH_CALLBACK, + FAIL_TRANSACTION_COMMIT + } + + private CountDownLatch expectFlush(FlushOutcome outcome, AtomicInteger flushCount) { + CountDownLatch result = new CountDownLatch(1); + org.easymock.IExpectationSetters flushBegin = EasyMock + .expect(offsetWriter.beginFlush()) + .andAnswer(() -> { + flushCount.incrementAndGet(); + result.countDown(); + return true; + }); + if (FlushOutcome.SUCCEED_ANY_TIMES.equals(outcome)) { + flushBegin.anyTimes(); + } + + Capture> flushCallback = EasyMock.newCapture(); + org.easymock.IExpectationSetters> offsetFlush = + EasyMock.expect(offsetWriter.doFlush(EasyMock.capture(flushCallback))); + switch (outcome) { + case SUCCEED: + // The worker task doesn't actually use the returned future + offsetFlush.andReturn(null); + expectCall(producer::commitTransaction); + expectCall(() -> sourceTask.commitRecord(EasyMock.anyObject(), EasyMock.anyObject())); + expectCall(sourceTask::commit); + break; + case SUCCEED_ANY_TIMES: + // The worker task doesn't actually use the returned future + offsetFlush.andReturn(null).anyTimes(); + expectCall(producer::commitTransaction).anyTimes(); + expectCall(() -> sourceTask.commitRecord(EasyMock.anyObject(), EasyMock.anyObject())).anyTimes(); + expectCall(sourceTask::commit).anyTimes(); + break; + case FAIL_FLUSH_CALLBACK: + expectCall(producer::commitTransaction); + offsetFlush.andAnswer(() -> { + flushCallback.getValue().onCompletion(new RecordTooLargeException(), null); + return null; + }); + expectCall(offsetWriter::cancelFlush); + break; + case FAIL_TRANSACTION_COMMIT: + offsetFlush.andReturn(null); + expectCall(producer::commitTransaction) + .andThrow(new RecordTooLargeException()); + expectCall(offsetWriter::cancelFlush); + break; + default: + fail("Unexpected flush outcome: " + outcome); + } + return result; + } + + private CountDownLatch expectFlush(FlushOutcome outcome) { + return expectFlush(outcome, new AtomicInteger()); + } + + private CountDownLatch expectAnyFlushes(AtomicInteger flushCount) { + EasyMock.expect(offsetWriter.willFlush()).andReturn(true).anyTimes(); + return expectFlush(FlushOutcome.SUCCEED_ANY_TIMES, flushCount); + } + + private void assertTransactionMetrics(int minimumMaxSizeExpected) { + MetricGroup transactionGroup = workerTask.transactionMetricsGroup().metricGroup(); + double actualMin = metrics.currentMetricValueAsDouble(transactionGroup, "transaction-size-min"); + double actualMax = metrics.currentMetricValueAsDouble(transactionGroup, "transaction-size-max"); + double actualAvg = metrics.currentMetricValueAsDouble(transactionGroup, "transaction-size-avg"); + assertTrue(actualMin >= 0); + assertTrue(actualMax >= minimumMaxSizeExpected); + + if (actualMax - actualMin <= 0.000001d) { + assertEquals(actualMax, actualAvg, 0.000002d); + } else { + assertTrue("Average transaction size should be greater than minimum transaction size", actualAvg > actualMin); + assertTrue("Average transaction size should be less than maximum transaction size", actualAvg < actualMax); + } + } + + private void assertPollMetrics(int minimumPollCountExpected) { + MetricGroup sourceTaskGroup = workerTask.sourceTaskMetricsGroup().metricGroup(); + MetricGroup taskGroup = workerTask.taskMetricsGroup().metricGroup(); + double pollRate = metrics.currentMetricValueAsDouble(sourceTaskGroup, "source-record-poll-rate"); + double pollTotal = metrics.currentMetricValueAsDouble(sourceTaskGroup, "source-record-poll-total"); + if (minimumPollCountExpected > 0) { + assertEquals(RECORDS.size(), metrics.currentMetricValueAsDouble(taskGroup, "batch-size-max"), 0.000001d); + assertEquals(RECORDS.size(), metrics.currentMetricValueAsDouble(taskGroup, "batch-size-avg"), 0.000001d); + assertTrue(pollRate > 0.0d); + } else { + assertTrue(pollRate == 0.0d); + } + assertTrue(pollTotal >= minimumPollCountExpected); + + double writeRate = metrics.currentMetricValueAsDouble(sourceTaskGroup, "source-record-write-rate"); + double writeTotal = metrics.currentMetricValueAsDouble(sourceTaskGroup, "source-record-write-total"); + if (minimumPollCountExpected > 0) { + assertTrue(writeRate > 0.0d); + } else { + assertTrue(writeRate == 0.0d); + } + assertTrue(writeTotal >= minimumPollCountExpected); + + double pollBatchTimeMax = metrics.currentMetricValueAsDouble(sourceTaskGroup, "poll-batch-max-time-ms"); + double pollBatchTimeAvg = metrics.currentMetricValueAsDouble(sourceTaskGroup, "poll-batch-avg-time-ms"); + if (minimumPollCountExpected > 0) { + assertTrue(pollBatchTimeMax >= 0.0d); + } + assertTrue(Double.isNaN(pollBatchTimeAvg) || pollBatchTimeAvg > 0.0d); + double activeCount = metrics.currentMetricValueAsDouble(sourceTaskGroup, "source-record-active-count"); + double activeCountMax = metrics.currentMetricValueAsDouble(sourceTaskGroup, "source-record-active-count-max"); + assertEquals(0, activeCount, 0.000001d); + if (minimumPollCountExpected > 0) { + assertEquals(RECORDS.size(), activeCountMax, 0.000001d); + } + } + + private RecordHeaders emptyHeaders() { + return new RecordHeaders(); + } + + private abstract static class TestSourceTask extends SourceTask { + } + + @FunctionalInterface + private interface MockedMethodCall { + void invoke() throws Exception; + } + + private static org.easymock.IExpectationSetters expectCall(MockedMethodCall call) { + try { + call.invoke(); + } catch (RuntimeException e) { + throw e; + } catch (Exception e) { + throw new RuntimeException("Mocked method invocation threw a checked exception", e); + } + return EasyMock.expectLastCall(); + } + + private void expectPreflight() { + expectCall(preProducerCheck::run); + expectCall(offsetStore::start); + expectCall(producer::initTransactions); + expectCall(postProducerCheck::run); + } + + private void expectStartup() { + expectCall(() -> sourceTask.initialize(EasyMock.anyObject(SourceTaskContext.class))); + expectCall(() -> sourceTask.start(TASK_PROPS)); + expectCall(() -> statusListener.onStartup(taskId)); + } + + private void expectClose() { + expectCall(offsetStore::stop); + expectCall(() -> producer.close(EasyMock.anyObject(Duration.class))); + expectCall(() -> admin.close(EasyMock.anyObject(Duration.class))); + expectCall(transformationChain::close); + expectCall(offsetReader::close); + } + + private void expectTopicCreation(String topic) { + if (config.topicCreationEnable()) { + EasyMock.expect(admin.describeTopics(topic)).andReturn(Collections.emptyMap()); + Capture newTopicCapture = EasyMock.newCapture(); + EasyMock.expect(admin.createOrFindTopics(EasyMock.capture(newTopicCapture))).andReturn(createdTopic(topic)); + } + } +} diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/SubmittedRecordsTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/SubmittedRecordsTest.java index 4028249a78ad..39d680a7d46b 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/SubmittedRecordsTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/SubmittedRecordsTest.java @@ -178,8 +178,8 @@ public void testRemoveLastSubmittedRecord() { assertEquals(Collections.emptyMap(), committableOffsets.offsets()); assertMetadata(committableOffsets, 0, 1, 1, 1, PARTITION1); - assertTrue("First attempt to remove record from submitted queue should succeed", submittedRecords.removeLastOccurrence(submittedRecord)); - assertFalse("Attempt to remove already-removed record from submitted queue should fail", submittedRecords.removeLastOccurrence(submittedRecord)); + assertTrue("First attempt to remove record from submitted queue should succeed", submittedRecord.drop()); + assertFalse("Attempt to remove already-removed record from submitted queue should fail", submittedRecord.drop()); committableOffsets = submittedRecords.committableOffsets(); // Even if SubmittedRecords::remove is broken, we haven't ack'd anything yet, so there should be no committable offsets @@ -203,7 +203,7 @@ public void testRemoveNotLastSubmittedRecord() { assertMetadata(committableOffsets, 0, 2, 2, 1, PARTITION1, PARTITION2); assertNoEmptyDeques(); - assertTrue("First attempt to remove record from submitted queue should succeed", submittedRecords.removeLastOccurrence(recordToRemove)); + assertTrue("First attempt to remove record from submitted queue should succeed", recordToRemove.drop()); committableOffsets = submittedRecords.committableOffsets(); // Even if SubmittedRecords::remove is broken, we haven't ack'd anything yet, so there should be no committable offsets @@ -269,20 +269,20 @@ public void testAwaitMessagesAfterAllRemoved() { submittedRecords.awaitAllMessages(0, TimeUnit.MILLISECONDS) ); - submittedRecords.removeLastOccurrence(recordToRemove1); + recordToRemove1.drop(); assertFalse( "Await should fail since only one of the two submitted records has been removed so far", submittedRecords.awaitAllMessages(0, TimeUnit.MILLISECONDS) ); - submittedRecords.removeLastOccurrence(recordToRemove1); + recordToRemove1.drop(); assertFalse( "Await should fail since only one of the two submitted records has been removed so far, " + "even though that record has been removed twice", submittedRecords.awaitAllMessages(0, TimeUnit.MILLISECONDS) ); - submittedRecords.removeLastOccurrence(recordToRemove2); + recordToRemove2.drop(); assertTrue( "Await should succeed since both submitted records have now been removed", submittedRecords.awaitAllMessages(0, TimeUnit.MILLISECONDS) diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerConnectorTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerConnectorTest.java index 29b85301e89a..253efff8c053 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerConnectorTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerConnectorTest.java @@ -24,7 +24,8 @@ import org.apache.kafka.connect.sink.SinkConnectorContext; import org.apache.kafka.connect.source.SourceConnector; import org.apache.kafka.connect.source.SourceConnectorContext; -import org.apache.kafka.connect.storage.OffsetStorageReader; +import org.apache.kafka.connect.storage.CloseableOffsetStorageReader; +import org.apache.kafka.connect.storage.ConnectorOffsetBackingStore; import org.easymock.Capture; import org.apache.kafka.connect.util.Callback; import org.easymock.EasyMock; @@ -65,7 +66,8 @@ public class WorkerConnectorTest extends EasyMockSupport { @Mock Connector connector; @Mock CloseableConnectorContext ctx; @Mock ConnectorStatus.Listener listener; - @Mock OffsetStorageReader offsetStorageReader; + @Mock CloseableOffsetStorageReader offsetStorageReader; + @Mock ConnectorOffsetBackingStore offsetStore; @Mock ClassLoader classLoader; @Before @@ -99,9 +101,15 @@ public void testInitializeFailure() { ctx.close(); expectLastCall(); + offsetStorageReader.close(); + expectLastCall(); + + offsetStore.stop(); + expectLastCall(); + replayAll(); - WorkerConnector workerConnector = new WorkerConnector(CONNECTOR, connector, connectorConfig, ctx, metrics, listener, offsetStorageReader, classLoader); + WorkerConnector workerConnector = new WorkerConnector(CONNECTOR, connector, connectorConfig, ctx, metrics, listener, offsetStorageReader, offsetStore, classLoader); workerConnector.initialize(); assertFailedMetric(workerConnector); @@ -134,13 +142,19 @@ public void testFailureIsFinalState() { ctx.close(); expectLastCall(); + offsetStorageReader.close(); + expectLastCall(); + + offsetStore.stop(); + expectLastCall(); + Callback onStateChange = createStrictMock(Callback.class); onStateChange.onCompletion(EasyMock.anyObject(Exception.class), EasyMock.isNull()); expectLastCall(); replayAll(); - WorkerConnector workerConnector = new WorkerConnector(CONNECTOR, connector, connectorConfig, ctx, metrics, listener, offsetStorageReader, classLoader); + WorkerConnector workerConnector = new WorkerConnector(CONNECTOR, connector, connectorConfig, ctx, metrics, listener, offsetStorageReader, offsetStore, classLoader); workerConnector.initialize(); assertFailedMetric(workerConnector); @@ -177,13 +191,19 @@ public void testStartupAndShutdown() { ctx.close(); expectLastCall(); + offsetStorageReader.close(); + expectLastCall(); + + offsetStore.stop(); + expectLastCall(); + Callback onStateChange = createStrictMock(Callback.class); onStateChange.onCompletion(EasyMock.isNull(), EasyMock.eq(TargetState.STARTED)); expectLastCall(); replayAll(); - WorkerConnector workerConnector = new WorkerConnector(CONNECTOR, connector, connectorConfig, ctx, metrics, listener, offsetStorageReader, classLoader); + WorkerConnector workerConnector = new WorkerConnector(CONNECTOR, connector, connectorConfig, ctx, metrics, listener, offsetStorageReader, offsetStore, classLoader); workerConnector.initialize(); assertInitializedSourceMetric(workerConnector); @@ -223,6 +243,12 @@ public void testStartupAndPause() { ctx.close(); expectLastCall(); + offsetStorageReader.close(); + expectLastCall(); + + offsetStore.stop(); + expectLastCall(); + Callback onStateChange = createStrictMock(Callback.class); onStateChange.onCompletion(EasyMock.isNull(), EasyMock.eq(TargetState.STARTED)); expectLastCall(); @@ -231,7 +257,7 @@ public void testStartupAndPause() { replayAll(); - WorkerConnector workerConnector = new WorkerConnector(CONNECTOR, connector, connectorConfig, ctx, metrics, listener, offsetStorageReader, classLoader); + WorkerConnector workerConnector = new WorkerConnector(CONNECTOR, connector, connectorConfig, ctx, metrics, listener, offsetStorageReader, offsetStore, classLoader); workerConnector.initialize(); assertInitializedSinkMetric(workerConnector); @@ -273,6 +299,12 @@ public void testOnResume() { ctx.close(); expectLastCall(); + offsetStorageReader.close(); + expectLastCall(); + + offsetStore.stop(); + expectLastCall(); + Callback onStateChange = createStrictMock(Callback.class); onStateChange.onCompletion(EasyMock.isNull(), EasyMock.eq(TargetState.PAUSED)); expectLastCall(); @@ -281,7 +313,7 @@ public void testOnResume() { replayAll(); - WorkerConnector workerConnector = new WorkerConnector(CONNECTOR, connector, connectorConfig, ctx, metrics, listener, offsetStorageReader, classLoader); + WorkerConnector workerConnector = new WorkerConnector(CONNECTOR, connector, connectorConfig, ctx, metrics, listener, offsetStorageReader, offsetStore, classLoader); workerConnector.initialize(); assertInitializedSourceMetric(workerConnector); @@ -316,13 +348,19 @@ public void testStartupPaused() { ctx.close(); expectLastCall(); + offsetStorageReader.close(); + expectLastCall(); + + offsetStore.stop(); + expectLastCall(); + Callback onStateChange = createStrictMock(Callback.class); onStateChange.onCompletion(EasyMock.isNull(), EasyMock.eq(TargetState.PAUSED)); expectLastCall(); replayAll(); - WorkerConnector workerConnector = new WorkerConnector(CONNECTOR, connector, connectorConfig, ctx, metrics, listener, offsetStorageReader, classLoader); + WorkerConnector workerConnector = new WorkerConnector(CONNECTOR, connector, connectorConfig, ctx, metrics, listener, offsetStorageReader, offsetStore, classLoader); workerConnector.initialize(); assertInitializedSinkMetric(workerConnector); @@ -358,13 +396,19 @@ public void testStartupFailure() { ctx.close(); expectLastCall(); + offsetStorageReader.close(); + expectLastCall(); + + offsetStore.stop(); + expectLastCall(); + Callback onStateChange = createStrictMock(Callback.class); onStateChange.onCompletion(EasyMock.anyObject(Exception.class), EasyMock.isNull()); expectLastCall(); replayAll(); - WorkerConnector workerConnector = new WorkerConnector(CONNECTOR, connector, connectorConfig, ctx, metrics, listener, offsetStorageReader, classLoader); + WorkerConnector workerConnector = new WorkerConnector(CONNECTOR, connector, connectorConfig, ctx, metrics, listener, offsetStorageReader, offsetStore, classLoader); workerConnector.initialize(); assertInitializedSinkMetric(workerConnector); @@ -407,9 +451,15 @@ public void testShutdownFailure() { ctx.close(); expectLastCall(); + offsetStorageReader.close(); + expectLastCall(); + + offsetStore.stop(); + expectLastCall(); + replayAll(); - WorkerConnector workerConnector = new WorkerConnector(CONNECTOR, connector, connectorConfig, ctx, metrics, listener, offsetStorageReader, classLoader); + WorkerConnector workerConnector = new WorkerConnector(CONNECTOR, connector, connectorConfig, ctx, metrics, listener, offsetStorageReader, offsetStore, classLoader); workerConnector.initialize(); assertInitializedSourceMetric(workerConnector); @@ -447,13 +497,19 @@ public void testTransitionStartedToStarted() { ctx.close(); expectLastCall(); + offsetStorageReader.close(); + expectLastCall(); + + offsetStore.stop(); + expectLastCall(); + Callback onStateChange = createStrictMock(Callback.class); onStateChange.onCompletion(EasyMock.isNull(), EasyMock.eq(TargetState.STARTED)); expectLastCall().times(2); replayAll(); - WorkerConnector workerConnector = new WorkerConnector(CONNECTOR, connector, connectorConfig, ctx, metrics, listener, offsetStorageReader, classLoader); + WorkerConnector workerConnector = new WorkerConnector(CONNECTOR, connector, connectorConfig, ctx, metrics, listener, offsetStorageReader, offsetStore, classLoader); workerConnector.initialize(); assertInitializedSourceMetric(workerConnector); @@ -495,6 +551,12 @@ public void testTransitionPausedToPaused() { ctx.close(); expectLastCall(); + offsetStorageReader.close(); + expectLastCall(); + + offsetStore.stop(); + expectLastCall(); + Callback onStateChange = createStrictMock(Callback.class); onStateChange.onCompletion(EasyMock.isNull(), EasyMock.eq(TargetState.STARTED)); expectLastCall(); @@ -503,7 +565,7 @@ public void testTransitionPausedToPaused() { replayAll(); - WorkerConnector workerConnector = new WorkerConnector(CONNECTOR, connector, connectorConfig, ctx, metrics, listener, offsetStorageReader, classLoader); + WorkerConnector workerConnector = new WorkerConnector(CONNECTOR, connector, connectorConfig, ctx, metrics, listener, offsetStorageReader, offsetStore, classLoader); workerConnector.initialize(); assertInitializedSourceMetric(workerConnector); @@ -531,7 +593,7 @@ public void testFailConnectorThatIsNeitherSourceNorSink() { replayAll(); - WorkerConnector workerConnector = new WorkerConnector(CONNECTOR, connector, connectorConfig, ctx, metrics, listener, offsetStorageReader, classLoader); + WorkerConnector workerConnector = new WorkerConnector(CONNECTOR, connector, connectorConfig, ctx, metrics, listener, offsetStorageReader, offsetStore, classLoader); workerConnector.initialize(); Throwable e = exceptionCapture.getValue(); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSourceTaskTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSourceTaskTest.java index 5aeff5e9d87e..322c37f05db8 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSourceTaskTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSourceTaskTest.java @@ -16,17 +16,12 @@ */ package org.apache.kafka.connect.runtime; -import java.util.Collection; import org.apache.kafka.clients.admin.NewTopic; -import org.apache.kafka.clients.admin.TopicDescription; import org.apache.kafka.clients.producer.KafkaProducer; import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.clients.producer.RecordMetadata; -import org.apache.kafka.common.InvalidRecordException; import org.apache.kafka.common.KafkaException; -import org.apache.kafka.common.MetricName; import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.common.TopicPartitionInfo; import org.apache.kafka.common.errors.InvalidTopicException; import org.apache.kafka.common.errors.TopicAuthorizationException; import org.apache.kafka.common.header.Header; @@ -34,13 +29,9 @@ import org.apache.kafka.common.header.internals.RecordHeaders; import org.apache.kafka.common.utils.Time; import org.apache.kafka.connect.data.Schema; -import org.apache.kafka.connect.data.SchemaAndValue; import org.apache.kafka.connect.errors.ConnectException; -import org.apache.kafka.connect.errors.RetriableException; -import org.apache.kafka.connect.header.ConnectHeaders; import org.apache.kafka.connect.integration.MonitorableSourceConnector; import org.apache.kafka.connect.runtime.ConnectMetrics.MetricGroup; -import org.apache.kafka.connect.runtime.WorkerSourceTask.SourceTaskMetricsGroup; import org.apache.kafka.connect.storage.ClusterConfigState; import org.apache.kafka.connect.runtime.errors.RetryWithToleranceOperator; import org.apache.kafka.connect.runtime.errors.RetryWithToleranceOperatorTest; @@ -50,6 +41,7 @@ import org.apache.kafka.connect.source.SourceTask; import org.apache.kafka.connect.source.SourceTaskContext; import org.apache.kafka.connect.storage.CloseableOffsetStorageReader; +import org.apache.kafka.connect.storage.ConnectorOffsetBackingStore; import org.apache.kafka.connect.storage.Converter; import org.apache.kafka.connect.storage.HeaderConverter; import org.apache.kafka.connect.storage.OffsetStorageWriter; @@ -76,10 +68,9 @@ import org.powermock.modules.junit4.PowerMockRunnerDelegate; import org.powermock.reflect.Whitebox; -import java.nio.ByteBuffer; import java.time.Duration; -import java.util.ArrayList; import java.util.Arrays; +import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.List; @@ -106,7 +97,6 @@ import static org.apache.kafka.connect.runtime.TopicCreationConfig.REPLICATION_FACTOR_CONFIG; import static org.apache.kafka.connect.runtime.WorkerConfig.TOPIC_CREATION_ENABLE_CONFIG; import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNull; import static org.junit.Assert.assertThrows; import static org.junit.Assert.assertTrue; @@ -147,6 +137,7 @@ public class WorkerSourceTaskTest extends ThreadedTest { @Mock private TopicAdmin admin; @Mock private CloseableOffsetStorageReader offsetReader; @Mock private OffsetStorageWriter offsetWriter; + @Mock private ConnectorOffsetBackingStore offsetStore; @Mock private ClusterConfigState clusterConfigState; private WorkerSourceTask workerTask; @Mock private Future sendFuture; @@ -235,16 +226,11 @@ private void createWorkerTask(TargetState initialState, RetryWithToleranceOperat createWorkerTask(initialState, keyConverter, valueConverter, headerConverter, retryWithToleranceOperator); } - private void createWorkerTask(TargetState initialState, Converter keyConverter, Converter valueConverter, - HeaderConverter headerConverter) { - createWorkerTask(initialState, keyConverter, valueConverter, headerConverter, RetryWithToleranceOperatorTest.NOOP_OPERATOR); - } - private void createWorkerTask(TargetState initialState, Converter keyConverter, Converter valueConverter, HeaderConverter headerConverter, RetryWithToleranceOperator retryWithToleranceOperator) { workerTask = new WorkerSourceTask(taskId, sourceTask, statusListener, initialState, keyConverter, valueConverter, headerConverter, - transformationChain, producer, admin, TopicCreationGroup.configuredGroups(sourceConfig), - offsetReader, offsetWriter, config, clusterConfigState, metrics, plugins.delegatingLoader(), Time.SYSTEM, + transformationChain, producer, admin, TopicCreationGroup.configuredGroups(sourceConfig), + offsetReader, offsetWriter, offsetStore, config, clusterConfigState, metrics, plugins.delegatingLoader(), Time.SYSTEM, retryWithToleranceOperator, statusBackingStore, Runnable::run); } @@ -644,93 +630,6 @@ public void testCommitFailure() throws Exception { PowerMock.verifyAll(); } - @Test - public void testSendRecordsConvertsData() throws Exception { - createWorkerTask(); - - List records = new ArrayList<>(); - // Can just use the same record for key and value - records.add(new SourceRecord(PARTITION, OFFSET, "topic", null, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD)); - - Capture> sent = expectSendRecordAnyTimes(); - - expectTopicCreation(TOPIC); - - PowerMock.replayAll(); - - Whitebox.setInternalState(workerTask, "toSend", records); - Whitebox.invokeMethod(workerTask, "sendRecords"); - assertEquals(SERIALIZED_KEY, sent.getValue().key()); - assertEquals(SERIALIZED_RECORD, sent.getValue().value()); - - PowerMock.verifyAll(); - } - - @Test - public void testSendRecordsPropagatesTimestamp() throws Exception { - final Long timestamp = System.currentTimeMillis(); - - createWorkerTask(); - - List records = Collections.singletonList( - new SourceRecord(PARTITION, OFFSET, "topic", null, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD, timestamp) - ); - - Capture> sent = expectSendRecordAnyTimes(); - - expectTopicCreation(TOPIC); - - PowerMock.replayAll(); - - Whitebox.setInternalState(workerTask, "toSend", records); - Whitebox.invokeMethod(workerTask, "sendRecords"); - assertEquals(timestamp, sent.getValue().timestamp()); - - PowerMock.verifyAll(); - } - - @Test - public void testSendRecordsCorruptTimestamp() throws Exception { - final Long timestamp = -3L; - createWorkerTask(); - - List records = Collections.singletonList( - new SourceRecord(PARTITION, OFFSET, "topic", null, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD, timestamp) - ); - - Capture> sent = expectSendRecordAnyTimes(); - - PowerMock.replayAll(); - - Whitebox.setInternalState(workerTask, "toSend", records); - assertThrows(InvalidRecordException.class, () -> Whitebox.invokeMethod(workerTask, "sendRecords")); - assertFalse(sent.hasCaptured()); - - PowerMock.verifyAll(); - } - - @Test - public void testSendRecordsNoTimestamp() throws Exception { - final Long timestamp = -1L; - createWorkerTask(); - - List records = Collections.singletonList( - new SourceRecord(PARTITION, OFFSET, "topic", null, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD, timestamp) - ); - - Capture> sent = expectSendRecordAnyTimes(); - - expectTopicCreation(TOPIC); - - PowerMock.replayAll(); - - Whitebox.setInternalState(workerTask, "toSend", records); - Whitebox.invokeMethod(workerTask, "sendRecords"); - assertNull(sent.getValue().timestamp()); - - PowerMock.verifyAll(); - } - @Test public void testSendRecordsRetries() throws Exception { createWorkerTask(); @@ -775,6 +674,8 @@ public void testSendRecordsProducerCallbackFail() throws Exception { expectTopicCreation(TOPIC); expectSendRecordProducerCallbackFail(); + expectApplyTransformationChain(false); + expectConvertHeadersAndKeyValue(false); PowerMock.replayAll(); @@ -842,7 +743,6 @@ public void testSourceTaskIgnoresProducerException() throws Exception { SourceRecord record1 = new SourceRecord(PARTITION, OFFSET, TOPIC, 1, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); SourceRecord record2 = new SourceRecord(PARTITION, OFFSET, TOPIC, 2, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); - expectSendRecordOnce(); expectSendRecordProducerCallbackFail(); sourceTask.commitRecord(EasyMock.anyObject(SourceRecord.class), EasyMock.isNull()); @@ -919,440 +819,12 @@ public void testCancel() { PowerMock.verifyAll(); } - @Test - public void testMetricsGroup() { - SourceTaskMetricsGroup group = new SourceTaskMetricsGroup(taskId, metrics); - SourceTaskMetricsGroup group1 = new SourceTaskMetricsGroup(taskId1, metrics); - for (int i = 0; i != 10; ++i) { - group.recordPoll(100, 1000 + i * 100); - group.recordWrite(10); - } - for (int i = 0; i != 20; ++i) { - group1.recordPoll(100, 1000 + i * 100); - group1.recordWrite(10); - } - assertEquals(1900.0, metrics.currentMetricValueAsDouble(group.metricGroup(), "poll-batch-max-time-ms"), 0.001d); - assertEquals(1450.0, metrics.currentMetricValueAsDouble(group.metricGroup(), "poll-batch-avg-time-ms"), 0.001d); - assertEquals(33.333, metrics.currentMetricValueAsDouble(group.metricGroup(), "source-record-poll-rate"), 0.001d); - assertEquals(1000, metrics.currentMetricValueAsDouble(group.metricGroup(), "source-record-poll-total"), 0.001d); - assertEquals(3.3333, metrics.currentMetricValueAsDouble(group.metricGroup(), "source-record-write-rate"), 0.001d); - assertEquals(100, metrics.currentMetricValueAsDouble(group.metricGroup(), "source-record-write-total"), 0.001d); - assertEquals(900.0, metrics.currentMetricValueAsDouble(group.metricGroup(), "source-record-active-count"), 0.001d); - - // Close the group - group.close(); - - for (MetricName metricName : group.metricGroup().metrics().metrics().keySet()) { - // Metrics for this group should no longer exist - assertFalse(group.metricGroup().groupId().includes(metricName)); - } - // Sensors for this group should no longer exist - assertNull(group.metricGroup().metrics().getSensor("sink-record-read")); - assertNull(group.metricGroup().metrics().getSensor("sink-record-send")); - assertNull(group.metricGroup().metrics().getSensor("sink-record-active-count")); - assertNull(group.metricGroup().metrics().getSensor("partition-count")); - assertNull(group.metricGroup().metrics().getSensor("offset-seq-number")); - assertNull(group.metricGroup().metrics().getSensor("offset-commit-completion")); - assertNull(group.metricGroup().metrics().getSensor("offset-commit-completion-skip")); - assertNull(group.metricGroup().metrics().getSensor("put-batch-time")); - - assertEquals(2900.0, metrics.currentMetricValueAsDouble(group1.metricGroup(), "poll-batch-max-time-ms"), 0.001d); - assertEquals(1950.0, metrics.currentMetricValueAsDouble(group1.metricGroup(), "poll-batch-avg-time-ms"), 0.001d); - assertEquals(66.667, metrics.currentMetricValueAsDouble(group1.metricGroup(), "source-record-poll-rate"), 0.001d); - assertEquals(2000, metrics.currentMetricValueAsDouble(group1.metricGroup(), "source-record-poll-total"), 0.001d); - assertEquals(6.667, metrics.currentMetricValueAsDouble(group1.metricGroup(), "source-record-write-rate"), 0.001d); - assertEquals(200, metrics.currentMetricValueAsDouble(group1.metricGroup(), "source-record-write-total"), 0.001d); - assertEquals(1800.0, metrics.currentMetricValueAsDouble(group1.metricGroup(), "source-record-active-count"), 0.001d); - } - - @Test - public void testHeaders() throws Exception { - Headers headers = new RecordHeaders(); - headers.add("header_key", "header_value".getBytes()); - - org.apache.kafka.connect.header.Headers connectHeaders = new ConnectHeaders(); - connectHeaders.add("header_key", new SchemaAndValue(Schema.STRING_SCHEMA, "header_value")); - - createWorkerTask(); - - List records = new ArrayList<>(); - records.add(new SourceRecord(PARTITION, OFFSET, TOPIC, null, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD, null, connectHeaders)); - - expectTopicCreation(TOPIC); - - Capture> sent = expectSendRecord(TOPIC, true, true, true, true, headers); - - PowerMock.replayAll(); - - Whitebox.setInternalState(workerTask, "toSend", records); - Whitebox.invokeMethod(workerTask, "sendRecords"); - assertEquals(SERIALIZED_KEY, sent.getValue().key()); - assertEquals(SERIALIZED_RECORD, sent.getValue().value()); - assertEquals(headers, sent.getValue().headers()); - - PowerMock.verifyAll(); - } - - @Test - public void testHeadersWithCustomConverter() throws Exception { - StringConverter stringConverter = new StringConverter(); - SampleConverterWithHeaders testConverter = new SampleConverterWithHeaders(); - - createWorkerTask(TargetState.STARTED, stringConverter, testConverter, stringConverter); - - List records = new ArrayList<>(); - - String stringA = "Árvíztűrő tükörfúrógép"; - org.apache.kafka.connect.header.Headers headersA = new ConnectHeaders(); - String encodingA = "latin2"; - headersA.addString("encoding", encodingA); - - records.add(new SourceRecord(PARTITION, OFFSET, "topic", null, Schema.STRING_SCHEMA, "a", Schema.STRING_SCHEMA, stringA, null, headersA)); - - String stringB = "Тестовое сообщение"; - org.apache.kafka.connect.header.Headers headersB = new ConnectHeaders(); - String encodingB = "koi8_r"; - headersB.addString("encoding", encodingB); - - records.add(new SourceRecord(PARTITION, OFFSET, "topic", null, Schema.STRING_SCHEMA, "b", Schema.STRING_SCHEMA, stringB, null, headersB)); - - expectTopicCreation(TOPIC); - - Capture> sentRecordA = expectSendRecord(TOPIC, false, true, true, false, null); - Capture> sentRecordB = expectSendRecord(TOPIC, false, true, true, false, null); - - PowerMock.replayAll(); - - Whitebox.setInternalState(workerTask, "toSend", records); - Whitebox.invokeMethod(workerTask, "sendRecords"); - - assertEquals(ByteBuffer.wrap("a".getBytes()), ByteBuffer.wrap(sentRecordA.getValue().key())); - assertEquals( - ByteBuffer.wrap(stringA.getBytes(encodingA)), - ByteBuffer.wrap(sentRecordA.getValue().value()) - ); - assertEquals(encodingA, new String(sentRecordA.getValue().headers().lastHeader("encoding").value())); - - assertEquals(ByteBuffer.wrap("b".getBytes()), ByteBuffer.wrap(sentRecordB.getValue().key())); - assertEquals( - ByteBuffer.wrap(stringB.getBytes(encodingB)), - ByteBuffer.wrap(sentRecordB.getValue().value()) - ); - assertEquals(encodingB, new String(sentRecordB.getValue().headers().lastHeader("encoding").value())); - - PowerMock.verifyAll(); - } - - @Test - public void testTopicCreateWhenTopicExists() throws Exception { - if (!enableTopicCreation) - // should only test with topic creation enabled - return; - - createWorkerTask(); - - SourceRecord record1 = new SourceRecord(PARTITION, OFFSET, TOPIC, 1, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); - SourceRecord record2 = new SourceRecord(PARTITION, OFFSET, TOPIC, 2, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); - - expectPreliminaryCalls(); - TopicPartitionInfo topicPartitionInfo = new TopicPartitionInfo(0, null, Collections.emptyList(), Collections.emptyList()); - TopicDescription topicDesc = new TopicDescription(TOPIC, false, Collections.singletonList(topicPartitionInfo)); - EasyMock.expect(admin.describeTopics(TOPIC)).andReturn(Collections.singletonMap(TOPIC, topicDesc)); - - expectSendRecordTaskCommitRecordSucceed(false); - expectSendRecordTaskCommitRecordSucceed(false); - - PowerMock.replayAll(); - - Whitebox.setInternalState(workerTask, "toSend", Arrays.asList(record1, record2)); - Whitebox.invokeMethod(workerTask, "sendRecords"); - } - - @Test - public void testSendRecordsTopicDescribeRetries() throws Exception { - if (!enableTopicCreation) - // should only test with topic creation enabled - return; - - createWorkerTask(); - - SourceRecord record1 = new SourceRecord(PARTITION, OFFSET, TOPIC, 1, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); - SourceRecord record2 = new SourceRecord(PARTITION, OFFSET, TOPIC, 2, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); - - expectPreliminaryCalls(); - // First round - call to describe the topic times out - EasyMock.expect(admin.describeTopics(TOPIC)) - .andThrow(new RetriableException(new TimeoutException("timeout"))); - - // Second round - calls to describe and create succeed - expectTopicCreation(TOPIC); - // Exactly two records are sent - expectSendRecordTaskCommitRecordSucceed(false); - expectSendRecordTaskCommitRecordSucceed(false); - - PowerMock.replayAll(); - - Whitebox.setInternalState(workerTask, "toSend", Arrays.asList(record1, record2)); - Whitebox.invokeMethod(workerTask, "sendRecords"); - assertEquals(Arrays.asList(record1, record2), Whitebox.getInternalState(workerTask, "toSend")); - - // Next they all succeed - Whitebox.invokeMethod(workerTask, "sendRecords"); - assertNull(Whitebox.getInternalState(workerTask, "toSend")); - } - - @Test - public void testSendRecordsTopicCreateRetries() throws Exception { - if (!enableTopicCreation) - // should only test with topic creation enabled - return; - - createWorkerTask(); - - SourceRecord record1 = new SourceRecord(PARTITION, OFFSET, TOPIC, 1, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); - SourceRecord record2 = new SourceRecord(PARTITION, OFFSET, TOPIC, 2, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); - - // First call to describe the topic times out - expectPreliminaryCalls(); - EasyMock.expect(admin.describeTopics(TOPIC)).andReturn(Collections.emptyMap()); - Capture newTopicCapture = EasyMock.newCapture(); - EasyMock.expect(admin.createOrFindTopics(EasyMock.capture(newTopicCapture))) - .andThrow(new RetriableException(new TimeoutException("timeout"))); - - // Second round - expectTopicCreation(TOPIC); - expectSendRecordTaskCommitRecordSucceed(false); - expectSendRecordTaskCommitRecordSucceed(false); - - PowerMock.replayAll(); - - Whitebox.setInternalState(workerTask, "toSend", Arrays.asList(record1, record2)); - Whitebox.invokeMethod(workerTask, "sendRecords"); - assertEquals(Arrays.asList(record1, record2), Whitebox.getInternalState(workerTask, "toSend")); - - // Next they all succeed - Whitebox.invokeMethod(workerTask, "sendRecords"); - assertNull(Whitebox.getInternalState(workerTask, "toSend")); - } - - @Test - public void testSendRecordsTopicDescribeRetriesMidway() throws Exception { - if (!enableTopicCreation) - // should only test with topic creation enabled - return; - - createWorkerTask(); - - // Differentiate only by Kafka partition so we can reuse conversion expectations - SourceRecord record1 = new SourceRecord(PARTITION, OFFSET, TOPIC, 1, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); - SourceRecord record2 = new SourceRecord(PARTITION, OFFSET, TOPIC, 2, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); - SourceRecord record3 = new SourceRecord(PARTITION, OFFSET, OTHER_TOPIC, 3, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); - - // First round - expectPreliminaryCalls(OTHER_TOPIC); - expectTopicCreation(TOPIC); - expectSendRecordTaskCommitRecordSucceed(false); - expectSendRecordTaskCommitRecordSucceed(false); - - // First call to describe the topic times out - EasyMock.expect(admin.describeTopics(OTHER_TOPIC)) - .andThrow(new RetriableException(new TimeoutException("timeout"))); - - // Second round - expectTopicCreation(OTHER_TOPIC); - expectSendRecord(OTHER_TOPIC, false, true, true, true, emptyHeaders()); - - PowerMock.replayAll(); - - // Try to send 3, make first pass, second fail. Should save last two - Whitebox.setInternalState(workerTask, "toSend", Arrays.asList(record1, record2, record3)); - Whitebox.invokeMethod(workerTask, "sendRecords"); - assertEquals(Arrays.asList(record3), Whitebox.getInternalState(workerTask, "toSend")); - - // Next they all succeed - Whitebox.invokeMethod(workerTask, "sendRecords"); - assertNull(Whitebox.getInternalState(workerTask, "toSend")); - - PowerMock.verifyAll(); - } - - @Test - public void testSendRecordsTopicCreateRetriesMidway() throws Exception { - if (!enableTopicCreation) - // should only test with topic creation enabled - return; - - createWorkerTask(); - - // Differentiate only by Kafka partition so we can reuse conversion expectations - SourceRecord record1 = new SourceRecord(PARTITION, OFFSET, TOPIC, 1, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); - SourceRecord record2 = new SourceRecord(PARTITION, OFFSET, TOPIC, 2, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); - SourceRecord record3 = new SourceRecord(PARTITION, OFFSET, OTHER_TOPIC, 3, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); - - // First round - expectPreliminaryCalls(OTHER_TOPIC); - expectTopicCreation(TOPIC); - expectSendRecordTaskCommitRecordSucceed(false); - expectSendRecordTaskCommitRecordSucceed(false); - - EasyMock.expect(admin.describeTopics(OTHER_TOPIC)).andReturn(Collections.emptyMap()); - // First call to create the topic times out - Capture newTopicCapture = EasyMock.newCapture(); - EasyMock.expect(admin.createOrFindTopics(EasyMock.capture(newTopicCapture))) - .andThrow(new RetriableException(new TimeoutException("timeout"))); - - // Second round - expectTopicCreation(OTHER_TOPIC); - expectSendRecord(OTHER_TOPIC, false, true, true, true, emptyHeaders()); - - PowerMock.replayAll(); - - // Try to send 3, make first pass, second fail. Should save last two - Whitebox.setInternalState(workerTask, "toSend", Arrays.asList(record1, record2, record3)); - Whitebox.invokeMethod(workerTask, "sendRecords"); - assertEquals(Arrays.asList(record3), Whitebox.getInternalState(workerTask, "toSend")); - - // Next they all succeed - Whitebox.invokeMethod(workerTask, "sendRecords"); - assertNull(Whitebox.getInternalState(workerTask, "toSend")); - - PowerMock.verifyAll(); - } - - @Test - public void testTopicDescribeFails() { - if (!enableTopicCreation) - // should only test with topic creation enabled - return; - - createWorkerTask(); - - SourceRecord record1 = new SourceRecord(PARTITION, OFFSET, TOPIC, 1, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); - SourceRecord record2 = new SourceRecord(PARTITION, OFFSET, TOPIC, 2, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); - - expectPreliminaryCalls(); - EasyMock.expect(admin.describeTopics(TOPIC)) - .andThrow(new ConnectException(new TopicAuthorizationException("unauthorized"))); - - PowerMock.replayAll(); - - Whitebox.setInternalState(workerTask, "toSend", Arrays.asList(record1, record2)); - assertThrows(ConnectException.class, () -> Whitebox.invokeMethod(workerTask, "sendRecords")); - } - - @Test - public void testTopicCreateFails() { - if (!enableTopicCreation) - // should only test with topic creation enabled - return; - - createWorkerTask(); - - SourceRecord record1 = new SourceRecord(PARTITION, OFFSET, TOPIC, 1, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); - SourceRecord record2 = new SourceRecord(PARTITION, OFFSET, TOPIC, 2, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); - - expectPreliminaryCalls(); - EasyMock.expect(admin.describeTopics(TOPIC)).andReturn(Collections.emptyMap()); - - Capture newTopicCapture = EasyMock.newCapture(); - EasyMock.expect(admin.createOrFindTopics(EasyMock.capture(newTopicCapture))) - .andThrow(new ConnectException(new TopicAuthorizationException("unauthorized"))); - - PowerMock.replayAll(); - - Whitebox.setInternalState(workerTask, "toSend", Arrays.asList(record1, record2)); - assertThrows(ConnectException.class, () -> Whitebox.invokeMethod(workerTask, "sendRecords")); - assertTrue(newTopicCapture.hasCaptured()); - } - - @Test - public void testTopicCreateFailsWithExceptionWhenCreateReturnsTopicNotCreatedOrFound() { - if (!enableTopicCreation) - // should only test with topic creation enabled - return; - - createWorkerTask(); - - SourceRecord record1 = new SourceRecord(PARTITION, OFFSET, TOPIC, 1, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); - SourceRecord record2 = new SourceRecord(PARTITION, OFFSET, TOPIC, 2, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); - - expectPreliminaryCalls(); - EasyMock.expect(admin.describeTopics(TOPIC)).andReturn(Collections.emptyMap()); - - Capture newTopicCapture = EasyMock.newCapture(); - EasyMock.expect(admin.createOrFindTopics(EasyMock.capture(newTopicCapture))).andReturn(TopicAdmin.EMPTY_CREATION); - - PowerMock.replayAll(); - - Whitebox.setInternalState(workerTask, "toSend", Arrays.asList(record1, record2)); - assertThrows(ConnectException.class, () -> Whitebox.invokeMethod(workerTask, "sendRecords")); - assertTrue(newTopicCapture.hasCaptured()); - } - - @Test - public void testTopicCreateSucceedsWhenCreateReturnsExistingTopicFound() throws Exception { - if (!enableTopicCreation) - // should only test with topic creation enabled - return; - - createWorkerTask(); - - SourceRecord record1 = new SourceRecord(PARTITION, OFFSET, TOPIC, 1, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); - SourceRecord record2 = new SourceRecord(PARTITION, OFFSET, TOPIC, 2, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); - - expectPreliminaryCalls(); - EasyMock.expect(admin.describeTopics(TOPIC)).andReturn(Collections.emptyMap()); - - Capture newTopicCapture = EasyMock.newCapture(); - EasyMock.expect(admin.createOrFindTopics(EasyMock.capture(newTopicCapture))).andReturn(foundTopic(TOPIC)); - - expectSendRecordTaskCommitRecordSucceed(false); - expectSendRecordTaskCommitRecordSucceed(false); - - PowerMock.replayAll(); - - Whitebox.setInternalState(workerTask, "toSend", Arrays.asList(record1, record2)); - Whitebox.invokeMethod(workerTask, "sendRecords"); - } - - @Test - public void testTopicCreateSucceedsWhenCreateReturnsNewTopicFound() throws Exception { - if (!enableTopicCreation) - // should only test with topic creation enabled - return; - - createWorkerTask(); - - SourceRecord record1 = new SourceRecord(PARTITION, OFFSET, TOPIC, 1, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); - SourceRecord record2 = new SourceRecord(PARTITION, OFFSET, TOPIC, 2, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); - - expectPreliminaryCalls(); - EasyMock.expect(admin.describeTopics(TOPIC)).andReturn(Collections.emptyMap()); - - Capture newTopicCapture = EasyMock.newCapture(); - EasyMock.expect(admin.createOrFindTopics(EasyMock.capture(newTopicCapture))).andReturn(createdTopic(TOPIC)); - - expectSendRecordTaskCommitRecordSucceed(false); - expectSendRecordTaskCommitRecordSucceed(false); - - PowerMock.replayAll(); - - Whitebox.setInternalState(workerTask, "toSend", Arrays.asList(record1, record2)); - Whitebox.invokeMethod(workerTask, "sendRecords"); - } - private TopicAdmin.TopicCreationResponse createdTopic(String topic) { Set created = Collections.singleton(topic); Set existing = Collections.emptySet(); return new TopicAdmin.TopicCreationResponse(created, existing); } - private TopicAdmin.TopicCreationResponse foundTopic(String topic) { - Set created = Collections.emptySet(); - Set existing = Collections.singleton(topic); - return new TopicAdmin.TopicCreationResponse(created, existing); - } - private void expectPreliminaryCalls() { expectPreliminaryCalls(TOPIC); } @@ -1625,6 +1097,12 @@ private void expectClose() { transformationChain.close(); EasyMock.expectLastCall(); + + offsetReader.close(); + EasyMock.expectLastCall(); + + offsetStore.stop(); + EasyMock.expectLastCall(); } private void expectTopicCreation(String topic) { diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerTest.java index b2d7e0a50074..a15cf77e322f 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerTest.java @@ -41,6 +41,7 @@ import org.apache.kafka.connect.runtime.ConnectMetrics.MetricGroup; import org.apache.kafka.connect.runtime.MockConnectMetrics.MockMetricsReporter; import org.apache.kafka.connect.storage.ClusterConfigState; +import org.apache.kafka.connect.runtime.distributed.DistributedConfig; import org.apache.kafka.connect.runtime.isolation.DelegatingClassLoader; import org.apache.kafka.connect.runtime.isolation.PluginClassLoader; import org.apache.kafka.connect.runtime.isolation.Plugins; @@ -48,6 +49,7 @@ import org.apache.kafka.connect.runtime.rest.entities.ConnectorStateInfo; import org.apache.kafka.connect.runtime.standalone.StandaloneConfig; import org.apache.kafka.connect.sink.SinkConnector; +import org.apache.kafka.connect.sink.SinkRecord; import org.apache.kafka.connect.sink.SinkTask; import org.apache.kafka.connect.source.SourceConnector; import org.apache.kafka.connect.source.SourceRecord; @@ -93,10 +95,18 @@ import static org.apache.kafka.clients.admin.AdminClientConfig.RETRY_BACKOFF_MS_CONFIG; import static org.apache.kafka.connect.runtime.ConnectorConfig.CONNECTOR_CLIENT_ADMIN_OVERRIDES_PREFIX; +import static org.apache.kafka.connect.runtime.ConnectorConfig.CONNECTOR_CLASS_CONFIG; import static org.apache.kafka.connect.runtime.TopicCreationConfig.DEFAULT_TOPIC_CREATION_PREFIX; import static org.apache.kafka.connect.runtime.TopicCreationConfig.PARTITIONS_CONFIG; import static org.apache.kafka.connect.runtime.TopicCreationConfig.REPLICATION_FACTOR_CONFIG; +import static org.apache.kafka.connect.runtime.WorkerConfig.BOOTSTRAP_SERVERS_CONFIG; import static org.apache.kafka.connect.runtime.WorkerConfig.TOPIC_CREATION_ENABLE_CONFIG; +import static org.apache.kafka.connect.runtime.distributed.DistributedConfig.CONFIG_TOPIC_CONFIG; +import static org.apache.kafka.connect.runtime.distributed.DistributedConfig.EXACTLY_ONCE_SOURCE_SUPPORT_CONFIG; +import static org.apache.kafka.connect.runtime.distributed.DistributedConfig.GROUP_ID_CONFIG; +import static org.apache.kafka.connect.runtime.distributed.DistributedConfig.OFFSET_STORAGE_TOPIC_CONFIG; +import static org.apache.kafka.connect.runtime.distributed.DistributedConfig.STATUS_STORAGE_TOPIC_CONFIG; +import static org.apache.kafka.connect.sink.SinkTask.TOPICS_CONFIG; import static org.hamcrest.CoreMatchers.instanceOf; import static org.hamcrest.MatcherAssert.assertThat; import static org.junit.Assert.assertEquals; @@ -281,9 +291,8 @@ public void teardown() { @Test public void testStartAndStopConnector() throws Throwable { - final String connectorClass = SampleSourceConnector.class.getName(); - connectorProps.put(ConnectorConfig.CONNECTOR_CLASS_CONFIG, connectorClass); + connectorProps.put(CONNECTOR_CLASS_CONFIG, connectorClass); // Create when(plugins.currentThreadLoader()).thenReturn(delegatingLoader); @@ -363,9 +372,8 @@ private void mockFileConfigProvider() { @Test public void testStartConnectorFailure() throws Exception { - final String nonConnectorClass = "java.util.HashMap"; - connectorProps.put(ConnectorConfig.CONNECTOR_CLASS_CONFIG, nonConnectorClass); // Bad connector class name + connectorProps.put(CONNECTOR_CLASS_CONFIG, nonConnectorClass); // Bad connector class name Exception exception = new ConnectException("Failed to find Connector"); @@ -413,7 +421,6 @@ public void testStartConnectorFailure() throws Exception { @Test public void testAddConnectorByAlias() throws Throwable { - final String connectorAlias = "SampleSourceConnector"; when(plugins.currentThreadLoader()).thenReturn(delegatingLoader); @@ -427,7 +434,7 @@ public void testAddConnectorByAlias() throws Throwable { connectUtilsMockedStatic.when(() -> ConnectUtils.lookupKafkaClusterId(any(WorkerConfig.class))) .thenReturn("test-cluster"); - connectorProps.put(ConnectorConfig.CONNECTOR_CLASS_CONFIG, connectorAlias); + connectorProps.put(CONNECTOR_CLASS_CONFIG, connectorAlias); connectorProps.put(SinkConnectorConfig.TOPICS_CONFIG, "gfieyls, wfru"); worker = new Worker(WORKER_ID, new MockTime(), plugins, config, offsetBackingStore, noneConnectorClientConfigOverridePolicy); @@ -471,7 +478,6 @@ public void testAddConnectorByAlias() throws Throwable { @Test public void testAddConnectorByShortAlias() throws Throwable { - final String shortConnectorAlias = "WorkerTest"; when(plugins.currentThreadLoader()).thenReturn(delegatingLoader); @@ -582,7 +588,7 @@ public void testReconfigureConnectorTasks() throws Throwable { Map expectedTaskProps = new HashMap<>(); expectedTaskProps.put("foo", "bar"); expectedTaskProps.put(TaskConfig.TASK_CLASS_CONFIG, TestSourceTask.class.getName()); - expectedTaskProps.put(SinkTask.TOPICS_CONFIG, "foo,bar"); + expectedTaskProps.put(TOPICS_CONFIG, "foo,bar"); assertEquals(2, taskConfigs.size()); assertEquals(expectedTaskProps, taskConfigs.get(0)); assertEquals(expectedTaskProps, taskConfigs.get(1)); @@ -615,7 +621,7 @@ public void testReconfigureConnectorTasks() throws Throwable { } @Test - public void testAddRemoveTask() { + public void testAddRemoveSourceTask() { when(plugins.currentThreadLoader()).thenReturn(delegatingLoader); when(plugins.delegatingLoader()).thenReturn(delegatingLoader); when(delegatingLoader.connectorLoader(SampleSourceConnector.class.getName())).thenReturn(pluginLoader); @@ -639,7 +645,7 @@ public void testAddRemoveTask() { assertStatistics(worker, 0, 0); assertEquals(Collections.emptySet(), worker.taskIds()); - worker.startTask(TASK_ID, ClusterConfigState.EMPTY, anyConnectorConfigMap(), origProps, taskStatusListener, TargetState.STARTED); + worker.startSourceTask(TASK_ID, ClusterConfigState.EMPTY, anyConnectorConfigMap(), origProps, taskStatusListener, TargetState.STARTED); assertStatistics(worker, 0, 1); assertEquals(Collections.singleton(TASK_ID), worker.taskIds()); worker.stopAndAwaitTask(TASK_ID); @@ -664,7 +670,136 @@ public void testAddRemoveTask() { pluginsMockedStatic.verify(() -> Plugins.compareAndSwapLoaders(pluginLoader), times(2)); pluginsMockedStatic.verify(() -> Plugins.compareAndSwapLoaders(delegatingLoader), times(2)); connectUtilsMockedStatic.verify(() -> ConnectUtils.lookupKafkaClusterId(any(WorkerConfig.class))); + } + + @Test + public void testAddRemoveSinkTask() { + // Most of the other cases use source tasks; we make sure to get code coverage for sink tasks here as well + when(plugins.currentThreadLoader()).thenReturn(delegatingLoader); + when(plugins.delegatingLoader()).thenReturn(delegatingLoader); + when(delegatingLoader.connectorLoader(SampleSinkConnector.class.getName())).thenReturn(pluginLoader); + + SinkTask task = mock(TestSinkTask.class); + when(plugins.newTask(TestSinkTask.class)).thenReturn(task); + when(task.version()).thenReturn("1.0"); + mockTaskConverter(ClassLoaderUsage.CURRENT_CLASSLOADER, WorkerConfig.KEY_CONVERTER_CLASS_CONFIG, taskKeyConverter); + mockTaskConverter(ClassLoaderUsage.CURRENT_CLASSLOADER, WorkerConfig.VALUE_CONVERTER_CLASS_CONFIG, taskValueConverter); + mockTaskHeaderConverter(ClassLoaderUsage.CURRENT_CLASSLOADER, taskHeaderConverter); + when(executorService.submit(any(WorkerSinkTask.class))).thenReturn(null); + doReturn(SampleSinkConnector.class).when(plugins).connectorClass(SampleSinkConnector.class.getName()); + pluginsMockedStatic.when(() -> Plugins.compareAndSwapLoaders(pluginLoader)).thenReturn(delegatingLoader); + pluginsMockedStatic.when(() -> Plugins.compareAndSwapLoaders(delegatingLoader)).thenReturn(pluginLoader); + + Map origProps = Collections.singletonMap(TaskConfig.TASK_CLASS_CONFIG, TestSinkTask.class.getName()); + + worker = new Worker(WORKER_ID, new MockTime(), plugins, config, offsetBackingStore, executorService, + noneConnectorClientConfigOverridePolicy); + worker.herder = herder; + worker.start(); + + assertStatistics(worker, 0, 0); + assertEquals(Collections.emptySet(), worker.taskIds()); + Map connectorConfigs = anyConnectorConfigMap(); + connectorConfigs.put(TOPICS_CONFIG, "t1"); + connectorConfigs.put(CONNECTOR_CLASS_CONFIG, SampleSinkConnector.class.getName()); + + worker.startSinkTask(TASK_ID, ClusterConfigState.EMPTY, connectorConfigs, origProps, taskStatusListener, TargetState.STARTED); + assertStatistics(worker, 0, 1); + assertEquals(Collections.singleton(TASK_ID), worker.taskIds()); + worker.stopAndAwaitTask(TASK_ID); + assertStatistics(worker, 0, 0); + assertEquals(Collections.emptySet(), worker.taskIds()); + // Nothing should be left, so this should effectively be a nop + worker.stop(); + assertStatistics(worker, 0, 0); + + verify(plugins, times(2)).currentThreadLoader(); + verify(plugins).newTask(TestSinkTask.class); + verify(task).version(); + verifyTaskConverter(WorkerConfig.KEY_CONVERTER_CLASS_CONFIG); + verifyTaskConverter(WorkerConfig.VALUE_CONVERTER_CLASS_CONFIG); + verifyTaskHeaderConverter(); + + verify(executorService).submit(any(WorkerSinkTask.class)); + verify(plugins).delegatingLoader(); + verify(delegatingLoader).connectorLoader(SampleSinkConnector.class.getName()); + verify(plugins).connectorClass(SampleSinkConnector.class.getName()); + + pluginsMockedStatic.verify(() -> Plugins.compareAndSwapLoaders(pluginLoader), times(2)); + pluginsMockedStatic.verify(() -> Plugins.compareAndSwapLoaders(delegatingLoader), times(2)); + connectUtilsMockedStatic.verify(() -> ConnectUtils.lookupKafkaClusterId(any(WorkerConfig.class))); + } + + @Test + public void testAddRemoveExactlyOnceSourceTask() { + Map workerProps = new HashMap<>(); + workerProps.put("key.converter", "org.apache.kafka.connect.json.JsonConverter"); + workerProps.put("value.converter", "org.apache.kafka.connect.json.JsonConverter"); + workerProps.put(CommonClientConfigs.METRIC_REPORTER_CLASSES_CONFIG, MockMetricsReporter.class.getName()); + workerProps.put("config.providers", "file"); + workerProps.put("config.providers.file.class", MockFileConfigProvider.class.getName()); + mockFileProviderTestId = UUID.randomUUID().toString(); + workerProps.put("config.providers.file.param.testId", mockFileProviderTestId); + workerProps.put(TOPIC_CREATION_ENABLE_CONFIG, String.valueOf(enableTopicCreation)); + workerProps.put(GROUP_ID_CONFIG, "connect-cluster"); + workerProps.put(BOOTSTRAP_SERVERS_CONFIG, "localhost:2606"); + workerProps.put(OFFSET_STORAGE_TOPIC_CONFIG, "connect-offsets"); + workerProps.put(CONFIG_TOPIC_CONFIG, "connect-configs"); + workerProps.put(STATUS_STORAGE_TOPIC_CONFIG, "connect-statuses"); + workerProps.put(EXACTLY_ONCE_SOURCE_SUPPORT_CONFIG, "enabled"); + config = new DistributedConfig(workerProps); + + when(plugins.currentThreadLoader()).thenReturn(delegatingLoader); + when(plugins.delegatingLoader()).thenReturn(delegatingLoader); + when(delegatingLoader.connectorLoader(SampleSourceConnector.class.getName())).thenReturn(pluginLoader); + + when(plugins.newTask(TestSourceTask.class)).thenReturn(task); + when(task.version()).thenReturn("1.0"); + mockTaskConverter(ClassLoaderUsage.CURRENT_CLASSLOADER, WorkerConfig.KEY_CONVERTER_CLASS_CONFIG, taskKeyConverter); + mockTaskConverter(ClassLoaderUsage.CURRENT_CLASSLOADER, WorkerConfig.VALUE_CONVERTER_CLASS_CONFIG, taskValueConverter); + mockTaskHeaderConverter(ClassLoaderUsage.CURRENT_CLASSLOADER, taskHeaderConverter); + when(executorService.submit(any(ExactlyOnceWorkerSourceTask.class))).thenReturn(null); + doReturn(SampleSourceConnector.class).when(plugins).connectorClass(SampleSourceConnector.class.getName()); + pluginsMockedStatic.when(() -> Plugins.compareAndSwapLoaders(pluginLoader)).thenReturn(delegatingLoader); + pluginsMockedStatic.when(() -> Plugins.compareAndSwapLoaders(delegatingLoader)).thenReturn(pluginLoader); + + Runnable preProducer = mock(Runnable.class); + Runnable postProducer = mock(Runnable.class); + + Map origProps = Collections.singletonMap(TaskConfig.TASK_CLASS_CONFIG, TestSourceTask.class.getName()); + + worker = new Worker(WORKER_ID, new MockTime(), plugins, config, offsetBackingStore, executorService, + noneConnectorClientConfigOverridePolicy); + worker.herder = herder; + worker.start(); + assertStatistics(worker, 0, 0); + assertEquals(Collections.emptySet(), worker.taskIds()); + worker.startExactlyOnceSourceTask(TASK_ID, ClusterConfigState.EMPTY, anyConnectorConfigMap(), origProps, taskStatusListener, TargetState.STARTED, preProducer, postProducer); + assertStatistics(worker, 0, 1); + assertEquals(Collections.singleton(TASK_ID), worker.taskIds()); + worker.stopAndAwaitTask(TASK_ID); + assertStatistics(worker, 0, 0); + assertEquals(Collections.emptySet(), worker.taskIds()); + // Nothing should be left, so this should effectively be a nop + worker.stop(); + assertStatistics(worker, 0, 0); + + verify(plugins, times(2)).currentThreadLoader(); + verify(plugins).newTask(TestSourceTask.class); + verify(task).version(); + verifyTaskConverter(WorkerConfig.KEY_CONVERTER_CLASS_CONFIG); + verifyTaskConverter(WorkerConfig.VALUE_CONVERTER_CLASS_CONFIG); + verifyTaskHeaderConverter(); + + verify(executorService).submit(any(ExactlyOnceWorkerSourceTask.class)); + verify(plugins).delegatingLoader(); + verify(delegatingLoader).connectorLoader(SampleSourceConnector.class.getName()); + verify(plugins).connectorClass(SampleSourceConnector.class.getName()); + + pluginsMockedStatic.verify(() -> Plugins.compareAndSwapLoaders(pluginLoader), times(2)); + pluginsMockedStatic.verify(() -> Plugins.compareAndSwapLoaders(delegatingLoader), times(2)); + connectUtilsMockedStatic.verify(() -> ConnectUtils.lookupKafkaClusterId(any(WorkerConfig.class))); } @Test @@ -674,7 +809,6 @@ public void testTaskStatusMetricsStatuses() { mockFileConfigProvider(); - when(plugins.currentThreadLoader()).thenReturn(delegatingLoader); Map origProps = Collections.singletonMap(TaskConfig.TASK_CLASS_CONFIG, TestSourceTask.class.getName()); @@ -701,7 +835,6 @@ public void testTaskStatusMetricsStatuses() { doReturn(SampleSourceConnector.class).when(plugins).connectorClass(SampleSourceConnector.class.getName()); - // Each time we check the task metrics, the worker will call the herder when(herder.taskStatus(TASK_ID)).thenReturn( new ConnectorStateInfo.TaskState(0, "RUNNING", "worker", "msg"), @@ -725,7 +858,7 @@ public void testTaskStatusMetricsStatuses() { assertStatistics(worker, 0, 0); assertStartupStatistics(worker, 0, 0, 0, 0); assertEquals(Collections.emptySet(), worker.taskIds()); - worker.startTask( + worker.startSourceTask( TASK_ID, ClusterConfigState.EMPTY, anyConnectorConfigMap(), @@ -766,7 +899,6 @@ public void testTaskStatusMetricsStatuses() { verifyTaskConverter(WorkerConfig.VALUE_CONVERTER_CLASS_CONFIG); verifyTaskHeaderConverter(); verify(plugins, times(2)).currentThreadLoader(); - } @Test @@ -819,7 +951,7 @@ public void testStartTaskFailure() { assertStatistics(worker, 0, 0); assertStartupStatistics(worker, 0, 0, 0, 0); - assertFalse(worker.startTask(TASK_ID, ClusterConfigState.EMPTY, anyConnectorConfigMap(), origProps, taskStatusListener, TargetState.STARTED)); + assertFalse(worker.startSourceTask(TASK_ID, ClusterConfigState.EMPTY, anyConnectorConfigMap(), origProps, taskStatusListener, TargetState.STARTED)); assertStartupStatistics(worker, 0, 0, 1, 1); assertStatistics(worker, 0, 0); @@ -870,7 +1002,7 @@ public void testCleanupTasksOnStop() { worker.herder = herder; worker.start(); assertStatistics(worker, 0, 0); - worker.startTask(TASK_ID, ClusterConfigState.EMPTY, anyConnectorConfigMap(), origProps, taskStatusListener, TargetState.STARTED); + worker.startSourceTask(TASK_ID, ClusterConfigState.EMPTY, anyConnectorConfigMap(), origProps, taskStatusListener, TargetState.STARTED); assertStatistics(worker, 0, 1); worker.stop(); assertStatistics(worker, 0, 0); @@ -939,7 +1071,7 @@ public void testConverterOverrides() { Map connProps = anyConnectorConfigMap(); connProps.put(ConnectorConfig.KEY_CONVERTER_CLASS_CONFIG, SampleConverterWithHeaders.class.getName()); connProps.put(ConnectorConfig.VALUE_CONVERTER_CLASS_CONFIG, SampleConverterWithHeaders.class.getName()); - worker.startTask(TASK_ID, ClusterConfigState.EMPTY, connProps, origProps, taskStatusListener, TargetState.STARTED); + worker.startSourceTask(TASK_ID, ClusterConfigState.EMPTY, connProps, origProps, taskStatusListener, TargetState.STARTED); assertStatistics(worker, 0, 1); assertEquals(Collections.singleton(TASK_ID), worker.taskIds()); worker.stopAndAwaitTask(TASK_ID); @@ -976,8 +1108,7 @@ public void testProducerConfigsWithoutOverrides() { expectedConfigs.put("client.id", "connector-producer-job-0"); expectedConfigs.put("metrics.context.connect.kafka.cluster.id", CLUSTER_ID); assertEquals(expectedConfigs, - Worker.producerConfigs(TASK_ID.connector(), "connector-producer-" + TASK_ID, config, connectorConfig, null, noneConnectorClientConfigOverridePolicy, CLUSTER_ID)); - + Worker.baseProducerConfigs(CONNECTOR_ID, "connector-producer-" + TASK_ID, config, connectorConfig, null, noneConnectorClientConfigOverridePolicy, CLUSTER_ID)); verify(connectorConfig).originalsWithPrefix(ConnectorConfig.CONNECTOR_CLIENT_PRODUCER_OVERRIDES_PREFIX); } @@ -996,8 +1127,9 @@ public void testProducerConfigsWithOverrides() { expectedConfigs.put("metrics.context.connect.kafka.cluster.id", CLUSTER_ID); when(connectorConfig.originalsWithPrefix(ConnectorConfig.CONNECTOR_CLIENT_PRODUCER_OVERRIDES_PREFIX)).thenReturn(new HashMap<>()); + assertEquals(expectedConfigs, - Worker.producerConfigs(TASK_ID.connector(), "connector-producer-" + TASK_ID, configWithOverrides, connectorConfig, null, allConnectorClientConfigOverridePolicy, CLUSTER_ID)); + Worker.baseProducerConfigs(CONNECTOR_ID, "connector-producer-" + TASK_ID, configWithOverrides, connectorConfig, null, allConnectorClientConfigOverridePolicy, CLUSTER_ID)); verify(connectorConfig).originalsWithPrefix(ConnectorConfig.CONNECTOR_CLIENT_PRODUCER_OVERRIDES_PREFIX); } @@ -1023,28 +1155,27 @@ public void testProducerConfigsWithClientOverrides() { when(connectorConfig.originalsWithPrefix(ConnectorConfig.CONNECTOR_CLIENT_PRODUCER_OVERRIDES_PREFIX)).thenReturn(connConfig); assertEquals(expectedConfigs, - Worker.producerConfigs(TASK_ID.connector(), "connector-producer-" + TASK_ID, configWithOverrides, connectorConfig, null, allConnectorClientConfigOverridePolicy, CLUSTER_ID)); - + Worker.baseProducerConfigs(CONNECTOR_ID, "connector-producer-" + TASK_ID, configWithOverrides, connectorConfig, null, allConnectorClientConfigOverridePolicy, CLUSTER_ID)); verify(connectorConfig).originalsWithPrefix(ConnectorConfig.CONNECTOR_CLIENT_PRODUCER_OVERRIDES_PREFIX); } @Test public void testConsumerConfigsWithoutOverrides() { Map expectedConfigs = new HashMap<>(defaultConsumerConfigs); - expectedConfigs.put("group.id", "connect-test"); - expectedConfigs.put("client.id", "connector-consumer-test-1"); + expectedConfigs.put("group.id", "connect-test-connector"); + expectedConfigs.put("client.id", "connector-consumer-job-0"); expectedConfigs.put("metrics.context.connect.kafka.cluster.id", CLUSTER_ID); when(connectorConfig.originalsWithPrefix(ConnectorConfig.CONNECTOR_CLIENT_CONSUMER_OVERRIDES_PREFIX)).thenReturn(new HashMap<>()); - assertEquals(expectedConfigs, Worker.consumerConfigs(new ConnectorTaskId("test", 1), config, connectorConfig, - null, noneConnectorClientConfigOverridePolicy, CLUSTER_ID)); - verify(connectorConfig).originalsWithPrefix(ConnectorConfig.CONNECTOR_CLIENT_CONSUMER_OVERRIDES_PREFIX); + assertEquals(expectedConfigs, Worker.baseConsumerConfigs(CONNECTOR_ID, "connector-consumer-" + TASK_ID, config, connectorConfig, + null, noneConnectorClientConfigOverridePolicy, CLUSTER_ID, ConnectorType.SINK)); } @Test public void testConsumerConfigsWithOverrides() { Map props = new HashMap<>(workerProps); + props.put("consumer.group.id", "connect-test"); props.put("consumer.auto.offset.reset", "latest"); props.put("consumer.max.poll.records", "1000"); props.put("consumer.client.id", "consumer-test-id"); @@ -1059,9 +1190,8 @@ public void testConsumerConfigsWithOverrides() { when(connectorConfig.originalsWithPrefix(ConnectorConfig.CONNECTOR_CLIENT_CONSUMER_OVERRIDES_PREFIX)).thenReturn(new HashMap<>()); - assertEquals(expectedConfigs, Worker.consumerConfigs(new ConnectorTaskId("test", 1), configWithOverrides, connectorConfig, - null, noneConnectorClientConfigOverridePolicy, CLUSTER_ID)); - + assertEquals(expectedConfigs, Worker.baseConsumerConfigs(CONNECTOR_ID, "connector-consumer-" + TASK_ID, configWithOverrides, connectorConfig, + null, noneConnectorClientConfigOverridePolicy, CLUSTER_ID, ConnectorType.SINK)); verify(connectorConfig).originalsWithPrefix(ConnectorConfig.CONNECTOR_CLIENT_CONSUMER_OVERRIDES_PREFIX); } @@ -1073,11 +1203,11 @@ public void testConsumerConfigsWithClientOverrides() { WorkerConfig configWithOverrides = new StandaloneConfig(props); Map expectedConfigs = new HashMap<>(defaultConsumerConfigs); - expectedConfigs.put("group.id", "connect-test"); + expectedConfigs.put("group.id", "connect-test-connector"); expectedConfigs.put("auto.offset.reset", "latest"); expectedConfigs.put("max.poll.records", "5000"); expectedConfigs.put("max.poll.interval.ms", "1000"); - expectedConfigs.put("client.id", "connector-consumer-test-1"); + expectedConfigs.put("client.id", "connector-consumer-job-0"); expectedConfigs.put("metrics.context.connect.kafka.cluster.id", CLUSTER_ID); Map connConfig = new HashMap<>(); @@ -1086,9 +1216,8 @@ public void testConsumerConfigsWithClientOverrides() { when(connectorConfig.originalsWithPrefix(ConnectorConfig.CONNECTOR_CLIENT_CONSUMER_OVERRIDES_PREFIX)).thenReturn(connConfig); - assertEquals(expectedConfigs, Worker.consumerConfigs(new ConnectorTaskId("test", 1), configWithOverrides, connectorConfig, - null, allConnectorClientConfigOverridePolicy, CLUSTER_ID)); - + assertEquals(expectedConfigs, Worker.baseConsumerConfigs(CONNECTOR_ID, "connector-consumer-" + TASK_ID, configWithOverrides, connectorConfig, + null, allConnectorClientConfigOverridePolicy, CLUSTER_ID, ConnectorType.SINK)); verify(connectorConfig).originalsWithPrefix(ConnectorConfig.CONNECTOR_CLIENT_CONSUMER_OVERRIDES_PREFIX); } @@ -1104,9 +1233,8 @@ public void testConsumerConfigsClientOverridesWithNonePolicy() { connConfig.put("max.poll.interval.ms", "1000"); when(connectorConfig.originalsWithPrefix(ConnectorConfig.CONNECTOR_CLIENT_CONSUMER_OVERRIDES_PREFIX)).thenReturn(connConfig); - assertThrows(ConnectException.class, () -> Worker.consumerConfigs(new ConnectorTaskId("test", 1), - configWithOverrides, connectorConfig, null, noneConnectorClientConfigOverridePolicy, CLUSTER_ID)); - + assertThrows(ConnectException.class, () -> Worker.baseConsumerConfigs(CONNECTOR_ID, "connector-consumer-" + TASK_ID, + configWithOverrides, connectorConfig, null, noneConnectorClientConfigOverridePolicy, CLUSTER_ID, ConnectorType.SINK)); verify(connectorConfig).originalsWithPrefix(ConnectorConfig.CONNECTOR_CLIENT_CONSUMER_OVERRIDES_PREFIX); } @@ -1129,9 +1257,9 @@ public void testAdminConfigsClientOverridesWithAllPolicy() { expectedConfigs.put("metrics.context.connect.kafka.cluster.id", CLUSTER_ID); when(connectorConfig.originalsWithPrefix(ConnectorConfig.CONNECTOR_CLIENT_ADMIN_OVERRIDES_PREFIX)).thenReturn(connConfig); - assertEquals(expectedConfigs, Worker.adminConfigs(CONNECTOR_ID, "", configWithOverrides, connectorConfig, - null, allConnectorClientConfigOverridePolicy, CLUSTER_ID, ConnectorType.SOURCE)); + assertEquals(expectedConfigs, Worker.adminConfigs(CONNECTOR_ID, "", configWithOverrides, connectorConfig, + null, allConnectorClientConfigOverridePolicy, CLUSTER_ID, ConnectorType.SINK)); verify(connectorConfig).originalsWithPrefix(ConnectorConfig.CONNECTOR_CLIENT_ADMIN_OVERRIDES_PREFIX); } @@ -1145,9 +1273,9 @@ public void testAdminConfigsClientOverridesWithNonePolicy() { Map connConfig = Collections.singletonMap("metadata.max.age.ms", "10000"); when(connectorConfig.originalsWithPrefix(ConnectorConfig.CONNECTOR_CLIENT_ADMIN_OVERRIDES_PREFIX)).thenReturn(connConfig); - assertThrows(ConnectException.class, () -> Worker.adminConfigs( - CONNECTOR_ID, "", configWithOverrides, connectorConfig, null, noneConnectorClientConfigOverridePolicy, CLUSTER_ID, ConnectorType.SOURCE)); + assertThrows(ConnectException.class, () -> Worker.adminConfigs("test", + "", configWithOverrides, connectorConfig, null, noneConnectorClientConfigOverridePolicy, CLUSTER_ID, ConnectorType.SINK)); verify(connectorConfig).originalsWithPrefix(ConnectorConfig.CONNECTOR_CLIENT_ADMIN_OVERRIDES_PREFIX); } @@ -1368,11 +1496,10 @@ private void verifyTaskHeaderConverter() { verify(plugins).newHeaderConverter(any(AbstractConfig.class), eq(WorkerConfig.HEADER_CONVERTER_CLASS_CONFIG), eq(ClassLoaderUsage.CURRENT_CLASSLOADER)); } - private Map anyConnectorConfigMap() { Map props = new HashMap<>(); props.put(ConnectorConfig.NAME_CONFIG, CONNECTOR_ID); - props.put(ConnectorConfig.CONNECTOR_CLASS_CONFIG, SampleSourceConnector.class.getName()); + props.put(CONNECTOR_CLASS_CONFIG, SampleSourceConnector.class.getName()); props.put(ConnectorConfig.TASKS_MAX_CONFIG, "1"); props.put(DEFAULT_TOPIC_CREATION_PREFIX + REPLICATION_FACTOR_CONFIG, String.valueOf(1)); props.put(DEFAULT_TOPIC_CREATION_PREFIX + PARTITIONS_CONFIG, String.valueOf(1)); @@ -1402,5 +1529,27 @@ public void stop() { } } + private static class TestSinkTask extends SinkTask { + public TestSinkTask() { + } + + @Override + public String version() { + return "1.0"; + } + + @Override + public void start(Map props) { + } + + @Override + public void put(Collection records) { + } + + @Override + public void stop() { + } + + } } diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerTransactionContextTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerTransactionContextTest.java new file mode 100644 index 000000000000..3bc2b2155d1f --- /dev/null +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerTransactionContextTest.java @@ -0,0 +1,110 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.connect.runtime; + +import org.apache.kafka.connect.source.SourceRecord; +import org.junit.Test; + +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertThrows; +import static org.junit.Assert.assertTrue; + +public class WorkerTransactionContextTest { + + private static final SourceRecord RECORD = new SourceRecord(null, null, "t", null, 0, null, null); + + private WorkerTransactionContext context = new WorkerTransactionContext(); + + @Test + public void shouldNotifyOfBatchCommit() { + context.commitTransaction(); + assertFalse(context.shouldAbortBatch()); + assertFalse(context.shouldAbortOn(RECORD)); + assertFalse(context.shouldCommitOn(RECORD)); + assertTrue(context.shouldCommitBatch()); + } + + @Test + public void shouldNotifyOfRecordCommit() { + context.commitTransaction(RECORD); + assertFalse(context.shouldAbortBatch()); + assertFalse(context.shouldAbortOn(RECORD)); + assertFalse(context.shouldCommitBatch()); + assertTrue(context.shouldCommitOn(RECORD)); + } + + @Test + public void shouldNotifyOfBatchAbort() { + context.abortTransaction(); + assertFalse(context.shouldAbortOn(RECORD)); + assertFalse(context.shouldCommitOn(RECORD)); + assertFalse(context.shouldCommitBatch()); + assertTrue(context.shouldAbortBatch()); + } + + @Test + public void shouldNotifyOfRecordAbort() { + context.abortTransaction(RECORD); + assertFalse(context.shouldAbortBatch()); + assertFalse(context.shouldCommitOn(RECORD)); + assertFalse(context.shouldCommitBatch()); + assertTrue(context.shouldAbortOn(RECORD)); + } + + @Test + public void shouldNotCommitBatchRepeatedly() { + context.commitTransaction(); + assertTrue(context.shouldCommitBatch()); + assertFalse(context.shouldCommitBatch()); + } + + @Test + public void shouldNotCommitRecordRepeatedly() { + context.commitTransaction(RECORD); + assertTrue(context.shouldCommitOn(RECORD)); + assertFalse(context.shouldCommitOn(RECORD)); + } + + @Test + public void shouldNotAbortBatchRepeatedly() { + context.abortTransaction(); + assertTrue(context.shouldAbortBatch()); + assertFalse(context.shouldAbortBatch()); + } + + @Test + public void shouldNotAbortRecordRepeatedly() { + context.abortTransaction(RECORD); + assertTrue(context.shouldAbortOn(RECORD)); + assertFalse(context.shouldAbortOn(RECORD)); + } + + @Test + public void shouldDisallowConflictingRequests() { + context.commitTransaction(); + context.abortTransaction(); + assertThrows(IllegalStateException.class, context::shouldCommitBatch); + assertThrows(IllegalStateException.class, context::shouldAbortBatch); + + context = new WorkerTransactionContext(); + context.commitTransaction(RECORD); + context.abortTransaction(RECORD); + assertThrows(IllegalStateException.class, () -> context.shouldCommitOn(RECORD)); + assertThrows(IllegalStateException.class, () -> context.shouldAbortOn(RECORD)); + } + +} diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/DistributedHerderTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/DistributedHerderTest.java index 71cdcc291554..3249412259e3 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/DistributedHerderTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/DistributedHerderTest.java @@ -287,7 +287,7 @@ public void testJoinAssignment() throws Exception { EasyMock.expect(worker.isRunning(CONN1)).andReturn(true); PowerMock.expectLastCall(); EasyMock.expect(worker.connectorTaskConfigs(CONN1, conn1SinkConfig)).andReturn(TASK_CONFIGS); - worker.startTask(EasyMock.eq(TASK1), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyObject(), + worker.startSourceTask(EasyMock.eq(TASK1), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED)); PowerMock.expectLastCall().andReturn(true); member.poll(EasyMock.anyInt()); @@ -321,7 +321,7 @@ public void testRebalance() throws Exception { PowerMock.expectLastCall(); EasyMock.expect(worker.isRunning(CONN1)).andReturn(true); EasyMock.expect(worker.connectorTaskConfigs(CONN1, conn1SinkConfig)).andReturn(TASK_CONFIGS); - worker.startTask(EasyMock.eq(TASK1), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyObject(), + worker.startSourceTask(EasyMock.eq(TASK1), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED)); PowerMock.expectLastCall().andReturn(true); member.poll(EasyMock.anyInt()); @@ -394,7 +394,7 @@ public void testIncrementalCooperativeRebalanceForNewMember() throws Exception { EasyMock.expect(worker.isRunning(CONN1)).andReturn(true); EasyMock.expect(worker.connectorTaskConfigs(CONN1, conn1SinkConfig)).andReturn(TASK_CONFIGS); - worker.startTask(EasyMock.eq(TASK1), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyObject(), + worker.startSourceTask(EasyMock.eq(TASK1), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED)); PowerMock.expectLastCall().andReturn(true); member.poll(EasyMock.anyInt()); @@ -467,7 +467,7 @@ public void testIncrementalCooperativeRebalanceWithDelay() throws Exception { rebalanceDelay); expectConfigRefreshAndSnapshot(SNAPSHOT); - worker.startTask(EasyMock.eq(TASK2), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyObject(), + worker.startSourceTask(EasyMock.eq(TASK2), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED)); PowerMock.expectLastCall().andReturn(true); member.poll(EasyMock.anyInt()); @@ -499,7 +499,7 @@ public void testIncrementalCooperativeRebalanceWithDelay() throws Exception { EasyMock.expect(worker.isRunning(CONN1)).andReturn(true); EasyMock.expect(worker.connectorTaskConfigs(CONN1, conn1SinkConfig)).andReturn(TASK_CONFIGS); - worker.startTask(EasyMock.eq(TASK1), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyObject(), + worker.startSourceTask(EasyMock.eq(TASK1), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED)); PowerMock.expectLastCall().andReturn(true); member.poll(EasyMock.anyInt()); @@ -538,7 +538,7 @@ public void testRebalanceFailedConnector() throws Exception { PowerMock.expectLastCall(); EasyMock.expect(worker.isRunning(CONN1)).andReturn(true); EasyMock.expect(worker.connectorTaskConfigs(CONN1, conn1SinkConfig)).andReturn(TASK_CONFIGS); - worker.startTask(EasyMock.eq(TASK1), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyObject(), + worker.startSourceTask(EasyMock.eq(TASK1), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED)); PowerMock.expectLastCall().andReturn(true); member.poll(EasyMock.anyInt()); @@ -609,8 +609,8 @@ public void revokeAndReassign(boolean incompleteRebalance) throws TimeoutExcepti PowerMock.expectLastCall(); EasyMock.expect(worker.isRunning(CONN1)).andReturn(true); EasyMock.expect(worker.connectorTaskConfigs(CONN1, conn1SinkConfig)).andReturn(TASK_CONFIGS); - worker.startTask(EasyMock.eq(TASK1), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyObject(), - EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED)); + worker.startSourceTask(EasyMock.eq(TASK1), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyObject(), + EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED)); PowerMock.expectLastCall().andReturn(true); member.poll(EasyMock.anyInt()); PowerMock.expectLastCall(); @@ -1566,13 +1566,15 @@ public void testDoRestartConnectorAndTasksOnlyTasks() { // But only one task is assigned to this worker EasyMock.expect(herder.assignment.tasks()).andReturn(Collections.singletonList(TASK0)).anyTimes(); + herder.configState = SNAPSHOT; + worker.stopAndAwaitTasks(Collections.singletonList(TASK0)); PowerMock.expectLastCall(); herder.onRestart(TASK0); EasyMock.expectLastCall(); - worker.startTask(EasyMock.eq(TASK0), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyObject(), + worker.startSourceTask(EasyMock.eq(TASK0), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.eq(herder), EasyMock.anyObject(TargetState.class)); PowerMock.expectLastCall().andReturn(true); @@ -1597,6 +1599,8 @@ public void testDoRestartConnectorAndTasksBoth() { EasyMock.expect(herder.assignment.connectors()).andReturn(Collections.singletonList(CONN1)).anyTimes(); EasyMock.expect(herder.assignment.tasks()).andReturn(Collections.singletonList(taskId)).anyTimes(); + herder.configState = SNAPSHOT; + worker.stopAndAwaitConnector(CONN1); PowerMock.expectLastCall(); @@ -1614,7 +1618,7 @@ public void testDoRestartConnectorAndTasksBoth() { herder.onRestart(taskId); EasyMock.expectLastCall(); - worker.startTask(EasyMock.eq(TASK0), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyObject(), + worker.startSourceTask(EasyMock.eq(TASK0), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.eq(herder), EasyMock.anyObject(TargetState.class)); PowerMock.expectLastCall().andReturn(true); @@ -1634,7 +1638,7 @@ public void testRestartTask() throws Exception { expectConfigRefreshAndSnapshot(SNAPSHOT); member.poll(EasyMock.anyInt()); PowerMock.expectLastCall(); - worker.startTask(EasyMock.eq(TASK0), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyObject(), + worker.startSourceTask(EasyMock.eq(TASK0), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED)); PowerMock.expectLastCall().andReturn(true); @@ -1648,7 +1652,7 @@ public void testRestartTask() throws Exception { worker.stopAndAwaitTask(TASK0); PowerMock.expectLastCall(); - worker.startTask(EasyMock.eq(TASK0), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyObject(), + worker.startSourceTask(EasyMock.eq(TASK0), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED)); PowerMock.expectLastCall().andReturn(true); @@ -2034,7 +2038,7 @@ public void testUnknownConnectorPaused() throws Exception { // join expectRebalance(1, Collections.emptyList(), singletonList(TASK0)); expectConfigRefreshAndSnapshot(SNAPSHOT); - worker.startTask(EasyMock.eq(TASK0), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyObject(), + worker.startSourceTask(EasyMock.eq(TASK0), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED)); PowerMock.expectLastCall().andReturn(true); member.poll(EasyMock.anyInt()); @@ -2073,7 +2077,7 @@ public void testConnectorPausedRunningTaskOnly() throws Exception { // join expectRebalance(1, Collections.emptyList(), singletonList(TASK0)); expectConfigRefreshAndSnapshot(SNAPSHOT); - worker.startTask(EasyMock.eq(TASK0), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyObject(), + worker.startSourceTask(EasyMock.eq(TASK0), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED)); PowerMock.expectLastCall().andReturn(true); member.poll(EasyMock.anyInt()); @@ -2121,7 +2125,7 @@ public void testConnectorResumedRunningTaskOnly() throws Exception { // join expectRebalance(1, Collections.emptyList(), singletonList(TASK0)); expectConfigRefreshAndSnapshot(SNAPSHOT_PAUSED_CONN1); - worker.startTask(EasyMock.eq(TASK0), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyObject(), + worker.startSourceTask(EasyMock.eq(TASK0), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.eq(herder), EasyMock.eq(TargetState.PAUSED)); PowerMock.expectLastCall().andReturn(true); member.poll(EasyMock.anyInt()); @@ -2190,7 +2194,7 @@ public void testTaskConfigAdded() { expectRebalance(Collections.emptyList(), Collections.emptyList(), ConnectProtocol.Assignment.NO_ERROR, 1, Collections.emptyList(), Arrays.asList(TASK0)); - worker.startTask(EasyMock.eq(TASK0), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyObject(), + worker.startSourceTask(EasyMock.eq(TASK0), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED)); PowerMock.expectLastCall().andReturn(true); member.poll(EasyMock.anyInt()); @@ -2237,7 +2241,7 @@ public void testJoinLeaderCatchUpFails() throws Exception { PowerMock.expectLastCall(); EasyMock.expect(worker.getPlugins()).andReturn(plugins); EasyMock.expect(worker.connectorTaskConfigs(CONN1, conn1SinkConfig)).andReturn(TASK_CONFIGS); - worker.startTask(EasyMock.eq(TASK1), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyObject(), + worker.startSourceTask(EasyMock.eq(TASK1), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED)); PowerMock.expectLastCall().andReturn(true); EasyMock.expect(worker.isRunning(CONN1)).andReturn(true); @@ -2304,7 +2308,7 @@ public void testJoinLeaderCatchUpRetriesForIncrementalCooperative() throws Excep EasyMock.expect(worker.isRunning(CONN1)).andReturn(true); EasyMock.expect(worker.connectorTaskConfigs(CONN1, conn1SinkConfig)).andReturn(TASK_CONFIGS); - worker.startTask(EasyMock.eq(TASK1), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyObject(), + worker.startSourceTask(EasyMock.eq(TASK1), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED)); PowerMock.expectLastCall().andReturn(true); member.poll(EasyMock.anyInt()); @@ -2396,7 +2400,7 @@ public void testJoinLeaderCatchUpFailsForIncrementalCooperative() throws Excepti EasyMock.expect(worker.isRunning(CONN1)).andReturn(true); EasyMock.expect(worker.connectorTaskConfigs(CONN1, conn1SinkConfig)).andReturn(TASK_CONFIGS); - worker.startTask(EasyMock.eq(TASK1), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyObject(), + worker.startSourceTask(EasyMock.eq(TASK1), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED)); PowerMock.expectLastCall().andReturn(true); member.poll(EasyMock.anyInt()); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/standalone/StandaloneHerderTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/standalone/StandaloneHerderTest.java index 99848d831474..ddc030ada4c9 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/standalone/StandaloneHerderTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/standalone/StandaloneHerderTest.java @@ -369,7 +369,7 @@ public void testRestartTask() throws Exception { new HashSet<>(), new HashSet<>(), transformer); - worker.startTask(taskId, configState, connectorConfig, taskConfig(SourceSink.SOURCE), herder, TargetState.STARTED); + worker.startSourceTask(taskId, configState, connectorConfig, taskConfig(SourceSink.SOURCE), herder, TargetState.STARTED); EasyMock.expectLastCall().andReturn(true); PowerMock.replayAll(); @@ -410,7 +410,7 @@ public void testRestartTaskFailureOnStart() throws Exception { new HashSet<>(), new HashSet<>(), transformer); - worker.startTask(taskId, configState, connectorConfig, taskConfig(SourceSink.SOURCE), herder, TargetState.STARTED); + worker.startSourceTask(taskId, configState, connectorConfig, taskConfig(SourceSink.SOURCE), herder, TargetState.STARTED); EasyMock.expectLastCall().andReturn(false); PowerMock.replayAll(); @@ -583,7 +583,7 @@ public void testRestartConnectorAndTasksOnlyTasks() throws Exception { new HashSet<>(), new HashSet<>(), transformer); - worker.startTask(taskId, configState, connectorConfig, taskConfig(SourceSink.SINK), herder, TargetState.STARTED); + worker.startSinkTask(taskId, configState, connectorConfig, taskConfig(SourceSink.SINK), herder, TargetState.STARTED); EasyMock.expectLastCall().andReturn(true); PowerMock.replayAll(); @@ -649,7 +649,7 @@ public void testRestartConnectorAndTasksBoth() throws Exception { new HashSet<>(), new HashSet<>(), transformer); - worker.startTask(taskId, configState, connectorConfig, taskConfig(SourceSink.SINK), herder, TargetState.STARTED); + worker.startSinkTask(taskId, configState, connectorConfig, taskConfig(SourceSink.SINK), herder, TargetState.STARTED); EasyMock.expectLastCall().andReturn(true); PowerMock.replayAll(); @@ -918,7 +918,11 @@ private void expectAdd(SourceSink sourceSink) { new HashSet<>(), new HashSet<>(), transformer); - worker.startTask(new ConnectorTaskId(CONNECTOR_NAME, 0), configState, connectorConfig(sourceSink), generatedTaskProps, herder, TargetState.STARTED); + if (sourceSink.equals(SourceSink.SOURCE)) { + worker.startSourceTask(new ConnectorTaskId(CONNECTOR_NAME, 0), configState, connectorConfig(sourceSink), generatedTaskProps, herder, TargetState.STARTED); + } else { + worker.startSinkTask(new ConnectorTaskId(CONNECTOR_NAME, 0), configState, connectorConfig(sourceSink), generatedTaskProps, herder, TargetState.STARTED); + } EasyMock.expectLastCall().andReturn(true); EasyMock.expect(herder.connectorTypeForClass(BogusSourceConnector.class.getName())) diff --git a/core/src/main/scala/kafka/cluster/Partition.scala b/core/src/main/scala/kafka/cluster/Partition.scala index 346d2ed18418..046e21c1f9cc 100755 --- a/core/src/main/scala/kafka/cluster/Partition.scala +++ b/core/src/main/scala/kafka/cluster/Partition.scala @@ -19,7 +19,6 @@ package kafka.cluster import java.util.concurrent.locks.ReentrantReadWriteLock import java.util.Optional import java.util.concurrent.CompletableFuture - import kafka.api.LeaderAndIsr import kafka.common.UnexpectedAppendOffsetException import kafka.controller.{KafkaController, StateChangeLogger} @@ -27,9 +26,11 @@ import kafka.log._ import kafka.metrics.KafkaMetricsGroup import kafka.server._ import kafka.server.checkpoints.OffsetCheckpoints +import kafka.server.metadata.KRaftMetadataCache import kafka.utils.CoreUtils.{inReadLock, inWriteLock} import kafka.utils._ import kafka.zookeeper.ZooKeeperClientException +import org.apache.kafka.common.TopicIdPartition import org.apache.kafka.common.errors._ import org.apache.kafka.common.message.{DescribeProducersResponseData, FetchResponseData} import org.apache.kafka.common.message.LeaderAndIsrRequestData.LeaderAndIsrPartitionState @@ -143,7 +144,6 @@ sealed trait PartitionState { * the high watermark as well as determining which replicas are required for acks=all produce requests. * * Only applicable as of IBP 2.7-IV2, for older versions this will return the committed ISR - * */ def maximalIsr: Set[Int] @@ -159,48 +159,61 @@ sealed trait PartitionState { } sealed trait PendingPartitionChange extends PartitionState { + def lastCommittedState: CommittedPartitionState def sentLeaderAndIsr: LeaderAndIsr override val leaderRecoveryState: LeaderRecoveryState = LeaderRecoveryState.RECOVERED + + def notifyListener(alterPartitionListener: AlterPartitionListener): Unit } case class PendingExpandIsr( - isr: Set[Int], newInSyncReplicaId: Int, - sentLeaderAndIsr: LeaderAndIsr + sentLeaderAndIsr: LeaderAndIsr, + lastCommittedState: CommittedPartitionState ) extends PendingPartitionChange { + val isr = lastCommittedState.isr val maximalIsr = isr + newInSyncReplicaId val isInflight = true + def notifyListener(alterPartitionListener: AlterPartitionListener): Unit = { + alterPartitionListener.markIsrExpand() + } + override def toString: String = { - s"PendingExpandIsr(isr=$isr" + - s", newInSyncReplicaId=$newInSyncReplicaId" + + s"PendingExpandIsr(newInSyncReplicaId=$newInSyncReplicaId" + s", sentLeaderAndIsr=$sentLeaderAndIsr" + s", leaderRecoveryState=$leaderRecoveryState" + + s", lastCommittedState=$lastCommittedState" + ")" } } case class PendingShrinkIsr( - isr: Set[Int], outOfSyncReplicaIds: Set[Int], - sentLeaderAndIsr: LeaderAndIsr + sentLeaderAndIsr: LeaderAndIsr, + lastCommittedState: CommittedPartitionState ) extends PendingPartitionChange { + val isr = lastCommittedState.isr val maximalIsr = isr val isInflight = true + def notifyListener(alterPartitionListener: AlterPartitionListener): Unit = { + alterPartitionListener.markIsrShrink() + } + override def toString: String = { - s"PendingShrinkIsr(isr=$isr" + - s", outOfSyncReplicaIds=$outOfSyncReplicaIds" + + s"PendingShrinkIsr(outOfSyncReplicaIds=$outOfSyncReplicaIds" + s", sentLeaderAndIsr=$sentLeaderAndIsr" + s", leaderRecoveryState=$leaderRecoveryState" + + s", lastCommittedState=$lastCommittedState" + ")" } } case class CommittedPartitionState( isr: Set[Int], - override val leaderRecoveryState: LeaderRecoveryState + leaderRecoveryState: LeaderRecoveryState ) extends PartitionState { val maximalIsr = isr val isInflight = false @@ -834,10 +847,11 @@ class Partition(val topicPartition: TopicPartition, if (needsIsrUpdate) { val alterIsrUpdateOpt = inWriteLock(leaderIsrUpdateLock) { // check if this replica needs to be added to the ISR - if (!partitionState.isInflight && needsExpandIsr(followerReplica)) { - Some(prepareIsrExpand(followerReplica.brokerId)) - } else { - None + partitionState match { + case currentState: CommittedPartitionState if needsExpandIsr(followerReplica) => + Some(prepareIsrExpand(currentState, followerReplica.brokerId)) + case _ => + None } } // Send the AlterPartition request outside of the LeaderAndIsr lock since the completion logic @@ -847,21 +861,35 @@ class Partition(val topicPartition: TopicPartition, } private def needsExpandIsr(followerReplica: Replica): Boolean = { - canAddReplicaToIsr(followerReplica.brokerId) && isFollowerAtHighwatermark(followerReplica) + canAddReplicaToIsr(followerReplica.brokerId) && isFollowerInSync(followerReplica) } private def canAddReplicaToIsr(followerReplicaId: Int): Boolean = { val current = partitionState - !current.isInflight && !current.isr.contains(followerReplicaId) + !current.isInflight && + !current.isr.contains(followerReplicaId) && + isReplicaIsrEligible(followerReplicaId) } - private def isFollowerAtHighwatermark(followerReplica: Replica): Boolean = { + private def isFollowerInSync(followerReplica: Replica): Boolean = { leaderLogIfLocal.exists { leaderLog => val followerEndOffset = followerReplica.stateSnapshot.logEndOffset followerEndOffset >= leaderLog.highWatermark && leaderEpochStartOffsetOpt.exists(followerEndOffset >= _) } } + private def isReplicaIsrEligible(followerReplicaId: Int): Boolean = { + metadataCache match { + // In KRaft mode, only replicas which are not fenced nor in controlled shutdown are + // allowed to join the ISR. This does not apply to ZK mode. + case kRaftMetadataCache: KRaftMetadataCache => + !kRaftMetadataCache.isBrokerFenced(followerReplicaId) && + !kRaftMetadataCache.isBrokerShuttingDown(followerReplicaId) + + case _ => true + } + } + /* * Returns a tuple where the first element is a boolean indicating whether enough replicas reached `requiredOffset` * and the second element is an error (which would be `Errors.NONE` for no error). @@ -1009,21 +1037,22 @@ class Partition(val topicPartition: TopicPartition, val alterIsrUpdateOpt = inWriteLock(leaderIsrUpdateLock) { leaderLogIfLocal.flatMap { leaderLog => val outOfSyncReplicaIds = getOutOfSyncReplicas(replicaLagTimeMaxMs) - if (!partitionState.isInflight && outOfSyncReplicaIds.nonEmpty) { - val outOfSyncReplicaLog = outOfSyncReplicaIds.map { replicaId => - val logEndOffsetMessage = getReplica(replicaId) - .map(_.stateSnapshot.logEndOffset.toString) - .getOrElse("unknown") - s"(brokerId: $replicaId, endOffset: $logEndOffsetMessage)" - }.mkString(" ") - val newIsrLog = (partitionState.isr -- outOfSyncReplicaIds).mkString(",") - info(s"Shrinking ISR from ${partitionState.isr.mkString(",")} to $newIsrLog. " + - s"Leader: (highWatermark: ${leaderLog.highWatermark}, " + - s"endOffset: ${leaderLog.logEndOffset}). " + - s"Out of sync replicas: $outOfSyncReplicaLog.") - Some(prepareIsrShrink(outOfSyncReplicaIds)) - } else { - None + partitionState match { + case currentState: CommittedPartitionState if outOfSyncReplicaIds.nonEmpty => + val outOfSyncReplicaLog = outOfSyncReplicaIds.map { replicaId => + val logEndOffsetMessage = getReplica(replicaId) + .map(_.stateSnapshot.logEndOffset.toString) + .getOrElse("unknown") + s"(brokerId: $replicaId, endOffset: $logEndOffsetMessage)" + }.mkString(" ") + val newIsrLog = (partitionState.isr -- outOfSyncReplicaIds).mkString(",") + info(s"Shrinking ISR from ${partitionState.isr.mkString(",")} to $newIsrLog. " + + s"Leader: (highWatermark: ${leaderLog.highWatermark}, " + + s"endOffset: ${leaderLog.logEndOffset}). " + + s"Out of sync replicas: $outOfSyncReplicaLog.") + Some(prepareIsrShrink(currentState, outOfSyncReplicaIds)) + case _ => + None } } } @@ -1496,33 +1525,63 @@ class Partition(val topicPartition: TopicPartition, } } - private def prepareIsrExpand(newInSyncReplicaId: Int): PendingExpandIsr = { + private def prepareIsrExpand( + currentState: CommittedPartitionState, + newInSyncReplicaId: Int + ): PendingExpandIsr = { // When expanding the ISR, we assume that the new replica will make it into the ISR // before we receive confirmation that it has. This ensures that the HW will already // reflect the updated ISR even if there is a delay before we receive the confirmation. // Alternatively, if the update fails, no harm is done since the expanded ISR puts // a stricter requirement for advancement of the HW. val isrToSend = partitionState.isr + newInSyncReplicaId - val newLeaderAndIsr = LeaderAndIsr(localBrokerId, leaderEpoch, isrToSend.toList, partitionState.leaderRecoveryState, partitionEpoch) - val updatedState = PendingExpandIsr(partitionState.isr, newInSyncReplicaId, newLeaderAndIsr) + val newLeaderAndIsr = LeaderAndIsr( + localBrokerId, + leaderEpoch, + isrToSend.toList, + partitionState.leaderRecoveryState, + partitionEpoch + ) + val updatedState = PendingExpandIsr( + newInSyncReplicaId, + newLeaderAndIsr, + currentState + ) partitionState = updatedState updatedState } - private[cluster] def prepareIsrShrink(outOfSyncReplicaIds: Set[Int]): PendingShrinkIsr = { + private[cluster] def prepareIsrShrink( + currentState: CommittedPartitionState, + outOfSyncReplicaIds: Set[Int] + ): PendingShrinkIsr = { // When shrinking the ISR, we cannot assume that the update will succeed as this could // erroneously advance the HW if the `AlterPartition` were to fail. Hence the "maximal ISR" // for `PendingShrinkIsr` is the the current ISR. val isrToSend = partitionState.isr -- outOfSyncReplicaIds - val newLeaderAndIsr = LeaderAndIsr(localBrokerId, leaderEpoch, isrToSend.toList, partitionState.leaderRecoveryState, partitionEpoch) - val updatedState = PendingShrinkIsr(partitionState.isr, outOfSyncReplicaIds, newLeaderAndIsr) + val newLeaderAndIsr = LeaderAndIsr( + localBrokerId, + leaderEpoch, + isrToSend.toList, + partitionState.leaderRecoveryState, + partitionEpoch + ) + val updatedState = PendingShrinkIsr( + outOfSyncReplicaIds, + newLeaderAndIsr, + currentState + ) partitionState = updatedState updatedState } private def submitAlterPartition(proposedIsrState: PendingPartitionChange): CompletableFuture[LeaderAndIsr] = { debug(s"Submitting ISR state change $proposedIsrState") - val future = alterIsrManager.submit(topicPartition, proposedIsrState.sentLeaderAndIsr, controllerEpoch) + val future = alterIsrManager.submit( + new TopicIdPartition(topicId.getOrElse(Uuid.ZERO_UUID), topicPartition), + proposedIsrState.sentLeaderAndIsr, + controllerEpoch + ) future.whenComplete { (leaderAndIsr, e) => var hwIncremented = false var shouldRetry = false @@ -1569,24 +1628,47 @@ class Partition(val topicPartition: TopicPartition, ): Boolean = { alterPartitionListener.markFailed() error match { - case Errors.OPERATION_NOT_ATTEMPTED => - // Since the operation was not attempted, it is safe to reset back to the committed state. - partitionState = CommittedPartitionState(proposedIsrState.isr, LeaderRecoveryState.RECOVERED) - debug(s"Failed to alter partition to $proposedIsrState since there is a pending AlterPartition still inflight. " + - s"partition state has been reset to the latest committed state $partitionState") + case Errors.OPERATION_NOT_ATTEMPTED | Errors.INELIGIBLE_REPLICA => + // Care must be taken when resetting to the last committed state since we may not + // know in general whether the request was applied or not taking into account retries + // and controller changes which might have occurred before we received the response. + // However, when the controller returns INELIGIBLE_REPLICA (or OPERATION_NOT_ATTEMPTED), + // the controller is explicitly telling us 1) that the current partition epoch is correct, + // and 2) that the request was not applied. Even if the controller that sent the response + // is stale, we are guaranteed from the monotonicity of the controller epoch that the + // request could not have been applied by any past or future controller. + partitionState = proposedIsrState.lastCommittedState + info(s"Failed to alter partition to $proposedIsrState since the controller rejected the request with $error. " + + s"Partition state has been reset to the latest committed state $partitionState.") false case Errors.UNKNOWN_TOPIC_OR_PARTITION => debug(s"Failed to alter partition to $proposedIsrState since the controller doesn't know about " + - "this topic or partition. Giving up.") + "this topic or partition. Partition state may be out of sync, awaiting new the latest metadata.") + false + case Errors.UNKNOWN_TOPIC_ID => + debug(s"Failed to alter partition to $proposedIsrState since the controller doesn't know about " + + "this topic. Partition state may be out of sync, awaiting new the latest metadata.") false case Errors.FENCED_LEADER_EPOCH => - debug(s"Failed to alter partition to $proposedIsrState since the leader epoch is old. Giving up.") + debug(s"Failed to alter partition to $proposedIsrState since the leader epoch is old. " + + "Partition state may be out of sync, awaiting new the latest metadata.") false case Errors.INVALID_UPDATE_VERSION => - debug(s"Failed to alter partition to $proposedIsrState because the partition epoch is invalid. Giving up.") + debug(s"Failed to alter partition to $proposedIsrState because the partition epoch is invalid. " + + "Partition state may be out of sync, awaiting new the latest metadata.") false case Errors.INVALID_REQUEST => - debug(s"Failed to alter partition to $proposedIsrState because the request is invalid. Giving up.") + debug(s"Failed to alter partition to $proposedIsrState because the request is invalid. " + + "Partition state may be out of sync, awaiting new the latest metadata.") + false + case Errors.NEW_LEADER_ELECTED => + // The operation completed successfully but this replica got removed from the replica set by the controller + // while completing a ongoing reassignment. This replica is no longer the leader but it does not know it + // yet. It should remain in the current pending state until the metadata overrides it. + // This is only raised in KRaft mode. + debug(s"The alter partition request successfully updated the partition state to $proposedIsrState but " + + "this replica got removed from the replica set while completing a reassignment. " + + "Waiting on new metadata to clean up this replica.") false case _ => warn(s"Failed to update ISR to $proposedIsrState due to unexpected $error. Retrying.") @@ -1624,10 +1706,7 @@ class Partition(val topicPartition: TopicPartition, partitionEpoch = leaderAndIsr.partitionEpoch info(s"ISR updated to ${partitionState.isr.mkString(",")} and version updated to $partitionEpoch") - proposedIsrState match { - case PendingExpandIsr(_, _, _) => alterPartitionListener.markIsrExpand() - case PendingShrinkIsr(_, _, _) => alterPartitionListener.markIsrShrink() - } + proposedIsrState.notifyListener(alterPartitionListener) // we may need to increment high watermark since ISR could be down to 1 leaderLogIfLocal.exists(log => maybeIncrementLeaderHW(log)) diff --git a/core/src/main/scala/kafka/controller/KafkaController.scala b/core/src/main/scala/kafka/controller/KafkaController.scala index 289de9ab2973..8d16eb7e1da8 100644 --- a/core/src/main/scala/kafka/controller/KafkaController.scala +++ b/core/src/main/scala/kafka/controller/KafkaController.scala @@ -16,12 +16,10 @@ */ package kafka.controller -import java.util import java.util.concurrent.TimeUnit import kafka.admin.AdminOperationException import kafka.api._ import kafka.common._ -import kafka.controller.KafkaController.AlterPartitionCallback import kafka.cluster.Broker import kafka.controller.KafkaController.{AlterReassignmentsCallback, ElectLeadersCallback, ListReassignmentsCallback, UpdateFeaturesCallback} import kafka.coordinator.transaction.ZkProducerIdManager @@ -38,6 +36,7 @@ import org.apache.kafka.clients.admin.FeatureUpdate.UpgradeType import org.apache.kafka.common.ElectionType import org.apache.kafka.common.KafkaException import org.apache.kafka.common.TopicPartition +import org.apache.kafka.common.Uuid import org.apache.kafka.common.errors.{BrokerNotAvailableException, ControllerMovedException, StaleBrokerEpochException} import org.apache.kafka.common.message.{AllocateProducerIdsRequestData, AllocateProducerIdsResponseData, AlterPartitionRequestData, AlterPartitionResponseData} import org.apache.kafka.common.metrics.Metrics @@ -66,7 +65,6 @@ object KafkaController extends Logging { type ElectLeadersCallback = Map[TopicPartition, Either[ApiError, Int]] => Unit type ListReassignmentsCallback = Either[Map[TopicPartition, ReplicaAssignment], ApiError] => Unit type AlterReassignmentsCallback = Either[Map[TopicPartition, ApiError], ApiError] => Unit - type AlterPartitionCallback = Either[Map[TopicPartition, Either[Errors, LeaderAndIsr]], Errors] => Unit type UpdateFeaturesCallback = Either[ApiError, Map[String, ApiError]] => Unit } @@ -2225,197 +2223,226 @@ class KafkaController(val config: KafkaConfig, } } - def alterPartitions(alterPartitionRequest: AlterPartitionRequestData, callback: AlterPartitionResponseData => Unit): Unit = { - val partitionsToAlter = mutable.Map[TopicPartition, LeaderAndIsr]() + def alterPartitions( + alterPartitionRequest: AlterPartitionRequestData, + alterPartitionRequestVersion: Short, + callback: AlterPartitionResponseData => Unit + ): Unit = { + eventManager.put(AlterPartitionReceived( + alterPartitionRequest, + alterPartitionRequestVersion, + callback + )) + } - alterPartitionRequest.topics.forEach { topicReq => - topicReq.partitions.forEach { partitionReq => - partitionsToAlter.put( - new TopicPartition(topicReq.name, partitionReq.partitionIndex), - LeaderAndIsr( - alterPartitionRequest.brokerId, - partitionReq.leaderEpoch, - partitionReq.newIsr().asScala.toList.map(_.toInt), - LeaderRecoveryState.of(partitionReq.leaderRecoveryState), - partitionReq.partitionEpoch - ) - ) - } + private def processAlterPartition( + alterPartitionRequest: AlterPartitionRequestData, + alterPartitionRequestVersion: Short, + callback: AlterPartitionResponseData => Unit + ): Unit = { + val partitionResponses = try { + tryProcessAlterPartition( + alterPartitionRequest, + alterPartitionRequestVersion, + callback + ) + } catch { + case e: Throwable => + error(s"Error when processing AlterPartition: $alterPartitionRequest", e) + callback(new AlterPartitionResponseData().setErrorCode(Errors.UNKNOWN_SERVER_ERROR.code)) + mutable.Map.empty } - def responseCallback(results: Either[Map[TopicPartition, Either[Errors, LeaderAndIsr]], Errors]): Unit = { - val resp = new AlterPartitionResponseData() - results match { - case Right(error) => - resp.setErrorCode(error.code) - case Left(partitionResults) => - resp.setTopics(new util.ArrayList()) - partitionResults - .groupBy { case (tp, _) => tp.topic } // Group by topic - .foreach { case (topic, partitions) => - // Add each topic part to the response - val topicResp = new AlterPartitionResponseData.TopicData() - .setName(topic) - .setPartitions(new util.ArrayList()) - resp.topics.add(topicResp) - partitions.foreach { case (tp, errorOrIsr) => - // Add each partition part to the response (new ISR or error) - errorOrIsr match { - case Left(error) => topicResp.partitions.add( - new AlterPartitionResponseData.PartitionData() - .setPartitionIndex(tp.partition) - .setErrorCode(error.code)) - case Right(leaderAndIsr) => - /* Setting the LeaderRecoveryState field is always safe because it will always be the same - * as the value set in the request. For version 0, that is always the default RECOVERED - * which is ignored when serializing to version 0. For any other version, the - * LeaderRecoveryState field is supported. - */ - topicResp.partitions.add( - new AlterPartitionResponseData.PartitionData() - .setPartitionIndex(tp.partition) - .setLeaderId(leaderAndIsr.leader) - .setLeaderEpoch(leaderAndIsr.leaderEpoch) - .setIsr(leaderAndIsr.isr.map(Integer.valueOf).asJava) - .setLeaderRecoveryState(leaderAndIsr.leaderRecoveryState.value) - .setPartitionEpoch(leaderAndIsr.partitionEpoch) - ) - } - } - } + // After we have returned the result of the `AlterPartition` request, we should check whether + // there are any reassignments which can be completed by a successful ISR expansion. + partitionResponses.forKeyValue { (topicPartition, partitionResponse) => + if (controllerContext.partitionsBeingReassigned.contains(topicPartition)) { + val isSuccessfulUpdate = partitionResponse.isRight + if (isSuccessfulUpdate) { + maybeCompleteReassignment(topicPartition) + } } - callback.apply(resp) } - - eventManager.put( - AlterPartitionReceived(alterPartitionRequest.brokerId, alterPartitionRequest.brokerEpoch, partitionsToAlter, responseCallback) - ) } - private def processAlterPartition( - brokerId: Int, - brokerEpoch: Long, - partitionsToAlter: Map[TopicPartition, LeaderAndIsr], - callback: AlterPartitionCallback - ): Unit = { + private def tryProcessAlterPartition( + alterPartitionRequest: AlterPartitionRequestData, + alterPartitionRequestVersion: Short, + callback: AlterPartitionResponseData => Unit + ): mutable.Map[TopicPartition, Either[Errors, LeaderAndIsr]] = { + val useTopicsIds = alterPartitionRequestVersion > 1 // Handle a few short-circuits if (!isActive) { - callback.apply(Right(Errors.NOT_CONTROLLER)) - return + callback(new AlterPartitionResponseData().setErrorCode(Errors.NOT_CONTROLLER.code)) + return mutable.Map.empty } + val brokerId = alterPartitionRequest.brokerId + val brokerEpoch = alterPartitionRequest.brokerEpoch val brokerEpochOpt = controllerContext.liveBrokerIdAndEpochs.get(brokerId) if (brokerEpochOpt.isEmpty) { info(s"Ignoring AlterPartition due to unknown broker $brokerId") - callback.apply(Right(Errors.STALE_BROKER_EPOCH)) - return + callback(new AlterPartitionResponseData().setErrorCode(Errors.STALE_BROKER_EPOCH.code)) + return mutable.Map.empty } if (!brokerEpochOpt.contains(brokerEpoch)) { info(s"Ignoring AlterPartition due to stale broker epoch $brokerEpoch and local broker epoch $brokerEpochOpt for broker $brokerId") - callback.apply(Right(Errors.STALE_BROKER_EPOCH)) - return + callback(new AlterPartitionResponseData().setErrorCode(Errors.STALE_BROKER_EPOCH.code)) + return mutable.Map.empty } - val response = try { - val partitionResponses = mutable.HashMap[TopicPartition, Either[Errors, LeaderAndIsr]]() - - // Determine which partitions we will accept the new ISR for - val adjustedIsrs: Map[TopicPartition, LeaderAndIsr] = partitionsToAlter.flatMap { - case (tp: TopicPartition, newLeaderAndIsr: LeaderAndIsr) => - controllerContext.partitionLeadershipInfo(tp) match { - case Some(leaderIsrAndControllerEpoch) => - val currentLeaderAndIsr = leaderIsrAndControllerEpoch.leaderAndIsr - if (newLeaderAndIsr.leaderEpoch != currentLeaderAndIsr.leaderEpoch) { - partitionResponses(tp) = Left(Errors.FENCED_LEADER_EPOCH) - None - } else if (newLeaderAndIsr.partitionEpoch < currentLeaderAndIsr.partitionEpoch) { - partitionResponses(tp) = Left(Errors.INVALID_UPDATE_VERSION) - None - } else if (newLeaderAndIsr.equalsIgnorePartitionEpoch(currentLeaderAndIsr)) { - // If a partition is already in the desired state, just return it - partitionResponses(tp) = Right(currentLeaderAndIsr) - None - } else if (newLeaderAndIsr.leaderRecoveryState == LeaderRecoveryState.RECOVERING && newLeaderAndIsr.isr.length > 1) { - partitionResponses(tp) = Left(Errors.INVALID_REQUEST) - info( - s"Rejecting AlterPartition from node $brokerId for $tp because leader is recovering and ISR is greater than 1: " + - s"$newLeaderAndIsr" - ) - None - } else if (currentLeaderAndIsr.leaderRecoveryState == LeaderRecoveryState.RECOVERED && - newLeaderAndIsr.leaderRecoveryState == LeaderRecoveryState.RECOVERING) { - - partitionResponses(tp) = Left(Errors.INVALID_REQUEST) - info( - s"Rejecting AlterPartition from node $brokerId for $tp because the leader recovery state cannot change from " + - s"RECOVERED to RECOVERING: $newLeaderAndIsr" - ) - None - } else { - Some(tp -> newLeaderAndIsr) - } - case None => - partitionResponses(tp) = Left(Errors.UNKNOWN_TOPIC_OR_PARTITION) - None - } + val partitionsToAlter = new mutable.HashMap[TopicPartition, LeaderAndIsr]() + val alterPartitionResponse = new AlterPartitionResponseData() + + alterPartitionRequest.topics.forEach { topicReq => + val topicNameOpt = if (useTopicsIds) { + controllerContext.topicName(topicReq.topicId) + } else { + Some(topicReq.topicName) } - // Do the updates in ZK - debug(s"Updating ISRs for partitions: ${adjustedIsrs.keySet}.") - val UpdateLeaderAndIsrResult(finishedUpdates, badVersionUpdates) = zkClient.updateLeaderAndIsr( - adjustedIsrs, controllerContext.epoch, controllerContext.epochZkVersion) - - val successfulUpdates: Map[TopicPartition, LeaderAndIsr] = finishedUpdates.flatMap { - case (partition: TopicPartition, isrOrError: Either[Throwable, LeaderAndIsr]) => - isrOrError match { - case Right(updatedIsr) => - debug(s"ISR for partition $partition updated to [${updatedIsr.isr.mkString(",")}] and zkVersion updated to [${updatedIsr.partitionEpoch}]") - partitionResponses(partition) = Right(updatedIsr) - Some(partition -> updatedIsr) - case Left(e) => - error(s"Failed to update ISR for partition $partition", e) - partitionResponses(partition) = Left(Errors.forException(e)) - None + topicNameOpt match { + case None => + val topicResponse = new AlterPartitionResponseData.TopicData() + .setTopicId(topicReq.topicId) + alterPartitionResponse.topics.add(topicResponse) + topicReq.partitions.forEach { partitionReq => + topicResponse.partitions.add(new AlterPartitionResponseData.PartitionData() + .setPartitionIndex(partitionReq.partitionIndex) + .setErrorCode(Errors.UNKNOWN_TOPIC_ID.code)) } - } - badVersionUpdates.foreach { partition => - info(s"Failed to update ISR to ${adjustedIsrs(partition)} for partition $partition, bad ZK version.") - partitionResponses(partition) = Left(Errors.INVALID_UPDATE_VERSION) + case Some(topicName) => + topicReq.partitions.forEach { partitionReq => + partitionsToAlter.put( + new TopicPartition(topicName, partitionReq.partitionIndex), + LeaderAndIsr( + alterPartitionRequest.brokerId, + partitionReq.leaderEpoch, + partitionReq.newIsr.asScala.toList.map(_.toInt), + LeaderRecoveryState.of(partitionReq.leaderRecoveryState), + partitionReq.partitionEpoch + ) + ) + } } + } + + val partitionResponses = mutable.HashMap[TopicPartition, Either[Errors, LeaderAndIsr]]() + // Determine which partitions we will accept the new ISR for + val adjustedIsrs = partitionsToAlter.flatMap { case (tp, newLeaderAndIsr) => + controllerContext.partitionLeadershipInfo(tp) match { + case Some(leaderIsrAndControllerEpoch) => + val currentLeaderAndIsr = leaderIsrAndControllerEpoch.leaderAndIsr + if (newLeaderAndIsr.leaderEpoch != currentLeaderAndIsr.leaderEpoch) { + partitionResponses(tp) = Left(Errors.FENCED_LEADER_EPOCH) + None + } else if (newLeaderAndIsr.partitionEpoch < currentLeaderAndIsr.partitionEpoch) { + partitionResponses(tp) = Left(Errors.INVALID_UPDATE_VERSION) + None + } else if (newLeaderAndIsr.equalsIgnorePartitionEpoch(currentLeaderAndIsr)) { + // If a partition is already in the desired state, just return it + partitionResponses(tp) = Right(currentLeaderAndIsr) + None + } else if (newLeaderAndIsr.leaderRecoveryState == LeaderRecoveryState.RECOVERING && newLeaderAndIsr.isr.length > 1) { + partitionResponses(tp) = Left(Errors.INVALID_REQUEST) + info( + s"Rejecting AlterPartition from node $brokerId for $tp because leader is recovering and ISR is greater than 1: " + + s"$newLeaderAndIsr" + ) + None + } else if (currentLeaderAndIsr.leaderRecoveryState == LeaderRecoveryState.RECOVERED && + newLeaderAndIsr.leaderRecoveryState == LeaderRecoveryState.RECOVERING) { + + partitionResponses(tp) = Left(Errors.INVALID_REQUEST) + info( + s"Rejecting AlterPartition from node $brokerId for $tp because the leader recovery state cannot change from " + + s"RECOVERED to RECOVERING: $newLeaderAndIsr" + ) + None + } else { + Some(tp -> newLeaderAndIsr) + } - def processUpdateNotifications(partitions: Seq[TopicPartition]): Unit = { - val liveBrokers: Seq[Int] = controllerContext.liveOrShuttingDownBrokerIds.toSeq - sendUpdateMetadataRequest(liveBrokers, partitions.toSet) + case None => + partitionResponses(tp) = Left(Errors.UNKNOWN_TOPIC_OR_PARTITION) + None } + } - // Update our cache and send out metadata updates - updateLeaderAndIsrCache(successfulUpdates.keys.toSeq) - processUpdateNotifications(partitionsToAlter.keys.toSeq) + // Do the updates in ZK + debug(s"Updating ISRs for partitions: ${adjustedIsrs.keySet}.") + val UpdateLeaderAndIsrResult(finishedUpdates, badVersionUpdates) = zkClient.updateLeaderAndIsr( + adjustedIsrs, controllerContext.epoch, controllerContext.epochZkVersion) + + val successfulUpdates = finishedUpdates.flatMap { case (partition, isrOrError) => + isrOrError match { + case Right(updatedIsr) => + debug(s"ISR for partition $partition updated to $updatedIsr.") + partitionResponses(partition) = Right(updatedIsr) + Some(partition -> updatedIsr) + case Left(e) => + error(s"Failed to update ISR for partition $partition", e) + partitionResponses(partition) = Left(Errors.forException(e)) + None + } + } - Left(partitionResponses) - } catch { - case e: Throwable => - error(s"Error when processing AlterPartition for partitions: ${partitionsToAlter.keys.toSeq}", e) - Right(Errors.UNKNOWN_SERVER_ERROR) + badVersionUpdates.foreach { partition => + info(s"Failed to update ISR to ${adjustedIsrs(partition)} for partition $partition, bad ZK version.") + partitionResponses(partition) = Left(Errors.INVALID_UPDATE_VERSION) } - callback.apply(response) + // Update our cache and send out metadata updates + updateLeaderAndIsrCache(successfulUpdates.keys.toSeq) + sendUpdateMetadataRequest( + controllerContext.liveOrShuttingDownBrokerIds.toSeq, + partitionsToAlter.keySet + ) - // After we have returned the result of the `AlterPartition` request, we should check whether - // there are any reassignments which can be completed by a successful ISR expansion. - response.left.foreach { alterPartitionResponses => - alterPartitionResponses.forKeyValue { (topicPartition, partitionResponse) => - if (controllerContext.partitionsBeingReassigned.contains(topicPartition)) { - val isSuccessfulUpdate = partitionResponse.isRight - if (isSuccessfulUpdate) { - maybeCompleteReassignment(topicPartition) - } + partitionResponses.groupBy(_._1.topic).forKeyValue { (topicName, partitionResponses) => + // Add each topic part to the response + val topicResponse = if (useTopicsIds) { + new AlterPartitionResponseData.TopicData() + .setTopicId(controllerContext.topicIds.getOrElse(topicName, Uuid.ZERO_UUID)) + } else { + new AlterPartitionResponseData.TopicData() + .setTopicName(topicName) + } + alterPartitionResponse.topics.add(topicResponse) + + partitionResponses.forKeyValue { (tp, errorOrIsr) => + // Add each partition part to the response (new ISR or error) + errorOrIsr match { + case Left(error) => + topicResponse.partitions.add( + new AlterPartitionResponseData.PartitionData() + .setPartitionIndex(tp.partition) + .setErrorCode(error.code)) + case Right(leaderAndIsr) => + /* Setting the LeaderRecoveryState field is always safe because it will always be the same + * as the value set in the request. For version 0, that is always the default RECOVERED + * which is ignored when serializing to version 0. For any other version, the + * LeaderRecoveryState field is supported. + */ + topicResponse.partitions.add( + new AlterPartitionResponseData.PartitionData() + .setPartitionIndex(tp.partition) + .setLeaderId(leaderAndIsr.leader) + .setLeaderEpoch(leaderAndIsr.leaderEpoch) + .setIsr(leaderAndIsr.isr.map(Integer.valueOf).asJava) + .setLeaderRecoveryState(leaderAndIsr.leaderRecoveryState.value) + .setPartitionEpoch(leaderAndIsr.partitionEpoch) + ) } } } + + callback(alterPartitionResponse) + + partitionResponses } def allocateProducerIds(allocateProducerIdsRequest: AllocateProducerIdsRequestData, @@ -2542,8 +2569,8 @@ class KafkaController(val config: KafkaConfig, processPartitionReassignmentIsrChange(partition) case IsrChangeNotification => processIsrChangeNotification() - case AlterPartitionReceived(brokerId, brokerEpoch, partitionsToAlter, callback) => - processAlterPartition(brokerId, brokerEpoch, partitionsToAlter, callback) + case AlterPartitionReceived(alterPartitionRequest, alterPartitionRequestVersion, callback) => + processAlterPartition(alterPartitionRequest, alterPartitionRequestVersion, callback) case AllocateProducerIds(brokerId, brokerEpoch, callback) => processAllocateProducerIds(brokerId, brokerEpoch, callback) case Startup => @@ -2806,7 +2833,9 @@ case object IsrChangeNotification extends ControllerEvent { } case class AlterPartitionReceived( - brokerId: Int, brokerEpoch: Long, partitionsToAlter: Map[TopicPartition, LeaderAndIsr], callback: AlterPartitionCallback + alterPartitionRequest: AlterPartitionRequestData, + alterPartitionRequestVersion: Short, + callback: AlterPartitionResponseData => Unit ) extends ControllerEvent { override def state: ControllerState = ControllerState.IsrChange override def preempt(): Unit = {} diff --git a/core/src/main/scala/kafka/server/AlterPartitionManager.scala b/core/src/main/scala/kafka/server/AlterPartitionManager.scala index ecae4dd27641..d791e0a0ddb0 100644 --- a/core/src/main/scala/kafka/server/AlterPartitionManager.scala +++ b/core/src/main/scala/kafka/server/AlterPartitionManager.scala @@ -19,17 +19,19 @@ package kafka.server import java.util import java.util.concurrent.atomic.AtomicBoolean import java.util.concurrent.{CompletableFuture, ConcurrentHashMap, TimeUnit} - import kafka.api.LeaderAndIsr import kafka.metrics.KafkaMetricsGroup import kafka.utils.{KafkaScheduler, Logging, Scheduler} import kafka.zk.KafkaZkClient import org.apache.kafka.clients.ClientResponse +import org.apache.kafka.common.TopicIdPartition import org.apache.kafka.common.TopicPartition +import org.apache.kafka.common.Uuid import org.apache.kafka.common.errors.OperationNotAttemptedException import org.apache.kafka.common.message.AlterPartitionRequestData import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.protocol.Errors +import org.apache.kafka.common.requests.RequestHeader import org.apache.kafka.common.requests.{AlterPartitionRequest, AlterPartitionResponse} import org.apache.kafka.common.utils.Time import org.apache.kafka.metadata.LeaderRecoveryState @@ -54,14 +56,14 @@ trait AlterPartitionManager { def shutdown(): Unit = {} def submit( - topicPartition: TopicPartition, + topicIdPartition: TopicIdPartition, leaderAndIsr: LeaderAndIsr, controllerEpoch: Int ): CompletableFuture[LeaderAndIsr] } case class AlterPartitionItem( - topicPartition: TopicPartition, + topicIdPartition: TopicIdPartition, leaderAndIsr: LeaderAndIsr, future: CompletableFuture[LeaderAndIsr], controllerEpoch: Int // controllerEpoch needed for `ZkAlterPartitionManager` @@ -112,7 +114,6 @@ object AlterPartitionManager { ): AlterPartitionManager = { new ZkAlterPartitionManager(scheduler, time, zkClient) } - } class DefaultAlterPartitionManager( @@ -124,7 +125,20 @@ class DefaultAlterPartitionManager( val metadataVersionSupplier: () => MetadataVersion ) extends AlterPartitionManager with Logging with KafkaMetricsGroup { - // Used to allow only one pending ISR update per partition (visible for testing) + // Used to allow only one pending ISR update per partition (visible for testing). + // Note that we key items by TopicPartition despite using TopicIdPartition while + // submitting changes. We do this to ensure that topics with the same name but + // with a different topic id or no topic id collide here. There are two cases to + // consider: + // 1) When the cluster is upgraded from IBP < 2.8 to IBP >= 2.8, the ZK controller + // assigns topic ids to the partitions. So partitions will start sending updates + // with a topic id while they might still have updates without topic ids in this + // Map. This would break the contract of only allowing one pending ISR update per + // partition. + // 2) When a topic is deleted and re-created, we cannot have two entries in this Map + // especially if we cannot use an AlterPartition request version which supports + // topic ids in the end because the two updates with the same name would be merged + // together. private[server] val unsentIsrUpdates: util.Map[TopicPartition, AlterPartitionItem] = new ConcurrentHashMap[TopicPartition, AlterPartitionItem]() // Used to allow only one in-flight request at a time @@ -139,18 +153,18 @@ class DefaultAlterPartitionManager( } override def submit( - topicPartition: TopicPartition, + topicIdPartition: TopicIdPartition, leaderAndIsr: LeaderAndIsr, controllerEpoch: Int ): CompletableFuture[LeaderAndIsr] = { val future = new CompletableFuture[LeaderAndIsr]() - val alterPartitionItem = AlterPartitionItem(topicPartition, leaderAndIsr, future, controllerEpoch) - val enqueued = unsentIsrUpdates.putIfAbsent(alterPartitionItem.topicPartition, alterPartitionItem) == null + val alterPartitionItem = AlterPartitionItem(topicIdPartition, leaderAndIsr, future, controllerEpoch) + val enqueued = unsentIsrUpdates.putIfAbsent(alterPartitionItem.topicIdPartition.topicPartition, alterPartitionItem) == null if (enqueued) { maybePropagateIsrChanges() } else { future.completeExceptionally(new OperationNotAttemptedException( - s"Failed to enqueue ISR change state $leaderAndIsr for partition $topicPartition")) + s"Failed to enqueue ISR change state $leaderAndIsr for partition $topicIdPartition")) } future } @@ -172,13 +186,14 @@ class DefaultAlterPartitionManager( } private def sendRequest(inflightAlterPartitionItems: Seq[AlterPartitionItem]): Unit = { - val message = buildRequest(inflightAlterPartitionItems) - debug(s"Sending AlterPartition to controller $message") + val brokerEpoch = brokerEpochSupplier() + val (request, topicNamesByIds) = buildRequest(inflightAlterPartitionItems, brokerEpoch) + debug(s"Sending AlterPartition to controller $request") // We will not timeout AlterPartition request, instead letting it retry indefinitely // until a response is received, or a new LeaderAndIsr overwrites the existing isrState // which causes the response for those partitions to be ignored. - controllerChannelManager.sendRequest(new AlterPartitionRequest.Builder(message), + controllerChannelManager.sendRequest(request, new ControllerRequestCompletionHandler { override def onComplete(response: ClientResponse): Unit = { debug(s"Received AlterPartition response $response") @@ -192,8 +207,13 @@ class DefaultAlterPartitionManager( } else if (response.versionMismatch != null) { Errors.UNSUPPORTED_VERSION } else { - val body = response.responseBody().asInstanceOf[AlterPartitionResponse] - handleAlterPartitionResponse(body, message.brokerEpoch, inflightAlterPartitionItems) + handleAlterPartitionResponse( + response.requestHeader, + response.responseBody.asInstanceOf[AlterPartitionResponse], + brokerEpoch, + inflightAlterPartitionItems, + topicNamesByIds + ) } } finally { // clear the flag so future requests can proceed @@ -217,36 +237,74 @@ class DefaultAlterPartitionManager( }) } - private def buildRequest(inflightAlterPartitionItems: Seq[AlterPartitionItem]): AlterPartitionRequestData = { + /** + * Builds an AlterPartition request. + * + * While building the request, we don't know which version of the AlterPartition API is + * supported by the controller. The final decision is taken when the AlterPartitionRequest + * is built in the network client based on the advertised api versions of the controller. + * + * We could use version 2 or above if all the pending changes have an topic id defined; + * otherwise we must use version 1 or below. + * + * @return A tuple containing the AlterPartitionRequest.Builder and a mapping from + * topic id to topic name. This mapping is used in the response handling. + */ + private def buildRequest( + inflightAlterPartitionItems: Seq[AlterPartitionItem], + brokerEpoch: Long + ): (AlterPartitionRequest.Builder, mutable.Map[Uuid, String]) = { + val metadataVersion = metadataVersionSupplier() + // We build this mapping in order to map topic id back to their name when we + // receive the response. We cannot rely on the metadata cache for this because + // the metadata cache is updated after the partition state so it might not know + // yet about a topic id already used here. + val topicNamesByIds = mutable.HashMap[Uuid, String]() + // We can use topic ids only if all the pending changed have one defined and + // we use IBP 2.8 or above. + var canUseTopicIds = metadataVersion.isTopicIdsSupported + val message = new AlterPartitionRequestData() .setBrokerId(brokerId) - .setBrokerEpoch(brokerEpochSupplier.apply()) + .setBrokerEpoch(brokerEpoch) - inflightAlterPartitionItems.groupBy(_.topicPartition.topic).foreach { case (topic, items) => + inflightAlterPartitionItems.groupBy(_.topicIdPartition.topic).foreach { case (topicName, items) => + val topicId = items.head.topicIdPartition.topicId + canUseTopicIds &= topicId != Uuid.ZERO_UUID + topicNamesByIds(topicId) = topicName + + // Both the topic name and the topic id are set here because at this stage + // we don't know which version of the request will be used. val topicData = new AlterPartitionRequestData.TopicData() - .setName(topic) + .setTopicName(topicName) + .setTopicId(topicId) message.topics.add(topicData) + items.foreach { item => val partitionData = new AlterPartitionRequestData.PartitionData() - .setPartitionIndex(item.topicPartition.partition) + .setPartitionIndex(item.topicIdPartition.partition) .setLeaderEpoch(item.leaderAndIsr.leaderEpoch) .setNewIsr(item.leaderAndIsr.isr.map(Integer.valueOf).asJava) .setPartitionEpoch(item.leaderAndIsr.partitionEpoch) - if (metadataVersionSupplier().isAtLeast(MetadataVersion.IBP_3_2_IV0)) { + if (metadataVersion.isLeaderRecoverySupported) { partitionData.setLeaderRecoveryState(item.leaderAndIsr.leaderRecoveryState.value) } topicData.partitions.add(partitionData) } } - message + + // If we cannot use topic ids, the builder will ensure that no version higher than 1 is used. + (new AlterPartitionRequest.Builder(message, canUseTopicIds), topicNamesByIds) } def handleAlterPartitionResponse( + requestHeader: RequestHeader, alterPartitionResp: AlterPartitionResponse, sentBrokerEpoch: Long, - inflightAlterPartitionItems: Seq[AlterPartitionItem] + inflightAlterPartitionItems: Seq[AlterPartitionItem], + topicNamesByIds: mutable.Map[Uuid, String] ): Errors = { val data = alterPartitionResp.data @@ -260,31 +318,37 @@ class DefaultAlterPartitionManager( case Errors.NONE => // Collect partition-level responses to pass to the callbacks - val partitionResponses = new mutable.HashMap[TopicPartition, Either[Errors, LeaderAndIsr]]() + val partitionResponses = new mutable.HashMap[TopicIdPartition, Either[Errors, LeaderAndIsr]]() data.topics.forEach { topic => - topic.partitions.forEach { partition => - val tp = new TopicPartition(topic.name, partition.partitionIndex) - val apiError = Errors.forCode(partition.errorCode) - debug(s"Controller successfully handled AlterPartition request for $tp: $partition") - if (apiError == Errors.NONE) { - LeaderRecoveryState.optionalOf(partition.leaderRecoveryState).asScala match { - case Some(leaderRecoveryState) => - partitionResponses(tp) = Right( - LeaderAndIsr( - partition.leaderId, - partition.leaderEpoch, - partition.isr.asScala.toList.map(_.toInt), - leaderRecoveryState, - partition.partitionEpoch + // Topic IDs are used since version 2 of the AlterPartition API. + val topicName = if (requestHeader.apiVersion > 1) topicNamesByIds.get(topic.topicId).orNull else topic.topicName + if (topicName == null || topicName.isEmpty) { + error(s"Received an unexpected topic $topic in the alter partition response, ignoring it.") + } else { + topic.partitions.forEach { partition => + val tp = new TopicIdPartition(topic.topicId, partition.partitionIndex, topicName) + val apiError = Errors.forCode(partition.errorCode) + debug(s"Controller successfully handled AlterPartition request for $tp: $partition") + if (apiError == Errors.NONE) { + LeaderRecoveryState.optionalOf(partition.leaderRecoveryState).asScala match { + case Some(leaderRecoveryState) => + partitionResponses(tp) = Right( + LeaderAndIsr( + partition.leaderId, + partition.leaderEpoch, + partition.isr.asScala.toList.map(_.toInt), + leaderRecoveryState, + partition.partitionEpoch + ) ) - ) - case None => - error(s"Controller returned an invalid leader recovery state (${partition.leaderRecoveryState}) for $tp: $partition") - partitionResponses(tp) = Left(Errors.UNKNOWN_SERVER_ERROR) + case None => + error(s"Controller returned an invalid leader recovery state (${partition.leaderRecoveryState}) for $tp: $partition") + partitionResponses(tp) = Left(Errors.UNKNOWN_SERVER_ERROR) + } + } else { + partitionResponses(tp) = Left(apiError) } - } else { - partitionResponses(tp) = Left(apiError) } } } @@ -293,7 +357,7 @@ class DefaultAlterPartitionManager( // partition was somehow erroneously excluded from the response. Note that these callbacks are run from // the leaderIsrUpdateLock write lock in Partition#sendAlterPartitionRequest inflightAlterPartitionItems.foreach { inflightAlterPartition => - partitionResponses.get(inflightAlterPartition.topicPartition) match { + partitionResponses.get(inflightAlterPartition.topicIdPartition) match { case Some(leaderAndIsrOrError) => try { leaderAndIsrOrError match { @@ -302,11 +366,11 @@ class DefaultAlterPartitionManager( } } finally { // Regardless of callback outcome, we need to clear from the unsent updates map to unblock further updates - unsentIsrUpdates.remove(inflightAlterPartition.topicPartition) + unsentIsrUpdates.remove(inflightAlterPartition.topicIdPartition.topicPartition) } case None => // Don't remove this partition from the update map so it will get re-sent - warn(s"Partition ${inflightAlterPartition.topicPartition} was sent but not included in the response") + warn(s"Partition ${inflightAlterPartition.topicIdPartition} was sent but not included in the response") } } diff --git a/core/src/main/scala/kafka/server/BrokerFeatures.scala b/core/src/main/scala/kafka/server/BrokerFeatures.scala index d385f1eb0775..70ef7c71cbba 100644 --- a/core/src/main/scala/kafka/server/BrokerFeatures.scala +++ b/core/src/main/scala/kafka/server/BrokerFeatures.scala @@ -73,7 +73,7 @@ object BrokerFeatures extends Logging { def createDefault(): BrokerFeatures = { new BrokerFeatures(Features.supportedFeatures( java.util.Collections.singletonMap(MetadataVersion.FEATURE_NAME, - new SupportedVersionRange(MetadataVersion.IBP_3_0_IV0.featureLevel(), MetadataVersion.latest().featureLevel())))) + new SupportedVersionRange(MetadataVersion.MINIMUM_KRAFT_VERSION.featureLevel(), MetadataVersion.latest().featureLevel())))) } def createEmpty(): BrokerFeatures = { diff --git a/core/src/main/scala/kafka/server/ControllerApis.scala b/core/src/main/scala/kafka/server/ControllerApis.scala index 0aaab0aefb82..85ae7d70b6d8 100644 --- a/core/src/main/scala/kafka/server/ControllerApis.scala +++ b/core/src/main/scala/kafka/server/ControllerApis.scala @@ -153,8 +153,8 @@ class ControllerApis(val requestChannel: RequestChannel, def handleDeleteTopics(request: RequestChannel.Request): Unit = { val deleteTopicsRequest = request.body[DeleteTopicsRequest] - val context = new ControllerRequestContext(request.context.principal, - requestTimeoutMsToDeadlineNs(time, deleteTopicsRequest.data().timeoutMs())) + val context = new ControllerRequestContext(request.context.header.data, request.context.principal, + requestTimeoutMsToDeadlineNs(time, deleteTopicsRequest.data.timeoutMs)) val future = deleteTopics(context, deleteTopicsRequest.data, request.context.apiVersion, @@ -317,10 +317,10 @@ class ControllerApis(val requestChannel: RequestChannel, def handleCreateTopics(request: RequestChannel.Request): Unit = { val createTopicsRequest = request.body[CreateTopicsRequest] - val context = new ControllerRequestContext(request.context.principal, - requestTimeoutMsToDeadlineNs(time, createTopicsRequest.data().timeoutMs())) + val context = new ControllerRequestContext(request.context.header.data, request.context.principal, + requestTimeoutMsToDeadlineNs(time, createTopicsRequest.data.timeoutMs)) val future = createTopics(context, - createTopicsRequest.data(), + createTopicsRequest.data, authHelper.authorize(request.context, CREATE, CLUSTER, CLUSTER_NAME, logIfDenied = false), names => authHelper.filterByAuthorized(request.context, CREATE, TOPIC, names)(identity), names => authHelper.filterByAuthorized(request.context, DESCRIBE_CONFIGS, TOPIC, @@ -429,7 +429,7 @@ class ControllerApis(val requestChannel: RequestChannel, def handleLegacyAlterConfigs(request: RequestChannel.Request): Unit = { val response = new AlterConfigsResponseData() val alterConfigsRequest = request.body[AlterConfigsRequest] - val context = new ControllerRequestContext(request.context.principal, OptionalLong.empty()) + val context = new ControllerRequestContext(request.context.header.data, request.context.principal, OptionalLong.empty()) val duplicateResources = new util.HashSet[ConfigResource] val configChanges = new util.HashMap[ConfigResource, util.Map[String, String]]() alterConfigsRequest.data.resources.forEach { resource => @@ -508,8 +508,8 @@ class ControllerApis(val requestChannel: RequestChannel, def handleElectLeaders(request: RequestChannel.Request): Unit = { authHelper.authorizeClusterOperation(request, ALTER) val electLeadersRequest = request.body[ElectLeadersRequest] - val context = new ControllerRequestContext(request.context.principal, - requestTimeoutMsToDeadlineNs(time, electLeadersRequest.data().timeoutMs())) + val context = new ControllerRequestContext(request.context.header.data, request.context.principal, + requestTimeoutMsToDeadlineNs(time, electLeadersRequest.data.timeoutMs)) val future = controller.electLeaders(context, electLeadersRequest.data) future.whenComplete { (responseData, exception) => if (exception != null) { @@ -526,7 +526,7 @@ class ControllerApis(val requestChannel: RequestChannel, def handleAlterPartitionRequest(request: RequestChannel.Request): Unit = { val alterPartitionRequest = request.body[AlterPartitionRequest] - val context = new ControllerRequestContext(request.context.principal, + val context = new ControllerRequestContext(request.context.header.data, request.context.principal, OptionalLong.empty()) authHelper.authorizeClusterOperation(request, CLUSTER_ACTION) val future = controller.alterPartition(context, alterPartitionRequest.data) @@ -543,7 +543,7 @@ class ControllerApis(val requestChannel: RequestChannel, def handleBrokerHeartBeatRequest(request: RequestChannel.Request): Unit = { val heartbeatRequest = request.body[BrokerHeartbeatRequest] authHelper.authorizeClusterOperation(request, CLUSTER_ACTION) - val context = new ControllerRequestContext(request.context.principal, + val context = new ControllerRequestContext(request.context.header.data, request.context.principal, requestTimeoutMsToDeadlineNs(time, config.brokerHeartbeatIntervalMs)) controller.processBrokerHeartbeat(context, heartbeatRequest.data).handle[Unit] { (reply, e) => def createResponseCallback(requestThrottleMs: Int, @@ -570,10 +570,10 @@ class ControllerApis(val requestChannel: RequestChannel, def handleUnregisterBroker(request: RequestChannel.Request): Unit = { val decommissionRequest = request.body[UnregisterBrokerRequest] authHelper.authorizeClusterOperation(request, ALTER) - val context = new ControllerRequestContext(request.context.principal, + val context = new ControllerRequestContext(request.context.header.data, request.context.principal, OptionalLong.empty()) - controller.unregisterBroker(context, decommissionRequest.data().brokerId()).handle[Unit] { (_, e) => + controller.unregisterBroker(context, decommissionRequest.data.brokerId).handle[Unit] { (_, e) => def createResponseCallback(requestThrottleMs: Int, e: Throwable): UnregisterBrokerResponse = { if (e != null) { @@ -593,7 +593,7 @@ class ControllerApis(val requestChannel: RequestChannel, def handleBrokerRegistration(request: RequestChannel.Request): Unit = { val registrationRequest = request.body[BrokerRegistrationRequest] authHelper.authorizeClusterOperation(request, CLUSTER_ACTION) - val context = new ControllerRequestContext(request.context.principal, + val context = new ControllerRequestContext(request.context.header.data, request.context.principal, OptionalLong.empty()) controller.registerBroker(context, registrationRequest.data).handle[Unit] { (reply, e) => @@ -634,7 +634,7 @@ class ControllerApis(val requestChannel: RequestChannel, def handleAlterClientQuotas(request: RequestChannel.Request): Unit = { val quotaRequest = request.body[AlterClientQuotasRequest] authHelper.authorizeClusterOperation(request, ALTER_CONFIGS) - val context = new ControllerRequestContext(request.context.principal, + val context = new ControllerRequestContext(request.context.header.data, request.context.principal, OptionalLong.empty()) controller.alterClientQuotas(context, quotaRequest.entries, quotaRequest.validateOnly) .whenComplete { (results, exception) => @@ -650,7 +650,7 @@ class ControllerApis(val requestChannel: RequestChannel, def handleIncrementalAlterConfigs(request: RequestChannel.Request): Unit = { val response = new IncrementalAlterConfigsResponseData() val alterConfigsRequest = request.body[IncrementalAlterConfigsRequest] - val context = new ControllerRequestContext(request.context.principal, + val context = new ControllerRequestContext(request.context.header.data, request.context.principal, OptionalLong.empty()) val duplicateResources = new util.HashSet[ConfigResource] val configChanges = new util.HashMap[ConfigResource, @@ -716,8 +716,8 @@ class ControllerApis(val requestChannel: RequestChannel, authHelper.filterByAuthorized(request.context, ALTER, TOPIC, topics)(n => n) } val createPartitionsRequest = request.body[CreatePartitionsRequest] - val context = new ControllerRequestContext(request.context.principal, - requestTimeoutMsToDeadlineNs(time, createPartitionsRequest.data().timeoutMs())) + val context = new ControllerRequestContext(request.context.header.data, request.context.principal, + requestTimeoutMsToDeadlineNs(time, createPartitionsRequest.data.timeoutMs)) val future = createPartitions(context, createPartitionsRequest.data(), filterAlterAuthorizedTopics) @@ -776,9 +776,9 @@ class ControllerApis(val requestChannel: RequestChannel, def handleAlterPartitionReassignments(request: RequestChannel.Request): Unit = { val alterRequest = request.body[AlterPartitionReassignmentsRequest] authHelper.authorizeClusterOperation(request, ALTER) - val context = new ControllerRequestContext(request.context.principal, - requestTimeoutMsToDeadlineNs(time, alterRequest.data().timeoutMs())) - val response = controller.alterPartitionReassignments(context, alterRequest.data()).get() + val context = new ControllerRequestContext(request.context.header.data, request.context.principal, + requestTimeoutMsToDeadlineNs(time, alterRequest.data.timeoutMs)) + val response = controller.alterPartitionReassignments(context, alterRequest.data).get() requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs => new AlterPartitionReassignmentsResponse(response.setThrottleTimeMs(requestThrottleMs))) } @@ -786,9 +786,9 @@ class ControllerApis(val requestChannel: RequestChannel, def handleListPartitionReassignments(request: RequestChannel.Request): Unit = { val listRequest = request.body[ListPartitionReassignmentsRequest] authHelper.authorizeClusterOperation(request, DESCRIBE) - val context = new ControllerRequestContext(request.context.principal, + val context = new ControllerRequestContext(request.context.header.data, request.context.principal, OptionalLong.empty()) - val response = controller.listPartitionReassignments(context, listRequest.data()).get() + val response = controller.listPartitionReassignments(context, listRequest.data).get() requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs => new ListPartitionReassignmentsResponse(response.setThrottleTimeMs(requestThrottleMs))) } @@ -796,7 +796,7 @@ class ControllerApis(val requestChannel: RequestChannel, def handleAllocateProducerIdsRequest(request: RequestChannel.Request): Unit = { val allocatedProducerIdsRequest = request.body[AllocateProducerIdsRequest] authHelper.authorizeClusterOperation(request, CLUSTER_ACTION) - val context = new ControllerRequestContext(request.context.principal, + val context = new ControllerRequestContext(request.context.header.data, request.context.principal, OptionalLong.empty()) controller.allocateProducerIds(context, allocatedProducerIdsRequest.data) .whenComplete((results, exception) => { @@ -814,7 +814,7 @@ class ControllerApis(val requestChannel: RequestChannel, def handleUpdateFeatures(request: RequestChannel.Request): Unit = { val updateFeaturesRequest = request.body[UpdateFeaturesRequest] authHelper.authorizeClusterOperation(request, ALTER) - val context = new ControllerRequestContext(request.context.principal, + val context = new ControllerRequestContext(request.context.header.data, request.context.principal, OptionalLong.empty()) controller.updateFeatures(context, updateFeaturesRequest.data) .whenComplete((response, exception) => { diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index e2537f4e2f97..b128db596cfd 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -3306,9 +3306,8 @@ class KafkaApis(val requestChannel: RequestChannel, requestHelper.sendResponseExemptThrottle(request, alterPartitionRequest.getErrorResponse( AbstractResponse.DEFAULT_THROTTLE_TIME, Errors.NOT_CONTROLLER.exception)) else - zkSupport.controller.alterPartitions(alterPartitionRequest.data, alterPartitionResp => - requestHelper.sendResponseExemptThrottle(request, new AlterPartitionResponse(alterPartitionResp)) - ) + zkSupport.controller.alterPartitions(alterPartitionRequest.data, request.context.apiVersion, alterPartitionResp => + requestHelper.sendResponseExemptThrottle(request, new AlterPartitionResponse(alterPartitionResp))) } def handleUpdateFeatures(request: RequestChannel.Request): Unit = { diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index 3d7df18cbb64..b4e0b9449c05 100755 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -1790,38 +1790,25 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami // We keep the user-provided String as `MetadataVersion.fromVersionString` can choose a slightly different version (eg if `0.10.0` // is passed, `0.10.0-IV0` may be picked) val interBrokerProtocolVersionString = getString(KafkaConfig.InterBrokerProtocolVersionProp) - val interBrokerProtocolVersion = MetadataVersion.fromVersionString(interBrokerProtocolVersionString) - - val fetchRequestVersion: Short = - if (interBrokerProtocolVersion.isAtLeast(IBP_3_1_IV0)) 13 - else if (interBrokerProtocolVersion.isAtLeast(IBP_2_7_IV1)) 12 - else if (interBrokerProtocolVersion.isAtLeast(IBP_2_3_IV1)) 11 - else if (interBrokerProtocolVersion.isAtLeast(IBP_2_1_IV2)) 10 - else if (interBrokerProtocolVersion.isAtLeast(IBP_2_0_IV1)) 8 - else if (interBrokerProtocolVersion.isAtLeast(IBP_1_1_IV0)) 7 - else if (interBrokerProtocolVersion.isAtLeast(IBP_0_11_0_IV1)) 5 - else if (interBrokerProtocolVersion.isAtLeast(IBP_0_11_0_IV0)) 4 - else if (interBrokerProtocolVersion.isAtLeast(IBP_0_10_1_IV1)) 3 - else if (interBrokerProtocolVersion.isAtLeast(IBP_0_10_0_IV0)) 2 - else if (interBrokerProtocolVersion.isAtLeast(IBP_0_9_0)) 1 - else 0 - - val offsetForLeaderEpochRequestVersion: Short = - if (interBrokerProtocolVersion.isAtLeast(IBP_2_8_IV0)) 4 - else if (interBrokerProtocolVersion.isAtLeast(IBP_2_3_IV1)) 3 - else if (interBrokerProtocolVersion.isAtLeast(IBP_2_1_IV1)) 2 - else if (interBrokerProtocolVersion.isAtLeast(IBP_2_0_IV0)) 1 - else 0 - - val listOffsetRequestVersion: Short = - if (interBrokerProtocolVersion.isAtLeast(IBP_3_0_IV1)) 7 - else if (interBrokerProtocolVersion.isAtLeast(IBP_2_8_IV0)) 6 - else if (interBrokerProtocolVersion.isAtLeast(IBP_2_2_IV1)) 5 - else if (interBrokerProtocolVersion.isAtLeast(IBP_2_1_IV1)) 4 - else if (interBrokerProtocolVersion.isAtLeast(IBP_2_0_IV1)) 3 - else if (interBrokerProtocolVersion.isAtLeast(IBP_0_11_0_IV0)) 2 - else if (interBrokerProtocolVersion.isAtLeast(IBP_0_10_1_IV2)) 1 - else 0 + val interBrokerProtocolVersion = if (processRoles.isEmpty) { + MetadataVersion.fromVersionString(interBrokerProtocolVersionString) + } else { + if (originals.containsKey(KafkaConfig.InterBrokerProtocolVersionProp)) { + // A user-supplied IBP was given + val configuredVersion = MetadataVersion.fromVersionString(interBrokerProtocolVersionString) + if (!configuredVersion.isKRaftSupported) { + throw new ConfigException(s"A non-KRaft version ${interBrokerProtocolVersionString} given for ${KafkaConfig.InterBrokerProtocolVersionProp}. " + + s"The minimum version is ${MetadataVersion.MINIMUM_KRAFT_VERSION}") + } else { + warn(s"${KafkaConfig.InterBrokerProtocolVersionProp} is deprecated in KRaft mode as of 3.3 and will only " + + s"be read when first upgrading from a KRaft prior to 3.3. See kafka-storage.sh help for details on setting " + + s"the metadata version for a new KRaft cluster.") + } + } + // In KRaft mode, we pin this value to the minimum KRaft-supported version. This prevents inadvertent usage of + // the static IBP config in broker components running in KRaft mode + MetadataVersion.MINIMUM_KRAFT_VERSION + } /** ********* Controlled shutdown configuration ***********/ val controlledShutdownMaxRetries = getInt(KafkaConfig.ControlledShutdownMaxRetriesProp) diff --git a/core/src/main/scala/kafka/server/KafkaRaftServer.scala b/core/src/main/scala/kafka/server/KafkaRaftServer.scala index 5a1c3087d384..f1474430b9b7 100644 --- a/core/src/main/scala/kafka/server/KafkaRaftServer.scala +++ b/core/src/main/scala/kafka/server/KafkaRaftServer.scala @@ -35,6 +35,7 @@ import org.apache.kafka.server.metrics.KafkaYammerMetrics import java.nio.file.Paths import scala.collection.Seq +import scala.compat.java8.FunctionConverters.asJavaSupplier import scala.jdk.CollectionConverters._ /** @@ -180,13 +181,16 @@ object KafkaRaftServer { "If you intend to create a new broker, you should remove all data in your data directories (log.dirs).") } - // Load the bootstrap metadata file or, in the case of an upgrade from KRaft preview, bootstrap the - // metadata.version corresponding to a user-configured IBP. - val bootstrapMetadata = if (config.originals.containsKey(KafkaConfig.InterBrokerProtocolVersionProp)) { - BootstrapMetadata.load(Paths.get(config.metadataLogDir), config.interBrokerProtocolVersion) - } else { - BootstrapMetadata.load(Paths.get(config.metadataLogDir), MetadataVersion.IBP_3_0_IV0) + // Load the bootstrap metadata file. In the case of an upgrade from older KRaft where there is no bootstrap metadata, + // read the IBP from config in order to bootstrap the equivalent metadata version. + def getUserDefinedIBPVersionOrThrow(): MetadataVersion = { + if (config.originals.containsKey(KafkaConfig.InterBrokerProtocolVersionProp)) { + MetadataVersion.fromVersionString(config.interBrokerProtocolVersionString) + } else { + throw new KafkaException(s"Cannot upgrade from KRaft version prior to 3.3 without first setting ${KafkaConfig.InterBrokerProtocolVersionProp} on each broker.") + } } + val bootstrapMetadata = BootstrapMetadata.load(Paths.get(config.metadataLogDir), asJavaSupplier(() => getUserDefinedIBPVersionOrThrow())) (metaProperties, bootstrapMetadata, offlineDirs.toSeq) } diff --git a/core/src/main/scala/kafka/server/RemoteLeaderEndPoint.scala b/core/src/main/scala/kafka/server/RemoteLeaderEndPoint.scala index a9ac51315a51..826643a0f5ec 100644 --- a/core/src/main/scala/kafka/server/RemoteLeaderEndPoint.scala +++ b/core/src/main/scala/kafka/server/RemoteLeaderEndPoint.scala @@ -31,6 +31,7 @@ import org.apache.kafka.common.message.OffsetForLeaderEpochRequestData.{OffsetFo import org.apache.kafka.common.message.OffsetForLeaderEpochResponseData.EpochEndOffset import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.requests.{FetchRequest, FetchResponse, ListOffsetsRequest, ListOffsetsResponse, OffsetsForLeaderEpochRequest, OffsetsForLeaderEpochResponse} +import org.apache.kafka.server.common.MetadataVersion import org.apache.kafka.server.common.MetadataVersion.IBP_0_10_1_IV2 import scala.jdk.CollectionConverters._ @@ -46,13 +47,16 @@ import scala.compat.java8.OptionConverters.RichOptionForJava8 * @param brokerConfig Broker configuration * @param replicaManager A ReplicaManager * @param quota The quota, used when building a fetch request + * @param metadataVersionSupplier A supplier that returns the current MetadataVersion. This can change during + * runtime in KRaft mode. */ class RemoteLeaderEndPoint(logPrefix: String, blockingSender: BlockingSend, private[server] val fetchSessionHandler: FetchSessionHandler, // visible for testing brokerConfig: KafkaConfig, replicaManager: ReplicaManager, - quota: ReplicaQuota) extends LeaderEndPoint with Logging { + quota: ReplicaQuota, + metadataVersionSupplier: () => MetadataVersion) extends LeaderEndPoint with Logging { this.logIdent = logPrefix @@ -61,7 +65,7 @@ class RemoteLeaderEndPoint(logPrefix: String, private val maxBytes = brokerConfig.replicaFetchResponseMaxBytes private val fetchSize = brokerConfig.replicaFetchMaxBytes - override val isTruncationOnFetchSupported = brokerConfig.interBrokerProtocolVersion.isTruncationOnFetchSupported + override def isTruncationOnFetchSupported = metadataVersionSupplier().isTruncationOnFetchSupported override def initiateClose(): Unit = blockingSender.initiateClose() @@ -106,7 +110,8 @@ class RemoteLeaderEndPoint(logPrefix: String, .setPartitionIndex(topicPartition.partition) .setCurrentLeaderEpoch(currentLeaderEpoch) .setTimestamp(earliestOrLatest))) - val requestBuilder = ListOffsetsRequest.Builder.forReplica(brokerConfig.listOffsetRequestVersion, brokerConfig.brokerId) + val metadataVersion = metadataVersionSupplier() + val requestBuilder = ListOffsetsRequest.Builder.forReplica(metadataVersion.listOffsetRequestVersion, brokerConfig.brokerId) .setTargetTimes(Collections.singletonList(topic)) val clientResponse = blockingSender.sendRequest(requestBuilder) @@ -116,7 +121,7 @@ class RemoteLeaderEndPoint(logPrefix: String, Errors.forCode(responsePartition.errorCode) match { case Errors.NONE => - if (brokerConfig.interBrokerProtocolVersion.isAtLeast(IBP_0_10_1_IV2)) + if (metadataVersion.isAtLeast(IBP_0_10_1_IV2)) responsePartition.offset else responsePartition.oldStyleOffsets.get(0) @@ -141,7 +146,7 @@ class RemoteLeaderEndPoint(logPrefix: String, } val epochRequest = OffsetsForLeaderEpochRequest.Builder.forFollower( - brokerConfig.offsetForLeaderEpochRequestVersion, topics, brokerConfig.brokerId) + metadataVersionSupplier().offsetForLeaderEpochRequestVersion, topics, brokerConfig.brokerId) debug(s"Sending offset for leader epoch request $epochRequest") try { @@ -201,7 +206,12 @@ class RemoteLeaderEndPoint(logPrefix: String, val fetchRequestOpt = if (fetchData.sessionPartitions.isEmpty && fetchData.toForget.isEmpty) { None } else { - val version: Short = if (brokerConfig.fetchRequestVersion >= 13 && !fetchData.canUseTopicIds) 12 else brokerConfig.fetchRequestVersion + val metadataVersion = metadataVersionSupplier() + val version: Short = if (metadataVersion.fetchRequestVersion >= 13 && !fetchData.canUseTopicIds) { + 12 + } else { + metadataVersion.fetchRequestVersion + } val requestBuilder = FetchRequest.Builder .forReplica(version, brokerConfig.brokerId, maxWait, minBytes, fetchData.toSend) .setMaxBytes(maxBytes) diff --git a/core/src/main/scala/kafka/server/ReplicaFetcherManager.scala b/core/src/main/scala/kafka/server/ReplicaFetcherManager.scala index feb082c5ae7e..33af5836cd14 100644 --- a/core/src/main/scala/kafka/server/ReplicaFetcherManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaFetcherManager.scala @@ -21,13 +21,15 @@ import kafka.cluster.BrokerEndPoint import org.apache.kafka.clients.FetchSessionHandler import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.utils.{LogContext, Time} +import org.apache.kafka.server.common.MetadataVersion class ReplicaFetcherManager(brokerConfig: KafkaConfig, protected val replicaManager: ReplicaManager, metrics: Metrics, time: Time, threadNamePrefix: Option[String] = None, - quotaManager: ReplicationQuotaManager) + quotaManager: ReplicationQuotaManager, + metadataVersionSupplier: () => MetadataVersion) extends AbstractFetcherManager[ReplicaFetcherThread]( name = "ReplicaFetcherManager on broker " + brokerConfig.brokerId, clientId = "Replica", @@ -41,9 +43,10 @@ class ReplicaFetcherManager(brokerConfig: KafkaConfig, val endpoint = new BrokerBlockingSender(sourceBroker, brokerConfig, metrics, time, fetcherId, s"broker-${brokerConfig.brokerId}-fetcher-$fetcherId", logContext) val fetchSessionHandler = new FetchSessionHandler(logContext, sourceBroker.id) - val leader = new RemoteLeaderEndPoint(logContext.logPrefix, endpoint, fetchSessionHandler, brokerConfig, replicaManager, quotaManager) + val leader = new RemoteLeaderEndPoint(logContext.logPrefix, endpoint, fetchSessionHandler, brokerConfig, + replicaManager, quotaManager, metadataVersionSupplier) new ReplicaFetcherThread(threadName, leader, brokerConfig, failedPartitions, replicaManager, - quotaManager, logContext.logPrefix) + quotaManager, logContext.logPrefix, metadataVersionSupplier) } def shutdown(): Unit = { diff --git a/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala b/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala index 86cc6b1b9db9..2e728ce8173a 100644 --- a/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala +++ b/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala @@ -21,6 +21,7 @@ import kafka.log.{LeaderOffsetIncremented, LogAppendInfo} import org.apache.kafka.common.record.MemoryRecords import org.apache.kafka.common.requests._ import org.apache.kafka.common.TopicPartition +import org.apache.kafka.server.common.MetadataVersion class ReplicaFetcherThread(name: String, leader: LeaderEndPoint, @@ -28,7 +29,8 @@ class ReplicaFetcherThread(name: String, failedPartitions: FailedPartitions, replicaMgr: ReplicaManager, quota: ReplicaQuota, - logPrefix: String) + logPrefix: String, + metadataVersionSupplier: () => MetadataVersion) extends AbstractFetcherThread(name = name, clientId = name, leader = leader, @@ -39,7 +41,7 @@ class ReplicaFetcherThread(name: String, this.logIdent = logPrefix - override protected val isOffsetForLeaderEpochSupported: Boolean = brokerConfig.interBrokerProtocolVersion.isOffsetForLeaderEpochSupported + override protected val isOffsetForLeaderEpochSupported: Boolean = metadataVersionSupplier().isOffsetForLeaderEpochSupported override protected def latestEpoch(topicPartition: TopicPartition): Option[Int] = { replicaMgr.localLogOrException(topicPartition).latestEpoch @@ -135,7 +137,7 @@ class ReplicaFetcherThread(name: String, def maybeWarnIfOversizedRecords(records: MemoryRecords, topicPartition: TopicPartition): Unit = { // oversized messages don't cause replication to fail from fetch request version 3 (KIP-74) - if (brokerConfig.fetchRequestVersion <= 2 && records.sizeInBytes > 0 && records.validBytes <= 0) + if (metadataVersionSupplier().fetchRequestVersion <= 2 && records.sizeInBytes > 0 && records.validBytes <= 0) error(s"Replication is failing due to a message that is greater than replica.fetch.max.bytes for partition $topicPartition. " + "This generally occurs when the max.message.bytes has been overridden to exceed this value and a suitably large " + "message has also been sent. To fix this problem increase replica.fetch.max.bytes in your broker config to be " + diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index 03983ad98d08..f60bd53a085e 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -62,6 +62,7 @@ import org.apache.kafka.image.{LocalReplicaChanges, MetadataImage, TopicsDelta} import org.apache.kafka.metadata.LeaderConstants.NO_LEADER import org.apache.kafka.server.common.MetadataVersion._ +import java.nio.file.{Files, Paths} import scala.jdk.CollectionConverters._ import scala.collection.{Map, Seq, Set, mutable} import scala.compat.java8.OptionConverters._ @@ -310,7 +311,7 @@ class ReplicaManager(val config: KafkaConfig, // If inter-broker protocol (IBP) < 1.0, the controller will send LeaderAndIsrRequest V0 which does not include isNew field. // In this case, the broker receiving the request cannot determine whether it is safe to create a partition if a log directory has failed. // Thus, we choose to halt the broker on any log directory failure if IBP < 1.0 - val haltBrokerOnFailure = config.interBrokerProtocolVersion.isLessThan(IBP_1_0_IV0) + val haltBrokerOnFailure = metadataCache.metadataVersion().isLessThan(IBP_1_0_IV0) logDirFailureHandler = new LogDirFailureHandler("LogDirFailureHandler", haltBrokerOnFailure) logDirFailureHandler.start() } @@ -790,11 +791,15 @@ class ReplicaManager(val config: KafkaConfig, val logsByDir = logManager.allLogs.groupBy(log => log.parentDir) config.logDirs.toSet.map { logDir: String => - val absolutePath = new File(logDir).getAbsolutePath + val file = Paths.get(logDir) + val absolutePath = file.toAbsolutePath.toString try { if (!logManager.isLogDirOnline(absolutePath)) throw new KafkaStorageException(s"Log directory $absolutePath is offline") + val fileStore = Files.getFileStore(file) + val totalBytes = adjustForLargeFileSystems(fileStore.getTotalSpace) + val usableBytes = adjustForLargeFileSystems(fileStore.getUsableSpace) logsByDir.get(absolutePath) match { case Some(logs) => val topicInfos = logs.groupBy(_.topicPartition.topic).map{case (topic, logs) => @@ -812,9 +817,11 @@ class ReplicaManager(val config: KafkaConfig, new DescribeLogDirsResponseData.DescribeLogDirsResult().setLogDir(absolutePath) .setErrorCode(Errors.NONE.code).setTopics(topicInfos) + .setTotalBytes(totalBytes).setUsableBytes(usableBytes) case None => new DescribeLogDirsResponseData.DescribeLogDirsResult().setLogDir(absolutePath) .setErrorCode(Errors.NONE.code) + .setTotalBytes(totalBytes).setUsableBytes(usableBytes) } } catch { @@ -832,6 +839,13 @@ class ReplicaManager(val config: KafkaConfig, }.toList } + // See: https://bugs.openjdk.java.net/browse/JDK-8162520 + def adjustForLargeFileSystems(space: Long): Long = { + if (space < 0) + return Long.MaxValue + space + } + def getLogEndOffsetLag(topicPartition: TopicPartition, logEndOffset: Long, isFuture: Boolean): Long = { localLog(topicPartition) match { case Some(log) => @@ -1773,7 +1787,7 @@ class ReplicaManager(val config: KafkaConfig, * OffsetForLeaderEpoch request. */ protected def initialFetchOffset(log: UnifiedLog): Long = { - if (config.interBrokerProtocolVersion.isTruncationOnFetchSupported() && log.latestEpoch.nonEmpty) + if (metadataCache.metadataVersion().isTruncationOnFetchSupported && log.latestEpoch.nonEmpty) log.logEndOffset else log.highWatermark @@ -1903,7 +1917,7 @@ class ReplicaManager(val config: KafkaConfig, } protected def createReplicaFetcherManager(metrics: Metrics, time: Time, threadNamePrefix: Option[String], quotaManager: ReplicationQuotaManager) = { - new ReplicaFetcherManager(config, this, metrics, time, threadNamePrefix, quotaManager) + new ReplicaFetcherManager(config, this, metrics, time, threadNamePrefix, quotaManager, () => metadataCache.metadataVersion()) } protected def createReplicaAlterLogDirsManager(quotaManager: ReplicationQuotaManager, brokerTopicStats: BrokerTopicStats) = { diff --git a/core/src/main/scala/kafka/server/ZkAlterPartitionManager.scala b/core/src/main/scala/kafka/server/ZkAlterPartitionManager.scala index c906ad6a7066..c3d842b79638 100644 --- a/core/src/main/scala/kafka/server/ZkAlterPartitionManager.scala +++ b/core/src/main/scala/kafka/server/ZkAlterPartitionManager.scala @@ -19,10 +19,11 @@ package kafka.server import kafka.utils.{Logging, ReplicationUtils, Scheduler} import kafka.zk.KafkaZkClient import org.apache.kafka.common.TopicPartition + import java.util.concurrent.atomic.AtomicLong import java.util.concurrent.{CompletableFuture, TimeUnit} - import kafka.api.LeaderAndIsr +import org.apache.kafka.common.TopicIdPartition import org.apache.kafka.common.errors.InvalidUpdateVersionException import org.apache.kafka.common.utils.Time @@ -58,21 +59,21 @@ class ZkAlterPartitionManager(scheduler: Scheduler, time: Time, zkClient: KafkaZ } override def submit( - topicPartition: TopicPartition, + topicIdPartition: TopicIdPartition, leaderAndIsr: LeaderAndIsr, controllerEpoch: Int ): CompletableFuture[LeaderAndIsr]= { debug(s"Writing new ISR ${leaderAndIsr.isr} to ZooKeeper with version " + - s"${leaderAndIsr.partitionEpoch} for partition $topicPartition") + s"${leaderAndIsr.partitionEpoch} for partition $topicIdPartition") - val (updateSucceeded, newVersion) = ReplicationUtils.updateLeaderAndIsr(zkClient, topicPartition, + val (updateSucceeded, newVersion) = ReplicationUtils.updateLeaderAndIsr(zkClient, topicIdPartition.topicPartition, leaderAndIsr, controllerEpoch) val future = new CompletableFuture[LeaderAndIsr]() if (updateSucceeded) { // Track which partitions need to be propagated to the controller isrChangeSet synchronized { - isrChangeSet += topicPartition + isrChangeSet += topicIdPartition.topicPartition lastIsrChangeMs.set(time.milliseconds()) } @@ -81,7 +82,7 @@ class ZkAlterPartitionManager(scheduler: Scheduler, time: Time, zkClient: KafkaZ future.complete(leaderAndIsr.withPartitionEpoch(newVersion)) } else { future.completeExceptionally(new InvalidUpdateVersionException( - s"ISR update $leaderAndIsr for partition $topicPartition with controller epoch $controllerEpoch " + + s"ISR update $leaderAndIsr for partition $topicIdPartition with controller epoch $controllerEpoch " + "failed with an invalid version error")) } future diff --git a/core/src/main/scala/kafka/server/metadata/BrokerMetadataPublisher.scala b/core/src/main/scala/kafka/server/metadata/BrokerMetadataPublisher.scala index 60165a399e57..212f188504e9 100644 --- a/core/src/main/scala/kafka/server/metadata/BrokerMetadataPublisher.scala +++ b/core/src/main/scala/kafka/server/metadata/BrokerMetadataPublisher.scala @@ -31,7 +31,6 @@ import org.apache.kafka.common.internals.Topic import org.apache.kafka.image.{MetadataDelta, MetadataImage, TopicDelta, TopicsImage} import org.apache.kafka.metadata.authorizer.ClusterMetadataAuthorizer import org.apache.kafka.server.authorizer.Authorizer -import org.apache.kafka.server.common.MetadataVersion import scala.collection.mutable @@ -132,10 +131,7 @@ class BrokerMetadataPublisher(conf: KafkaConfig, // Publish the new metadata image to the metadata cache. metadataCache.setImage(newImage) - val metadataVersionLogMsg = newImage.features().metadataVersion() match { - case MetadataVersion.UNINITIALIZED => "un-initialized metadata.version" - case mv: MetadataVersion => s"metadata.version ${mv.featureLevel()}" - } + val metadataVersionLogMsg = s"metadata.version ${newImage.features().metadataVersion()}" if (_firstPublish) { info(s"Publishing initial metadata at offset $highestOffsetAndEpoch with $metadataVersionLogMsg.") diff --git a/core/src/main/scala/kafka/server/metadata/KRaftMetadataCache.scala b/core/src/main/scala/kafka/server/metadata/KRaftMetadataCache.scala index e3270fa70aba..ae2e65235739 100644 --- a/core/src/main/scala/kafka/server/metadata/KRaftMetadataCache.scala +++ b/core/src/main/scala/kafka/server/metadata/KRaftMetadataCache.scala @@ -199,7 +199,15 @@ class KRaftMetadataCache(val brokerId: Int) extends MetadataCache with Logging w override def getTopicName(topicId: Uuid): Option[String] = _currentImage.topics().topicsById.asScala.get(topicId).map(_.name()) override def hasAliveBroker(brokerId: Int): Boolean = { - Option(_currentImage.cluster().broker(brokerId)).count(!_.fenced()) == 1 + Option(_currentImage.cluster.broker(brokerId)).count(!_.fenced()) == 1 + } + + def isBrokerFenced(brokerId: Int): Boolean = { + Option(_currentImage.cluster.broker(brokerId)).count(_.fenced) == 1 + } + + def isBrokerShuttingDown(brokerId: Int): Boolean = { + Option(_currentImage.cluster.broker(brokerId)).count(_.inControlledShutdown) == 1 } override def getAliveBrokers(): Iterable[BrokerMetadata] = getAliveBrokers(_currentImage) diff --git a/core/src/main/scala/kafka/tools/StorageTool.scala b/core/src/main/scala/kafka/tools/StorageTool.scala index 333af863145d..a96275cc27cd 100644 --- a/core/src/main/scala/kafka/tools/StorageTool.scala +++ b/core/src/main/scala/kafka/tools/StorageTool.scala @@ -49,7 +49,7 @@ object StorageTool extends Logging { val clusterId = namespace.getString("cluster_id") val metadataVersion = getMetadataVersion(namespace) if (!metadataVersion.isKRaftSupported) { - throw new TerseFailure(s"Must specify a metadata version of at least 1.") + throw new TerseFailure(s"Must specify a valid KRaft metadata version of at least 3.0.") } val metaProperties = buildMetadataProperties(clusterId, config.get) val ignoreFormatted = namespace.getBoolean("ignore_formatted") @@ -99,7 +99,7 @@ object StorageTool extends Logging { action(storeTrue()) formatParser.addArgument("--release-version", "-r"). action(store()). - help(s"A release version to use for the initial metadata.version. The default is (${MetadataVersion.latest().version()})") + help(s"A KRaft release version to use for the initial metadata version. The minimum is 3.0, the default is ${MetadataVersion.latest().version()}") parser.parseArgsOrFail(args) } diff --git a/core/src/test/java/kafka/test/ClusterConfig.java b/core/src/test/java/kafka/test/ClusterConfig.java index 5830959283ba..8e9f7de96abb 100644 --- a/core/src/test/java/kafka/test/ClusterConfig.java +++ b/core/src/test/java/kafka/test/ClusterConfig.java @@ -122,8 +122,8 @@ public Optional trustStoreFile() { return Optional.ofNullable(trustStoreFile); } - public Optional metadataVersion() { - return Optional.ofNullable(metadataVersion); + public MetadataVersion metadataVersion() { + return metadataVersion; } public Properties brokerServerProperties(int brokerId) { @@ -133,7 +133,7 @@ public Properties brokerServerProperties(int brokerId) { public Map nameTags() { Map tags = new LinkedHashMap<>(4); name().ifPresent(name -> tags.put("Name", name)); - metadataVersion().ifPresent(mv -> tags.put("MetadataVersion", mv.toString())); + tags.put("MetadataVersion", metadataVersion.toString()); tags.put("Security", securityProtocol.name()); listenerName().ifPresent(listener -> tags.put("Listener", listener)); return tags; @@ -150,11 +150,12 @@ public ClusterConfig copyOf() { } public static Builder defaultClusterBuilder() { - return new Builder(Type.ZK, 1, 1, true, SecurityProtocol.PLAINTEXT); + return new Builder(Type.ZK, 1, 1, true, SecurityProtocol.PLAINTEXT, MetadataVersion.latest()); } - public static Builder clusterBuilder(Type type, int brokers, int controllers, boolean autoStart, SecurityProtocol securityProtocol) { - return new Builder(type, brokers, controllers, autoStart, securityProtocol); + public static Builder clusterBuilder(Type type, int brokers, int controllers, boolean autoStart, + SecurityProtocol securityProtocol, MetadataVersion metadataVersion) { + return new Builder(type, brokers, controllers, autoStart, securityProtocol, metadataVersion); } public static class Builder { @@ -168,12 +169,13 @@ public static class Builder { private File trustStoreFile; private MetadataVersion metadataVersion; - Builder(Type type, int brokers, int controllers, boolean autoStart, SecurityProtocol securityProtocol) { + Builder(Type type, int brokers, int controllers, boolean autoStart, SecurityProtocol securityProtocol, MetadataVersion metadataVersion) { this.type = type; this.brokers = brokers; this.controllers = controllers; this.autoStart = autoStart; this.securityProtocol = securityProtocol; + this.metadataVersion = metadataVersion; } public Builder type(Type type) { diff --git a/core/src/test/java/kafka/test/ClusterTestExtensionsTest.java b/core/src/test/java/kafka/test/ClusterTestExtensionsTest.java index 8c18451efd06..33780f795eb8 100644 --- a/core/src/test/java/kafka/test/ClusterTestExtensionsTest.java +++ b/core/src/test/java/kafka/test/ClusterTestExtensionsTest.java @@ -25,6 +25,7 @@ import kafka.test.annotation.ClusterTests; import kafka.test.annotation.Type; import kafka.test.junit.ClusterTestExtensions; +import org.apache.kafka.server.common.MetadataVersion; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.BeforeEach; @@ -109,4 +110,9 @@ public void testNoAutoStart() { clusterInstance.start(); Assertions.assertNotNull(clusterInstance.anyBrokerSocketServer()); } + + @ClusterTest + public void testDefaults(ClusterConfig config) { + Assertions.assertEquals(MetadataVersion.IBP_3_3_IV3, config.metadataVersion()); + } } diff --git a/core/src/test/java/kafka/test/annotation/ClusterTest.java b/core/src/test/java/kafka/test/annotation/ClusterTest.java index b83df127f5a5..d1d3222a25b8 100644 --- a/core/src/test/java/kafka/test/annotation/ClusterTest.java +++ b/core/src/test/java/kafka/test/annotation/ClusterTest.java @@ -41,6 +41,6 @@ String name() default ""; SecurityProtocol securityProtocol() default SecurityProtocol.PLAINTEXT; String listener() default ""; - MetadataVersion metadataVersion() default MetadataVersion.UNINITIALIZED; + MetadataVersion metadataVersion() default MetadataVersion.IBP_3_3_IV3; ClusterConfigProperty[] serverProperties() default {}; } diff --git a/core/src/test/java/kafka/test/junit/ClusterTestExtensions.java b/core/src/test/java/kafka/test/junit/ClusterTestExtensions.java index f0c1d9bbda3c..bd69109c4b75 100644 --- a/core/src/test/java/kafka/test/junit/ClusterTestExtensions.java +++ b/core/src/test/java/kafka/test/junit/ClusterTestExtensions.java @@ -25,7 +25,6 @@ import kafka.test.annotation.ClusterTest; import kafka.test.annotation.ClusterTests; import kafka.test.annotation.Type; -import org.apache.kafka.server.common.MetadataVersion; import org.junit.jupiter.api.extension.ExtensionContext; import org.junit.jupiter.api.extension.TestTemplateInvocationContext; import org.junit.jupiter.api.extension.TestTemplateInvocationContextProvider; @@ -180,7 +179,8 @@ private void processClusterTest(ExtensionContext context, ClusterTest annot, Clu throw new IllegalStateException(); } - ClusterConfig.Builder builder = ClusterConfig.clusterBuilder(type, brokers, controllers, autoStart, annot.securityProtocol()); + ClusterConfig.Builder builder = ClusterConfig.clusterBuilder(type, brokers, controllers, autoStart, + annot.securityProtocol(), annot.metadataVersion()); if (!annot.name().isEmpty()) { builder.name(annot.name()); } else { @@ -195,10 +195,6 @@ private void processClusterTest(ExtensionContext context, ClusterTest annot, Clu properties.put(property.key(), property.value()); } - if (!annot.metadataVersion().equals(MetadataVersion.UNINITIALIZED)) { - builder.metadataVersion(annot.metadataVersion()); - } - ClusterConfig config = builder.build(); config.serverProperties().putAll(properties); type.invocationContexts(config, testInvocations); diff --git a/core/src/test/java/kafka/test/junit/RaftClusterInvocationContext.java b/core/src/test/java/kafka/test/junit/RaftClusterInvocationContext.java index f0ca98a5f2d7..73fe67836a3f 100644 --- a/core/src/test/java/kafka/test/junit/RaftClusterInvocationContext.java +++ b/core/src/test/java/kafka/test/junit/RaftClusterInvocationContext.java @@ -30,7 +30,6 @@ import org.apache.kafka.common.network.ListenerName; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.metadata.BrokerState; -import org.apache.kafka.server.common.MetadataVersion; import org.junit.jupiter.api.extension.AfterTestExecutionCallback; import org.junit.jupiter.api.extension.BeforeTestExecutionCallback; import org.junit.jupiter.api.extension.Extension; @@ -86,7 +85,7 @@ public List getAdditionalExtensions() { return Arrays.asList( (BeforeTestExecutionCallback) context -> { TestKitNodes nodes = new TestKitNodes.Builder(). - setBootstrapMetadataVersion(clusterConfig.metadataVersion().orElse(MetadataVersion.latest())). + setBootstrapMetadataVersion(clusterConfig.metadataVersion()). setNumBrokerNodes(clusterConfig.numBrokers()). setNumControllerNodes(clusterConfig.numControllers()).build(); nodes.brokerNodes().forEach((brokerId, brokerNode) -> { diff --git a/core/src/test/java/kafka/test/junit/ZkClusterInvocationContext.java b/core/src/test/java/kafka/test/junit/ZkClusterInvocationContext.java index 02f21906ed4a..d8375b012796 100644 --- a/core/src/test/java/kafka/test/junit/ZkClusterInvocationContext.java +++ b/core/src/test/java/kafka/test/junit/ZkClusterInvocationContext.java @@ -108,7 +108,7 @@ public void modifyConfigs(Seq props) { @Override public Properties serverConfig() { Properties props = clusterConfig.serverProperties(); - clusterConfig.metadataVersion().ifPresent(mv -> props.put(KafkaConfig.InterBrokerProtocolVersionProp(), mv.version())); + props.put(KafkaConfig.InterBrokerProtocolVersionProp(), metadataVersion().version()); return props; } diff --git a/core/src/test/java/kafka/testkit/KafkaClusterTestKit.java b/core/src/test/java/kafka/testkit/KafkaClusterTestKit.java index 1924579e1726..a930bafde637 100644 --- a/core/src/test/java/kafka/testkit/KafkaClusterTestKit.java +++ b/core/src/test/java/kafka/testkit/KafkaClusterTestKit.java @@ -340,7 +340,7 @@ private void formatNodeAndLog(MetaProperties properties, String metadataLogDir, StorageTool.formatCommand(out, JavaConverters.asScalaBuffer(Collections.singletonList(metadataLogDir)).toSeq(), properties, - MetadataVersion.IBP_3_0_IV0, + MetadataVersion.MINIMUM_KRAFT_VERSION, false); } finally { for (String line : stream.toString().split(String.format("%n"))) { diff --git a/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala index d563bcb0c82c..203c04a68a7a 100644 --- a/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala @@ -265,6 +265,8 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { assertEquals(expectedPartitions.toSet, replicaInfos.keys.map(_.partition).toSet) logDirInfos.forEach { (logDir, logDirInfo) => + assertTrue(logDirInfo.totalBytes.isPresent) + assertTrue(logDirInfo.usableBytes.isPresent) logDirInfo.replicaInfos.asScala.keys.foreach(tp => assertEquals(server.logManager.getLog(tp).get.dir.getParent, logDir) ) diff --git a/core/src/test/scala/integration/kafka/coordinator/transaction/ProducerIdsIntegrationTest.scala b/core/src/test/scala/integration/kafka/coordinator/transaction/ProducerIdsIntegrationTest.scala index 3b97ee8398b2..7d3203e93095 100644 --- a/core/src/test/scala/integration/kafka/coordinator/transaction/ProducerIdsIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/coordinator/transaction/ProducerIdsIntegrationTest.scala @@ -46,7 +46,7 @@ class ProducerIdsIntegrationTest { @ClusterTests(Array( new ClusterTest(clusterType = Type.ZK, brokers = 3, metadataVersion = MetadataVersion.IBP_2_8_IV1), new ClusterTest(clusterType = Type.ZK, brokers = 3, metadataVersion = MetadataVersion.IBP_3_0_IV0), - new ClusterTest(clusterType = Type.KRAFT, brokers = 3, metadataVersion = MetadataVersion.IBP_3_0_IV0) + new ClusterTest(clusterType = Type.KRAFT, brokers = 3, metadataVersion = MetadataVersion.IBP_3_0_IV1) )) def testUniqueProducerIds(clusterInstance: ClusterInstance): Unit = { verifyUniqueIds(clusterInstance) diff --git a/core/src/test/scala/integration/kafka/server/MetadataVersionIntegrationTest.scala b/core/src/test/scala/integration/kafka/server/MetadataVersionIntegrationTest.scala index b671e5d5e350..c060e3a6daae 100644 --- a/core/src/test/scala/integration/kafka/server/MetadataVersionIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/server/MetadataVersionIntegrationTest.scala @@ -32,7 +32,7 @@ import scala.jdk.CollectionConverters._ @ExtendWith(value = Array(classOf[ClusterTestExtensions])) class MetadataVersionIntegrationTest { @ClusterTests(value = Array( - new ClusterTest(clusterType = Type.KRAFT, metadataVersion = MetadataVersion.IBP_3_0_IV0), + new ClusterTest(clusterType = Type.KRAFT, metadataVersion = MetadataVersion.IBP_3_0_IV1), new ClusterTest(clusterType = Type.KRAFT, metadataVersion = MetadataVersion.IBP_3_1_IV0), new ClusterTest(clusterType = Type.KRAFT, metadataVersion = MetadataVersion.IBP_3_2_IV0) )) @@ -40,8 +40,8 @@ class MetadataVersionIntegrationTest { val admin = clusterInstance.createAdminClient() val describeResult = admin.describeFeatures() val ff = describeResult.featureMetadata().get().finalizedFeatures().get(MetadataVersion.FEATURE_NAME) - assertEquals(ff.minVersionLevel(), clusterInstance.config().metadataVersion().get().featureLevel()) - assertEquals(ff.maxVersionLevel(), clusterInstance.config().metadataVersion().get().featureLevel()) + assertEquals(ff.minVersionLevel(), clusterInstance.config().metadataVersion().featureLevel()) + assertEquals(ff.maxVersionLevel(), clusterInstance.config().metadataVersion().featureLevel()) // Update to new version val updateVersion = MetadataVersion.IBP_3_3_IV0.featureLevel.shortValue @@ -71,7 +71,8 @@ class MetadataVersionIntegrationTest { val admin = clusterInstance.createAdminClient() val describeResult = admin.describeFeatures() val ff = describeResult.featureMetadata().get().finalizedFeatures().get(MetadataVersion.FEATURE_NAME) - assertEquals(ff.minVersionLevel(), MetadataVersion.latest().featureLevel()) + assertEquals(ff.minVersionLevel(), MetadataVersion.latest().featureLevel(), + "If this test fails, check the default MetadataVersion in the @ClusterTest annotation") assertEquals(ff.maxVersionLevel(), MetadataVersion.latest().featureLevel()) } } diff --git a/core/src/test/scala/unit/kafka/cluster/AbstractPartitionTest.scala b/core/src/test/scala/unit/kafka/cluster/AbstractPartitionTest.scala index 147743a77d09..13e627b52985 100644 --- a/core/src/test/scala/unit/kafka/cluster/AbstractPartitionTest.scala +++ b/core/src/test/scala/unit/kafka/cluster/AbstractPartitionTest.scala @@ -65,7 +65,7 @@ class AbstractPartitionTest { val logProps = createLogProperties(Map.empty) logConfig = LogConfig(logProps) - configRepository = MockConfigRepository.forTopic(topicPartition.topic(), logProps) + configRepository = MockConfigRepository.forTopic(topicPartition.topic, logProps) tmpDir = TestUtils.tempDir() logDir1 = TestUtils.randomPartitionLogDir(tmpDir) diff --git a/core/src/test/scala/unit/kafka/cluster/PartitionLockTest.scala b/core/src/test/scala/unit/kafka/cluster/PartitionLockTest.scala index 1cf66a9b4c5d..6c374fe3c1d8 100644 --- a/core/src/test/scala/unit/kafka/cluster/PartitionLockTest.scala +++ b/core/src/test/scala/unit/kafka/cluster/PartitionLockTest.scala @@ -20,7 +20,6 @@ package kafka.cluster import java.util.{Optional, Properties} import java.util.concurrent._ import java.util.concurrent.atomic.AtomicBoolean - import kafka.api.LeaderAndIsr import kafka.log._ import kafka.server._ @@ -28,6 +27,7 @@ import kafka.server.checkpoints.OffsetCheckpoints import kafka.server.epoch.LeaderEpochFileCache import kafka.server.metadata.MockConfigRepository import kafka.utils._ +import org.apache.kafka.common.TopicIdPartition import org.apache.kafka.common.message.LeaderAndIsrRequestData.LeaderAndIsrPartitionState import org.apache.kafka.common.protocol.ApiKeys import org.apache.kafka.common.record.{MemoryRecords, SimpleRecord} @@ -276,10 +276,13 @@ class PartitionLockTest extends Logging { logManager, alterIsrManager) { - override def prepareIsrShrink(outOfSyncReplicaIds: Set[Int]): PendingShrinkIsr = { + override def prepareIsrShrink( + currentState: CommittedPartitionState, + outOfSyncReplicaIds: Set[Int] + ): PendingShrinkIsr = { shrinkIsrSemaphore.acquire() try { - super.prepareIsrShrink(outOfSyncReplicaIds) + super.prepareIsrShrink(currentState, outOfSyncReplicaIds) } finally { shrinkIsrSemaphore.release() } @@ -319,12 +322,14 @@ class PartitionLockTest extends Logging { new SlowLog(log, offsets.logStartOffset, localLog, leaderEpochCache, producerStateManager, appendSemaphore) } } + + val topicIdPartition = new TopicIdPartition(partition.topicId.getOrElse(Uuid.ZERO_UUID), topicPartition) when(offsetCheckpoints.fetch( ArgumentMatchers.anyString, ArgumentMatchers.eq(topicPartition) )).thenReturn(None) when(alterIsrManager.submit( - ArgumentMatchers.eq(topicPartition), + ArgumentMatchers.eq(topicIdPartition), ArgumentMatchers.any[LeaderAndIsr], ArgumentMatchers.anyInt() )).thenReturn(new CompletableFuture[LeaderAndIsr]()) diff --git a/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala b/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala index 04d2b15c6034..7dcead062bc7 100644 --- a/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala +++ b/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala @@ -46,6 +46,7 @@ import java.nio.ByteBuffer import java.util.Optional import java.util.concurrent.{CountDownLatch, Semaphore} import kafka.server.epoch.LeaderEpochFileCache +import kafka.server.metadata.KRaftMetadataCache import org.apache.kafka.common.network.ListenerName import org.apache.kafka.common.replica.ClientMetadata import org.apache.kafka.common.replica.ClientMetadata.DefaultClientMetadata @@ -1332,6 +1333,202 @@ class PartitionTest extends AbstractPartitionTest { assertEquals(alterPartitionListener.failures.get, 1) } + @Test + def testIsrNotExpandedIfReplicaIsFenced(): Unit = { + val log = logManager.getOrCreateLog(topicPartition, topicId = None) + seedLogData(log, numRecords = 10, leaderEpoch = 4) + + val controllerEpoch = 0 + val leaderEpoch = 5 + val remoteBrokerId = brokerId + 1 + val replicas = List(brokerId, remoteBrokerId) + val isr = Set(brokerId) + + val metadataCache = mock(classOf[KRaftMetadataCache]) + val partition = new Partition( + topicPartition, + replicaLagTimeMaxMs = Defaults.ReplicaLagTimeMaxMs, + interBrokerProtocolVersion = MetadataVersion.latest, + localBrokerId = brokerId, + time, + alterPartitionListener, + delayedOperations, + metadataCache, + logManager, + alterPartitionManager + ) + + partition.createLogIfNotExists(isNew = false, isFutureReplica = false, offsetCheckpoints, None) + assertTrue(partition.makeLeader( + new LeaderAndIsrPartitionState() + .setControllerEpoch(controllerEpoch) + .setLeader(brokerId) + .setLeaderEpoch(leaderEpoch) + .setIsr(isr.toList.map(Int.box).asJava) + .setPartitionEpoch(1) + .setReplicas(replicas.map(Int.box).asJava) + .setIsNew(true), + offsetCheckpoints, None), "Expected become leader transition to succeed") + assertEquals(isr, partition.partitionState.isr) + assertEquals(isr, partition.partitionState.maximalIsr) + + // Fetch to let the follower catch up to the log end offset and + // to check if an expansion is possible. + fetchFollower(partition, replicaId = remoteBrokerId, fetchOffset = log.logEndOffset) + + // Follower fetches and catches up to the log end offset. + assertReplicaState(partition, remoteBrokerId, + lastCaughtUpTimeMs = time.milliseconds(), + logStartOffset = 0L, + logEndOffset = log.logEndOffset + ) + + // Expansion is triggered. + assertEquals(isr, partition.partitionState.isr) + assertEquals(replicas.toSet, partition.partitionState.maximalIsr) + assertEquals(1, alterPartitionManager.isrUpdates.size) + + // Controller rejects the expansion because the broker is fenced. + alterPartitionManager.failIsrUpdate(Errors.INELIGIBLE_REPLICA) + + // The leader reverts back to the previous ISR. + assertEquals(isr, partition.partitionState.isr) + assertEquals(isr, partition.partitionState.maximalIsr) + assertFalse(partition.partitionState.isInflight) + assertEquals(0, alterPartitionManager.isrUpdates.size) + + // The leader eventually learns about the fenced broker. + when(metadataCache.isBrokerFenced(remoteBrokerId)).thenReturn(true) + + // The follower fetches again. + fetchFollower(partition, replicaId = remoteBrokerId, fetchOffset = log.logEndOffset) + + // Expansion is not triggered because the follower is fenced. + assertEquals(isr, partition.partitionState.isr) + assertEquals(isr, partition.partitionState.maximalIsr) + assertFalse(partition.partitionState.isInflight) + assertEquals(0, alterPartitionManager.isrUpdates.size) + + // The broker is eventually unfenced. + when(metadataCache.isBrokerFenced(remoteBrokerId)).thenReturn(false) + + // The follower fetches again. + fetchFollower(partition, replicaId = remoteBrokerId, fetchOffset = log.logEndOffset) + + // Expansion is triggered. + assertEquals(isr, partition.partitionState.isr) + assertEquals(replicas.toSet, partition.partitionState.maximalIsr) + assertTrue(partition.partitionState.isInflight) + assertEquals(1, alterPartitionManager.isrUpdates.size) + + // Expansion succeeds. + alterPartitionManager.completeIsrUpdate(newPartitionEpoch = 1) + + // ISR is committed. + assertEquals(replicas.toSet, partition.partitionState.isr) + assertEquals(replicas.toSet, partition.partitionState.maximalIsr) + assertFalse(partition.partitionState.isInflight) + assertEquals(0, alterPartitionManager.isrUpdates.size) + } + + @Test + def testIsrNotExpandedIfReplicaIsInControlledShutdown(): Unit = { + val log = logManager.getOrCreateLog(topicPartition, topicId = None) + seedLogData(log, numRecords = 10, leaderEpoch = 4) + + val controllerEpoch = 0 + val leaderEpoch = 5 + val remoteBrokerId = brokerId + 1 + val replicas = List(brokerId, remoteBrokerId) + val isr = Set(brokerId) + + val metadataCache = mock(classOf[KRaftMetadataCache]) + val partition = new Partition( + topicPartition, + replicaLagTimeMaxMs = Defaults.ReplicaLagTimeMaxMs, + interBrokerProtocolVersion = MetadataVersion.latest, + localBrokerId = brokerId, + time, + alterPartitionListener, + delayedOperations, + metadataCache, + logManager, + alterPartitionManager + ) + + partition.createLogIfNotExists(isNew = false, isFutureReplica = false, offsetCheckpoints, None) + assertTrue(partition.makeLeader( + new LeaderAndIsrPartitionState() + .setControllerEpoch(controllerEpoch) + .setLeader(brokerId) + .setLeaderEpoch(leaderEpoch) + .setIsr(isr.toList.map(Int.box).asJava) + .setPartitionEpoch(1) + .setReplicas(replicas.map(Int.box).asJava) + .setIsNew(true), + offsetCheckpoints, None), "Expected become leader transition to succeed") + assertEquals(isr, partition.partitionState.isr) + assertEquals(isr, partition.partitionState.maximalIsr) + + // Fetch to let the follower catch up to the log end offset and + // to check if an expansion is possible. + fetchFollower(partition, replicaId = remoteBrokerId, fetchOffset = log.logEndOffset) + + // Follower fetches and catches up to the log end offset. + assertReplicaState(partition, remoteBrokerId, + lastCaughtUpTimeMs = time.milliseconds(), + logStartOffset = 0L, + logEndOffset = log.logEndOffset + ) + + // Expansion is triggered. + assertEquals(isr, partition.partitionState.isr) + assertEquals(replicas.toSet, partition.partitionState.maximalIsr) + assertEquals(1, alterPartitionManager.isrUpdates.size) + + // Controller rejects the expansion because the broker is in controlled shutdown. + alterPartitionManager.failIsrUpdate(Errors.INELIGIBLE_REPLICA) + + // The leader reverts back to the previous ISR. + assertEquals(isr, partition.partitionState.isr) + assertEquals(isr, partition.partitionState.maximalIsr) + assertFalse(partition.partitionState.isInflight) + assertEquals(0, alterPartitionManager.isrUpdates.size) + + // The leader eventually learns about the in controlled shutdown broker. + when(metadataCache.isBrokerShuttingDown(remoteBrokerId)).thenReturn(true) + + // The follower fetches again. + fetchFollower(partition, replicaId = remoteBrokerId, fetchOffset = log.logEndOffset) + + // Expansion is not triggered because the follower is fenced. + assertEquals(isr, partition.partitionState.isr) + assertEquals(isr, partition.partitionState.maximalIsr) + assertFalse(partition.partitionState.isInflight) + assertEquals(0, alterPartitionManager.isrUpdates.size) + + // The broker eventually comes back. + when(metadataCache.isBrokerShuttingDown(remoteBrokerId)).thenReturn(false) + + // The follower fetches again. + fetchFollower(partition, replicaId = remoteBrokerId, fetchOffset = log.logEndOffset) + + // Expansion is triggered. + assertEquals(isr, partition.partitionState.isr) + assertEquals(replicas.toSet, partition.partitionState.maximalIsr) + assertTrue(partition.partitionState.isInflight) + assertEquals(1, alterPartitionManager.isrUpdates.size) + + // Expansion succeeds. + alterPartitionManager.completeIsrUpdate(newPartitionEpoch= 1) + + // ISR is committed. + assertEquals(replicas.toSet, partition.partitionState.isr) + assertEquals(replicas.toSet, partition.partitionState.maximalIsr) + assertFalse(partition.partitionState.isInflight) + assertEquals(0, alterPartitionManager.isrUpdates.size) + } + @Test def testRetryShrinkIsr(): Unit = { val log = logManager.getOrCreateLog(topicPartition, topicId = None) @@ -1652,6 +1849,16 @@ class PartitionTest extends AbstractPartitionTest { assertEquals(0L, partition.localLogOrException.highWatermark) } + @Test + def testAlterIsrNewLeaderElected(): Unit = { + handleAlterIsrFailure(Errors.NEW_LEADER_ELECTED, + (brokerId: Int, remoteBrokerId: Int, partition: Partition) => { + assertEquals(partition.partitionState.isr, Set(brokerId)) + assertEquals(partition.partitionState.maximalIsr, Set(brokerId, remoteBrokerId)) + assertEquals(alterPartitionManager.isrUpdates.size, 0) + }) + } + @Test def testAlterIsrUnknownTopic(): Unit = { handleAlterIsrFailure(Errors.UNKNOWN_TOPIC_OR_PARTITION, diff --git a/core/src/test/scala/unit/kafka/controller/ControllerIntegrationTest.scala b/core/src/test/scala/unit/kafka/controller/ControllerIntegrationTest.scala index d49502dd628a..57cbeafd4d03 100644 --- a/core/src/test/scala/unit/kafka/controller/ControllerIntegrationTest.scala +++ b/core/src/test/scala/unit/kafka/controller/ControllerIntegrationTest.scala @@ -19,14 +19,17 @@ package kafka.controller import java.util.Properties import java.util.concurrent.{CompletableFuture, CountDownLatch, LinkedBlockingQueue, TimeUnit} +import java.util.stream.{Stream => JStream} import com.yammer.metrics.core.Timer import kafka.api.LeaderAndIsr -import kafka.controller.KafkaController.AlterPartitionCallback import kafka.server.{KafkaConfig, KafkaServer, QuorumTestHarness} import kafka.utils.{LogCaptureAppender, TestUtils} import kafka.zk.{FeatureZNodeStatus, _} import org.apache.kafka.common.errors.{ControllerMovedException, StaleBrokerEpochException} +import org.apache.kafka.common.message.AlterPartitionRequestData +import org.apache.kafka.common.message.AlterPartitionResponseData import org.apache.kafka.common.metrics.KafkaMetric +import org.apache.kafka.common.protocol.ApiKeys import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.{ElectionType, TopicPartition, Uuid} import org.apache.kafka.metadata.LeaderRecoveryState @@ -36,6 +39,9 @@ import org.apache.kafka.server.metrics.KafkaYammerMetrics import org.apache.log4j.Level import org.junit.jupiter.api.Assertions.{assertEquals, assertNotEquals, assertTrue} import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, TestInfo} +import org.junit.jupiter.params.ParameterizedTest +import org.junit.jupiter.params.provider.Arguments +import org.junit.jupiter.params.provider.MethodSource import org.mockito.Mockito.{doAnswer, spy, verify} import org.mockito.invocation.InvocationOnMock @@ -43,6 +49,16 @@ import scala.collection.{Map, Seq, mutable} import scala.jdk.CollectionConverters._ import scala.util.{Failure, Success, Try} +object ControllerIntegrationTest { + def testAlterPartitionSource(): JStream[Arguments] = { + Seq(MetadataVersion.IBP_2_7_IV0, MetadataVersion.latest).asJava.stream.flatMap { metadataVersion => + ApiKeys.ALTER_PARTITION.allVersions.stream.map { alterPartitionVersion => + Arguments.of(metadataVersion, alterPartitionVersion) + } + } + } +} + class ControllerIntegrationTest extends QuorumTestHarness { var servers = Seq.empty[KafkaServer] val firstControllerEpoch = KafkaController.InitialControllerEpoch + 1 @@ -846,6 +862,135 @@ class ControllerIntegrationTest extends QuorumTestHarness { } } + @ParameterizedTest + @MethodSource(Array("testAlterPartitionSource")) + def testAlterPartition(metadataVersion: MetadataVersion, alterPartitionVersion: Short): Unit = { + if (!metadataVersion.isTopicIdsSupported && alterPartitionVersion > 1) { + // This combination is not valid. We cannot use alter partition version > 1 + // if the broker is on an IBP < 2.8 because topics don't have id in this case. + return + } + + servers = makeServers(1, interBrokerProtocolVersion = Some(metadataVersion)) + + val controllerId = TestUtils.waitUntilControllerElected(zkClient) + val tp = new TopicPartition("t", 0) + val assignment = Map(tp.partition -> Seq(controllerId)) + TestUtils.createTopic(zkClient, tp.topic, partitionReplicaAssignment = assignment, servers = servers) + + val controller = getController().kafkaController + val leaderIsrAndControllerEpochMap = zkClient.getTopicPartitionStates(Seq(tp)) + val newLeaderAndIsr = leaderIsrAndControllerEpochMap(tp).leaderAndIsr + val topicId = controller.controllerContext.topicIds.getOrElse(tp.topic, Uuid.ZERO_UUID) + val brokerId = controllerId + val brokerEpoch = controller.controllerContext.liveBrokerIdAndEpochs(controllerId) + + // The caller of the AlterPartition API can only use topics ids iff 1) the controller is + // on IBP >= 2.8 and 2) the AlterPartition version 2 and above is used. + val canCallerUseTopicIds = metadataVersion.isTopicIdsSupported && alterPartitionVersion > 1 + + val alterPartitionRequest = new AlterPartitionRequestData() + .setBrokerId(brokerId) + .setBrokerEpoch(brokerEpoch) + .setTopics(Seq(new AlterPartitionRequestData.TopicData() + .setTopicName(if (!canCallerUseTopicIds) tp.topic else "") + .setTopicId(if (canCallerUseTopicIds) topicId else Uuid.ZERO_UUID) + .setPartitions(Seq(new AlterPartitionRequestData.PartitionData() + .setPartitionIndex(tp.partition) + .setLeaderEpoch(newLeaderAndIsr.leaderEpoch) + .setPartitionEpoch(newLeaderAndIsr.partitionEpoch) + .setNewIsr(newLeaderAndIsr.isr.map(Int.box).asJava) + .setLeaderRecoveryState(newLeaderAndIsr.leaderRecoveryState.value) + ).asJava) + ).asJava) + + val future = new CompletableFuture[AlterPartitionResponseData]() + controller.eventManager.put(AlterPartitionReceived( + alterPartitionRequest, + alterPartitionVersion, + future.complete + )) + + val expectedAlterPartitionResponse = new AlterPartitionResponseData() + .setTopics(Seq(new AlterPartitionResponseData.TopicData() + .setTopicName(if (!canCallerUseTopicIds) tp.topic else "") + .setTopicId(if (canCallerUseTopicIds) topicId else Uuid.ZERO_UUID) + .setPartitions(Seq(new AlterPartitionResponseData.PartitionData() + .setPartitionIndex(tp.partition) + .setLeaderId(brokerId) + .setLeaderEpoch(newLeaderAndIsr.leaderEpoch) + .setPartitionEpoch(newLeaderAndIsr.partitionEpoch) + .setIsr(newLeaderAndIsr.isr.map(Int.box).asJava) + .setLeaderRecoveryState(newLeaderAndIsr.leaderRecoveryState.value) + ).asJava) + ).asJava) + + assertEquals(expectedAlterPartitionResponse, future.get(10, TimeUnit.SECONDS)) + } + + @Test + def testAlterPartitionVersion2KeepWorkingWhenControllerDowngradeToPre28IBP(): Unit = { + // When the controller downgrades from IBP >= 2.8 to IBP < 2.8, it does not assign + // topic ids anymore. However, the already assigned topic ids are kept. This means + // that using AlterPartition version 2 should still work assuming that it only + // contains topic with topics ids. + servers = makeServers(1, interBrokerProtocolVersion = Some(MetadataVersion.latest)) + + val controllerId = TestUtils.waitUntilControllerElected(zkClient) + val tp = new TopicPartition("t", 0) + val assignment = Map(tp.partition -> Seq(controllerId)) + TestUtils.createTopic(zkClient, tp.topic, partitionReplicaAssignment = assignment, servers = servers) + + // Downgrade controller to IBP 2.7 + servers(0).shutdown() + servers(0).awaitShutdown() + servers = makeServers(1, interBrokerProtocolVersion = Some(IBP_2_7_IV0)) + TestUtils.waitUntilControllerElected(zkClient) + + val controller = getController().kafkaController + val leaderIsrAndControllerEpochMap = zkClient.getTopicPartitionStates(Seq(tp)) + val newLeaderAndIsr = leaderIsrAndControllerEpochMap(tp).leaderAndIsr + val topicId = controller.controllerContext.topicIds.getOrElse(tp.topic, Uuid.ZERO_UUID) + val brokerId = controllerId + val brokerEpoch = controller.controllerContext.liveBrokerIdAndEpochs(controllerId) + + val alterPartitionRequest = new AlterPartitionRequestData() + .setBrokerId(brokerId) + .setBrokerEpoch(brokerEpoch) + .setTopics(Seq(new AlterPartitionRequestData.TopicData() + .setTopicId(topicId) + .setPartitions(Seq(new AlterPartitionRequestData.PartitionData() + .setPartitionIndex(tp.partition) + .setLeaderEpoch(newLeaderAndIsr.leaderEpoch) + .setPartitionEpoch(newLeaderAndIsr.partitionEpoch) + .setNewIsr(newLeaderAndIsr.isr.map(Int.box).asJava) + .setLeaderRecoveryState(newLeaderAndIsr.leaderRecoveryState.value) + ).asJava) + ).asJava) + + val future = new CompletableFuture[AlterPartitionResponseData]() + controller.eventManager.put(AlterPartitionReceived( + alterPartitionRequest, + ApiKeys.ALTER_PARTITION.latestVersion, + future.complete + )) + + val expectedAlterPartitionResponse = new AlterPartitionResponseData() + .setTopics(Seq(new AlterPartitionResponseData.TopicData() + .setTopicId(topicId) + .setPartitions(Seq(new AlterPartitionResponseData.PartitionData() + .setPartitionIndex(tp.partition) + .setLeaderId(brokerId) + .setLeaderEpoch(newLeaderAndIsr.leaderEpoch) + .setPartitionEpoch(newLeaderAndIsr.partitionEpoch) + .setIsr(newLeaderAndIsr.isr.map(Int.box).asJava) + .setLeaderRecoveryState(newLeaderAndIsr.leaderRecoveryState.value) + ).asJava) + ).asJava) + + assertEquals(expectedAlterPartitionResponse, future.get(10, TimeUnit.SECONDS)) + } + @Test def testIdempotentAlterPartition(): Unit = { servers = makeServers(2) @@ -855,29 +1000,49 @@ class ControllerIntegrationTest extends QuorumTestHarness { val assignment = Map(tp.partition -> Seq(otherBroker.config.brokerId, controllerId)) TestUtils.createTopic(zkClient, tp.topic, partitionReplicaAssignment = assignment, servers = servers) - val latch = new CountDownLatch(1) val controller = getController().kafkaController - val leaderIsrAndControllerEpochMap = zkClient.getTopicPartitionStates(Seq(tp)) val newLeaderAndIsr = leaderIsrAndControllerEpochMap(tp).leaderAndIsr + val topicId = controller.controllerContext.topicIds(tp.topic) + val brokerId = otherBroker.config.brokerId + val brokerEpoch = controller.controllerContext.liveBrokerIdAndEpochs(otherBroker.config.brokerId) - val callback = (result: Either[Map[TopicPartition, Either[Errors, LeaderAndIsr]], Errors]) => { - result match { - case Left(partitionResults: Map[TopicPartition, Either[Errors, LeaderAndIsr]]) => - partitionResults.get(tp) match { - case Some(Left(error: Errors)) => throw new AssertionError(s"Should not have seen error for $tp") - case Some(Right(leaderAndIsr: LeaderAndIsr)) => assertEquals(leaderAndIsr, newLeaderAndIsr, "ISR should remain unchanged") - case None => throw new AssertionError(s"Should have seen $tp in result") - } - case Right(_: Errors) => throw new AssertionError("Should not have had top-level error here") - } - latch.countDown() - } - - val brokerEpoch = controller.controllerContext.liveBrokerIdAndEpochs.get(otherBroker.config.brokerId).get // When re-sending the current ISR, we should not get and error or any ISR changes - controller.eventManager.put(AlterPartitionReceived(otherBroker.config.brokerId, brokerEpoch, Map(tp -> newLeaderAndIsr), callback)) - latch.await() + val alterPartitionRequest = new AlterPartitionRequestData() + .setBrokerId(brokerId) + .setBrokerEpoch(brokerEpoch) + .setTopics(Seq(new AlterPartitionRequestData.TopicData() + .setTopicId(topicId) + .setPartitions(Seq(new AlterPartitionRequestData.PartitionData() + .setPartitionIndex(tp.partition) + .setLeaderEpoch(newLeaderAndIsr.leaderEpoch) + .setPartitionEpoch(newLeaderAndIsr.partitionEpoch) + .setNewIsr(newLeaderAndIsr.isr.map(Int.box).asJava) + .setLeaderRecoveryState(newLeaderAndIsr.leaderRecoveryState.value) + ).asJava) + ).asJava) + + val future = new CompletableFuture[AlterPartitionResponseData]() + controller.eventManager.put(AlterPartitionReceived( + alterPartitionRequest, + AlterPartitionRequestData.HIGHEST_SUPPORTED_VERSION, + future.complete + )) + + val expectedAlterPartitionResponse = new AlterPartitionResponseData() + .setTopics(Seq(new AlterPartitionResponseData.TopicData() + .setTopicId(topicId) + .setPartitions(Seq(new AlterPartitionResponseData.PartitionData() + .setPartitionIndex(tp.partition) + .setLeaderId(brokerId) + .setLeaderEpoch(newLeaderAndIsr.leaderEpoch) + .setPartitionEpoch(newLeaderAndIsr.partitionEpoch) + .setIsr(newLeaderAndIsr.isr.map(Int.box).asJava) + .setLeaderRecoveryState(newLeaderAndIsr.leaderRecoveryState.value) + ).asJava) + ).asJava) + + assertEquals(expectedAlterPartitionResponse, future.get(10, TimeUnit.SECONDS)) } @Test @@ -895,20 +1060,23 @@ class ControllerIntegrationTest extends QuorumTestHarness { val leaderBrokerEpoch = servers(leaderId).kafkaController.brokerEpoch val leaderEpoch = partitionState.leaderAndIsr.leaderEpoch val partitionEpoch = partitionState.leaderAndIsr.partitionEpoch + val topicId = controller.controllerContext.topicIds.get(tp.topic) def assertAlterPartition( topLevelError: Errors = Errors.NONE, partitionError: Errors = Errors.NONE, topicPartition: TopicPartition = tp, + topicIdOpt: Option[Uuid] = topicId, leaderId: Int = leaderId, brokerEpoch: Long = leaderBrokerEpoch, leaderEpoch: Int = leaderEpoch, partitionEpoch: Int = partitionEpoch, isr: Set[Int] = replicas.toSet, - leaderRecoveryState: LeaderRecoveryState = LeaderRecoveryState.RECOVERED + leaderRecoveryState: Byte = LeaderRecoveryState.RECOVERED.value ): Unit = { assertAlterPartitionError( topicPartition = topicPartition, + topicIdOpt = topicIdOpt, leaderId = leaderId, brokerEpoch = brokerEpoch, leaderEpoch = leaderEpoch, @@ -930,14 +1098,22 @@ class ControllerIntegrationTest extends QuorumTestHarness { leaderId = 99, ) + assertAlterPartition( + partitionError = Errors.UNKNOWN_TOPIC_ID, + topicPartition = tp, + topicIdOpt = Some(Uuid.randomUuid()) + ) + assertAlterPartition( partitionError = Errors.UNKNOWN_TOPIC_OR_PARTITION, - topicPartition = new TopicPartition("unknown", 0) + topicPartition = new TopicPartition("unknown", 0), + topicIdOpt = None ) assertAlterPartition( partitionError = Errors.UNKNOWN_TOPIC_OR_PARTITION, - topicPartition = new TopicPartition(tp.topic, 1) + topicPartition = new TopicPartition(tp.topic, 1), + topicIdOpt = None ) assertAlterPartition( @@ -957,12 +1133,12 @@ class ControllerIntegrationTest extends QuorumTestHarness { assertAlterPartition( partitionError = Errors.INVALID_REQUEST, - leaderRecoveryState = LeaderRecoveryState.RECOVERING + leaderRecoveryState = LeaderRecoveryState.RECOVERING.value ) assertAlterPartition( partitionError = Errors.INVALID_REQUEST, - leaderRecoveryState = LeaderRecoveryState.RECOVERING, + leaderRecoveryState = LeaderRecoveryState.RECOVERING.value, isr = Set(controllerId) ) @@ -971,21 +1147,27 @@ class ControllerIntegrationTest extends QuorumTestHarness { assertAlterPartition( partitionError = Errors.INVALID_UPDATE_VERSION, - leaderRecoveryState = LeaderRecoveryState.RECOVERING, + leaderRecoveryState = LeaderRecoveryState.RECOVERING.value, partitionEpoch = partitionEpoch - 1 ) assertAlterPartition( partitionError = Errors.FENCED_LEADER_EPOCH, - leaderRecoveryState = LeaderRecoveryState.RECOVERING, + leaderRecoveryState = LeaderRecoveryState.RECOVERING.value, leaderEpoch = leaderEpoch - 1 ) assertAlterPartition( partitionError = Errors.FENCED_LEADER_EPOCH, - leaderRecoveryState = LeaderRecoveryState.RECOVERING, + leaderRecoveryState = LeaderRecoveryState.RECOVERING.value, leaderEpoch = leaderEpoch + 1 ) + + // Validate that unexpected exceptions are handled correctly. + assertAlterPartition( + topLevelError = Errors.UNKNOWN_SERVER_ERROR, + leaderRecoveryState = 25, // Invalid recovery state. + ) } @Test @@ -1008,6 +1190,8 @@ class ControllerIntegrationTest extends QuorumTestHarness { val replica1 :: replica2 :: Nil = replicas TestUtils.createTopic(zkClient, tp.topic, partitionReplicaAssignment = assignment, servers = servers) + val topicIdOpt = controller.controllerContext.topicIds.get(tp.topic) + servers(replica1).shutdown() servers(replica1).awaitShutdown() @@ -1042,10 +1226,11 @@ class ControllerIntegrationTest extends QuorumTestHarness { brokerEpoch: Long = leaderBrokerEpoch, leaderEpoch: Int = leaderEpoch, partitionEpoch: Int = partitionEpoch, - leaderRecoveryState: LeaderRecoveryState = LeaderRecoveryState.RECOVERED + leaderRecoveryState: Byte = LeaderRecoveryState.RECOVERED.value ): Unit = { assertAlterPartitionError( topicPartition = tp, + topicIdOpt = topicIdOpt, leaderId = leaderId, brokerEpoch = brokerEpoch, leaderEpoch = leaderEpoch, @@ -1084,7 +1269,7 @@ class ControllerIntegrationTest extends QuorumTestHarness { assertAlterPartition( partitionError = Errors.INVALID_REQUEST, - leaderRecoveryState = LeaderRecoveryState.RECOVERING + leaderRecoveryState = LeaderRecoveryState.RECOVERING.value ) // Version/epoch errors take precedence over other validations since @@ -1093,86 +1278,70 @@ class ControllerIntegrationTest extends QuorumTestHarness { assertAlterPartition( partitionError = Errors.INVALID_UPDATE_VERSION, partitionEpoch = partitionEpoch - 1, - leaderRecoveryState = LeaderRecoveryState.RECOVERING + leaderRecoveryState = LeaderRecoveryState.RECOVERING.value ) assertAlterPartition( partitionError = Errors.FENCED_LEADER_EPOCH, leaderEpoch = leaderEpoch - 1, - leaderRecoveryState = LeaderRecoveryState.RECOVERING + leaderRecoveryState = LeaderRecoveryState.RECOVERING.value ) assertAlterPartition( partitionError = Errors.FENCED_LEADER_EPOCH, leaderEpoch = leaderEpoch + 1, - leaderRecoveryState = LeaderRecoveryState.RECOVERING + leaderRecoveryState = LeaderRecoveryState.RECOVERING.value ) } def assertAlterPartitionError( topicPartition: TopicPartition, + topicIdOpt: Option[Uuid], leaderId: Int, brokerEpoch: Long, leaderEpoch: Int, partitionEpoch: Int, isr: Set[Int], - leaderRecoveryState: LeaderRecoveryState, + leaderRecoveryState: Byte, topLevelError: Errors, partitionError: Errors, ): Unit = { - val leaderAndIsr = LeaderAndIsr( - leader = leaderId, - leaderEpoch = leaderEpoch, - isr = isr.toList, - partitionEpoch = partitionEpoch, - leaderRecoveryState = leaderRecoveryState - ) - - val future = captureAlterPartitionError( - brokerId = leaderId, - brokerEpoch = brokerEpoch, - topicPartition = topicPartition, - leaderAndIsr = leaderAndIsr - ) - - val errors = future.get(10, TimeUnit.SECONDS) - assertEquals(topLevelError, errors.topLevelError) - - if (topLevelError == Errors.NONE) { - assertEquals(Some(partitionError), errors.partitionError) - } - } - - private case class AlterPartitionError(topLevelError: Errors, partitionError: Option[Errors]) - - private def captureAlterPartitionError( - brokerId: Int, - brokerEpoch: Long, - topicPartition: TopicPartition, - leaderAndIsr: LeaderAndIsr - ): CompletableFuture[AlterPartitionError] = { - val future = new CompletableFuture[AlterPartitionError]() - val callback: AlterPartitionCallback = { - case Left(partitionResults: Map[TopicPartition, Either[Errors, LeaderAndIsr]]) => - partitionResults.get(topicPartition) match { - case Some(Left(error: Errors)) => - future.complete(AlterPartitionError(topLevelError = Errors.NONE, partitionError = Some(error))) - - case Some(Right(_: LeaderAndIsr)) => - future.complete(AlterPartitionError(topLevelError = Errors.NONE, partitionError = Some(Errors.NONE))) - - case None => - future.completeExceptionally(new AssertionError(s"Should have seen $topicPartition in result")) - } - - case Right(error: Errors) => - future.complete(AlterPartitionError(topLevelError = error, partitionError = None)) + val topicName = if (topicIdOpt.isEmpty) topicPartition.topic else "" + val topicId = topicIdOpt.getOrElse(Uuid.ZERO_UUID) + + val alterPartitionRequest = new AlterPartitionRequestData() + .setBrokerId(leaderId) + .setBrokerEpoch(brokerEpoch) + .setTopics(Seq(new AlterPartitionRequestData.TopicData() + .setTopicId(topicId) + .setTopicName(topicName) + .setPartitions(Seq(new AlterPartitionRequestData.PartitionData() + .setPartitionIndex(topicPartition.partition) + .setLeaderEpoch(leaderEpoch) + .setPartitionEpoch(partitionEpoch) + .setNewIsr(isr.toList.map(Int.box).asJava) + .setLeaderRecoveryState(leaderRecoveryState)).asJava)).asJava) + + val future = new CompletableFuture[AlterPartitionResponseData]() + getController().kafkaController.eventManager.put(AlterPartitionReceived( + alterPartitionRequest, + if (topicIdOpt.isDefined) AlterPartitionRequestData.HIGHEST_SUPPORTED_VERSION else 1, + future.complete + )) + + val expectedAlterPartitionResponse = if (topLevelError != Errors.NONE) { + new AlterPartitionResponseData().setErrorCode(topLevelError.code) + } else { + new AlterPartitionResponseData() + .setTopics(Seq(new AlterPartitionResponseData.TopicData() + .setTopicId(topicId) + .setTopicName(topicName) + .setPartitions(Seq(new AlterPartitionResponseData.PartitionData() + .setPartitionIndex(topicPartition.partition) + .setErrorCode(partitionError.code)).asJava)).asJava) } - val partitionsToAlter = Map(topicPartition -> leaderAndIsr) - val controller = getController().kafkaController - controller.eventManager.put(AlterPartitionReceived(brokerId, brokerEpoch, partitionsToAlter, callback)) - future + assertEquals(expectedAlterPartitionResponse, future.get(10, TimeUnit.SECONDS)) } @Test diff --git a/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala b/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala index 29aebd55cd82..af1762e31941 100755 --- a/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala +++ b/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala @@ -20,7 +20,6 @@ package kafka.integration import java.io.File import java.util import java.util.Arrays - import kafka.server.QuorumTestHarness import kafka.server._ import kafka.utils.TestUtils @@ -30,13 +29,12 @@ import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo} import scala.collection.{Seq, mutable} import scala.jdk.CollectionConverters._ import java.util.Properties - import kafka.utils.TestUtils.{createAdminClient, resource} import org.apache.kafka.common.{KafkaException, Uuid} import org.apache.kafka.common.network.ListenerName import org.apache.kafka.common.security.scram.ScramCredential import org.apache.kafka.common.utils.Time -import org.apache.kafka.controller.ControllerRequestContext.ANONYMOUS_CONTEXT +import org.apache.kafka.controller.ControllerRequestContextUtil.ANONYMOUS_CONTEXT /** * A test harness that brings up some number of broker nodes diff --git a/core/src/test/scala/unit/kafka/server/AlterPartitionManagerTest.scala b/core/src/test/scala/unit/kafka/server/AlterPartitionManagerTest.scala index dd3bfdd62da2..c35bdc0193bd 100644 --- a/core/src/test/scala/unit/kafka/server/AlterPartitionManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/AlterPartitionManagerTest.scala @@ -19,16 +19,18 @@ package kafka.server import java.util.Collections import java.util.stream.{Stream => JStream} - import kafka.api.LeaderAndIsr import kafka.utils.{MockScheduler, MockTime} import kafka.zk.KafkaZkClient import org.apache.kafka.clients.ClientResponse -import org.apache.kafka.common.TopicPartition +import org.apache.kafka.common.TopicIdPartition +import org.apache.kafka.common.Uuid import org.apache.kafka.common.errors.{AuthenticationException, InvalidUpdateVersionException, OperationNotAttemptedException, UnknownServerException, UnsupportedVersionException} import org.apache.kafka.common.message.AlterPartitionResponseData import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.protocol.{ApiKeys, Errors} +import org.apache.kafka.common.requests.AbstractResponse +import org.apache.kafka.common.requests.RequestHeader import org.apache.kafka.common.requests.{AbstractRequest, AlterPartitionRequest, AlterPartitionResponse} import org.apache.kafka.metadata.LeaderRecoveryState import org.apache.kafka.server.common.MetadataVersion @@ -40,6 +42,7 @@ import org.junit.jupiter.api.Test import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.provider.Arguments import org.junit.jupiter.params.provider.MethodSource +import org.mockito.ArgumentMatcher import org.mockito.ArgumentMatchers.{any, anyString} import org.mockito.Mockito.{mock, reset, times, verify} import org.mockito.{ArgumentCaptor, ArgumentMatchers, Mockito} @@ -49,15 +52,16 @@ import scala.jdk.CollectionConverters._ class AlterPartitionManagerTest { val topic = "test-topic" + val topicId = Uuid.randomUuid() val time = new MockTime val metrics = new Metrics val brokerId = 1 var brokerToController: BrokerToControllerChannelManager = _ - val tp0 = new TopicPartition(topic, 0) - val tp1 = new TopicPartition(topic, 1) - val tp2 = new TopicPartition(topic, 2) + val tp0 = new TopicIdPartition(topicId, 0, topic) + val tp1 = new TopicIdPartition(topicId, 1, topic) + val tp2 = new TopicIdPartition(topicId, 2, topic) @BeforeEach def setup(): Unit = { @@ -68,9 +72,9 @@ class AlterPartitionManagerTest { @MethodSource(Array("provideMetadataVersions")) def testBasic(metadataVersion: MetadataVersion): Unit = { val scheduler = new MockScheduler(time) - val alterIsrManager = new DefaultAlterPartitionManager(brokerToController, scheduler, time, brokerId, () => 2, () => metadataVersion) - alterIsrManager.start() - alterIsrManager.submit(tp0, new LeaderAndIsr(1, 1, List(1,2,3), LeaderRecoveryState.RECOVERED, 10), 0) + val alterPartitionManager = new DefaultAlterPartitionManager(brokerToController, scheduler, time, brokerId, () => 2, () => metadataVersion) + alterPartitionManager.start() + alterPartitionManager.submit(tp0, new LeaderAndIsr(1, 1, List(1,2,3), LeaderRecoveryState.RECOVERED, 10), 0) verify(brokerToController).start() verify(brokerToController).sendRequest(any(), any()) } @@ -84,9 +88,9 @@ class AlterPartitionManagerTest { val requestCapture = ArgumentCaptor.forClass(classOf[AbstractRequest.Builder[AlterPartitionRequest]]) val scheduler = new MockScheduler(time) - val alterIsrManager = new DefaultAlterPartitionManager(brokerToController, scheduler, time, brokerId, () => 2, () => metadataVersion) - alterIsrManager.start() - alterIsrManager.submit(tp0, new LeaderAndIsr(1, 1, List(1), leaderRecoveryState, 10), 0) + val alterPartitionManager = new DefaultAlterPartitionManager(brokerToController, scheduler, time, brokerId, () => 2, () => metadataVersion) + alterPartitionManager.start() + alterPartitionManager.submit(tp0, new LeaderAndIsr(1, 1, List(1), leaderRecoveryState, 10), 0) verify(brokerToController).start() verify(brokerToController).sendRequest(requestCapture.capture(), any()) @@ -98,30 +102,33 @@ class AlterPartitionManagerTest { @ParameterizedTest @MethodSource(Array("provideMetadataVersions")) def testOverwriteWithinBatch(metadataVersion: MetadataVersion): Unit = { + val canUseTopicIds = metadataVersion.isAtLeast(MetadataVersion.IBP_2_8_IV0) val capture: ArgumentCaptor[AbstractRequest.Builder[AlterPartitionRequest]] = ArgumentCaptor.forClass(classOf[AbstractRequest.Builder[AlterPartitionRequest]]) val callbackCapture: ArgumentCaptor[ControllerRequestCompletionHandler] = ArgumentCaptor.forClass(classOf[ControllerRequestCompletionHandler]) val scheduler = new MockScheduler(time) - val alterIsrManager = new DefaultAlterPartitionManager(brokerToController, scheduler, time, brokerId, () => 2, () => metadataVersion) - alterIsrManager.start() + val alterPartitionManager = new DefaultAlterPartitionManager(brokerToController, scheduler, time, brokerId, () => 2, () => metadataVersion) + alterPartitionManager.start() // Only send one ISR update for a given topic+partition - val firstSubmitFuture = alterIsrManager.submit(tp0, LeaderAndIsr(1, 1, List(1,2,3), LeaderRecoveryState.RECOVERED, 10), 0) + val firstSubmitFuture = alterPartitionManager.submit(tp0, LeaderAndIsr(1, 1, List(1,2,3), LeaderRecoveryState.RECOVERED, 10), 0) assertFalse(firstSubmitFuture.isDone) - val failedSubmitFuture = alterIsrManager.submit(tp0, LeaderAndIsr(1, 1, List(1,2), LeaderRecoveryState.RECOVERED, 10), 0) + val failedSubmitFuture = alterPartitionManager.submit(tp0, LeaderAndIsr(1, 1, List(1,2), LeaderRecoveryState.RECOVERED, 10), 0) assertTrue(failedSubmitFuture.isCompletedExceptionally) assertFutureThrows(failedSubmitFuture, classOf[OperationNotAttemptedException]) // Simulate response val alterPartitionResp = partitionResponse(tp0, Errors.NONE) - val resp = new ClientResponse(null, null, "", 0L, 0L, - false, null, null, alterPartitionResp) + val resp = makeClientResponse( + response = alterPartitionResp, + version = if (canUseTopicIds) ApiKeys.ALTER_PARTITION.latestVersion else 1 + ) verify(brokerToController).sendRequest(capture.capture(), callbackCapture.capture()) callbackCapture.getValue.onComplete(resp) // Now we can submit this partition again - val newSubmitFuture = alterIsrManager.submit(tp0, LeaderAndIsr(1, 1, List(1), LeaderRecoveryState.RECOVERED, 10), 0) + val newSubmitFuture = alterPartitionManager.submit(tp0, LeaderAndIsr(1, 1, List(1), LeaderRecoveryState.RECOVERED, 10), 0) assertFalse(newSubmitFuture.isDone) verify(brokerToController).start() @@ -140,16 +147,16 @@ class AlterPartitionManagerTest { val callbackCapture: ArgumentCaptor[ControllerRequestCompletionHandler] = ArgumentCaptor.forClass(classOf[ControllerRequestCompletionHandler]) val scheduler = new MockScheduler(time) - val alterIsrManager = new DefaultAlterPartitionManager(brokerToController, scheduler, time, brokerId, () => 2, () => metadataVersion) - alterIsrManager.start() + val alterPartitionManager = new DefaultAlterPartitionManager(brokerToController, scheduler, time, brokerId, () => 2, () => metadataVersion) + alterPartitionManager.start() // First request will send batch of one - alterIsrManager.submit(new TopicPartition(topic, 0), + alterPartitionManager.submit(new TopicIdPartition(topicId, 0, topic), LeaderAndIsr(1, 1, List(1,2,3), LeaderRecoveryState.RECOVERED, 10), 0) // Other submissions will queue up until a response for (i <- 1 to 9) { - alterIsrManager.submit(new TopicPartition(topic, i), + alterPartitionManager.submit(new TopicIdPartition(topicId, i, topic), LeaderAndIsr(1, 1, List(1,2,3), LeaderRecoveryState.RECOVERED, 10), 0) } @@ -200,8 +207,7 @@ class AlterPartitionManagerTest { private def testRetryOnTopLevelError(error: Errors): Unit = { val alterPartitionResp = new AlterPartitionResponse(new AlterPartitionResponseData().setErrorCode(error.code)) - val response = new ClientResponse(null, null, "", 0L, 0L, - false, null, null, alterPartitionResp) + val response = makeClientResponse(alterPartitionResp, ApiKeys.ALTER_PARTITION.latestVersion) testRetryOnErrorResponse(response) } @@ -210,16 +216,16 @@ class AlterPartitionManagerTest { val callbackCapture: ArgumentCaptor[ControllerRequestCompletionHandler] = ArgumentCaptor.forClass(classOf[ControllerRequestCompletionHandler]) val scheduler = new MockScheduler(time) - val alterIsrManager = new DefaultAlterPartitionManager(brokerToController, scheduler, time, brokerId, () => 2, () => IBP_3_2_IV0) - alterIsrManager.start() - alterIsrManager.submit(tp0, leaderAndIsr, 0) + val alterPartitionManager = new DefaultAlterPartitionManager(brokerToController, scheduler, time, brokerId, () => 2, () => IBP_3_2_IV0) + alterPartitionManager.start() + alterPartitionManager.submit(tp0, leaderAndIsr, 0) verify(brokerToController).start() verify(brokerToController).sendRequest(any(), callbackCapture.capture()) callbackCapture.getValue.onComplete(response) // Any top-level error, we want to retry, so we don't clear items from the pending map - assertTrue(alterIsrManager.unsentIsrUpdates.containsKey(tp0)) + assertTrue(alterPartitionManager.unsentIsrUpdates.containsKey(tp0.topicPartition)) reset(brokerToController) @@ -229,13 +235,12 @@ class AlterPartitionManagerTest { // After a successful response, we can submit another AlterIsrItem val retryAlterPartitionResponse = partitionResponse(tp0, Errors.NONE) - val retryResponse = new ClientResponse(null, null, "", 0L, 0L, - false, null, null, retryAlterPartitionResponse) + val retryResponse = makeClientResponse(retryAlterPartitionResponse, ApiKeys.ALTER_PARTITION.latestVersion) verify(brokerToController).sendRequest(any(), callbackCapture.capture()) callbackCapture.getValue.onComplete(retryResponse) - assertFalse(alterIsrManager.unsentIsrUpdates.containsKey(tp0)) + assertFalse(alterPartitionManager.unsentIsrUpdates.containsKey(tp0.topicPartition)) } @Test @@ -259,33 +264,32 @@ class AlterPartitionManagerTest { } private def checkPartitionError(error: Errors): Unit = { - val alterIsrManager = testPartitionError(tp0, error) + val alterPartitionManager = testPartitionError(tp0, error) // Any partition-level error should clear the item from the pending queue allowing for future updates - val future = alterIsrManager.submit(tp0, LeaderAndIsr(1, 1, List(1,2,3), LeaderRecoveryState.RECOVERED, 10), 0) + val future = alterPartitionManager.submit(tp0, LeaderAndIsr(1, 1, List(1,2,3), LeaderRecoveryState.RECOVERED, 10), 0) assertFalse(future.isDone) } - private def testPartitionError(tp: TopicPartition, error: Errors): AlterPartitionManager = { + private def testPartitionError(tp: TopicIdPartition, error: Errors): AlterPartitionManager = { val callbackCapture: ArgumentCaptor[ControllerRequestCompletionHandler] = ArgumentCaptor.forClass(classOf[ControllerRequestCompletionHandler]) reset(brokerToController) val scheduler = new MockScheduler(time) - val alterIsrManager = new DefaultAlterPartitionManager(brokerToController, scheduler, time, brokerId, () => 2, () => IBP_3_2_IV0) - alterIsrManager.start() + val alterPartitionManager = new DefaultAlterPartitionManager(brokerToController, scheduler, time, brokerId, () => 2, () => IBP_3_2_IV0) + alterPartitionManager.start() - val future = alterIsrManager.submit(tp, LeaderAndIsr(1, 1, List(1,2,3), LeaderRecoveryState.RECOVERED, 10), 0) + val future = alterPartitionManager.submit(tp, LeaderAndIsr(1, 1, List(1,2,3), LeaderRecoveryState.RECOVERED, 10), 0) verify(brokerToController).start() verify(brokerToController).sendRequest(any(), callbackCapture.capture()) reset(brokerToController) val alterPartitionResp = partitionResponse(tp, error) - val resp = new ClientResponse(null, null, "", 0L, 0L, - false, null, null, alterPartitionResp) + val resp = makeClientResponse(alterPartitionResp, ApiKeys.ALTER_PARTITION.latestVersion) callbackCapture.getValue.onComplete(resp) assertTrue(future.isCompletedExceptionally) assertFutureThrows(future, error.exception.getClass) - alterIsrManager + alterPartitionManager } @ParameterizedTest @@ -294,15 +298,15 @@ class AlterPartitionManagerTest { val callbackCapture: ArgumentCaptor[ControllerRequestCompletionHandler] = ArgumentCaptor.forClass(classOf[ControllerRequestCompletionHandler]) val scheduler = new MockScheduler(time) - val alterIsrManager = new DefaultAlterPartitionManager(brokerToController, scheduler, time, brokerId, () => 2, () => metadataVersion) - alterIsrManager.start() + val alterPartitionManager = new DefaultAlterPartitionManager(brokerToController, scheduler, time, brokerId, () => 2, () => metadataVersion) + alterPartitionManager.start() // First submit will send the request - alterIsrManager.submit(tp0, LeaderAndIsr(1, 1, List(1,2,3), LeaderRecoveryState.RECOVERED, 10), 0) + alterPartitionManager.submit(tp0, LeaderAndIsr(1, 1, List(1,2,3), LeaderRecoveryState.RECOVERED, 10), 0) // These will become pending unsent items - alterIsrManager.submit(tp1, LeaderAndIsr(1, 1, List(1,2,3), LeaderRecoveryState.RECOVERED, 10), 0) - alterIsrManager.submit(tp2, LeaderAndIsr(1, 1, List(1,2,3), LeaderRecoveryState.RECOVERED, 10), 0) + alterPartitionManager.submit(tp1, LeaderAndIsr(1, 1, List(1,2,3), LeaderRecoveryState.RECOVERED, 10), 0) + alterPartitionManager.submit(tp2, LeaderAndIsr(1, 1, List(1,2,3), LeaderRecoveryState.RECOVERED, 10), 0) verify(brokerToController).start() verify(brokerToController).sendRequest(any(), callbackCapture.capture()) @@ -311,78 +315,209 @@ class AlterPartitionManagerTest { reset(brokerToController) val alterPartitionResp = new AlterPartitionResponse(new AlterPartitionResponseData()) - val resp = new ClientResponse(null, null, "", 0L, 0L, - false, null, null, alterPartitionResp) + val resp = makeClientResponse(alterPartitionResp, ApiKeys.ALTER_PARTITION.latestVersion) callbackCapture.getValue.onComplete(resp) } @ParameterizedTest @MethodSource(Array("provideMetadataVersions")) def testPartitionMissingInResponse(metadataVersion: MetadataVersion): Unit = { - brokerToController = Mockito.mock(classOf[BrokerToControllerChannelManager]) - + val expectedVersion = if (metadataVersion.isTopicIdsSupported) { + ApiKeys.ALTER_PARTITION.latestVersion + } else { + 1.toShort + } + val leaderAndIsr = LeaderAndIsr(1, 1, List(1,2,3), LeaderRecoveryState.RECOVERED, 10) + val controlledEpoch = 0 val brokerEpoch = 2 val scheduler = new MockScheduler(time) - val alterIsrManager = new DefaultAlterPartitionManager(brokerToController, scheduler, time, brokerId, () => brokerEpoch, () => metadataVersion) - alterIsrManager.start() - - def matchesAlterIsr(topicPartitions: Set[TopicPartition]): AbstractRequest.Builder[_ <: AbstractRequest] = { - ArgumentMatchers.argThat[AbstractRequest.Builder[_ <: AbstractRequest]] { request => - assertEquals(ApiKeys.ALTER_PARTITION, request.apiKey()) - val alterPartitionRequest = request.asInstanceOf[AlterPartitionRequest.Builder].build() - - val requestTopicPartitions = alterPartitionRequest.data.topics.asScala.flatMap { topicData => - val topic = topicData.name - topicData.partitions.asScala.map(partitionData => new TopicPartition(topic, partitionData.partitionIndex)) - }.toSet - - topicPartitions == requestTopicPartitions - } - } - - def verifySendAlterIsr(topicPartitions: Set[TopicPartition]): ControllerRequestCompletionHandler = { - val callbackCapture: ArgumentCaptor[ControllerRequestCompletionHandler] = - ArgumentCaptor.forClass(classOf[ControllerRequestCompletionHandler]) - Mockito.verify(brokerToController).sendRequest( - matchesAlterIsr(topicPartitions), - callbackCapture.capture() - ) - Mockito.reset(brokerToController) - callbackCapture.getValue - } - - def clientResponse(topicPartition: TopicPartition, error: Errors): ClientResponse = { - val alterIsrResponse = partitionResponse(topicPartition, error) - new ClientResponse(null, null, "", 0L, 0L, - false, null, null, alterIsrResponse) - } + val brokerToController = Mockito.mock(classOf[BrokerToControllerChannelManager]) + val alterPartitionManager = new DefaultAlterPartitionManager( + brokerToController, + scheduler, + time, + brokerId, + () => brokerEpoch, + () => metadataVersion + ) + alterPartitionManager.start() // The first `submit` will send the `AlterIsr` request - val future1 = alterIsrManager.submit(tp0, LeaderAndIsr(1, 1, List(1,2,3), LeaderRecoveryState.RECOVERED, 10), 0) - val callback1 = verifySendAlterIsr(Set(tp0)) + val future1 = alterPartitionManager.submit(tp0, leaderAndIsr, controlledEpoch) + val callback1 = verifySendRequest(brokerToController, alterPartitionRequestMatcher( + expectedTopicPartitions = Set(tp0), + expectedVersion = expectedVersion + )) // Additional calls while the `AlterIsr` request is inflight will be queued - val future2 = alterIsrManager.submit(tp1, LeaderAndIsr(1, 1, List(1,2,3), LeaderRecoveryState.RECOVERED, 10), 0) - val future3 = alterIsrManager.submit(tp2, LeaderAndIsr(1, 1, List(1,2,3), LeaderRecoveryState.RECOVERED, 10), 0) + val future2 = alterPartitionManager.submit(tp1, leaderAndIsr, controlledEpoch) + val future3 = alterPartitionManager.submit(tp2, leaderAndIsr, controlledEpoch) // Respond to the first request, which will also allow the next request to get sent - callback1.onComplete(clientResponse(tp0, Errors.UNKNOWN_SERVER_ERROR)) + callback1.onComplete(makeClientResponse( + response = partitionResponse(tp0, Errors.UNKNOWN_SERVER_ERROR), + version = expectedVersion + )) assertFutureThrows(future1, classOf[UnknownServerException]) assertFalse(future2.isDone) assertFalse(future3.isDone) // Verify the second request includes both expected partitions, but only respond with one of them - val callback2 = verifySendAlterIsr(Set(tp1, tp2)) - callback2.onComplete(clientResponse(tp2, Errors.UNKNOWN_SERVER_ERROR)) + val callback2 = verifySendRequest(brokerToController, alterPartitionRequestMatcher( + expectedTopicPartitions = Set(tp1, tp2), + expectedVersion = expectedVersion + )) + callback2.onComplete(makeClientResponse( + response = partitionResponse(tp2, Errors.UNKNOWN_SERVER_ERROR), + version = expectedVersion + )) assertFutureThrows(future3, classOf[UnknownServerException]) assertFalse(future2.isDone) // The missing partition should be retried - val callback3 = verifySendAlterIsr(Set(tp1)) - callback3.onComplete(clientResponse(tp1, Errors.UNKNOWN_SERVER_ERROR)) + val callback3 = verifySendRequest(brokerToController, alterPartitionRequestMatcher( + expectedTopicPartitions = Set(tp1), + expectedVersion = expectedVersion + )) + callback3.onComplete(makeClientResponse( + response = partitionResponse(tp1, Errors.UNKNOWN_SERVER_ERROR), + version = expectedVersion + )) assertFutureThrows(future2, classOf[UnknownServerException]) } + @ParameterizedTest + @MethodSource(Array("provideMetadataVersions")) + def testPartialTopicIds(metadataVersion: MetadataVersion): Unit = { + val canUseTopicIds = metadataVersion.isAtLeast(MetadataVersion.IBP_2_8_IV0) + val foo = new TopicIdPartition(Uuid.ZERO_UUID, 0, "foo") + val bar = new TopicIdPartition(Uuid.randomUuid(), 0, "bar") + val zar = new TopicIdPartition(Uuid.randomUuid(), 0, "zar") + + val leaderAndIsr = LeaderAndIsr(1, 1, List(1,2,3), LeaderRecoveryState.RECOVERED, 10) + val controlledEpoch = 0 + val brokerEpoch = 2 + val scheduler = new MockScheduler(time) + val brokerToController = Mockito.mock(classOf[BrokerToControllerChannelManager]) + val alterPartitionManager = new DefaultAlterPartitionManager( + brokerToController, + scheduler, + time, + brokerId, + () => brokerEpoch, + () => metadataVersion + ) + alterPartitionManager.start() + + // Submits an alter isr update with zar, which has a topic id. + val future1 = alterPartitionManager.submit(zar, leaderAndIsr, controlledEpoch) + + // The latest version is expected if all the submitted partitions + // have topic ids and IBP >= 2.8; version 1 should be used otherwise. + val callback1 = verifySendRequest(brokerToController, alterPartitionRequestMatcher( + expectedTopicPartitions = Set(zar), + expectedVersion = if (canUseTopicIds) ApiKeys.ALTER_PARTITION.latestVersion else 1 + )) + + // Submits two additional alter isr changes with foo and bar while the previous one + // is still inflight. foo has no topic id, bar has one. + val future2 = alterPartitionManager.submit(foo, leaderAndIsr, controlledEpoch) + val future3 = alterPartitionManager.submit(bar, leaderAndIsr, controlledEpoch) + + // Completes the first request. That triggers the next one. + callback1.onComplete(makeClientResponse( + response = makeAlterPartition(Seq(makeAlterPartitionTopicData(zar, Errors.NONE))), + version = if (canUseTopicIds) ApiKeys.ALTER_PARTITION.latestVersion else 1 + )) + + assertTrue(future1.isDone) + assertFalse(future2.isDone) + assertFalse(future3.isDone) + + // Version 1 is expected because foo does not have a topic id. + val callback2 = verifySendRequest(brokerToController, alterPartitionRequestMatcher( + expectedTopicPartitions = Set(foo, bar), + expectedVersion = 1 + )) + + // Completes the second request. + callback2.onComplete(makeClientResponse( + response = makeAlterPartition(Seq( + makeAlterPartitionTopicData(foo, Errors.NONE), + makeAlterPartitionTopicData(bar, Errors.NONE), + )), + version = 1 + )) + + assertTrue(future1.isDone) + assertTrue(future2.isDone) + assertTrue(future3.isDone) + } + + private def verifySendRequest( + brokerToController: BrokerToControllerChannelManager, + expectedRequest: ArgumentMatcher[AbstractRequest.Builder[_ <: AbstractRequest]] + ): ControllerRequestCompletionHandler = { + val callbackCapture = ArgumentCaptor.forClass(classOf[ControllerRequestCompletionHandler]) + + Mockito.verify(brokerToController).sendRequest( + ArgumentMatchers.argThat(expectedRequest), + callbackCapture.capture() + ) + + Mockito.reset(brokerToController) + + callbackCapture.getValue + } + + private def alterPartitionRequestMatcher( + expectedTopicPartitions: Set[TopicIdPartition], + expectedVersion: Short + ): ArgumentMatcher[AbstractRequest.Builder[_ <: AbstractRequest]] = { + request => { + assertEquals(ApiKeys.ALTER_PARTITION, request.apiKey) + + val alterPartitionRequest = request.asInstanceOf[AlterPartitionRequest.Builder].build() + assertEquals(expectedVersion, alterPartitionRequest.version) + + val requestTopicPartitions = alterPartitionRequest.data.topics.asScala.flatMap { topicData => + topicData.partitions.asScala.map { partitionData => + new TopicIdPartition(topicData.topicId, partitionData.partitionIndex, topicData.topicName) + } + }.toSet + + expectedTopicPartitions == requestTopicPartitions + } + } + + private def makeClientResponse( + response: AbstractResponse, + version: Short + ): ClientResponse = { + val requestHeader = new RequestHeader(response.apiKey, version, "", 0) + new ClientResponse(requestHeader, null, "", 0L, 0L, + false, null, null, response) + } + + private def makeAlterPartition( + topics: Seq[AlterPartitionResponseData.TopicData] + ): AlterPartitionResponse = { + new AlterPartitionResponse(new AlterPartitionResponseData().setTopics(topics.asJava)) + } + + private def makeAlterPartitionTopicData( + topicIdPartition: TopicIdPartition, + error: Errors + ): AlterPartitionResponseData.TopicData = { + new AlterPartitionResponseData.TopicData() + .setTopicName(topicIdPartition.topic) + .setTopicId(topicIdPartition.topicId) + .setPartitions(Collections.singletonList( + new AlterPartitionResponseData.PartitionData() + .setPartitionIndex(topicIdPartition.partition) + .setErrorCode(error.code))) + } + @Test def testZkBasic(): Unit = { val scheduler = new MockScheduler(time) @@ -410,11 +545,12 @@ class AlterPartitionManagerTest { assertFutureThrows(future2, classOf[InvalidUpdateVersionException]) } - private def partitionResponse(tp: TopicPartition, error: Errors): AlterPartitionResponse = { + private def partitionResponse(tp: TopicIdPartition, error: Errors): AlterPartitionResponse = { new AlterPartitionResponse(new AlterPartitionResponseData() .setTopics(Collections.singletonList( new AlterPartitionResponseData.TopicData() - .setName(tp.topic()) + .setTopicName(tp.topic) + .setTopicId(tp.topicId) .setPartitions(Collections.singletonList( new AlterPartitionResponseData.PartitionData() .setPartitionIndex(tp.partition()) diff --git a/core/src/test/scala/unit/kafka/server/ControllerApisTest.scala b/core/src/test/scala/unit/kafka/server/ControllerApisTest.scala index 9f0ae482c5e7..0c6f979874f6 100644 --- a/core/src/test/scala/unit/kafka/server/ControllerApisTest.scala +++ b/core/src/test/scala/unit/kafka/server/ControllerApisTest.scala @@ -52,8 +52,8 @@ import org.apache.kafka.common.requests._ import org.apache.kafka.common.resource.{PatternType, Resource, ResourcePattern, ResourceType} import org.apache.kafka.common.security.auth.{KafkaPrincipal, SecurityProtocol} import org.apache.kafka.common.{ElectionType, Uuid} +import org.apache.kafka.controller.ControllerRequestContextUtil.ANONYMOUS_CONTEXT import org.apache.kafka.controller.{Controller, ControllerRequestContext} -import org.apache.kafka.controller.ControllerRequestContext.ANONYMOUS_CONTEXT import org.apache.kafka.server.authorizer.{Action, AuthorizableRequestContext, AuthorizationResult, Authorizer} import org.apache.kafka.server.common.ApiMessageAndVersion import org.junit.jupiter.api.Assertions._ @@ -298,7 +298,7 @@ class ControllerApisTest { assertThrows(classOf[ClusterAuthorizationException], () => createControllerApis( Some(createDenyAllAuthorizer()), new MockController.Builder().build()). handleAlterPartitionRequest(buildRequest(new AlterPartitionRequest.Builder( - new AlterPartitionRequestData()).build(0)))) + new AlterPartitionRequestData(), false).build(0)))) } @Test diff --git a/core/src/test/scala/unit/kafka/server/DescribeLogDirsRequestTest.scala b/core/src/test/scala/unit/kafka/server/DescribeLogDirsRequestTest.scala index 9ab3f86d7709..0ad110dcdbfa 100644 --- a/core/src/test/scala/unit/kafka/server/DescribeLogDirsRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/DescribeLogDirsRequestTest.scala @@ -53,9 +53,13 @@ class DescribeLogDirsRequestTest extends BaseRequestTest { val offlineResult = response.data.results.asScala.find(logDirResult => logDirResult.logDir == offlineDir).get assertEquals(Errors.KAFKA_STORAGE_ERROR.code, offlineResult.errorCode) assertEquals(0, offlineResult.topics.asScala.map(t => t.partitions().size()).sum) + assertEquals(DescribeLogDirsResponse.UNKNOWN_VOLUME_BYTES, offlineResult.totalBytes) + assertEquals(DescribeLogDirsResponse.UNKNOWN_VOLUME_BYTES, offlineResult.usableBytes) val onlineResult = response.data.results.asScala.find(logDirResult => logDirResult.logDir == onlineDir).get assertEquals(Errors.NONE.code, onlineResult.errorCode) + assertTrue(onlineResult.totalBytes > 0) + assertTrue(onlineResult.usableBytes > 0) val onlinePartitionsMap = onlineResult.topics.asScala.flatMap { topic => topic.partitions().asScala.map { partitionResult => new TopicPartition(topic.name, partitionResult.partitionIndex) -> partitionResult diff --git a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala index 1dd0a9ebc85f..ee638ba893d9 100755 --- a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala @@ -1570,4 +1570,32 @@ class KafkaConfigTest { "contained in listeners or controller.listener.names", assertThrows(classOf[ConfigException], () => new KafkaConfig(props)).getMessage) } + + @Test + def testIgnoreUserInterBrokerProtocolVersionKRaft(): Unit = { + for (ibp <- Seq("3.0", "3.1", "3.2")) { + val props = new Properties() + props.putAll(kraftProps()) + props.setProperty(KafkaConfig.InterBrokerProtocolVersionProp, ibp) + val config = new KafkaConfig(props) + assertEquals(config.interBrokerProtocolVersion, MetadataVersion.MINIMUM_KRAFT_VERSION) + } + } + + @Test + def testInvalidInterBrokerProtocolVersionKRaft(): Unit = { + val props = new Properties() + props.putAll(kraftProps()) + props.setProperty(KafkaConfig.InterBrokerProtocolVersionProp, "2.8") + assertEquals("A non-KRaft version 2.8 given for inter.broker.protocol.version. The minimum version is 3.0-IV1", + assertThrows(classOf[ConfigException], () => new KafkaConfig(props)).getMessage) + } + + @Test + def testDefaultInterBrokerProtocolVersionKRaft(): Unit = { + val props = new Properties() + props.putAll(kraftProps()) + val config = new KafkaConfig(props) + assertEquals(config.interBrokerProtocolVersion, MetadataVersion.MINIMUM_KRAFT_VERSION) + } } diff --git a/core/src/test/scala/unit/kafka/server/KafkaRaftServerTest.scala b/core/src/test/scala/unit/kafka/server/KafkaRaftServerTest.scala index f997455f0b92..17483e58a6a0 100644 --- a/core/src/test/scala/unit/kafka/server/KafkaRaftServerTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaRaftServerTest.scala @@ -24,6 +24,7 @@ import kafka.log.UnifiedLog import org.apache.kafka.common.{KafkaException, Uuid} import org.apache.kafka.common.utils.Utils import org.apache.kafka.controller.BootstrapMetadata +import org.apache.kafka.server.common.MetadataVersion import org.apache.kafka.test.TestUtils import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Test @@ -71,12 +72,13 @@ class KafkaRaftServerTest { private def invokeLoadMetaProperties( metaProperties: MetaProperties, - configProperties: Properties + configProperties: Properties, + metadataVersion: Option[MetadataVersion] = Some(MetadataVersion.latest()) ): (MetaProperties, BootstrapMetadata, collection.Seq[String]) = { val tempLogDir = TestUtils.tempDirectory() try { writeMetaProperties(tempLogDir, metaProperties) - + metadataVersion.foreach(mv => writeBootstrapMetadata(tempLogDir, mv)) configProperties.put(KafkaConfig.LogDirProp, tempLogDir.getAbsolutePath) val config = KafkaConfig.fromProps(configProperties) KafkaRaftServer.initializeLogDirs(config) @@ -94,6 +96,11 @@ class KafkaRaftServerTest { checkpoint.write(metaProperties.toProperties) } + private def writeBootstrapMetadata(logDir: File, metadataVersion: MetadataVersion): Unit = { + val bootstrapMetadata = BootstrapMetadata.create(metadataVersion) + BootstrapMetadata.write(bootstrapMetadata, logDir.toPath) + } + @Test def testStartupFailsIfMetaPropertiesMissingInSomeLogDir(): Unit = { val clusterId = clusterIdBase64 @@ -147,6 +154,7 @@ class KafkaRaftServerTest { // One log dir is online and has properly formatted `meta.properties` val validDir = TestUtils.tempDirectory() writeMetaProperties(validDir, MetaProperties(clusterId, nodeId)) + writeBootstrapMetadata(validDir, MetadataVersion.latest()) // Use a regular file as an invalid log dir to trigger an IO error val invalidDir = TestUtils.tempFile("blah") @@ -215,4 +223,47 @@ class KafkaRaftServerTest { () => KafkaRaftServer.initializeLogDirs(config)) } + @Test + def testKRaftUpdateWithIBP(): Unit = { + val clusterId = clusterIdBase64 + val nodeId = 0 + val metaProperties = MetaProperties(clusterId, nodeId) + + val configProperties = new Properties + configProperties.put(KafkaConfig.ProcessRolesProp, "broker,controller") + configProperties.put(KafkaConfig.NodeIdProp, nodeId.toString) + configProperties.put(KafkaConfig.ListenersProp, "PLAINTEXT://127.0.0.1:9092,SSL://127.0.0.1:9093") + configProperties.put(KafkaConfig.QuorumVotersProp, s"$nodeId@localhost:9093") + configProperties.put(KafkaConfig.ControllerListenerNamesProp, "SSL") + configProperties.put(KafkaConfig.InterBrokerProtocolVersionProp, "3.2") + + val (loadedMetaProperties, bootstrapMetadata, offlineDirs) = + invokeLoadMetaProperties(metaProperties, configProperties, None) + + assertEquals(metaProperties, loadedMetaProperties) + assertEquals(Seq.empty, offlineDirs) + assertEquals(bootstrapMetadata.metadataVersion(), MetadataVersion.IBP_3_2_IV0) + } + + @Test + def testKRaftUpdateWithoutIBP(): Unit = { + val clusterId = clusterIdBase64 + val nodeId = 0 + val metaProperties = MetaProperties(clusterId, nodeId) + + val logDir = TestUtils.tempDirectory() + writeMetaProperties(logDir, metaProperties) + + val configProperties = new Properties + configProperties.put(KafkaConfig.ProcessRolesProp, "broker,controller") + configProperties.put(KafkaConfig.NodeIdProp, nodeId.toString) + configProperties.put(KafkaConfig.ListenersProp, "PLAINTEXT://127.0.0.1:9092,SSL://127.0.0.1:9093") + configProperties.put(KafkaConfig.QuorumVotersProp, s"$nodeId@localhost:9093") + configProperties.put(KafkaConfig.ControllerListenerNamesProp, "SSL") + configProperties.put(KafkaConfig.LogDirProp, logDir.getAbsolutePath) + + val config = KafkaConfig.fromProps(configProperties) + assertEquals("Cannot upgrade from KRaft version prior to 3.3 without first setting inter.broker.protocol.version on each broker.", + assertThrows(classOf[KafkaException], () => KafkaRaftServer.initializeLogDirs(config)).getMessage) + } } diff --git a/core/src/test/scala/unit/kafka/server/MetadataCacheTest.scala b/core/src/test/scala/unit/kafka/server/MetadataCacheTest.scala index 524aded1f44b..d92c76f71189 100644 --- a/core/src/test/scala/unit/kafka/server/MetadataCacheTest.scala +++ b/core/src/test/scala/unit/kafka/server/MetadataCacheTest.scala @@ -19,25 +19,27 @@ package kafka.server import org.apache.kafka.common.{Node, TopicPartition, Uuid} import java.util -import util.Arrays.asList +import java.util.Arrays.asList +import java.util.Collections + +import kafka.api.LeaderAndIsr +import kafka.server.metadata.{KRaftMetadataCache, ZkMetadataCache} import org.apache.kafka.common.message.UpdateMetadataRequestData.{UpdateMetadataBroker, UpdateMetadataEndpoint, UpdateMetadataPartitionState, UpdateMetadataTopicState} import org.apache.kafka.common.network.ListenerName import org.apache.kafka.common.protocol.{ApiKeys, ApiMessage, Errors} import org.apache.kafka.common.record.RecordBatch import org.apache.kafka.common.requests.UpdateMetadataRequest import org.apache.kafka.common.security.auth.SecurityProtocol +import org.apache.kafka.common.metadata.{BrokerRegistrationChangeRecord, PartitionRecord, RegisterBrokerRecord, RemoveTopicRecord, TopicRecord} +import org.apache.kafka.common.metadata.RegisterBrokerRecord.{BrokerEndpoint, BrokerEndpointCollection} +import org.apache.kafka.image.{ClusterImage, MetadataDelta, MetadataImage} +import org.apache.kafka.server.common.MetadataVersion import org.apache.kafka.raft.{OffsetAndEpoch => RaftOffsetAndEpoch} + import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.provider.MethodSource - -import java.util.Collections -import kafka.api.LeaderAndIsr -import kafka.server.metadata.{KRaftMetadataCache, ZkMetadataCache} -import org.apache.kafka.common.metadata.{PartitionRecord, RegisterBrokerRecord, RemoveTopicRecord, TopicRecord} -import org.apache.kafka.common.metadata.RegisterBrokerRecord.{BrokerEndpoint, BrokerEndpointCollection} -import org.apache.kafka.image.{ClusterImage, MetadataDelta, MetadataImage} -import org.apache.kafka.server.common.MetadataVersion +import org.junit.jupiter.api.Test import scala.collection.{Seq, mutable} import scala.jdk.CollectionConverters._ @@ -639,4 +641,48 @@ class MetadataCacheTest { assertEquals(Seq(expectedNode0, expectedNode1), partitionInfo.inSyncReplicas.toSeq) assertEquals(Seq(expectedNode1), partitionInfo.offlineReplicas.toSeq) } + + @Test + def testIsBrokerFenced(): Unit = { + val metadataCache = MetadataCache.kRaftMetadataCache(0) + + val delta = new MetadataDelta(MetadataImage.EMPTY) + delta.replay(new RegisterBrokerRecord() + .setBrokerId(0) + .setFenced(false)) + + metadataCache.setImage(delta.apply()) + + assertFalse(metadataCache.isBrokerFenced(0)) + + delta.replay(new BrokerRegistrationChangeRecord() + .setBrokerId(0) + .setFenced(1.toByte)) + + metadataCache.setImage(delta.apply()) + + assertTrue(metadataCache.isBrokerFenced(0)) + } + + @Test + def testIsBrokerInControlledShutdown(): Unit = { + val metadataCache = MetadataCache.kRaftMetadataCache(0) + + val delta = new MetadataDelta(MetadataImage.EMPTY) + delta.replay(new RegisterBrokerRecord() + .setBrokerId(0) + .setInControlledShutdown(false)) + + metadataCache.setImage(delta.apply()) + + assertFalse(metadataCache.isBrokerShuttingDown(0)) + + delta.replay(new BrokerRegistrationChangeRecord() + .setBrokerId(0) + .setInControlledShutdown(1.toByte)) + + metadataCache.setImage(delta.apply()) + + assertTrue(metadataCache.isBrokerShuttingDown(0)) + } } diff --git a/core/src/test/scala/unit/kafka/server/ReplicaFetcherThreadTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaFetcherThreadTest.scala index e18dd29d6945..c7a222c2d547 100644 --- a/core/src/test/scala/unit/kafka/server/ReplicaFetcherThreadTest.scala +++ b/core/src/test/scala/unit/kafka/server/ReplicaFetcherThreadTest.scala @@ -103,14 +103,16 @@ class ReplicaFetcherThreadTest { leaderEndpointBlockingSend: BlockingSend): ReplicaFetcherThread = { val logContext = new LogContext(s"[ReplicaFetcher replicaId=${brokerConfig.brokerId}, leaderId=${leaderEndpointBlockingSend.brokerEndPoint().id}, fetcherId=$fetcherId] ") val fetchSessionHandler = new FetchSessionHandler(logContext, leaderEndpointBlockingSend.brokerEndPoint().id) - val leader = new RemoteLeaderEndPoint(logContext.logPrefix, leaderEndpointBlockingSend, fetchSessionHandler, brokerConfig, replicaMgr, quota) + val leader = new RemoteLeaderEndPoint(logContext.logPrefix, leaderEndpointBlockingSend, fetchSessionHandler, + brokerConfig, replicaMgr, quota, () => brokerConfig.interBrokerProtocolVersion) new ReplicaFetcherThread(name, leader, brokerConfig, failedPartitions, replicaMgr, quota, - logContext.logPrefix) + logContext.logPrefix, + () => brokerConfig.interBrokerProtocolVersion) } @Test @@ -121,9 +123,9 @@ class ReplicaFetcherThreadTest { val replicaManager: ReplicaManager = mock(classOf[ReplicaManager]) when(replicaManager.brokerTopicStats).thenReturn(mock(classOf[BrokerTopicStats])) - assertEquals(ApiKeys.FETCH.latestVersion, config.fetchRequestVersion) - assertEquals(ApiKeys.OFFSET_FOR_LEADER_EPOCH.latestVersion, config.offsetForLeaderEpochRequestVersion) - assertEquals(ApiKeys.LIST_OFFSETS.latestVersion, config.listOffsetRequestVersion) + assertEquals(ApiKeys.FETCH.latestVersion, config.interBrokerProtocolVersion.fetchRequestVersion()) + assertEquals(ApiKeys.OFFSET_FOR_LEADER_EPOCH.latestVersion, config.interBrokerProtocolVersion.offsetForLeaderEpochRequestVersion) + assertEquals(ApiKeys.LIST_OFFSETS.latestVersion, config.interBrokerProtocolVersion.listOffsetRequestVersion) } @Test @@ -581,8 +583,10 @@ class ReplicaFetcherThreadTest { val mockNetwork = new MockBlockingSender(Collections.emptyMap(), brokerEndPoint, new SystemTime()) val logContext = new LogContext(s"[ReplicaFetcher replicaId=${config.brokerId}, leaderId=${brokerEndPoint.id}, fetcherId=0] ") val fetchSessionHandler = new FetchSessionHandler(logContext, brokerEndPoint.id) - val leader = new RemoteLeaderEndPoint(logContext.logPrefix, mockNetwork, fetchSessionHandler, config, replicaManager, quota) - val thread = new ReplicaFetcherThread("bob", leader, config, failedPartitions, replicaManager, quota, logContext.logPrefix) { + val leader = new RemoteLeaderEndPoint(logContext.logPrefix, mockNetwork, fetchSessionHandler, config, + replicaManager, quota, () => config.interBrokerProtocolVersion) + val thread = new ReplicaFetcherThread("bob", leader, config, failedPartitions, + replicaManager, quota, logContext.logPrefix, () => config.interBrokerProtocolVersion) { override def processPartitionData(topicPartition: TopicPartition, fetchOffset: Long, partitionData: FetchData): Option[LogAppendInfo] = None } thread.addPartitions(Map(t1p0 -> initialFetchState(Some(topicId1), initialLEO), t1p1 -> initialFetchState(Some(topicId1), initialLEO))) @@ -1036,14 +1040,16 @@ class ReplicaFetcherThreadTest { val logContext = new LogContext(s"[ReplicaFetcher replicaId=${config.brokerId}, leaderId=${brokerEndPoint.id}, fetcherId=0] ") val fetchSessionHandler = new FetchSessionHandler(logContext, brokerEndPoint.id) - val leader = new RemoteLeaderEndPoint(logContext.logPrefix, mockBlockingSend, fetchSessionHandler, config, replicaManager, replicaQuota) + val leader = new RemoteLeaderEndPoint(logContext.logPrefix, mockBlockingSend, fetchSessionHandler, config, + replicaManager, replicaQuota, () => config.interBrokerProtocolVersion) val thread = new ReplicaFetcherThread("bob", leader, config, failedPartitions, replicaManager, replicaQuota, - logContext.logPrefix) + logContext.logPrefix, + () => config.interBrokerProtocolVersion) val leaderEpoch = 1 diff --git a/core/src/test/scala/unit/kafka/server/ReplicaManagerConcurrencyTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaManagerConcurrencyTest.scala index df95f701c571..651451afad43 100644 --- a/core/src/test/scala/unit/kafka/server/ReplicaManagerConcurrencyTest.scala +++ b/core/src/test/scala/unit/kafka/server/ReplicaManagerConcurrencyTest.scala @@ -20,12 +20,13 @@ import java.net.InetAddress import java.util import java.util.concurrent.{CompletableFuture, Executors, LinkedBlockingQueue, TimeUnit} import java.util.{Optional, Properties} - import kafka.api.LeaderAndIsr import kafka.log.{AppendOrigin, LogConfig} +import kafka.server.metadata.KRaftMetadataCache import kafka.server.metadata.MockConfigRepository import kafka.utils.TestUtils.waitUntilTrue import kafka.utils.{MockTime, ShutdownableThread, TestUtils} +import org.apache.kafka.common.metadata.RegisterBrokerRecord import org.apache.kafka.common.metadata.{PartitionChangeRecord, PartitionRecord, TopicRecord} import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.protocol.{ApiKeys, Errors} @@ -70,8 +71,9 @@ class ReplicaManagerConcurrencyTest { def testIsrExpandAndShrinkWithConcurrentProduce(): Unit = { val localId = 0 val remoteId = 1 + val metadataCache = MetadataCache.kRaftMetadataCache(localId) val channel = new ControllerChannel - val replicaManager = buildReplicaManager(localId, channel) + val replicaManager = buildReplicaManager(localId, channel, metadataCache) // Start with the remote replica out of the ISR val initialPartitionRegistration = registration( @@ -84,7 +86,7 @@ class ReplicaManagerConcurrencyTest { val topicModel = new TopicModel(Uuid.randomUuid(), "foo", Map(0 -> initialPartitionRegistration)) val topicPartition = new TopicPartition(topicModel.name, 0) val topicIdPartition = new TopicIdPartition(topicModel.topicId, topicPartition) - val controller = new ControllerModel(topicModel, channel, replicaManager) + val controller = new ControllerModel(Seq(localId, remoteId), topicModel, channel, replicaManager, metadataCache) submit(new Clock(time)) replicaManager.startup() @@ -140,7 +142,8 @@ class ReplicaManagerConcurrencyTest { private def buildReplicaManager( localId: Int, - channel: ControllerChannel + channel: ControllerChannel, + metadataCache: MetadataCache, ): ReplicaManager = { val logDir = TestUtils.tempDir() @@ -168,7 +171,7 @@ class ReplicaManagerConcurrencyTest { scheduler = time.scheduler, logManager = logManager, quotaManagers = QuotaFactory.instantiate(config, metrics, time, ""), - metadataCache = MetadataCache.kRaftMetadataCache(config.brokerId), + metadataCache = metadataCache, logDirFailureChannel = new LogDirFailureChannel(config.logDirs.size), alterPartitionManager = new MockAlterPartitionManager(channel) ) { @@ -295,7 +298,7 @@ class ReplicaManagerConcurrencyTest { case object ShutdownEvent extends ControllerEvent case class AlterIsrEvent( future: CompletableFuture[LeaderAndIsr], - topicPartition: TopicPartition, + topicPartition: TopicIdPartition, leaderAndIsr: LeaderAndIsr ) extends ControllerEvent @@ -307,7 +310,7 @@ class ReplicaManagerConcurrencyTest { } def alterIsr( - topicPartition: TopicPartition, + topicPartition: TopicIdPartition, leaderAndIsr: LeaderAndIsr ): CompletableFuture[LeaderAndIsr] = { val future = new CompletableFuture[LeaderAndIsr]() @@ -325,9 +328,11 @@ class ReplicaManagerConcurrencyTest { } private class ControllerModel( + brokerIds: Seq[Int], topic: TopicModel, channel: ControllerChannel, - replicaManager: ReplicaManager + replicaManager: ReplicaManager, + metadataCache: KRaftMetadataCache ) extends ShutdownableThread(name = "controller", isInterruptible = false) { private var latestImage = MetadataImage.EMPTY @@ -345,8 +350,15 @@ class ReplicaManagerConcurrencyTest { channel.poll() match { case InitializeEvent => val delta = new MetadataDelta(latestImage) + brokerIds.foreach { brokerId => + delta.replay(new RegisterBrokerRecord() + .setBrokerId(brokerId) + .setFenced(false) + ) + } topic.initialize(delta) latestImage = delta.apply() + metadataCache.setImage(latestImage) replicaManager.applyDelta(delta.topicsDelta, latestImage) case AlterIsrEvent(future, topicPartition, leaderAndIsr) => @@ -380,7 +392,7 @@ class ReplicaManagerConcurrencyTest { } def alterIsr( - topicPartition: TopicPartition, + topicPartition: TopicIdPartition, leaderAndIsr: LeaderAndIsr, delta: MetadataDelta ): LeaderAndIsr = { @@ -433,7 +445,7 @@ class ReplicaManagerConcurrencyTest { private class MockAlterPartitionManager(channel: ControllerChannel) extends AlterPartitionManager { override def submit( - topicPartition: TopicPartition, + topicPartition: TopicIdPartition, leaderAndIsr: LeaderAndIsr, controllerEpoch: Int ): CompletableFuture[LeaderAndIsr] = { diff --git a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala index c2c8b4ddc791..8050a36d1d94 100644 --- a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala @@ -32,7 +32,7 @@ import kafka.server.QuotaFactory.{QuotaManagers, UnboundedQuota} import kafka.server.checkpoints.{LazyOffsetCheckpoints, OffsetCheckpointFile} import kafka.server.epoch.util.MockBlockingSender import kafka.utils.timer.MockTimer -import kafka.utils.{MockScheduler, MockTime, TestUtils} +import kafka.utils.{MockScheduler, MockTime, Pool, TestUtils} import org.apache.kafka.clients.FetchSessionHandler import org.apache.kafka.common.errors.KafkaStorageException import org.apache.kafka.common.message.FetchResponseData @@ -65,7 +65,7 @@ import org.junit.jupiter.params.provider.ValueSource import org.mockito.invocation.InvocationOnMock import org.mockito.stubbing.Answer import org.mockito.ArgumentMatchers -import org.mockito.ArgumentMatchers.{any, anyInt} +import org.mockito.ArgumentMatchers.{any, anyInt, anyString} import org.mockito.Mockito.{mock, never, reset, times, verify, when} import scala.collection.{Map, Seq, mutable} @@ -214,6 +214,7 @@ class ReplicaManagerTest { val aliveBrokers = Seq(new Node(0, "host0", 0), new Node(1, "host1", 1)) val metadataCache: MetadataCache = mock(classOf[MetadataCache]) mockGetAliveBrokerFunctions(metadataCache, aliveBrokers) + when(metadataCache.metadataVersion()).thenReturn(config.interBrokerProtocolVersion) val rm = new ReplicaManager( metrics = metrics, config = config, @@ -1187,8 +1188,6 @@ class ReplicaManagerTest { val tp0 = new TopicPartition(topic, 0) val tidp0 = new TopicIdPartition(topicId, tp0) - initializeLogAndTopicId(replicaManager, tp0, topicId) - // Make this replica the follower val leaderAndIsrRequest2 = new LeaderAndIsrRequest.Builder(ApiKeys.LEADER_AND_ISR.latestVersion, 0, 0, brokerEpoch, Seq(new LeaderAndIsrPartitionState() @@ -1245,8 +1244,6 @@ class ReplicaManagerTest { val tp0 = new TopicPartition(topic, 0) val tidp0 = new TopicIdPartition(topicId, tp0) - initializeLogAndTopicId(replicaManager, tp0, topicId) - // Make this replica the leader val leaderAndIsrRequest2 = new LeaderAndIsrRequest.Builder(ApiKeys.LEADER_AND_ISR.latestVersion, 0, 0, brokerEpoch, Seq(new LeaderAndIsrPartitionState() @@ -1293,7 +1290,6 @@ class ReplicaManagerTest { val topicId = Uuid.randomUuid() val tp0 = new TopicPartition(topic, 0) val tidp0 = new TopicIdPartition(topicId, tp0) - initializeLogAndTopicId(replicaManager, tp0, topicId) // Make this replica the follower val leaderAndIsrRequest = new LeaderAndIsrRequest.Builder(ApiKeys.LEADER_AND_ISR.latestVersion, 0, 0, brokerEpoch, @@ -1343,8 +1339,6 @@ class ReplicaManagerTest { val tp0 = new TopicPartition(topic, 0) val tidp0 = new TopicIdPartition(topicId, tp0) - initializeLogAndTopicId(replicaManager, tp0, topicId) - when(replicaManager.metadataCache.getPartitionReplicaEndpoints( tp0, new ListenerName("default") @@ -1418,8 +1412,6 @@ class ReplicaManagerTest { val tp0 = new TopicPartition(topic, 0) val tidp0 = new TopicIdPartition(topicId, tp0) - initializeLogAndTopicId(replicaManager, tp0, topicId) - // Make this replica the follower val leaderAndIsrRequest2 = new LeaderAndIsrRequest.Builder(ApiKeys.LEADER_AND_ISR.latestVersion, 0, 0, brokerEpoch, Seq(new LeaderAndIsrPartitionState() @@ -1489,15 +1481,6 @@ class ReplicaManagerTest { leaderBrokerId, countDownLatch, expectTruncation = true, extraProps = props)) } - // Due to some limitations to EasyMock, we need to create the log so that the Partition.topicId does not call - // LogManager.getLog with a default argument - // TODO: convert tests to using Mockito to avoid this issue. - private def initializeLogAndTopicId(replicaManager: ReplicaManager, topicPartition: TopicPartition, topicId: Uuid): Unit = { - val partition = replicaManager.createPartition(new TopicPartition(topic, 0)) - val log = replicaManager.logManager.getOrCreateLog(topicPartition, false, false, Some(topicId)) - partition.log = Some(log) - } - @Test def testDefaultReplicaSelector(): Unit = { val topicPartition = 0 @@ -1953,7 +1936,12 @@ class ReplicaManagerTest { val mockLogMgr: LogManager = mock(classOf[LogManager]) when(mockLogMgr.liveLogDirs).thenReturn(config.logDirs.map(new File(_).getAbsoluteFile)) when(mockLogMgr.getOrCreateLog(ArgumentMatchers.eq(topicPartitionObj), ArgumentMatchers.eq(false), ArgumentMatchers.eq(false), any())).thenReturn(mockLog) + when(mockLogMgr.getLog(topicPartitionObj, isFuture = false)).thenReturn(Some(mockLog)) when(mockLogMgr.getLog(topicPartitionObj, isFuture = true)).thenReturn(None) + val allLogs = new Pool[TopicPartition, UnifiedLog]() + allLogs.put(topicPartitionObj, mockLog) + when(mockLogMgr.allLogs).thenReturn(allLogs.values) + when(mockLogMgr.isLogDirOnline(anyString)).thenReturn(true) val aliveBrokerIds = Seq[Integer](followerBrokerId, leaderBrokerId) val aliveBrokers = aliveBrokerIds.map(brokerId => new Node(brokerId, s"host$brokerId", brokerId)) @@ -1964,7 +1952,7 @@ class ReplicaManagerTest { any[TopicPartition], any[ListenerName])). thenReturn(Map(leaderBrokerId -> new Node(leaderBrokerId, "host1", 9092, "rack-a"), followerBrokerId -> new Node(followerBrokerId, "host2", 9092, "rack-b")).toMap) - + when(metadataCache.metadataVersion()).thenReturn(config.interBrokerProtocolVersion) val mockProducePurgatory = new DelayedOperationPurgatory[DelayedProduce]( purgatoryName = "Produce", timer, reaperEnabled = false) val mockFetchPurgatory = new DelayedOperationPurgatory[DelayedFetch]( @@ -2003,15 +1991,16 @@ class ReplicaManagerTest { time: Time, threadNamePrefix: Option[String], replicationQuotaManager: ReplicationQuotaManager): ReplicaFetcherManager = { - new ReplicaFetcherManager(config, this, metrics, time, threadNamePrefix, replicationQuotaManager) { + new ReplicaFetcherManager(config, this, metrics, time, threadNamePrefix, replicationQuotaManager, () => metadataCache.metadataVersion()) { override def createFetcherThread(fetcherId: Int, sourceBroker: BrokerEndPoint): ReplicaFetcherThread = { val logContext = new LogContext(s"[ReplicaFetcher replicaId=${config.brokerId}, leaderId=${sourceBroker.id}, " + s"fetcherId=$fetcherId] ") val fetchSessionHandler = new FetchSessionHandler(logContext, sourceBroker.id) - val leader = new RemoteLeaderEndPoint(logContext.logPrefix, blockingSend, fetchSessionHandler, config, replicaManager, quotaManager.follower) + val leader = new RemoteLeaderEndPoint(logContext.logPrefix, blockingSend, fetchSessionHandler, config, + replicaManager, quotaManager.follower, () => config.interBrokerProtocolVersion) new ReplicaFetcherThread(s"ReplicaFetcherThread-$fetcherId", leader, config, failedPartitions, replicaManager, - quotaManager.follower, logContext.logPrefix) { + quotaManager.follower, logContext.logPrefix, () => config.interBrokerProtocolVersion) { override def doWork(): Unit = { // In case the thread starts before the partition is added by AbstractFetcherManager, // add it here (it's a no-op if already added) @@ -2235,6 +2224,7 @@ class ReplicaManagerTest { when(metadataCache.topicIdInfo()).thenReturn((topicIds.asJava, topicNames.asJava)) when(metadataCache.topicNamesToIds()).thenReturn(topicIds.asJava) when(metadataCache.topicIdsToNames()).thenReturn(topicNames.asJava) + when(metadataCache.metadataVersion()).thenReturn(config.interBrokerProtocolVersion) mockGetAliveBrokerFunctions(metadataCache, aliveBrokers) val mockProducePurgatory = new DelayedOperationPurgatory[DelayedProduce]( purgatoryName = "Produce", timer, reaperEnabled = false) @@ -2487,6 +2477,8 @@ class ReplicaManagerTest { val aliveBrokers = Seq(new Node(0, "host0", 0), new Node(1, "host1", 1)) mockGetAliveBrokerFunctions(metadataCache0, aliveBrokers) mockGetAliveBrokerFunctions(metadataCache1, aliveBrokers) + when(metadataCache0.metadataVersion()).thenReturn(config0.interBrokerProtocolVersion) + when(metadataCache1.metadataVersion()).thenReturn(config1.interBrokerProtocolVersion) // each replica manager is for a broker val rm0 = new ReplicaManager( @@ -4144,6 +4136,31 @@ class ReplicaManagerTest { replicaManager.shutdown(checkpointHW = false) } } + + @Test + def testDescribeLogDirs(): Unit = { + val topicPartition = 0 + val topicId = Uuid.randomUuid() + val followerBrokerId = 0 + val leaderBrokerId = 1 + val leaderEpoch = 1 + val leaderEpochIncrement = 2 + val countDownLatch = new CountDownLatch(1) + val offsetFromLeader = 5 + + // Prepare the mocked components for the test + val (replicaManager, mockLogMgr) = prepareReplicaManagerAndLogManager(new MockTimer(time), + topicPartition, leaderEpoch + leaderEpochIncrement, followerBrokerId, leaderBrokerId, countDownLatch, + expectTruncation = false, localLogOffset = Some(10), offsetFromLeader = offsetFromLeader, topicId = Some(topicId)) + + val responses = replicaManager.describeLogDirs(Set(new TopicPartition(topic, topicPartition))) + assertEquals(mockLogMgr.liveLogDirs.size, responses.size) + responses.foreach { response => + assertEquals(Errors.NONE.code, response.errorCode) + assertTrue(response.totalBytes > 0) + assertTrue(response.usableBytes >= 0) + } + } } class MockReplicaSelector extends ReplicaSelector { diff --git a/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala b/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala index a5a820d651ed..6d17e937824d 100644 --- a/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala +++ b/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala @@ -599,7 +599,7 @@ class RequestQuotaTest extends BaseRequestTest { tp, 10, 5, Collections.singletonList(3))) case ApiKeys.ALTER_PARTITION => - new AlterPartitionRequest.Builder(new AlterPartitionRequestData()) + new AlterPartitionRequest.Builder(new AlterPartitionRequestData(), true) case ApiKeys.UPDATE_FEATURES => new UpdateFeaturesRequest.Builder(new UpdateFeaturesRequestData()) diff --git a/core/src/test/scala/unit/kafka/server/metadata/BrokerMetadataListenerTest.scala b/core/src/test/scala/unit/kafka/server/metadata/BrokerMetadataListenerTest.scala index 948e05133701..f75823a02928 100644 --- a/core/src/test/scala/unit/kafka/server/metadata/BrokerMetadataListenerTest.scala +++ b/core/src/test/scala/unit/kafka/server/metadata/BrokerMetadataListenerTest.scala @@ -74,7 +74,7 @@ class BrokerMetadataListenerTest { ) ) val imageRecords = listener.getImageRecords().get() - assertEquals(0, imageRecords.size()) + assertEquals(1, imageRecords.size()) assertEquals(100L, listener.highestMetadataOffset) assertEquals(0L, metrics.lastAppliedRecordOffset.get) assertEquals(0L, metrics.lastAppliedRecordTimestamp.get) diff --git a/core/src/test/scala/unit/kafka/utils/TestUtils.scala b/core/src/test/scala/unit/kafka/utils/TestUtils.scala index c6fa9d7db16f..bb307e7f4ccd 100755 --- a/core/src/test/scala/unit/kafka/utils/TestUtils.scala +++ b/core/src/test/scala/unit/kafka/utils/TestUtils.scala @@ -28,8 +28,8 @@ import java.util import java.util.concurrent.atomic.{AtomicBoolean, AtomicInteger} import java.util.concurrent.{Callable, CompletableFuture, ExecutionException, Executors, TimeUnit} import java.util.{Arrays, Collections, Optional, Properties} - import com.yammer.metrics.core.{Gauge, Meter} + import javax.net.ssl.X509TrustManager import kafka.api._ import kafka.cluster.{AlterPartitionListener, Broker, EndPoint} @@ -47,6 +47,7 @@ import org.apache.kafka.clients.admin._ import org.apache.kafka.clients.consumer._ import org.apache.kafka.clients.consumer.internals.AbstractCoordinator import org.apache.kafka.clients.producer.{KafkaProducer, ProducerConfig, ProducerRecord} +import org.apache.kafka.common.TopicIdPartition import org.apache.kafka.common.acl.{AccessControlEntry, AccessControlEntryFilter, AclBinding, AclBindingFilter} import org.apache.kafka.common.config.{ConfigException, ConfigResource} import org.apache.kafka.common.config.ConfigResource.Type.TOPIC @@ -1318,13 +1319,18 @@ object TestUtils extends Logging { override def submit( - topicPartition: TopicPartition, + topicPartition: TopicIdPartition, leaderAndIsr: LeaderAndIsr, controllerEpoch: Int ): CompletableFuture[LeaderAndIsr]= { val future = new CompletableFuture[LeaderAndIsr]() if (inFlight.compareAndSet(false, true)) { - isrUpdates += AlterPartitionItem(topicPartition, leaderAndIsr, future, controllerEpoch) + isrUpdates += AlterPartitionItem( + topicPartition, + leaderAndIsr, + future, + controllerEpoch + ) } else { future.completeExceptionally(new OperationNotAttemptedException( s"Failed to enqueue AlterIsr request for $topicPartition since there is already an inflight request")) diff --git a/docs/ops.html b/docs/ops.html index 5ace510c4ce9..76fee9cc868f 100644 --- a/docs/ops.html +++ b/docs/ops.html @@ -1401,18 +1401,18 @@

+ @@ -311,16 +311,6 @@ For a detailed description of spotbugs bug categories, see https://spotbugs.read - - - - - - - diff --git a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/fetcher/ReplicaFetcherThreadBenchmark.java b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/fetcher/ReplicaFetcherThreadBenchmark.java index 86400ba1e906..b0f36522f324 100644 --- a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/fetcher/ReplicaFetcherThreadBenchmark.java +++ b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/fetcher/ReplicaFetcherThreadBenchmark.java @@ -325,7 +325,8 @@ static class ReplicaFetcherBenchThread extends ReplicaFetcherThread { new LogContext(String.format("[ReplicaFetcher replicaId=%d, leaderId=%d, fetcherId=%d", config.brokerId(), 3, 3)), 3), config, replicaManager, - replicaQuota + replicaQuota, + config::interBrokerProtocolVersion ) { @Override public long fetchEarliestOffset(TopicPartition topicPartition, int currentLeaderEpoch) { @@ -356,7 +357,8 @@ public Map fetch(FetchRequest.B new FailedPartitions(), replicaManager, replicaQuota, - String.format("[ReplicaFetcher replicaId=%d, leaderId=%d, fetcherId=%d", config.brokerId(), 3, 3) + String.format("[ReplicaFetcher replicaId=%d, leaderId=%d, fetcherId=%d", config.brokerId(), 3, 3), + config::interBrokerProtocolVersion ); pool = partitions; diff --git a/metadata/src/main/java/org/apache/kafka/controller/BootstrapMetadata.java b/metadata/src/main/java/org/apache/kafka/controller/BootstrapMetadata.java index fa031c525f1e..d9d0651a193b 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/BootstrapMetadata.java +++ b/metadata/src/main/java/org/apache/kafka/controller/BootstrapMetadata.java @@ -39,6 +39,7 @@ import java.util.Objects; import java.util.Optional; import java.util.concurrent.ExecutionException; +import java.util.function.Supplier; import java.util.stream.Stream; @@ -128,7 +129,9 @@ public static BootstrapMetadata create(MetadataVersion metadataVersion) { public static BootstrapMetadata create(MetadataVersion metadataVersion, List records) { if (!metadataVersion.isKRaftSupported()) { - throw new IllegalArgumentException("Cannot create BootstrapMetadata with a non-KRaft metadata version."); + throw new IllegalArgumentException(String.format( + "Cannot create BootstrapMetadata with a non-KRaft metadata version %s. Minimum version is %s", + metadataVersion, MetadataVersion.MINIMUM_KRAFT_VERSION)); } records.add(new ApiMessageAndVersion( new FeatureLevelRecord() @@ -142,18 +145,24 @@ public static BootstrapMetadata create(MetadataVersion metadataVersion, List fallbackVersionSupplier) throws Exception { final Path bootstrapPath = bootstrapDir.resolve(BOOTSTRAP_FILE); if (!Files.exists(bootstrapPath)) { - log.debug("Missing bootstrap file, this appears to be a KRaft preview cluster. Setting metadata.version to {}.", - fallbackPreviewVersion.featureLevel()); - return BootstrapMetadata.create(fallbackPreviewVersion); + // Upgrade scenario from KRaft prior to 3.3 (i.e., no bootstrap metadata present) + MetadataVersion fallbackVersion = fallbackVersionSupplier.get(); + if (fallbackVersion.isKRaftSupported()) { + log.debug("Missing bootstrap file, this appears to be a KRaft cluster older than 3.3. Setting metadata.version to {}.", + fallbackVersion.featureLevel()); + return BootstrapMetadata.create(fallbackVersion); + } else { + throw new Exception(String.format("Could not set fallback bootstrap metadata with non-KRaft metadata version of %s", fallbackVersion)); + } } BootstrapListener listener = new BootstrapListener(); @@ -182,7 +191,7 @@ public static BootstrapMetadata load(Path bootstrapDir, MetadataVersion fallback if (metadataVersionRecord.isPresent()) { return new BootstrapMetadata(MetadataVersion.fromFeatureLevel(metadataVersionRecord.get().featureLevel()), listener.records); } else { - throw new RuntimeException("Expected a metadata.version to exist in the snapshot " + bootstrapPath + ", but none was found"); + throw new Exception("Expected a metadata.version to exist in the snapshot " + bootstrapPath + ", but none was found"); } } diff --git a/metadata/src/main/java/org/apache/kafka/controller/ClusterControlManager.java b/metadata/src/main/java/org/apache/kafka/controller/ClusterControlManager.java index 21bd01c3b92f..859dedda2200 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/ClusterControlManager.java +++ b/metadata/src/main/java/org/apache/kafka/controller/ClusterControlManager.java @@ -590,11 +590,7 @@ class ClusterControlIterator implements Iterator> { ClusterControlIterator(long epoch) { this.iterator = brokerRegistrations.entrySet(epoch).iterator(); - if (featureControl.metadataVersion().equals(MetadataVersion.UNINITIALIZED)) { - this.metadataVersion = MetadataVersion.IBP_3_0_IV1; - } else { - this.metadataVersion = featureControl.metadataVersion(); - } + this.metadataVersion = featureControl.metadataVersion(); } @Override diff --git a/metadata/src/main/java/org/apache/kafka/controller/ControllerRequestContext.java b/metadata/src/main/java/org/apache/kafka/controller/ControllerRequestContext.java index e18d68c5875c..e4bc2f3eb4a8 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/ControllerRequestContext.java +++ b/metadata/src/main/java/org/apache/kafka/controller/ControllerRequestContext.java @@ -17,8 +17,11 @@ package org.apache.kafka.controller; + +import org.apache.kafka.common.message.RequestHeaderData; import org.apache.kafka.common.security.auth.KafkaPrincipal; import org.apache.kafka.common.utils.Time; +import org.apache.kafka.server.authorizer.AuthorizableRequestContext; import java.util.OptionalLong; @@ -27,9 +30,6 @@ public class ControllerRequestContext { - public static final ControllerRequestContext ANONYMOUS_CONTEXT = - new ControllerRequestContext(KafkaPrincipal.ANONYMOUS, - OptionalLong.empty()); public static OptionalLong requestTimeoutMsToDeadlineNs( Time time, @@ -39,17 +39,38 @@ public static OptionalLong requestTimeoutMsToDeadlineNs( } private final KafkaPrincipal principal; - private final OptionalLong deadlineNs; + private final RequestHeaderData requestHeader; public ControllerRequestContext( + RequestHeaderData requestHeader, KafkaPrincipal principal, OptionalLong deadlineNs ) { + this.requestHeader = requestHeader; this.principal = principal; this.deadlineNs = deadlineNs; } + public ControllerRequestContext( + AuthorizableRequestContext requestContext, + OptionalLong deadlineNs + ) { + this( + new RequestHeaderData() + .setRequestApiKey((short) requestContext.requestType()) + .setRequestApiVersion((short) requestContext.requestVersion()) + .setCorrelationId(requestContext.correlationId()) + .setClientId(requestContext.clientId()), + requestContext.principal(), + deadlineNs + ); + } + + public RequestHeaderData requestHeader() { + return requestHeader; + } + public KafkaPrincipal principal() { return principal; } diff --git a/metadata/src/main/java/org/apache/kafka/controller/FeatureControlManager.java b/metadata/src/main/java/org/apache/kafka/controller/FeatureControlManager.java index 19127afa72ff..c092abcdccaa 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/FeatureControlManager.java +++ b/metadata/src/main/java/org/apache/kafka/controller/FeatureControlManager.java @@ -52,7 +52,7 @@ public static class Builder { private LogContext logContext = null; private SnapshotRegistry snapshotRegistry = null; private QuorumFeatures quorumFeatures = null; - private MetadataVersion metadataVersion = MetadataVersion.UNINITIALIZED; + private MetadataVersion metadataVersion = MetadataVersion.MINIMUM_KRAFT_VERSION; Builder setLogContext(LogContext logContext) { this.logContext = logContext; @@ -105,6 +105,10 @@ public FeatureControlManager build() { */ private final TimelineObject metadataVersion; + /** + * A boolean to see if we have encountered a metadata.version or not. + */ + private final TimelineObject sawMetadataVersion; private FeatureControlManager( LogContext logContext, @@ -116,6 +120,7 @@ private FeatureControlManager( this.quorumFeatures = quorumFeatures; this.finalizedVersions = new TimelineHashMap<>(snapshotRegistry, 0); this.metadataVersion = new TimelineObject<>(snapshotRegistry, metadataVersion); + this.sawMetadataVersion = new TimelineObject<>(snapshotRegistry, false); } ControllerResult> updateFeatures( @@ -226,7 +231,7 @@ private ApiError updateMetadataVersion( return invalidMetadataVersion(newVersionLevel, "Unknown metadata.version."); } - if (!currentVersion.equals(MetadataVersion.UNINITIALIZED) && newVersion.isLessThan(currentVersion)) { + if (newVersion.isLessThan(currentVersion)) { // This is a downgrade boolean metadataChanged = MetadataVersion.checkIfMetadataChanged(currentVersion, newVersion); if (!metadataChanged) { @@ -257,15 +262,20 @@ private ApiError invalidMetadataVersion(short version, String message) { FinalizedControllerFeatures finalizedFeatures(long epoch) { Map features = new HashMap<>(); - if (!metadataVersion.get(epoch).equals(MetadataVersion.UNINITIALIZED)) { - features.put(MetadataVersion.FEATURE_NAME, metadataVersion.get(epoch).featureLevel()); - } + features.put(MetadataVersion.FEATURE_NAME, metadataVersion.get(epoch).featureLevel()); for (Entry entry : finalizedVersions.entrySet(epoch)) { features.put(entry.getKey(), entry.getValue()); } return new FinalizedControllerFeatures(features, epoch); } + /** + * @return true if a FeatureLevelRecord for "metadata.version" has been replayed. False otherwise + */ + boolean sawMetadataVersion() { + return this.sawMetadataVersion.get(); + } + public void replay(FeatureLevelRecord record) { VersionRange range = quorumFeatures.localSupportedFeature(record.name()); if (!range.contains(record.featureLevel())) { @@ -275,6 +285,7 @@ public void replay(FeatureLevelRecord record) { if (record.name().equals(MetadataVersion.FEATURE_NAME)) { log.info("Setting metadata.version to {}", record.featureLevel()); metadataVersion.set(MetadataVersion.fromFeatureLevel(record.featureLevel())); + sawMetadataVersion.set(true); } else { if (record.featureLevel() == 0) { log.info("Removing feature {}", record.name()); @@ -294,9 +305,6 @@ class FeatureControlIterator implements Iterator> { FeatureControlIterator(long epoch) { this.iterator = finalizedVersions.entrySet(epoch).iterator(); this.metadataVersion = FeatureControlManager.this.metadataVersion.get(epoch); - if (this.metadataVersion.equals(MetadataVersion.UNINITIALIZED)) { - this.wroteVersion = true; - } } @Override diff --git a/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java b/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java index 4a8c8c96618c..97c0cdd78294 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java +++ b/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java @@ -283,7 +283,7 @@ public Builder setStaticConfig(Map staticConfig) { public QuorumController build() throws Exception { if (raftClient == null) { throw new IllegalStateException("You must set a raft client."); - } else if (bootstrapMetadata == null || bootstrapMetadata.metadataVersion().equals(MetadataVersion.UNINITIALIZED)) { + } else if (bootstrapMetadata == null) { throw new IllegalStateException("You must specify an initial metadata.version using the kafka-storage tool."); } else if (quorumFeatures == null) { throw new IllegalStateException("You must specify the quorum features"); @@ -932,21 +932,21 @@ public void handleLeaderChange(LeaderAndEpoch newLeader) { // write any other records to the log since we need the metadata.version to determine the correct // record version final MetadataVersion metadataVersion; - if (featureControl.metadataVersion().equals(MetadataVersion.UNINITIALIZED)) { + if (!featureControl.sawMetadataVersion()) { final CompletableFuture> future; if (!bootstrapMetadata.metadataVersion().isKRaftSupported()) { - metadataVersion = MetadataVersion.UNINITIALIZED; + metadataVersion = MetadataVersion.MINIMUM_KRAFT_VERSION; future = new CompletableFuture<>(); future.completeExceptionally( - new IllegalStateException("Cannot become leader without an initial metadata.version of " + - "at least 1. Got " + bootstrapMetadata.metadataVersion().featureLevel())); + new IllegalStateException("Cannot become leader without a KRaft supported version. " + + "Got " + bootstrapMetadata.metadataVersion())); } else { metadataVersion = bootstrapMetadata.metadataVersion(); future = appendWriteEvent("bootstrapMetadata", OptionalLong.empty(), () -> { if (metadataVersion.isAtLeast(MetadataVersion.IBP_3_3_IV0)) { log.info("Initializing metadata.version to {}", metadataVersion.featureLevel()); } else { - log.info("Upgrading from KRaft preview. Initializing metadata.version to {}", + log.info("Upgrading KRaft cluster and initializing metadata.version to {}", metadataVersion.featureLevel()); } return ControllerResult.atomicOf(bootstrapMetadata.records(), null); @@ -1619,7 +1619,7 @@ public CompletableFuture alterPartition( return CompletableFuture.completedFuture(new AlterPartitionResponseData()); } return appendWriteEvent("alterPartition", context.deadlineNs(), - () -> replicationControl.alterPartition(request)); + () -> replicationControl.alterPartition(context, request)); } @Override @@ -1964,6 +1964,11 @@ public int curClaimEpoch() { return curClaimEpoch; } + // Visible for testing + MetadataVersion metadataVersion() { + return featureControl.metadataVersion(); + } + @Override public void close() throws InterruptedException { queue.close(); diff --git a/metadata/src/main/java/org/apache/kafka/controller/QuorumFeatures.java b/metadata/src/main/java/org/apache/kafka/controller/QuorumFeatures.java index 9b723515bd4a..36725c251857 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/QuorumFeatures.java +++ b/metadata/src/main/java/org/apache/kafka/controller/QuorumFeatures.java @@ -72,7 +72,9 @@ public static QuorumFeatures create( public static Map defaultFeatureMap() { Map features = new HashMap<>(1); - features.put(MetadataVersion.FEATURE_NAME, VersionRange.of(MetadataVersion.IBP_3_0_IV0.featureLevel(), MetadataVersion.latest().featureLevel())); + features.put(MetadataVersion.FEATURE_NAME, VersionRange.of( + MetadataVersion.MINIMUM_KRAFT_VERSION.featureLevel(), + MetadataVersion.latest().featureLevel())); return features; } diff --git a/metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java b/metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java index 1c81954617ad..8382cd9c16b7 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java +++ b/metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java @@ -120,10 +120,13 @@ import static org.apache.kafka.common.metadata.MetadataRecordType.REMOVE_TOPIC_RECORD; import static org.apache.kafka.common.metadata.MetadataRecordType.TOPIC_RECORD; import static org.apache.kafka.common.protocol.Errors.FENCED_LEADER_EPOCH; +import static org.apache.kafka.common.protocol.Errors.INELIGIBLE_REPLICA; import static org.apache.kafka.common.protocol.Errors.INVALID_REQUEST; import static org.apache.kafka.common.protocol.Errors.INVALID_UPDATE_VERSION; +import static org.apache.kafka.common.protocol.Errors.NEW_LEADER_ELECTED; import static org.apache.kafka.common.protocol.Errors.NONE; import static org.apache.kafka.common.protocol.Errors.NO_REASSIGNMENT_IN_PROGRESS; +import static org.apache.kafka.common.protocol.Errors.OPERATION_NOT_ATTEMPTED; import static org.apache.kafka.common.protocol.Errors.TOPIC_AUTHORIZATION_FAILED; import static org.apache.kafka.common.protocol.Errors.UNKNOWN_TOPIC_ID; import static org.apache.kafka.common.protocol.Errors.UNKNOWN_TOPIC_OR_PARTITION; @@ -915,22 +918,31 @@ Set imbalancedPartitions() { return new HashSet<>(imbalancedPartitions); } - ControllerResult alterPartition(AlterPartitionRequestData request) { + ControllerResult alterPartition( + ControllerRequestContext context, + AlterPartitionRequestData request + ) { + short requestVersion = context.requestHeader().requestApiVersion(); clusterControl.checkBrokerEpoch(request.brokerId(), request.brokerEpoch()); AlterPartitionResponseData response = new AlterPartitionResponseData(); List records = new ArrayList<>(); for (AlterPartitionRequestData.TopicData topicData : request.topics()) { AlterPartitionResponseData.TopicData responseTopicData = - new AlterPartitionResponseData.TopicData().setName(topicData.name()); + new AlterPartitionResponseData.TopicData(). + setTopicName(topicData.topicName()). + setTopicId(topicData.topicId()); response.topics().add(responseTopicData); - Uuid topicId = topicsByName.get(topicData.name()); - if (topicId == null || !topics.containsKey(topicId)) { + + Uuid topicId = requestVersion > 1 ? topicData.topicId() : topicsByName.get(topicData.topicName()); + if (topicId == null || topicId.equals(Uuid.ZERO_UUID) || !topics.containsKey(topicId)) { + Errors error = requestVersion > 1 ? UNKNOWN_TOPIC_ID : UNKNOWN_TOPIC_OR_PARTITION; for (AlterPartitionRequestData.PartitionData partitionData : topicData.partitions()) { responseTopicData.partitions().add(new AlterPartitionResponseData.PartitionData(). setPartitionIndex(partitionData.partitionIndex()). - setErrorCode(UNKNOWN_TOPIC_OR_PARTITION.code())); + setErrorCode(error.code())); } - log.info("Rejecting AlterPartition request for unknown topic ID {}.", topicId); + log.info("Rejecting AlterPartition request for unknown topic ID {} or name {}.", + topicData.topicId(), topicData.topicName()); continue; } @@ -939,7 +951,15 @@ ControllerResult alterPartition(AlterPartitionReques int partitionId = partitionData.partitionIndex(); PartitionRegistration partition = topic.parts.get(partitionId); - Errors validationError = validateAlterPartitionData(request.brokerId(), topic, partitionId, partition, partitionData); + Errors validationError = validateAlterPartitionData( + request.brokerId(), + topic, + partitionId, + partition, + clusterControl::active, + context.requestHeader().requestApiVersion(), + partitionData); + if (validationError != Errors.NONE) { responseTopicData.partitions().add( new AlterPartitionResponseData.PartitionData() @@ -956,7 +976,7 @@ ControllerResult alterPartition(AlterPartitionReques partitionId, clusterControl::active, featureControl.metadataVersion().isLeaderRecoverySupported()); - if (configurationControl.uncleanLeaderElectionEnabledForTopic(topicData.name())) { + if (configurationControl.uncleanLeaderElectionEnabledForTopic(topic.name())) { builder.setElection(PartitionChangeBuilder.Election.UNCLEAN); } builder.setTargetIsr(partitionData.newIsr()); @@ -979,18 +999,20 @@ ControllerResult alterPartition(AlterPartitionReques // ISR change completes it, then the leader may change as part of // the changes made during reassignment cleanup. // - // In this case, we report back FENCED_LEADER_EPOCH to the leader + // In this case, we report back NEW_LEADER_ELECTED to the leader // which made the AlterPartition request. This lets it know that it must // fetch new metadata before trying again. This return code is // unusual because we both return an error and generate a new // metadata record. We usually only do one or the other. + // FENCED_LEADER_EPOCH is used for request version below or equal to 1. + Errors error = requestVersion > 1 ? NEW_LEADER_ELECTED : FENCED_LEADER_EPOCH; log.info("AlterPartition request from node {} for {}-{} completed " + "the ongoing partition reassignment and triggered a " + - "leadership change. Returning FENCED_LEADER_EPOCH.", - request.brokerId(), topic.name, partitionId); + "leadership change. Returning {}.", + request.brokerId(), topic.name, partitionId, error); responseTopicData.partitions().add(new AlterPartitionResponseData.PartitionData(). setPartitionIndex(partitionId). - setErrorCode(FENCED_LEADER_EPOCH.code())); + setErrorCode(error.code())); continue; } else if (change.removingReplicas() != null || change.addingReplicas() != null) { @@ -1026,6 +1048,7 @@ ControllerResult alterPartition(AlterPartitionReques * @param topic current topic information store by the replication manager * @param partitionId partition id being altered * @param partition current partition registration for the partition being altered + * @param isEligibleReplica function telling if the replica is acceptable to join the ISR * @param partitionData partition data from the alter partition request * * @return Errors.NONE for valid alter partition data; otherwise the validation error @@ -1035,6 +1058,8 @@ private Errors validateAlterPartitionData( TopicControlInfo topic, int partitionId, PartitionRegistration partition, + Function isEligibleReplica, + short requestApiVersion, AlterPartitionRequestData.PartitionData partitionData ) { if (partition == null) { @@ -1085,7 +1110,7 @@ private Errors validateAlterPartitionData( if (leaderRecoveryState == LeaderRecoveryState.RECOVERING && newIsr.length > 1) { log.info("Rejecting AlterPartition request from node {} for {}-{} because " + "the ISR {} had more than one replica while the leader was still " + - "recovering from an unlcean leader election {}.", + "recovering from an unclean leader election {}.", brokerId, topic.name, partitionId, partitionData.newIsr(), leaderRecoveryState); @@ -1093,7 +1118,6 @@ private Errors validateAlterPartitionData( } if (partition.leaderRecoveryState == LeaderRecoveryState.RECOVERED && leaderRecoveryState == LeaderRecoveryState.RECOVERING) { - log.info("Rejecting AlterPartition request from node {} for {}-{} because " + "the leader recovery state cannot change from RECOVERED to RECOVERING.", brokerId, topic.name, partitionId); @@ -1101,6 +1125,21 @@ private Errors validateAlterPartitionData( return INVALID_REQUEST; } + List ineligibleReplicas = partitionData.newIsr().stream() + .filter(replica -> !isEligibleReplica.apply(replica)) + .collect(Collectors.toList()); + if (!ineligibleReplicas.isEmpty()) { + log.info("Rejecting AlterPartition request from node {} for {}-{} because " + + "it specified ineligible replicas {} in the new ISR {}.", + brokerId, topic.name, partitionId, ineligibleReplicas, partitionData.newIsr()); + + if (requestApiVersion > 1) { + return INELIGIBLE_REPLICA; + } else { + return OPERATION_NOT_ATTEMPTED; + } + } + return Errors.NONE; } @@ -1113,7 +1152,6 @@ private Errors validateAlterPartitionData( * @param brokerId The broker id. * @param records The record list to append to. */ - void handleBrokerFenced(int brokerId, List records) { BrokerRegistration brokerRegistration = clusterControl.brokerRegistrations().get(brokerId); if (brokerRegistration == null) { diff --git a/metadata/src/main/java/org/apache/kafka/image/FeaturesImage.java b/metadata/src/main/java/org/apache/kafka/image/FeaturesImage.java index 3843b59bc781..4cfb1260f1dc 100644 --- a/metadata/src/main/java/org/apache/kafka/image/FeaturesImage.java +++ b/metadata/src/main/java/org/apache/kafka/image/FeaturesImage.java @@ -38,7 +38,7 @@ * This class is thread-safe. */ public final class FeaturesImage { - public static final FeaturesImage EMPTY = new FeaturesImage(Collections.emptyMap(), MetadataVersion.UNINITIALIZED); + public static final FeaturesImage EMPTY = new FeaturesImage(Collections.emptyMap(), MetadataVersion.MINIMUM_KRAFT_VERSION); private final Map finalizedVersions; @@ -68,11 +68,10 @@ private Optional finalizedVersion(String feature) { public void write(Consumer> out) { List batch = new ArrayList<>(); // Write out the metadata.version record first, and then the rest of the finalized features - if (!metadataVersion().equals(MetadataVersion.UNINITIALIZED)) { - batch.add(new ApiMessageAndVersion(new FeatureLevelRecord(). - setName(MetadataVersion.FEATURE_NAME). - setFeatureLevel(metadataVersion.featureLevel()), FEATURE_LEVEL_RECORD.lowestSupportedVersion())); - } + batch.add(new ApiMessageAndVersion(new FeatureLevelRecord(). + setName(MetadataVersion.FEATURE_NAME). + setFeatureLevel(metadataVersion.featureLevel()), FEATURE_LEVEL_RECORD.lowestSupportedVersion())); + for (Entry entry : finalizedVersions.entrySet()) { if (entry.getKey().equals(MetadataVersion.FEATURE_NAME)) { continue; diff --git a/metadata/src/main/java/org/apache/kafka/image/MetadataImage.java b/metadata/src/main/java/org/apache/kafka/image/MetadataImage.java index e3cd94a0cb5b..55d572127e97 100644 --- a/metadata/src/main/java/org/apache/kafka/image/MetadataImage.java +++ b/metadata/src/main/java/org/apache/kafka/image/MetadataImage.java @@ -121,12 +121,7 @@ public AclsImage acls() { } public void write(Consumer> out) { - // We use the minimum KRaft metadata version if this image does - // not have a specific version set. MetadataVersion metadataVersion = features.metadataVersion(); - if (metadataVersion.equals(MetadataVersion.UNINITIALIZED)) { - metadataVersion = MetadataVersion.IBP_3_0_IV1; - } // Features should be written out first so we can include the metadata.version at the beginning of the // snapshot features.write(out); diff --git a/metadata/src/main/java/org/apache/kafka/metadata/authorizer/ClusterMetadataAuthorizer.java b/metadata/src/main/java/org/apache/kafka/metadata/authorizer/ClusterMetadataAuthorizer.java index d76730228977..710a975778c0 100644 --- a/metadata/src/main/java/org/apache/kafka/metadata/authorizer/ClusterMetadataAuthorizer.java +++ b/metadata/src/main/java/org/apache/kafka/metadata/authorizer/ClusterMetadataAuthorizer.java @@ -103,7 +103,7 @@ default List> createAcls( AclMutator aclMutator = aclMutatorOrException(); aclBindings.forEach(b -> futures.add(new CompletableFuture<>())); ControllerRequestContext context = new ControllerRequestContext( - requestContext.principal(), OptionalLong.empty()); + requestContext, OptionalLong.empty()); aclMutator.createAcls(context, aclBindings).whenComplete((results, throwable) -> { if (throwable == null && results.size() != futures.size()) { throwable = new UnknownServerException("Invalid size " + @@ -143,7 +143,7 @@ default List> deleteAcls( AclMutator aclMutator = aclMutatorOrException(); filters.forEach(b -> futures.add(new CompletableFuture<>())); ControllerRequestContext context = new ControllerRequestContext( - requestContext.principal(), OptionalLong.empty()); + requestContext, OptionalLong.empty()); aclMutator.deleteAcls(context, filters).whenComplete((results, throwable) -> { if (throwable == null && results.size() != futures.size()) { throwable = new UnknownServerException("Invalid size " + diff --git a/metadata/src/test/java/org/apache/kafka/controller/BootstrapMetadataTest.java b/metadata/src/test/java/org/apache/kafka/controller/BootstrapMetadataTest.java index 16e03a050101..4eb5d4747813 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/BootstrapMetadataTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/BootstrapMetadataTest.java @@ -34,28 +34,28 @@ public class BootstrapMetadataTest { @Test public void testWriteAndReadBootstrapFile() throws Exception { Path tmpDir = Files.createTempDirectory("BootstrapMetadataTest"); - BootstrapMetadata metadata = BootstrapMetadata.create(MetadataVersion.IBP_3_0_IV0); + BootstrapMetadata metadata = BootstrapMetadata.create(MetadataVersion.MINIMUM_KRAFT_VERSION); BootstrapMetadata.write(metadata, tmpDir); assertTrue(Files.exists(tmpDir.resolve(BootstrapMetadata.BOOTSTRAP_FILE))); - BootstrapMetadata newMetadata = BootstrapMetadata.load(tmpDir, MetadataVersion.IBP_3_0_IV0); + BootstrapMetadata newMetadata = BootstrapMetadata.load(tmpDir, () -> MetadataVersion.MINIMUM_KRAFT_VERSION); assertEquals(metadata, newMetadata); } @Test public void testNoBootstrapFile() throws Exception { Path tmpDir = Files.createTempDirectory("BootstrapMetadataTest"); - BootstrapMetadata metadata = BootstrapMetadata.load(tmpDir, MetadataVersion.IBP_3_0_IV0); - assertEquals(MetadataVersion.IBP_3_0_IV0, metadata.metadataVersion()); - metadata = BootstrapMetadata.load(tmpDir, MetadataVersion.IBP_3_2_IV0); + BootstrapMetadata metadata = BootstrapMetadata.load(tmpDir, () -> MetadataVersion.MINIMUM_KRAFT_VERSION); + assertEquals(MetadataVersion.MINIMUM_KRAFT_VERSION, metadata.metadataVersion()); + metadata = BootstrapMetadata.load(tmpDir, () -> MetadataVersion.IBP_3_2_IV0); assertEquals(MetadataVersion.IBP_3_2_IV0, metadata.metadataVersion()); } @Test public void testExistingBootstrapFile() throws Exception { Path tmpDir = Files.createTempDirectory("BootstrapMetadataTest"); - BootstrapMetadata.write(BootstrapMetadata.create(MetadataVersion.IBP_3_0_IV0), tmpDir); + BootstrapMetadata.write(BootstrapMetadata.create(MetadataVersion.MINIMUM_KRAFT_VERSION), tmpDir); assertThrows(IOException.class, () -> { BootstrapMetadata.write(BootstrapMetadata.create(MetadataVersion.IBP_3_1_IV0), tmpDir); }); @@ -65,7 +65,7 @@ public void testExistingBootstrapFile() throws Exception { public void testEmptyBootstrapFile() throws Exception { Path tmpDir = Files.createTempDirectory("BootstrapMetadataTest"); Files.createFile(tmpDir.resolve(BootstrapMetadata.BOOTSTRAP_FILE)); - assertThrows(RuntimeException.class, () -> BootstrapMetadata.load(tmpDir, MetadataVersion.IBP_3_0_IV0), + assertThrows(Exception.class, () -> BootstrapMetadata.load(tmpDir, () -> MetadataVersion.MINIMUM_KRAFT_VERSION), "Should fail to load if no metadata.version is set"); } @@ -77,7 +77,7 @@ public void testGarbageBootstrapFile() throws Exception { byte[] data = new byte[100]; random.nextBytes(data); Files.write(tmpDir.resolve(BootstrapMetadata.BOOTSTRAP_FILE), data, StandardOpenOption.CREATE, StandardOpenOption.WRITE); - assertThrows(RuntimeException.class, () -> BootstrapMetadata.load(tmpDir, MetadataVersion.IBP_3_0_IV0), + assertThrows(Exception.class, () -> BootstrapMetadata.load(tmpDir, () -> MetadataVersion.MINIMUM_KRAFT_VERSION), "Should fail on invalid data"); } } diff --git a/metadata/src/test/java/org/apache/kafka/controller/ClusterControlManagerTest.java b/metadata/src/test/java/org/apache/kafka/controller/ClusterControlManagerTest.java index cd44e2e678cd..c0b163180e59 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/ClusterControlManagerTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/ClusterControlManagerTest.java @@ -68,7 +68,7 @@ @Timeout(value = 40) public class ClusterControlManagerTest { @ParameterizedTest - @EnumSource(value = MetadataVersion.class, names = {"IBP_3_0_IV0", "IBP_3_3_IV2"}) + @EnumSource(value = MetadataVersion.class, names = {"IBP_3_0_IV1", "IBP_3_3_IV2"}) public void testReplay(MetadataVersion metadataVersion) { MockTime time = new MockTime(0, 0, 0); diff --git a/metadata/src/test/java/org/apache/kafka/controller/ControllerRequestContextUtil.java b/metadata/src/test/java/org/apache/kafka/controller/ControllerRequestContextUtil.java new file mode 100644 index 000000000000..8d70a2d82f53 --- /dev/null +++ b/metadata/src/test/java/org/apache/kafka/controller/ControllerRequestContextUtil.java @@ -0,0 +1,48 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.controller; + +import java.util.OptionalLong; +import org.apache.kafka.common.message.RequestHeaderData; +import org.apache.kafka.common.protocol.ApiKeys; +import org.apache.kafka.common.security.auth.KafkaPrincipal; + +public class ControllerRequestContextUtil { + public static final ControllerRequestContext ANONYMOUS_CONTEXT = + new ControllerRequestContext( + new RequestHeaderData(), + KafkaPrincipal.ANONYMOUS, + OptionalLong.empty()); + + public static ControllerRequestContext anonymousContextFor(ApiKeys apiKeys) { + return anonymousContextFor(apiKeys, apiKeys.latestVersion()); + } + + public static ControllerRequestContext anonymousContextFor( + ApiKeys apiKeys, + short version + ) { + return new ControllerRequestContext( + new RequestHeaderData() + .setRequestApiKey(apiKeys.id) + .setRequestApiVersion(version), + KafkaPrincipal.ANONYMOUS, + OptionalLong.empty() + ); + } +} diff --git a/metadata/src/test/java/org/apache/kafka/controller/FeatureControlManagerTest.java b/metadata/src/test/java/org/apache/kafka/controller/FeatureControlManagerTest.java index fd1e5af39ee6..4d4c4719945f 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/FeatureControlManagerTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/FeatureControlManagerTest.java @@ -93,7 +93,7 @@ public void testUpdateFeatures() { setSnapshotRegistry(snapshotRegistry). build(); snapshotRegistry.getOrCreateSnapshot(-1); - assertEquals(new FinalizedControllerFeatures(Collections.emptyMap(), -1), + assertEquals(new FinalizedControllerFeatures(Collections.singletonMap("metadata.version", (short) 1), -1), manager.finalizedFeatures(-1)); assertEquals(ControllerResult.atomicOf(emptyList(), Collections. singletonMap("foo", new ApiError(Errors.INVALID_UPDATE_VERSION, @@ -131,7 +131,7 @@ public void testReplay() { build(); manager.replay(record); snapshotRegistry.getOrCreateSnapshot(123); - assertEquals(new FinalizedControllerFeatures(versionMap("foo", 2), 123), + assertEquals(new FinalizedControllerFeatures(versionMap("metadata.version", 1, "foo", 2), 123), manager.finalizedFeatures(123)); } @@ -210,6 +210,9 @@ public void testFeatureControlIterator() throws Exception { Collections.emptyMap(), Collections.emptyMap(), false); RecordTestUtils.replayAll(manager, result.records()); RecordTestUtils.assertBatchIteratorContains(Arrays.asList( + Arrays.asList(new ApiMessageAndVersion(new FeatureLevelRecord(). + setName("metadata.version"). + setFeatureLevel((short) 1), (short) 0)), Arrays.asList(new ApiMessageAndVersion(new FeatureLevelRecord(). setName("foo"). setFeatureLevel((short) 5), (short) 0)), @@ -222,13 +225,13 @@ public void testFeatureControlIterator() throws Exception { @Test public void testApplyMetadataVersionChangeRecord() { QuorumFeatures features = features(MetadataVersion.FEATURE_NAME, - MetadataVersion.IBP_3_0_IV0.featureLevel(), MetadataVersion.IBP_3_3_IV0.featureLevel()); + MetadataVersion.IBP_3_0_IV1.featureLevel(), MetadataVersion.IBP_3_3_IV0.featureLevel()); FeatureControlManager manager = new FeatureControlManager.Builder(). setQuorumFeatures(features).build(); manager.replay(new FeatureLevelRecord(). setName(MetadataVersion.FEATURE_NAME). - setFeatureLevel(MetadataVersion.IBP_3_0_IV0.featureLevel())); - assertEquals(MetadataVersion.IBP_3_0_IV0, manager.metadataVersion()); + setFeatureLevel(MetadataVersion.IBP_3_0_IV1.featureLevel())); + assertEquals(MetadataVersion.IBP_3_0_IV1, manager.metadataVersion()); } @Test @@ -258,12 +261,12 @@ public void testDowngradeMetadataVersion() { assertEquals(Errors.INVALID_UPDATE_VERSION, result.response().get(MetadataVersion.FEATURE_NAME).error()); result = manager.updateFeatures( - Collections.singletonMap(MetadataVersion.FEATURE_NAME, MetadataVersion.IBP_3_0_IV0.featureLevel()), + Collections.singletonMap(MetadataVersion.FEATURE_NAME, MetadataVersion.MINIMUM_KRAFT_VERSION.featureLevel()), Collections.singletonMap(MetadataVersion.FEATURE_NAME, FeatureUpdate.UpgradeType.SAFE_DOWNGRADE), Collections.emptyMap(), true); assertEquals(Errors.INVALID_UPDATE_VERSION, result.response().get(MetadataVersion.FEATURE_NAME).error()); - assertEquals("Invalid update version 1 for feature metadata.version. Local controller 0 only supports versions 4-5", + assertEquals("Invalid update version 1 for feature metadata.version. Local controller 0 only supports versions 3-4", result.response().get(MetadataVersion.FEATURE_NAME).message()); } @@ -271,7 +274,7 @@ public void testDowngradeMetadataVersion() { public void testCreateFeatureLevelRecords() { Map localSupportedFeatures = new HashMap<>(); localSupportedFeatures.put(MetadataVersion.FEATURE_NAME, VersionRange.of( - MetadataVersion.IBP_3_0_IV0.featureLevel(), MetadataVersion.latest().featureLevel())); + MetadataVersion.IBP_3_0_IV1.featureLevel(), MetadataVersion.latest().featureLevel())); localSupportedFeatures.put("foo", VersionRange.of(0, 2)); FeatureControlManager manager = new FeatureControlManager.Builder(). setQuorumFeatures(new QuorumFeatures(0, new ApiVersions(), localSupportedFeatures, emptyList())). diff --git a/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTest.java b/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTest.java index 87afbf4199f7..34ad4d9727d1 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTest.java @@ -43,6 +43,7 @@ import org.apache.kafka.common.Uuid; import org.apache.kafka.common.errors.BrokerIdNotRegisteredException; import org.apache.kafka.common.errors.UnknownTopicOrPartitionException; +import org.apache.kafka.common.message.RequestHeaderData; import org.apache.kafka.common.security.auth.KafkaPrincipal; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.common.config.ConfigResource; @@ -96,6 +97,7 @@ import org.apache.kafka.test.TestUtils; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; +import org.mockito.Mockito; import static java.util.function.Function.identity; import static org.apache.kafka.clients.admin.AlterConfigOp.OpType.SET; @@ -104,7 +106,7 @@ import static org.apache.kafka.controller.ConfigurationControlManagerTest.BROKER0; import static org.apache.kafka.controller.ConfigurationControlManagerTest.SCHEMA; import static org.apache.kafka.controller.ConfigurationControlManagerTest.entry; -import static org.apache.kafka.controller.ControllerRequestContext.ANONYMOUS_CONTEXT; +import static org.apache.kafka.controller.ControllerRequestContextUtil.ANONYMOUS_CONTEXT; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertNotEquals; @@ -214,7 +216,7 @@ public void testFenceMultipleBrokers() throws Throwable { LocalLogManagerTestEnv logEnv = new LocalLogManagerTestEnv(1, Optional.empty()); QuorumControllerTestEnv controlEnv = new QuorumControllerTestEnv(logEnv, b -> { b.setConfigSchema(SCHEMA); - }, OptionalLong.of(sessionTimeoutMillis), OptionalLong.empty(), MetadataVersion.latest()); + }, OptionalLong.of(sessionTimeoutMillis), OptionalLong.empty(), BootstrapMetadata.create(MetadataVersion.latest())); ) { ListenerCollection listeners = new ListenerCollection(); listeners.add(new Listener().setName("PLAINTEXT").setHost("localhost").setPort(9092)); @@ -306,7 +308,7 @@ public void testBalancePartitionLeaders() throws Throwable { LocalLogManagerTestEnv logEnv = new LocalLogManagerTestEnv(1, Optional.empty()); QuorumControllerTestEnv controlEnv = new QuorumControllerTestEnv(logEnv, b -> { b.setConfigSchema(SCHEMA); - }, OptionalLong.of(sessionTimeoutMillis), OptionalLong.of(leaderImbalanceCheckIntervalNs), MetadataVersion.latest()); + }, OptionalLong.of(sessionTimeoutMillis), OptionalLong.of(leaderImbalanceCheckIntervalNs), BootstrapMetadata.create(MetadataVersion.latest())); ) { ListenerCollection listeners = new ListenerCollection(); listeners.add(new Listener().setName("PLAINTEXT").setHost("localhost").setPort(9092)); @@ -399,7 +401,7 @@ public void testBalancePartitionLeaders() throws Throwable { .setNewIsr(Arrays.asList(1, 2, 3)); AlterPartitionRequestData.TopicData topicData = new AlterPartitionRequestData.TopicData() - .setName("foo"); + .setTopicName("foo"); topicData.partitions().add(partitionData); AlterPartitionRequestData alterPartitionRequest = new AlterPartitionRequestData() @@ -537,7 +539,7 @@ private BrokerRegistrationRequestData.FeatureCollection brokerFeatures() { BrokerRegistrationRequestData.FeatureCollection features = new BrokerRegistrationRequestData.FeatureCollection(); features.add(new BrokerRegistrationRequestData.Feature() .setName(MetadataVersion.FEATURE_NAME) - .setMinSupportedVersion(MetadataVersion.IBP_3_0_IV0.featureLevel()) + .setMinSupportedVersion(MetadataVersion.MINIMUM_KRAFT_VERSION.featureLevel()) .setMaxSupportedVersion(MetadataVersion.latest().featureLevel())); return features; } @@ -856,7 +858,7 @@ public void testTimeouts() throws Throwable { CountDownLatch countDownLatch = controller.pause(); long now = controller.time().nanoseconds(); ControllerRequestContext context0 = new ControllerRequestContext( - KafkaPrincipal.ANONYMOUS, OptionalLong.of(now)); + new RequestHeaderData(), KafkaPrincipal.ANONYMOUS, OptionalLong.of(now)); CompletableFuture createFuture = controller.createTopics(context0, new CreateTopicsRequestData().setTimeoutMs(0). setTopics(new CreatableTopicCollection(Collections.singleton( @@ -992,7 +994,7 @@ public void testMissingInMemorySnapshot() throws Exception { .collect(Collectors.toList()); AlterPartitionRequestData.TopicData topicData = new AlterPartitionRequestData.TopicData() - .setName(topicName); + .setTopicName(topicName); topicData.partitions().addAll(alterPartitions); int leaderId = 0; @@ -1179,4 +1181,21 @@ private static void assertInitialLoadFuturesNotComplete(List "authorizer " + i + " should not have completed loading."); } } + + @Test + public void testInvalidBootstrapMetadata() throws Exception { + // We can't actually create a BootstrapMetadata with an invalid version, so we have to fake it + BootstrapMetadata bootstrapMetadata = Mockito.mock(BootstrapMetadata.class); + Mockito.when(bootstrapMetadata.metadataVersion()).thenReturn(MetadataVersion.IBP_2_8_IV0); + try ( + LocalLogManagerTestEnv logEnv = new LocalLogManagerTestEnv(1, Optional.empty()); + QuorumControllerTestEnv controlEnv = new QuorumControllerTestEnv(logEnv, b -> { + b.setConfigSchema(SCHEMA); + }, OptionalLong.empty(), OptionalLong.empty(), bootstrapMetadata); + ) { + QuorumController active = controlEnv.activeController(); + TestUtils.waitForCondition(() -> !active.isActive(), + "Timed out waiting for controller to renounce itself after bad bootstrap metadata version."); + } + } } diff --git a/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTestEnv.java b/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTestEnv.java index e1481b9fe55f..68b11ae3ece0 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTestEnv.java +++ b/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTestEnv.java @@ -49,7 +49,7 @@ public QuorumControllerTestEnv( LocalLogManagerTestEnv logEnv, Consumer builderConsumer ) throws Exception { - this(logEnv, builderConsumer, OptionalLong.empty(), OptionalLong.empty(), MetadataVersion.latest()); + this(logEnv, builderConsumer, OptionalLong.empty(), OptionalLong.empty(), BootstrapMetadata.create(MetadataVersion.latest())); } public QuorumControllerTestEnv( @@ -57,7 +57,7 @@ public QuorumControllerTestEnv( Consumer builderConsumer, OptionalLong sessionTimeoutMillis, OptionalLong leaderImbalanceCheckIntervalNs, - MetadataVersion metadataVersion + BootstrapMetadata bootstrapMetadata ) throws Exception { this.logEnv = logEnv; int numControllers = logEnv.logManagers().size(); @@ -68,7 +68,7 @@ public QuorumControllerTestEnv( for (int i = 0; i < numControllers; i++) { QuorumController.Builder builder = new QuorumController.Builder(i, logEnv.clusterId()); builder.setRaftClient(logEnv.logManagers().get(i)); - builder.setBootstrapMetadata(BootstrapMetadata.create(metadataVersion)); + builder.setBootstrapMetadata(bootstrapMetadata); builder.setLeaderImbalanceCheckIntervalNs(leaderImbalanceCheckIntervalNs); builder.setQuorumFeatures(new QuorumFeatures(i, apiVersions, QuorumFeatures.defaultFeatureMap(), nodeIds)); sessionTimeoutMillis.ifPresent(timeout -> { diff --git a/metadata/src/test/java/org/apache/kafka/controller/ReplicationControlManagerTest.java b/metadata/src/test/java/org/apache/kafka/controller/ReplicationControlManagerTest.java index 59b5488f6a10..d33776ca10e5 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/ReplicationControlManagerTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/ReplicationControlManagerTest.java @@ -61,12 +61,14 @@ import org.apache.kafka.common.metadata.PartitionRecord; import org.apache.kafka.common.metadata.RegisterBrokerRecord; import org.apache.kafka.common.metadata.TopicRecord; +import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.requests.ApiError; import org.apache.kafka.common.security.auth.SecurityProtocol; import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.common.utils.annotation.ApiKeyVersionsSource; import org.apache.kafka.controller.ReplicationControlManager.KRaftClusterDescriber; import org.apache.kafka.metadata.BrokerHeartbeatReply; import org.apache.kafka.metadata.BrokerRegistration; @@ -113,16 +115,20 @@ import static org.apache.kafka.common.protocol.Errors.ELECTION_NOT_NEEDED; import static org.apache.kafka.common.protocol.Errors.ELIGIBLE_LEADERS_NOT_AVAILABLE; import static org.apache.kafka.common.protocol.Errors.FENCED_LEADER_EPOCH; +import static org.apache.kafka.common.protocol.Errors.INELIGIBLE_REPLICA; import static org.apache.kafka.common.protocol.Errors.INVALID_PARTITIONS; import static org.apache.kafka.common.protocol.Errors.INVALID_REPLICATION_FACTOR; import static org.apache.kafka.common.protocol.Errors.INVALID_REPLICA_ASSIGNMENT; import static org.apache.kafka.common.protocol.Errors.INVALID_TOPIC_EXCEPTION; +import static org.apache.kafka.common.protocol.Errors.NEW_LEADER_ELECTED; import static org.apache.kafka.common.protocol.Errors.NONE; import static org.apache.kafka.common.protocol.Errors.NO_REASSIGNMENT_IN_PROGRESS; +import static org.apache.kafka.common.protocol.Errors.OPERATION_NOT_ATTEMPTED; import static org.apache.kafka.common.protocol.Errors.POLICY_VIOLATION; import static org.apache.kafka.common.protocol.Errors.PREFERRED_LEADER_NOT_AVAILABLE; import static org.apache.kafka.common.protocol.Errors.UNKNOWN_TOPIC_ID; import static org.apache.kafka.common.protocol.Errors.UNKNOWN_TOPIC_OR_PARTITION; +import static org.apache.kafka.controller.ControllerRequestContextUtil.anonymousContextFor; import static org.apache.kafka.metadata.LeaderConstants.NO_LEADER; import static org.junit.jupiter.api.Assertions.assertArrayEquals; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -319,10 +325,14 @@ void alterPartition( String topicName = replicationControl.getTopic(topicIdPartition.topicId()).name(); TopicData topicData = new TopicData() - .setName(topicName) + .setTopicName(topicName) + .setTopicId(topicIdPartition.topicId()) .setPartitions(singletonList(partitionData)); + ControllerRequestContext requestContext = + anonymousContextFor(ApiKeys.ALTER_PARTITION); ControllerResult alterPartition = replicationControl.alterPartition( + requestContext, new AlterPartitionRequestData() .setBrokerId(leaderId) .setBrokerEpoch(registration.epoch()) @@ -857,26 +867,63 @@ public void testShrinkAndExpandIsr() throws Exception { new int[][] {new int[] {0, 1, 2}}); TopicIdPartition topicIdPartition = new TopicIdPartition(createTopicResult.topicId(), 0); - TopicPartition topicPartition = new TopicPartition("foo", 0); assertEquals(OptionalInt.of(0), ctx.currentLeader(topicIdPartition)); long brokerEpoch = ctx.currentBrokerEpoch(0); PartitionData shrinkIsrRequest = newAlterPartition( replicationControl, topicIdPartition, asList(0, 1), LeaderRecoveryState.RECOVERED); - ControllerResult shrinkIsrResult = sendAlterIsr( - replicationControl, 0, brokerEpoch, "foo", shrinkIsrRequest); + ControllerResult shrinkIsrResult = sendAlterPartition( + replicationControl, 0, brokerEpoch, topicIdPartition.topicId(), shrinkIsrRequest); AlterPartitionResponseData.PartitionData shrinkIsrResponse = assertAlterPartitionResponse( - shrinkIsrResult, topicPartition, NONE); + shrinkIsrResult, topicIdPartition, NONE); assertConsistentAlterPartitionResponse(replicationControl, topicIdPartition, shrinkIsrResponse); PartitionData expandIsrRequest = newAlterPartition( replicationControl, topicIdPartition, asList(0, 1, 2), LeaderRecoveryState.RECOVERED); - ControllerResult expandIsrResult = sendAlterIsr( - replicationControl, 0, brokerEpoch, "foo", expandIsrRequest); + ControllerResult expandIsrResult = sendAlterPartition( + replicationControl, 0, brokerEpoch, topicIdPartition.topicId(), expandIsrRequest); AlterPartitionResponseData.PartitionData expandIsrResponse = assertAlterPartitionResponse( - expandIsrResult, topicPartition, NONE); + expandIsrResult, topicIdPartition, NONE); assertConsistentAlterPartitionResponse(replicationControl, topicIdPartition, expandIsrResponse); } + @ParameterizedTest + @ApiKeyVersionsSource(apiKey = ApiKeys.ALTER_PARTITION) + public void testAlterPartitionHandleUnknownTopicIdOrName(short version) throws Exception { + ReplicationControlTestContext ctx = new ReplicationControlTestContext(); + ReplicationControlManager replicationControl = ctx.replicationControl; + ctx.registerBrokers(0, 1, 2); + ctx.unfenceBrokers(0, 1, 2); + + String topicName = "foo"; + Uuid topicId = Uuid.randomUuid(); + + AlterPartitionRequestData request = new AlterPartitionRequestData() + .setBrokerId(0) + .setBrokerEpoch(100) + .setTopics(asList(new AlterPartitionRequestData.TopicData() + .setTopicName(version <= 1 ? topicName : "") + .setTopicId(version > 1 ? topicId : Uuid.ZERO_UUID) + .setPartitions(asList(new PartitionData() + .setPartitionIndex(0))))); + + ControllerRequestContext requestContext = + anonymousContextFor(ApiKeys.ALTER_PARTITION, version); + + ControllerResult result = + replicationControl.alterPartition(requestContext, request); + + Errors expectedError = version > 1 ? UNKNOWN_TOPIC_ID : UNKNOWN_TOPIC_OR_PARTITION; + AlterPartitionResponseData expectedResponse = new AlterPartitionResponseData() + .setTopics(asList(new AlterPartitionResponseData.TopicData() + .setTopicName(version <= 1 ? topicName : "") + .setTopicId(version > 1 ? topicId : Uuid.ZERO_UUID) + .setPartitions(asList(new AlterPartitionResponseData.PartitionData() + .setPartitionIndex(0) + .setErrorCode(expectedError.code()))))); + + assertEquals(expectedResponse, result.response()); + } + @Test public void testInvalidAlterPartitionRequests() throws Exception { ReplicationControlTestContext ctx = new ReplicationControlTestContext(); @@ -887,7 +934,6 @@ public void testInvalidAlterPartitionRequests() throws Exception { new int[][] {new int[] {0, 1, 2}}); TopicIdPartition topicIdPartition = new TopicIdPartition(createTopicResult.topicId(), 0); - TopicPartition topicPartition = new TopicPartition("foo", 0); int leaderId = 0; int notLeaderId = 1; assertEquals(OptionalInt.of(leaderId), ctx.currentLeader(topicIdPartition)); @@ -896,57 +942,57 @@ public void testInvalidAlterPartitionRequests() throws Exception { // Invalid leader PartitionData invalidLeaderRequest = newAlterPartition( replicationControl, topicIdPartition, asList(0, 1), LeaderRecoveryState.RECOVERED); - ControllerResult invalidLeaderResult = sendAlterIsr( + ControllerResult invalidLeaderResult = sendAlterPartition( replicationControl, notLeaderId, ctx.currentBrokerEpoch(notLeaderId), - "foo", invalidLeaderRequest); - assertAlterPartitionResponse(invalidLeaderResult, topicPartition, Errors.INVALID_REQUEST); + topicIdPartition.topicId(), invalidLeaderRequest); + assertAlterPartitionResponse(invalidLeaderResult, topicIdPartition, Errors.INVALID_REQUEST); // Stale broker epoch PartitionData invalidBrokerEpochRequest = newAlterPartition( replicationControl, topicIdPartition, asList(0, 1), LeaderRecoveryState.RECOVERED); - assertThrows(StaleBrokerEpochException.class, () -> sendAlterIsr( - replicationControl, leaderId, brokerEpoch - 1, "foo", invalidBrokerEpochRequest)); + assertThrows(StaleBrokerEpochException.class, () -> sendAlterPartition( + replicationControl, leaderId, brokerEpoch - 1, topicIdPartition.topicId(), invalidBrokerEpochRequest)); // Invalid leader epoch PartitionData invalidLeaderEpochRequest = newAlterPartition( replicationControl, topicIdPartition, asList(0, 1), LeaderRecoveryState.RECOVERED); invalidLeaderEpochRequest.setLeaderEpoch(500); - ControllerResult invalidLeaderEpochResult = sendAlterIsr( + ControllerResult invalidLeaderEpochResult = sendAlterPartition( replicationControl, leaderId, ctx.currentBrokerEpoch(leaderId), - "foo", invalidLeaderEpochRequest); - assertAlterPartitionResponse(invalidLeaderEpochResult, topicPartition, FENCED_LEADER_EPOCH); + topicIdPartition.topicId(), invalidLeaderEpochRequest); + assertAlterPartitionResponse(invalidLeaderEpochResult, topicIdPartition, FENCED_LEADER_EPOCH); // Invalid ISR (3 is not a valid replica) PartitionData invalidIsrRequest1 = newAlterPartition( replicationControl, topicIdPartition, asList(0, 1, 3), LeaderRecoveryState.RECOVERED); - ControllerResult invalidIsrResult1 = sendAlterIsr( + ControllerResult invalidIsrResult1 = sendAlterPartition( replicationControl, leaderId, ctx.currentBrokerEpoch(leaderId), - "foo", invalidIsrRequest1); - assertAlterPartitionResponse(invalidIsrResult1, topicPartition, Errors.INVALID_REQUEST); + topicIdPartition.topicId(), invalidIsrRequest1); + assertAlterPartitionResponse(invalidIsrResult1, topicIdPartition, Errors.INVALID_REQUEST); // Invalid ISR (does not include leader 0) PartitionData invalidIsrRequest2 = newAlterPartition( replicationControl, topicIdPartition, asList(1, 2), LeaderRecoveryState.RECOVERED); - ControllerResult invalidIsrResult2 = sendAlterIsr( + ControllerResult invalidIsrResult2 = sendAlterPartition( replicationControl, leaderId, ctx.currentBrokerEpoch(leaderId), - "foo", invalidIsrRequest2); - assertAlterPartitionResponse(invalidIsrResult2, topicPartition, Errors.INVALID_REQUEST); + topicIdPartition.topicId(), invalidIsrRequest2); + assertAlterPartitionResponse(invalidIsrResult2, topicIdPartition, Errors.INVALID_REQUEST); // Invalid ISR length and recovery state PartitionData invalidIsrRecoveryRequest = newAlterPartition( replicationControl, topicIdPartition, asList(0, 1), LeaderRecoveryState.RECOVERING); - ControllerResult invalidIsrRecoveryResult = sendAlterIsr( + ControllerResult invalidIsrRecoveryResult = sendAlterPartition( replicationControl, leaderId, ctx.currentBrokerEpoch(leaderId), - "foo", invalidIsrRecoveryRequest); - assertAlterPartitionResponse(invalidIsrRecoveryResult, topicPartition, Errors.INVALID_REQUEST); + topicIdPartition.topicId(), invalidIsrRecoveryRequest); + assertAlterPartitionResponse(invalidIsrRecoveryResult, topicIdPartition, Errors.INVALID_REQUEST); - // Invalid recovery state transtion from RECOVERED to RECOVERING + // Invalid recovery state transition from RECOVERED to RECOVERING PartitionData invalidRecoveryRequest = newAlterPartition( replicationControl, topicIdPartition, asList(0), LeaderRecoveryState.RECOVERING); - ControllerResult invalidRecoveryResult = sendAlterIsr( + ControllerResult invalidRecoveryResult = sendAlterPartition( replicationControl, leaderId, ctx.currentBrokerEpoch(leaderId), - "foo", invalidRecoveryRequest); - assertAlterPartitionResponse(invalidRecoveryResult, topicPartition, Errors.INVALID_REQUEST); + topicIdPartition.topicId(), invalidRecoveryRequest); + assertAlterPartitionResponse(invalidRecoveryResult, topicIdPartition, Errors.INVALID_REQUEST); } private PartitionData newAlterPartition( @@ -965,11 +1011,11 @@ private PartitionData newAlterPartition( .setLeaderRecoveryState(leaderRecoveryState.value()); } - private ControllerResult sendAlterIsr( + private ControllerResult sendAlterPartition( ReplicationControlManager replicationControl, int brokerId, long brokerEpoch, - String topic, + Uuid topicId, AlterPartitionRequestData.PartitionData partitionData ) throws Exception { AlterPartitionRequestData request = new AlterPartitionRequestData() @@ -977,29 +1023,30 @@ private ControllerResult sendAlterIsr( .setBrokerEpoch(brokerEpoch); AlterPartitionRequestData.TopicData topicData = new AlterPartitionRequestData.TopicData() - .setName(topic); + .setTopicId(topicId); request.topics().add(topicData); topicData.partitions().add(partitionData); - ControllerResult result = replicationControl.alterPartition(request); + ControllerRequestContext requestContext = anonymousContextFor(ApiKeys.ALTER_PARTITION); + ControllerResult result = replicationControl.alterPartition(requestContext, request); RecordTestUtils.replayAll(replicationControl, result.records()); return result; } private AlterPartitionResponseData.PartitionData assertAlterPartitionResponse( ControllerResult alterPartitionResult, - TopicPartition topicPartition, + TopicIdPartition topicIdPartition, Errors expectedError ) { AlterPartitionResponseData response = alterPartitionResult.response(); assertEquals(1, response.topics().size()); AlterPartitionResponseData.TopicData topicData = response.topics().get(0); - assertEquals(topicPartition.topic(), topicData.name()); + assertEquals(topicIdPartition.topicId(), topicData.topicId()); assertEquals(1, topicData.partitions().size()); AlterPartitionResponseData.PartitionData partitionData = topicData.partitions().get(0); - assertEquals(topicPartition.partition(), partitionData.partitionIndex()); + assertEquals(topicIdPartition.partitionId(), partitionData.partitionIndex()); assertEquals(expectedError, Errors.forCode(partitionData.errorCode())); return partitionData; } @@ -1312,8 +1359,9 @@ public void testValidateBadManualPartitionAssignments() throws Exception { private final static ListPartitionReassignmentsResponseData NONE_REASSIGNING = new ListPartitionReassignmentsResponseData().setErrorMessage(null); - @Test - public void testReassignPartitions() throws Exception { + @ParameterizedTest + @ApiKeyVersionsSource(apiKey = ApiKeys.ALTER_PARTITION) + public void testReassignPartitions(short version) throws Exception { ReplicationControlTestContext ctx = new ReplicationControlTestContext(); ReplicationControlManager replication = ctx.replicationControl; ctx.registerBrokers(0, 1, 2, 3); @@ -1398,21 +1446,167 @@ public void testReassignPartitions() throws Exception { setErrorMessage(null)))))), cancelResult); log.info("running final alterPartition..."); + ControllerRequestContext requestContext = + anonymousContextFor(ApiKeys.ALTER_PARTITION, version); ControllerResult alterPartitionResult = replication.alterPartition( + requestContext, new AlterPartitionRequestData().setBrokerId(3).setBrokerEpoch(103). - setTopics(asList(new TopicData().setName("foo").setPartitions(asList( - new PartitionData().setPartitionIndex(1).setPartitionEpoch(1). - setLeaderEpoch(0).setNewIsr(asList(3, 0, 2, 1))))))); + setTopics(asList(new TopicData(). + setTopicName(version <= 1 ? "foo" : ""). + setTopicId(version > 1 ? fooId : Uuid.ZERO_UUID). + setPartitions(asList(new PartitionData(). + setPartitionIndex(1). + setPartitionEpoch(1). + setLeaderEpoch(0). + setNewIsr(asList(3, 0, 2, 1))))))); + Errors expectedError = version > 1 ? NEW_LEADER_ELECTED : FENCED_LEADER_EPOCH; assertEquals(new AlterPartitionResponseData().setTopics(asList( - new AlterPartitionResponseData.TopicData().setName("foo").setPartitions(asList( + new AlterPartitionResponseData.TopicData(). + setTopicName(version <= 1 ? "foo" : ""). + setTopicId(version > 1 ? fooId : Uuid.ZERO_UUID). + setPartitions(asList( new AlterPartitionResponseData.PartitionData(). setPartitionIndex(1). - setErrorCode(FENCED_LEADER_EPOCH.code()))))), + setErrorCode(expectedError.code()))))), alterPartitionResult.response()); ctx.replay(alterPartitionResult.records()); assertEquals(NONE_REASSIGNING, replication.listPartitionReassignments(null)); } + @ParameterizedTest + @ApiKeyVersionsSource(apiKey = ApiKeys.ALTER_PARTITION) + public void testAlterPartitionShouldRejectFencedBrokers(short version) throws Exception { + ReplicationControlTestContext ctx = new ReplicationControlTestContext(); + ReplicationControlManager replication = ctx.replicationControl; + ctx.registerBrokers(0, 1, 2, 3, 4); + ctx.unfenceBrokers(0, 1, 2, 3, 4); + Uuid fooId = ctx.createTestTopic( + "foo", + new int[][] {new int[] {1, 2, 3, 4}} + ).topicId(); + + List fenceRecords = new ArrayList<>(); + replication.handleBrokerFenced(3, fenceRecords); + ctx.replay(fenceRecords); + + assertEquals( + new PartitionRegistration( + new int[] {1, 2, 3, 4}, + new int[] {1, 2, 4}, + new int[] {}, + new int[] {}, + 1, + LeaderRecoveryState.RECOVERED, + 1, + 1), + replication.getPartition(fooId, 0)); + + AlterPartitionRequestData alterIsrRequest = new AlterPartitionRequestData() + .setBrokerId(1) + .setBrokerEpoch(101) + .setTopics(asList(new TopicData() + .setTopicName(version <= 1 ? "foo" : "") + .setTopicId(version > 1 ? fooId : Uuid.ZERO_UUID) + .setPartitions(asList(new PartitionData() + .setPartitionIndex(0) + .setPartitionEpoch(1) + .setLeaderEpoch(1) + .setNewIsr(asList(1, 2, 3, 4)))))); + + ControllerRequestContext requestContext = + anonymousContextFor(ApiKeys.ALTER_PARTITION, version); + + ControllerResult alterPartitionResult = + replication.alterPartition(requestContext, alterIsrRequest); + + Errors expectedError = version <= 1 ? OPERATION_NOT_ATTEMPTED : INELIGIBLE_REPLICA; + assertEquals( + new AlterPartitionResponseData() + .setTopics(asList(new AlterPartitionResponseData.TopicData() + .setTopicName(version <= 1 ? "foo" : "") + .setTopicId(version > 1 ? fooId : Uuid.ZERO_UUID) + .setPartitions(asList(new AlterPartitionResponseData.PartitionData() + .setPartitionIndex(0) + .setErrorCode(expectedError.code()))))), + alterPartitionResult.response()); + + fenceRecords = new ArrayList<>(); + replication.handleBrokerUnfenced(3, 103, fenceRecords); + ctx.replay(fenceRecords); + + alterPartitionResult = replication.alterPartition(requestContext, alterIsrRequest); + + assertEquals( + new AlterPartitionResponseData() + .setTopics(asList(new AlterPartitionResponseData.TopicData() + .setTopicName(version <= 1 ? "foo" : "") + .setTopicId(version > 1 ? fooId : Uuid.ZERO_UUID) + .setPartitions(asList(new AlterPartitionResponseData.PartitionData() + .setPartitionIndex(0) + .setLeaderId(1) + .setLeaderEpoch(1) + .setIsr(asList(1, 2, 3, 4)) + .setPartitionEpoch(2) + .setErrorCode(NONE.code()))))), + alterPartitionResult.response()); + } + + @ParameterizedTest + @ApiKeyVersionsSource(apiKey = ApiKeys.ALTER_PARTITION) + public void testAlterPartitionShouldRejectShuttingDownBrokers(short version) throws Exception { + ReplicationControlTestContext ctx = new ReplicationControlTestContext(); + ReplicationControlManager replication = ctx.replicationControl; + ctx.registerBrokers(0, 1, 2, 3, 4); + ctx.unfenceBrokers(0, 1, 2, 3, 4); + Uuid fooId = ctx.createTestTopic( + "foo", + new int[][] {new int[] {1, 2, 3, 4}} + ).topicId(); + + assertEquals( + new PartitionRegistration( + new int[] {1, 2, 3, 4}, + new int[] {1, 2, 3, 4}, + new int[] {}, + new int[] {}, + 1, + LeaderRecoveryState.RECOVERED, + 0, + 0), + replication.getPartition(fooId, 0)); + + ctx.inControlledShutdownBrokers(3); + + AlterPartitionRequestData alterIsrRequest = new AlterPartitionRequestData() + .setBrokerId(1) + .setBrokerEpoch(101) + .setTopics(asList(new TopicData() + .setTopicName(version <= 1 ? "foo" : "") + .setTopicId(version > 1 ? fooId : Uuid.ZERO_UUID) + .setPartitions(asList(new PartitionData() + .setPartitionIndex(0) + .setPartitionEpoch(0) + .setLeaderEpoch(0) + .setNewIsr(asList(1, 2, 3, 4)))))); + + ControllerRequestContext requestContext = + anonymousContextFor(ApiKeys.ALTER_PARTITION, version); + + ControllerResult alterPartitionResult = + replication.alterPartition(requestContext, alterIsrRequest); + + Errors expectedError = version <= 1 ? OPERATION_NOT_ATTEMPTED : INELIGIBLE_REPLICA; + assertEquals( + new AlterPartitionResponseData() + .setTopics(asList(new AlterPartitionResponseData.TopicData() + .setTopicName(version <= 1 ? "foo" : "") + .setTopicId(version > 1 ? fooId : Uuid.ZERO_UUID) + .setPartitions(asList(new AlterPartitionResponseData.PartitionData() + .setPartitionIndex(0) + .setErrorCode(expectedError.code()))))), + alterPartitionResult.response()); + } + @Test public void testCancelReassignPartitions() throws Exception { ReplicationControlTestContext ctx = new ReplicationControlTestContext(); @@ -1487,17 +1681,18 @@ public void testCancelReassignPartitions() throws Exception { new ListPartitionReassignmentsTopics().setName("bar"). setPartitionIndexes(asList(0, 1, 2))))); ControllerResult alterPartitionResult = replication.alterPartition( + anonymousContextFor(ApiKeys.ALTER_PARTITION), new AlterPartitionRequestData().setBrokerId(4).setBrokerEpoch(104). - setTopics(asList(new TopicData().setName("bar").setPartitions(asList( + setTopics(asList(new TopicData().setTopicId(barId).setPartitions(asList( new PartitionData().setPartitionIndex(0).setPartitionEpoch(2). - setLeaderEpoch(1).setNewIsr(asList(4, 1, 2, 3, 0))))))); + setLeaderEpoch(1).setNewIsr(asList(4, 1, 2, 0))))))); assertEquals(new AlterPartitionResponseData().setTopics(asList( - new AlterPartitionResponseData.TopicData().setName("bar").setPartitions(asList( + new AlterPartitionResponseData.TopicData().setTopicId(barId).setPartitions(asList( new AlterPartitionResponseData.PartitionData(). setPartitionIndex(0). setLeaderId(4). setLeaderEpoch(1). - setIsr(asList(4, 1, 2, 3, 0)). + setIsr(asList(4, 1, 2, 0)). setPartitionEpoch(3). setErrorCode(NONE.code()))))), alterPartitionResult.response()); @@ -1785,8 +1980,9 @@ public void testElectPreferredLeaders() throws Exception { ctx.unfenceBrokers(0, 1); ControllerResult alterPartitionResult = replication.alterPartition( + anonymousContextFor(ApiKeys.ALTER_PARTITION), new AlterPartitionRequestData().setBrokerId(2).setBrokerEpoch(102). - setTopics(asList(new AlterPartitionRequestData.TopicData().setName("foo"). + setTopics(asList(new AlterPartitionRequestData.TopicData().setTopicId(fooId). setPartitions(asList( new AlterPartitionRequestData.PartitionData(). setPartitionIndex(0).setPartitionEpoch(0). @@ -1795,7 +1991,7 @@ public void testElectPreferredLeaders() throws Exception { setPartitionIndex(2).setPartitionEpoch(0). setLeaderEpoch(0).setNewIsr(asList(0, 2, 1))))))); assertEquals(new AlterPartitionResponseData().setTopics(asList( - new AlterPartitionResponseData.TopicData().setName("foo").setPartitions(asList( + new AlterPartitionResponseData.TopicData().setTopicId(fooId).setPartitions(asList( new AlterPartitionResponseData.PartitionData(). setPartitionIndex(0). setLeaderId(2). @@ -1871,13 +2067,14 @@ public void testBalancePartitionLeaders() throws Exception { ctx.unfenceBrokers(1); ControllerResult alterPartitionResult = replication.alterPartition( + anonymousContextFor(ApiKeys.ALTER_PARTITION), new AlterPartitionRequestData().setBrokerId(2).setBrokerEpoch(102). - setTopics(asList(new AlterPartitionRequestData.TopicData().setName("foo"). + setTopics(asList(new AlterPartitionRequestData.TopicData().setTopicId(fooId). setPartitions(asList(new AlterPartitionRequestData.PartitionData(). setPartitionIndex(0).setPartitionEpoch(0). setLeaderEpoch(0).setNewIsr(asList(1, 2, 3))))))); assertEquals(new AlterPartitionResponseData().setTopics(asList( - new AlterPartitionResponseData.TopicData().setName("foo").setPartitions(asList( + new AlterPartitionResponseData.TopicData().setTopicId(fooId).setPartitions(asList( new AlterPartitionResponseData.PartitionData(). setPartitionIndex(0). setLeaderId(2). @@ -1903,13 +2100,14 @@ public void testBalancePartitionLeaders() throws Exception { ctx.unfenceBrokers(0); alterPartitionResult = replication.alterPartition( + anonymousContextFor(ApiKeys.ALTER_PARTITION), new AlterPartitionRequestData().setBrokerId(2).setBrokerEpoch(102). - setTopics(asList(new AlterPartitionRequestData.TopicData().setName("foo"). + setTopics(asList(new AlterPartitionRequestData.TopicData().setTopicId(fooId). setPartitions(asList(new AlterPartitionRequestData.PartitionData(). setPartitionIndex(2).setPartitionEpoch(0). setLeaderEpoch(0).setNewIsr(asList(0, 2, 1))))))); assertEquals(new AlterPartitionResponseData().setTopics(asList( - new AlterPartitionResponseData.TopicData().setName("foo").setPartitions(asList( + new AlterPartitionResponseData.TopicData().setTopicId(fooId).setPartitions(asList( new AlterPartitionResponseData.PartitionData(). setPartitionIndex(2). setLeaderId(2). diff --git a/server-common/src/main/java/org/apache/kafka/server/common/MetadataVersion.java b/server-common/src/main/java/org/apache/kafka/server/common/MetadataVersion.java index aa1a416c2943..55916470cbd8 100644 --- a/server-common/src/main/java/org/apache/kafka/server/common/MetadataVersion.java +++ b/server-common/src/main/java/org/apache/kafka/server/common/MetadataVersion.java @@ -43,7 +43,6 @@ * released version, they can use "0.10.0" when upgrading to the 0.10.0 release. */ public enum MetadataVersion { - UNINITIALIZED(-1, "0.0", ""), IBP_0_8_0(-1, "0.8.0", ""), IBP_0_8_1(-1, "0.8.1", ""), @@ -140,32 +139,36 @@ public enum MetadataVersion { IBP_2_8_IV1(-1, "2.8", "IV1"), // Introduce AllocateProducerIds (KIP-730) - IBP_3_0_IV0(1, "3.0", "IV0", true), + IBP_3_0_IV0(-1, "3.0", "IV0"), // Introduce ListOffsets V7 which supports listing offsets by max timestamp (KIP-734) // Assume message format version is 3.0 (KIP-724) - IBP_3_0_IV1(2, "3.0", "IV1", false), + IBP_3_0_IV1(1, "3.0", "IV1", true), // Adds topic IDs to Fetch requests/responses (KIP-516) - IBP_3_1_IV0(3, "3.1", "IV0", false), + IBP_3_1_IV0(2, "3.1", "IV0", false), // Support for leader recovery for unclean leader election (KIP-704) - IBP_3_2_IV0(4, "3.2", "IV0", true), + IBP_3_2_IV0(3, "3.2", "IV0", true), // Support for metadata.version feature flag and Removes min_version_level from the finalized version range that is written to ZooKeeper (KIP-778) - IBP_3_3_IV0(5, "3.3", "IV0", false), + IBP_3_3_IV0(4, "3.3", "IV0", false), // Support NoopRecord for the cluster metadata log (KIP-835) - IBP_3_3_IV1(6, "3.3", "IV1", true), + IBP_3_3_IV1(5, "3.3", "IV1", true), // In KRaft mode, use BrokerRegistrationChangeRecord instead of UnfenceBrokerRecord and FenceBrokerRecord. - IBP_3_3_IV2(7, "3.3", "IV2", true), + IBP_3_3_IV2(6, "3.3", "IV2", true), // Adds InControlledShutdown state to RegisterBrokerRecord and BrokerRegistrationChangeRecord (KIP-841). - IBP_3_3_IV3(8, "3.3", "IV3", true); + IBP_3_3_IV3(7, "3.3", "IV3", true); + // NOTE: update the default version in @ClusterTest annotation to point to the latest version + public static final String FEATURE_NAME = "metadata.version"; + public static final MetadataVersion MINIMUM_KRAFT_VERSION = IBP_3_0_IV1; + public static final MetadataVersion[] VERSIONS; private final short featureLevel; @@ -258,12 +261,73 @@ public short registerBrokerRecordVersion() { } } + public short fetchRequestVersion() { + if (this.isAtLeast(IBP_3_1_IV0)) { + return 13; + } else if (this.isAtLeast(IBP_2_7_IV1)) { + return 12; + } else if (this.isAtLeast(IBP_2_3_IV1)) { + return 11; + } else if (this.isAtLeast(IBP_2_1_IV2)) { + return 10; + } else if (this.isAtLeast(IBP_2_0_IV1)) { + return 8; + } else if (this.isAtLeast(IBP_1_1_IV0)) { + return 7; + } else if (this.isAtLeast(IBP_0_11_0_IV1)) { + return 5; + } else if (this.isAtLeast(IBP_0_11_0_IV0)) { + return 4; + } else if (this.isAtLeast(IBP_0_10_1_IV1)) { + return 3; + } else if (this.isAtLeast(IBP_0_10_0_IV0)) { + return 2; + } else if (this.isAtLeast(IBP_0_9_0)) { + return 1; + } else { + return 0; + } + } + + public short offsetForLeaderEpochRequestVersion() { + if (this.isAtLeast(IBP_2_8_IV0)) { + return 4; + } else if (this.isAtLeast(IBP_2_3_IV1)) { + return 3; + } else if (this.isAtLeast(IBP_2_1_IV1)) { + return 2; + } else if (this.isAtLeast(IBP_2_0_IV0)) { + return 1; + } else { + return 0; + } + } + + public short listOffsetRequestVersion() { + if (this.isAtLeast(IBP_3_0_IV1)) { + return 7; + } else if (this.isAtLeast(IBP_2_8_IV0)) { + return 6; + } else if (this.isAtLeast(IBP_2_2_IV1)) { + return 5; + } else if (this.isAtLeast(IBP_2_1_IV1)) { + return 4; + } else if (this.isAtLeast(IBP_2_0_IV1)) { + return 3; + } else if (this.isAtLeast(IBP_0_11_0_IV0)) { + return 2; + } else if (this.isAtLeast(IBP_0_10_1_IV2)) { + return 1; + } else { + return 0; + } + } + private static final Map IBP_VERSIONS; static { { - // Make a copy of values() and omit UNINITIALIZED MetadataVersion[] enumValues = MetadataVersion.values(); - VERSIONS = Arrays.copyOfRange(enumValues, 1, enumValues.length); + VERSIONS = Arrays.copyOf(enumValues, enumValues.length); IBP_VERSIONS = new HashMap<>(); Map maxInterVersion = new HashMap<>(); @@ -289,8 +353,8 @@ public boolean didMetadataChange() { Optional previous() { int idx = this.ordinal(); - if (idx > 1) { - return Optional.of(VERSIONS[idx - 2]); + if (idx > 0) { + return Optional.of(VERSIONS[idx - 1]); } else { return Optional.empty(); } diff --git a/server-common/src/main/java/org/apache/kafka/server/metrics/KafkaYammerMetrics.java b/server-common/src/main/java/org/apache/kafka/server/metrics/KafkaYammerMetrics.java index e4b7afc3eac9..329083350cdc 100644 --- a/server-common/src/main/java/org/apache/kafka/server/metrics/KafkaYammerMetrics.java +++ b/server-common/src/main/java/org/apache/kafka/server/metrics/KafkaYammerMetrics.java @@ -22,6 +22,7 @@ import org.apache.kafka.common.Reconfigurable; import org.apache.kafka.common.config.ConfigException; import org.apache.kafka.common.metrics.JmxReporter; +import org.apache.kafka.common.utils.Exit; import org.apache.kafka.common.utils.Sanitizer; import java.util.Iterator; @@ -57,7 +58,7 @@ public static MetricsRegistry defaultRegistry() { private KafkaYammerMetrics() { jmxReporter.start(); - Runtime.getRuntime().addShutdownHook(new Thread(jmxReporter::shutdown)); + Exit.addShutdownHook("kafka-jmx-shutdown-hook", jmxReporter::shutdown); } @Override diff --git a/server-common/src/test/java/org/apache/kafka/server/common/MetadataVersionTest.java b/server-common/src/test/java/org/apache/kafka/server/common/MetadataVersionTest.java index 2f96d5fd04fb..99f9cc3515ce 100644 --- a/server-common/src/test/java/org/apache/kafka/server/common/MetadataVersionTest.java +++ b/server-common/src/test/java/org/apache/kafka/server/common/MetadataVersionTest.java @@ -76,7 +76,7 @@ class MetadataVersionTest { @Test public void testFeatureLevel() { MetadataVersion[] metadataVersions = MetadataVersion.VERSIONS; - int firstFeatureLevelIndex = Arrays.asList(metadataVersions).indexOf(IBP_3_0_IV0); + int firstFeatureLevelIndex = Arrays.asList(metadataVersions).indexOf(MetadataVersion.MINIMUM_KRAFT_VERSION); for (int i = 0; i < firstFeatureLevelIndex; i++) { assertTrue(metadataVersions[i].featureLevel() < 0); } @@ -287,7 +287,7 @@ public void testVersion() { public void testPrevious() { for (int i = 1; i < MetadataVersion.VERSIONS.length - 2; i++) { MetadataVersion version = MetadataVersion.VERSIONS[i]; - assertTrue(version.previous().isPresent()); + assertTrue(version.previous().isPresent(), version.toString()); assertEquals(MetadataVersion.VERSIONS[i - 1], version.previous().get()); } } @@ -317,8 +317,8 @@ public void testKRaftVersions() { } for (MetadataVersion metadataVersion : MetadataVersion.VERSIONS) { - if (metadataVersion.isAtLeast(IBP_3_0_IV0)) { - assertTrue(metadataVersion.isKRaftSupported()); + if (metadataVersion.isAtLeast(IBP_3_0_IV1)) { + assertTrue(metadataVersion.isKRaftSupported(), metadataVersion.toString()); } else { assertFalse(metadataVersion.isKRaftSupported()); } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ClientUtils.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ClientUtils.java index ea48c6461732..1177e29d8259 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ClientUtils.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ClientUtils.java @@ -174,7 +174,7 @@ public static String extractThreadId(final String fullThreadName) { public static long producerRecordSizeInBytes(final ProducerRecord record) { return recordSizeInBytes( - record.key().length, + record.key() == null ? 0 : record.key().length, record.value() == null ? 0 : record.value().length, record.topic(), record.headers() @@ -190,10 +190,10 @@ public static long consumerRecordSizeInBytes(final ConsumerRecord 12 bytes - new RecordHeader("h2", "headerVal2".getBytes()) - )); // 2 + 10 --> 12 bytes + new RecordHeader("h2", "headerVal2".getBytes()) // 2 + 10 --> 12 bytes + )); private static final int HEADERS_BYTES = 24; + // 20 bytes private static final int RECORD_METADATA_BYTES = 8 + // timestamp 8 + // offset @@ -86,6 +87,14 @@ public class ClientUtilsTest { HEADERS_BYTES + RECORD_METADATA_BYTES; + // 54 bytes + private static final long NULL_KEY_SIZE_IN_BYTES = + VALUE_BYTES + + TOPIC_BYTES + + HEADERS_BYTES + + RECORD_METADATA_BYTES; + + // 52 bytes private static final long TOMBSTONE_SIZE_IN_BYTES = KEY_BYTES + TOPIC_BYTES + @@ -202,6 +211,37 @@ public void shouldComputeSizeInBytesForProducerRecord() { assertThat(producerRecordSizeInBytes(record), equalTo(SIZE_IN_BYTES)); } + @Test + public void shouldComputeSizeInBytesForConsumerRecordWithNullKey() { + final ConsumerRecord record = new ConsumerRecord<>( + TOPIC, + 1, + 0, + 0L, + TimestampType.CREATE_TIME, + 0, + 5, + null, + VALUE, + HEADERS, + Optional.empty() + ); + assertThat(consumerRecordSizeInBytes(record), equalTo(NULL_KEY_SIZE_IN_BYTES)); + } + + @Test + public void shouldComputeSizeInBytesForProducerRecordWithNullKey() { + final ProducerRecord record = new ProducerRecord<>( + TOPIC, + 1, + 0L, + null, + VALUE, + HEADERS + ); + assertThat(producerRecordSizeInBytes(record), equalTo(NULL_KEY_SIZE_IN_BYTES)); + } + @Test public void shouldComputeSizeInBytesForConsumerRecordWithNullValue() { final ConsumerRecord record = new ConsumerRecord<>( diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/metrics/StreamsMetricsImplTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/metrics/StreamsMetricsImplTest.java index b8d3d92e6257..5ec834a11f87 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/metrics/StreamsMetricsImplTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/metrics/StreamsMetricsImplTest.java @@ -45,6 +45,7 @@ import java.util.List; import java.util.Map; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicReference; import static org.apache.kafka.common.utils.Utils.mkEntry; import static org.apache.kafka.common.utils.Utils.mkMap; @@ -84,6 +85,8 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertThrows; import static org.junit.Assert.assertTrue; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertNotNull; import static org.powermock.api.easymock.PowerMock.createMock; @RunWith(PowerMockRunner.class) @@ -497,6 +500,17 @@ public void shouldAddNewStoreLevelMutableMetric() { verify(metrics); } + @Test + public void shouldCreateNewStoreLevelMutableMetric() { + final MetricName metricName = + new MetricName(METRIC_NAME1, STATE_STORE_LEVEL_GROUP, DESCRIPTION1, STORE_LEVEL_TAG_MAP); + final MetricConfig metricConfig = new MetricConfig().recordLevel(INFO_RECORDING_LEVEL); + final Metrics metrics = new Metrics(metricConfig); + assertNull(metrics.metric(metricName)); + metrics.addMetricIfAbsent(metricName, metricConfig, VALUE_PROVIDER); + assertNotNull(metrics.metric(metricName)); + } + @Test public void shouldNotAddStoreLevelMutableMetricIfAlreadyExists() { final Metrics metrics = mock(Metrics.class); @@ -521,6 +535,38 @@ public void shouldNotAddStoreLevelMutableMetricIfAlreadyExists() { verify(metrics); } + @Test + public void shouldReturnSameMetricIfAlreadyCreated() { + final MetricName metricName = + new MetricName(METRIC_NAME1, STATE_STORE_LEVEL_GROUP, DESCRIPTION1, STORE_LEVEL_TAG_MAP); + final MetricConfig metricConfig = new MetricConfig().recordLevel(INFO_RECORDING_LEVEL); + final Metrics metrics = new Metrics(metricConfig); + assertNull(metrics.metric(metricName)); + final KafkaMetric kafkaMetric = metrics.addMetricIfAbsent(metricName, metricConfig, VALUE_PROVIDER); + assertEquals(kafkaMetric, metrics.addMetricIfAbsent(metricName, metricConfig, VALUE_PROVIDER)); + } + + @Test + public void shouldCreateMetricOnceDuringConcurrentMetricCreationRequest() throws InterruptedException { + final MetricName metricName = + new MetricName(METRIC_NAME1, STATE_STORE_LEVEL_GROUP, DESCRIPTION1, STORE_LEVEL_TAG_MAP); + final MetricConfig metricConfig = new MetricConfig().recordLevel(INFO_RECORDING_LEVEL); + final Metrics metrics = new Metrics(metricConfig); + assertNull(metrics.metric(metricName)); + final AtomicReference metricCreatedViaThread1 = new AtomicReference<>(); + final AtomicReference metricCreatedViaThread2 = new AtomicReference<>(); + + final Thread thread1 = new Thread(() -> metricCreatedViaThread1.set(metrics.addMetricIfAbsent(metricName, metricConfig, VALUE_PROVIDER))); + final Thread thread2 = new Thread(() -> metricCreatedViaThread2.set(metrics.addMetricIfAbsent(metricName, metricConfig, VALUE_PROVIDER))); + + thread1.start(); + thread2.start(); + + thread1.join(); + thread2.join(); + assertEquals(metricCreatedViaThread1.get(), metricCreatedViaThread2.get()); + } + @Test public void shouldRemoveStateStoreLevelSensors() { final Metrics metrics = niceMock(Metrics.class); diff --git a/streams/test-utils/src/test/java/org/apache/kafka/streams/TestTopicsTest.java b/streams/test-utils/src/test/java/org/apache/kafka/streams/TestTopicsTest.java index f39b2b554a63..84b8c5ebd602 100644 --- a/streams/test-utils/src/test/java/org/apache/kafka/streams/TestTopicsTest.java +++ b/streams/test-utils/src/test/java/org/apache/kafka/streams/TestTopicsTest.java @@ -31,8 +31,6 @@ import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import java.time.Duration; import java.time.Instant; @@ -55,8 +53,6 @@ import static org.junit.jupiter.api.Assertions.assertThrows; public class TestTopicsTest { - private static final Logger log = LoggerFactory.getLogger(TestTopicsTest.class); - private final static String INPUT_TOPIC = "input"; private final static String OUTPUT_TOPIC = "output1"; private final static String INPUT_TOPIC_MAP = OUTPUT_TOPIC; @@ -171,15 +167,13 @@ public void testKeyValuesToMap() { } @Test - public void testPipeInputWithNullKey() { + public void testKeyValuesToMapWithNull() { final TestInputTopic inputTopic = testDriver.createInputTopic(INPUT_TOPIC, longSerde.serializer(), stringSerde.serializer()); final TestOutputTopic outputTopic = testDriver.createOutputTopic(OUTPUT_TOPIC, longSerde.deserializer(), stringSerde.deserializer()); - final StreamsException exception = assertThrows(StreamsException.class, () -> inputTopic.pipeInput("value")); - assertThat(exception.getCause() instanceof NullPointerException, is(true)); - assertThat(outputTopic.readKeyValuesToMap().isEmpty(), is(true)); - + inputTopic.pipeInput("value"); + assertThrows(IllegalStateException.class, outputTopic::readKeyValuesToMap); } @Test