From 9543310f9ffc306737bc0498904ad43e6db43524 Mon Sep 17 00:00:00 2001 From: Antoine Pourchet Date: Tue, 11 Jun 2024 14:16:05 -0600 Subject: [PATCH 1/5] KAFKA-15045: (KIP-924 pt. 22) More TaskAssignmentUtils tests Also moved the assignment validation test from StreamsPartitionAssignorTest to TaskAssignmentUtilsTest. --- .../assignors/StickyTaskAssignor.java | 2 + .../StreamsPartitionAssignorTest.java | 126 ---------- .../assignment/TaskAssignmentUtilsTest.java | 231 +++++++++++++++++- 3 files changed, 232 insertions(+), 127 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/assignment/assignors/StickyTaskAssignor.java b/streams/src/main/java/org/apache/kafka/streams/processor/assignment/assignors/StickyTaskAssignor.java index 827c9138c996a..31cf0d13cbd59 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/assignment/assignors/StickyTaskAssignor.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/assignment/assignors/StickyTaskAssignor.java @@ -105,6 +105,8 @@ private void optimizeActive(final ApplicationState applicationState, .filter(task -> !task.isStateful()) .map(TaskInfo::id) .collect(Collectors.toSet()); + + // TODO: We need to use stateless traffic costs and non overlap costs here. final Map optimizedAssignmentsForAllTasks = TaskAssignmentUtils.optimizeRackAwareActiveTasks( applicationState, optimizedAssignmentsForStatefulTasks, new TreeSet<>(statelessTasks)); diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignorTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignorTest.java index 9c7338f748dcb..43c07ee3c2811 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignorTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignorTest.java @@ -18,7 +18,6 @@ import java.util.Arrays; import java.util.Optional; -import java.util.UUID; import java.util.concurrent.TimeUnit; import org.apache.kafka.clients.admin.Admin; import org.apache.kafka.clients.admin.AdminClient; @@ -60,20 +59,12 @@ import org.apache.kafka.streams.kstream.internals.InternalStreamsBuilder; import org.apache.kafka.streams.kstream.internals.MaterializedInternal; import org.apache.kafka.streams.processor.TaskId; -import org.apache.kafka.streams.processor.assignment.ApplicationState; -import org.apache.kafka.streams.processor.assignment.AssignmentConfigs; -import org.apache.kafka.streams.processor.assignment.KafkaStreamsAssignment; import org.apache.kafka.streams.processor.assignment.ProcessId; -import org.apache.kafka.streams.processor.assignment.TaskAssignmentUtils; -import org.apache.kafka.streams.processor.assignment.TaskInfo; import org.apache.kafka.streams.processor.internals.TopologyMetadata.Subtopology; import org.apache.kafka.streams.processor.internals.assignment.AssignmentInfo; import org.apache.kafka.streams.processor.internals.assignment.AssignorConfiguration; import org.apache.kafka.streams.processor.internals.assignment.AssignorError; import org.apache.kafka.streams.processor.internals.assignment.ClientState; -import org.apache.kafka.streams.processor.internals.assignment.DefaultApplicationState; -import org.apache.kafka.streams.processor.internals.assignment.DefaultTaskInfo; -import org.apache.kafka.streams.processor.internals.assignment.DefaultTaskTopicPartition; import org.apache.kafka.streams.processor.internals.assignment.FallbackPriorTaskAssignor; import org.apache.kafka.streams.processor.internals.assignment.HighAvailabilityTaskAssignor; import org.apache.kafka.streams.processor.internals.assignment.ReferenceContainer; @@ -2616,123 +2607,6 @@ public void testClientTags() { assertEquals(clientTags, partitionAssignor.clientTags()); } - @Test - public void testValidateTaskAssignment() { - createDefaultMockTaskManager(); - configureDefaultPartitionAssignor(); - - final StreamsConfig streamsConfig = new StreamsConfig(configProps()); - final AssignmentConfigs assignmentConfigs = AssignmentConfigs.of(streamsConfig); - final Set tasks = mkSet( - new DefaultTaskInfo( - new TaskId(1, 1), - false, - mkSet(), - mkSet( - new DefaultTaskTopicPartition( - new TopicPartition("t1", 1), - true, - false, - () -> { } - ) - ) - ) - ); - - final ProcessId clientUuid1 = new ProcessId(UUID.randomUUID()); - final ProcessId clientUuid2 = new ProcessId(UUID.randomUUID()); - final Map clients = mkMap( - mkEntry(clientUuid1, new StreamsPartitionAssignor.ClientMetadata(clientUuid1, "endpoint1:80", mkMap(), Optional.empty())), - mkEntry(clientUuid2, new StreamsPartitionAssignor.ClientMetadata(clientUuid1, "endpoint2:80", mkMap(), Optional.empty())) - ); - final ApplicationState applicationState = new DefaultApplicationState( - assignmentConfigs, - tasks.stream().collect(Collectors.toMap( - TaskInfo::id, - t -> t - )), - clients - ); - - // **** - final org.apache.kafka.streams.processor.assignment.TaskAssignor.TaskAssignment noError = new org.apache.kafka.streams.processor.assignment.TaskAssignor.TaskAssignment( - mkSet( - KafkaStreamsAssignment.of(clientUuid1, mkSet( - new KafkaStreamsAssignment.AssignedTask( - new TaskId(1, 1), KafkaStreamsAssignment.AssignedTask.Type.ACTIVE - ) - )), - KafkaStreamsAssignment.of(clientUuid2, mkSet()) - ) - ); - org.apache.kafka.streams.processor.assignment.TaskAssignor.AssignmentError error = TaskAssignmentUtils.validateTaskAssignment(applicationState, noError); - assertEquals(org.apache.kafka.streams.processor.assignment.TaskAssignor.AssignmentError.NONE, error); - - // **** - final org.apache.kafka.streams.processor.assignment.TaskAssignor.TaskAssignment missingProcessId = new org.apache.kafka.streams.processor.assignment.TaskAssignor.TaskAssignment( - mkSet( - KafkaStreamsAssignment.of(clientUuid1, mkSet( - new KafkaStreamsAssignment.AssignedTask( - new TaskId(1, 1), KafkaStreamsAssignment.AssignedTask.Type.ACTIVE - ) - )) - ) - ); - error = TaskAssignmentUtils.validateTaskAssignment(applicationState, missingProcessId); - assertEquals(org.apache.kafka.streams.processor.assignment.TaskAssignor.AssignmentError.MISSING_PROCESS_ID, error); - - // **** - final org.apache.kafka.streams.processor.assignment.TaskAssignor.TaskAssignment unknownProcessId = new org.apache.kafka.streams.processor.assignment.TaskAssignor.TaskAssignment( - mkSet( - KafkaStreamsAssignment.of(clientUuid1, mkSet( - new KafkaStreamsAssignment.AssignedTask( - new TaskId(1, 1), KafkaStreamsAssignment.AssignedTask.Type.ACTIVE - ) - )), - KafkaStreamsAssignment.of(clientUuid2, mkSet()), - KafkaStreamsAssignment.of(new ProcessId(UUID.randomUUID()), mkSet()) - ) - ); - error = TaskAssignmentUtils.validateTaskAssignment(applicationState, unknownProcessId); - assertEquals(org.apache.kafka.streams.processor.assignment.TaskAssignor.AssignmentError.UNKNOWN_PROCESS_ID, error); - - // **** - final org.apache.kafka.streams.processor.assignment.TaskAssignor.TaskAssignment unknownTaskId = new org.apache.kafka.streams.processor.assignment.TaskAssignor.TaskAssignment( - mkSet( - KafkaStreamsAssignment.of(clientUuid1, mkSet( - new KafkaStreamsAssignment.AssignedTask( - new TaskId(1, 1), KafkaStreamsAssignment.AssignedTask.Type.ACTIVE - ) - )), - KafkaStreamsAssignment.of(clientUuid2, mkSet( - new KafkaStreamsAssignment.AssignedTask( - new TaskId(13, 13), KafkaStreamsAssignment.AssignedTask.Type.ACTIVE - ) - )) - ) - ); - error = TaskAssignmentUtils.validateTaskAssignment(applicationState, unknownTaskId); - assertEquals(org.apache.kafka.streams.processor.assignment.TaskAssignor.AssignmentError.UNKNOWN_TASK_ID, error); - - // **** - final org.apache.kafka.streams.processor.assignment.TaskAssignor.TaskAssignment activeTaskDuplicated = new org.apache.kafka.streams.processor.assignment.TaskAssignor.TaskAssignment( - mkSet( - KafkaStreamsAssignment.of(clientUuid1, mkSet( - new KafkaStreamsAssignment.AssignedTask( - new TaskId(1, 1), KafkaStreamsAssignment.AssignedTask.Type.ACTIVE - ) - )), - KafkaStreamsAssignment.of(clientUuid2, mkSet( - new KafkaStreamsAssignment.AssignedTask( - new TaskId(1, 1), KafkaStreamsAssignment.AssignedTask.Type.ACTIVE - ) - )) - ) - ); - error = TaskAssignmentUtils.validateTaskAssignment(applicationState, activeTaskDuplicated); - assertEquals(org.apache.kafka.streams.processor.assignment.TaskAssignor.AssignmentError.ACTIVE_TASK_ASSIGNED_MULTIPLE_TIMES, error); - } - private static class CorruptedInternalTopologyBuilder extends InternalTopologyBuilder { private Map corruptedTopicGroups; diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/TaskAssignmentUtilsTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/TaskAssignmentUtilsTest.java index 7bcf386ab2ab1..7fdbd468cbb97 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/TaskAssignmentUtilsTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/TaskAssignmentUtilsTest.java @@ -25,10 +25,12 @@ import static org.apache.kafka.streams.processor.internals.assignment.AssignmentTestUtils.TASK_0_3; import static org.apache.kafka.streams.processor.internals.assignment.AssignmentTestUtils.TASK_0_4; import static org.apache.kafka.streams.processor.internals.assignment.AssignmentTestUtils.TASK_0_5; +import static org.apache.kafka.streams.processor.internals.assignment.AssignmentTestUtils.TASK_1_1; import static org.apache.kafka.streams.processor.internals.assignment.AssignmentTestUtils.processIdForInt; import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.MatcherAssert.assertThat; +import java.util.Arrays; import java.util.Collections; import java.util.HashSet; import java.util.List; @@ -38,6 +40,7 @@ import java.util.Set; import java.util.TreeMap; import java.util.TreeSet; +import java.util.UUID; import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; import org.apache.kafka.common.TopicPartition; @@ -50,10 +53,11 @@ import org.apache.kafka.streams.processor.assignment.KafkaStreamsState; import org.apache.kafka.streams.processor.assignment.ProcessId; import org.apache.kafka.streams.processor.assignment.TaskAssignmentUtils; +import org.apache.kafka.streams.processor.assignment.TaskAssignor; import org.apache.kafka.streams.processor.assignment.TaskInfo; import org.apache.kafka.streams.processor.assignment.TaskTopicPartition; import org.junit.Rule; -import org.junit.Test; +import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.ValueSource; import org.junit.rules.Timeout; @@ -207,6 +211,219 @@ public void shouldAssignStandbyTasksByClientLoad() { assertThat(assignments.get(processId(5)).tasks().keySet(), equalTo(mkSet(TASK_0_0))); } + @ParameterizedTest + @ValueSource(strings = { + StreamsConfig.RACK_AWARE_ASSIGNMENT_STRATEGY_MIN_TRAFFIC, + StreamsConfig.RACK_AWARE_ASSIGNMENT_STRATEGY_BALANCE_SUBTOPOLOGY, + }) + public void shouldNotViolateClientTagsAssignmentDuringStandbyOptimization(final String strategy) { + final AssignmentConfigs assignmentConfigs = defaultAssignmentConfigs( + strategy, 100, 1, 2, Collections.singletonList("az")); + final Map tasks = mkMap( + mkTaskInfo(TASK_0_0, true, mkSet("r1")), + mkTaskInfo(TASK_0_1, true, mkSet("r1")) + ); + final Map kafkaStreamsStates = mkMap( + mkStreamState(1, 2, Optional.of("r1"), mkSet(), mkSet(), mkMap( + mkEntry("az", "1") + )), + mkStreamState(2, 2, Optional.of("r1"), mkSet(), mkSet(), mkMap( + mkEntry("az", "2") + )), + mkStreamState(3, 2, Optional.of("r1"), mkSet(), mkSet(), mkMap( + mkEntry("az", "3") + )), + mkStreamState(4, 2, Optional.of("r1"), mkSet(), mkSet(), mkMap( + mkEntry("az", "2") + )) + ); + final ApplicationState applicationState = new TestApplicationState( + assignmentConfigs, kafkaStreamsStates, tasks); + + final Map assignments = mkMap( + mkAssignment( + 1, + new AssignedTask(TASK_0_0, AssignedTask.Type.ACTIVE), + new AssignedTask(TASK_0_1, AssignedTask.Type.STANDBY) + ), + mkAssignment( + 2, + new AssignedTask(TASK_0_0, AssignedTask.Type.STANDBY), + new AssignedTask(TASK_0_1, AssignedTask.Type.ACTIVE) + ), + mkAssignment( + 3, + new AssignedTask(TASK_0_0, AssignedTask.Type.STANDBY), + new AssignedTask(TASK_0_1, AssignedTask.Type.STANDBY) + ), + mkAssignment(4) + ); + + TaskAssignmentUtils.optimizeRackAwareStandbyTasks(applicationState, assignments); + assertThat(assignments.size(), equalTo(4)); + assertThat(assignments.get(processId(1)).tasks().keySet(), equalTo(mkSet(TASK_0_0, TASK_0_1))); + assertThat(assignments.get(processId(2)).tasks().keySet(), equalTo(mkSet(TASK_0_0, TASK_0_1))); + assertThat(assignments.get(processId(3)).tasks().keySet(), equalTo(mkSet(TASK_0_0, TASK_0_1))); + assertThat(assignments.get(processId(4)).tasks().keySet(), equalTo(mkSet())); + } + + @ParameterizedTest + @ValueSource(strings = { + StreamsConfig.RACK_AWARE_ASSIGNMENT_STRATEGY_MIN_TRAFFIC, + StreamsConfig.RACK_AWARE_ASSIGNMENT_STRATEGY_BALANCE_SUBTOPOLOGY, + }) + public void shouldOptimizeStandbyTasksWithMultipleRacks(final String strategy) { + final AssignmentConfigs assignmentConfigs = defaultAssignmentConfigs( + strategy, 100, 1, 1, Collections.emptyList()); + final Map tasks = mkMap( + mkTaskInfo(TASK_0_0, true, mkSet("rack-1", "rack-2")), + mkTaskInfo(TASK_0_1, true, mkSet("rack-2", "rack-3")), + mkTaskInfo(TASK_0_2, true, mkSet("rack-3", "rack-4")) + ); + final Map kafkaStreamsStates = mkMap( + mkStreamState(1, 2, Optional.of("rack-1")), + mkStreamState(2, 2, Optional.of("rack-2")), + mkStreamState(3, 2, Optional.of("rack-3")) + ); + final ApplicationState applicationState = new TestApplicationState( + assignmentConfigs, kafkaStreamsStates, tasks); + + final Map assignments = mkMap( + mkAssignment(AssignedTask.Type.ACTIVE, 1, TASK_0_0), + mkAssignment(AssignedTask.Type.ACTIVE, 2, TASK_0_1), + mkAssignment(AssignedTask.Type.ACTIVE, 3, TASK_0_2) + ); + + TaskAssignmentUtils.optimizeRackAwareActiveTasks(applicationState, assignments, + new TreeSet<>(mkSet(TASK_0_0, TASK_0_1, TASK_0_2))); + assertThat(assignments.size(), equalTo(3)); + assertThat(assignments.get(processId(1)).tasks().keySet(), equalTo(mkSet(TASK_0_0))); + assertThat(assignments.get(processId(2)).tasks().keySet(), equalTo(mkSet(TASK_0_1))); + assertThat(assignments.get(processId(3)).tasks().keySet(), equalTo(mkSet(TASK_0_2))); + } + + @Test + public void shouldCorrectlyReturnIdentityAssignment() { + final AssignmentConfigs assignmentConfigs = defaultAssignmentConfigs( + StreamsConfig.RACK_AWARE_ASSIGNMENT_STRATEGY_NONE, 100, 1, 1, Collections.emptyList()); + final Map tasks = mkMap( + mkTaskInfo(TASK_0_0, true), + mkTaskInfo(TASK_0_1, true), + mkTaskInfo(TASK_0_2, true) + ); + final Map kafkaStreamsStates = mkMap( + mkStreamState(1, 5, Optional.empty(), mkSet(TASK_0_0, TASK_0_1, TASK_0_2), mkSet()), + mkStreamState(2, 5, Optional.empty(), mkSet(), mkSet(TASK_0_0, TASK_0_1, TASK_0_2)), + mkStreamState(3, 5, Optional.empty(), mkSet(), mkSet()), + mkStreamState(4, 5, Optional.empty(), mkSet(), mkSet()), + mkStreamState(5, 5, Optional.empty(), mkSet(), mkSet()) + ); + final ApplicationState applicationState = new TestApplicationState( + assignmentConfigs, kafkaStreamsStates, tasks); + + + final Map assignments = TaskAssignmentUtils.identityAssignment(applicationState); + assertThat(assignments.size(), equalTo(5)); + assertThat(assignments.get(processId(1)).tasks().keySet(), equalTo(mkSet(TASK_0_0, TASK_0_1, TASK_0_2))); + assertThat(assignments.get(processId(2)).tasks().keySet(), equalTo(mkSet(TASK_0_0, TASK_0_1, TASK_0_2))); + assertThat(assignments.get(processId(3)).tasks().keySet(), equalTo(mkSet())); + assertThat(assignments.get(processId(4)).tasks().keySet(), equalTo(mkSet())); + assertThat(assignments.get(processId(5)).tasks().keySet(), equalTo(mkSet())); + } + + @Test + public void testValidateTaskAssignment() { + final AssignmentConfigs assignmentConfigs = defaultAssignmentConfigs( + StreamsConfig.RACK_AWARE_ASSIGNMENT_STRATEGY_NONE, 100, 1, 1, Collections.emptyList()); + final Map tasks = mkMap( + mkTaskInfo(TASK_1_1, false) + ); + final Map kafkaStreamsStates = mkMap( + mkStreamState(1, 5, Optional.empty()), + mkStreamState(2, 5, Optional.empty()) + ); + final ApplicationState applicationState = new TestApplicationState( + assignmentConfigs, kafkaStreamsStates, tasks); + + // **** + final org.apache.kafka.streams.processor.assignment.TaskAssignor.TaskAssignment noError = new org.apache.kafka.streams.processor.assignment.TaskAssignor.TaskAssignment( + mkSet( + KafkaStreamsAssignment.of(processId(1), mkSet( + new KafkaStreamsAssignment.AssignedTask( + new TaskId(1, 1), KafkaStreamsAssignment.AssignedTask.Type.ACTIVE + ) + )), + KafkaStreamsAssignment.of(processId(2), mkSet()) + ) + ); + org.apache.kafka.streams.processor.assignment.TaskAssignor.AssignmentError error = TaskAssignmentUtils.validateTaskAssignment(applicationState, noError); + assertThat(error, equalTo(TaskAssignor.AssignmentError.NONE)); + + // **** + final org.apache.kafka.streams.processor.assignment.TaskAssignor.TaskAssignment missingProcessId = new org.apache.kafka.streams.processor.assignment.TaskAssignor.TaskAssignment( + mkSet( + KafkaStreamsAssignment.of(processId(1), mkSet( + new KafkaStreamsAssignment.AssignedTask( + new TaskId(1, 1), KafkaStreamsAssignment.AssignedTask.Type.ACTIVE + ) + )) + ) + ); + error = TaskAssignmentUtils.validateTaskAssignment(applicationState, missingProcessId); + assertThat(error, equalTo(TaskAssignor.AssignmentError.MISSING_PROCESS_ID)); + + // **** + final org.apache.kafka.streams.processor.assignment.TaskAssignor.TaskAssignment unknownProcessId = new org.apache.kafka.streams.processor.assignment.TaskAssignor.TaskAssignment( + mkSet( + KafkaStreamsAssignment.of(processId(1), mkSet( + new KafkaStreamsAssignment.AssignedTask( + new TaskId(1, 1), KafkaStreamsAssignment.AssignedTask.Type.ACTIVE + ) + )), + KafkaStreamsAssignment.of(processId(2), mkSet()), + KafkaStreamsAssignment.of(new ProcessId(UUID.randomUUID()), mkSet()) + ) + ); + error = TaskAssignmentUtils.validateTaskAssignment(applicationState, unknownProcessId); + assertThat(error, equalTo(TaskAssignor.AssignmentError.UNKNOWN_PROCESS_ID)); + + // **** + final org.apache.kafka.streams.processor.assignment.TaskAssignor.TaskAssignment unknownTaskId = new org.apache.kafka.streams.processor.assignment.TaskAssignor.TaskAssignment( + mkSet( + KafkaStreamsAssignment.of(processId(1), mkSet( + new KafkaStreamsAssignment.AssignedTask( + new TaskId(1, 1), KafkaStreamsAssignment.AssignedTask.Type.ACTIVE + ) + )), + KafkaStreamsAssignment.of(processId(2), mkSet( + new KafkaStreamsAssignment.AssignedTask( + new TaskId(13, 13), KafkaStreamsAssignment.AssignedTask.Type.ACTIVE + ) + )) + ) + ); + error = TaskAssignmentUtils.validateTaskAssignment(applicationState, unknownTaskId); + assertThat(error, equalTo(TaskAssignor.AssignmentError.UNKNOWN_TASK_ID)); + + // **** + final org.apache.kafka.streams.processor.assignment.TaskAssignor.TaskAssignment activeTaskDuplicated = new org.apache.kafka.streams.processor.assignment.TaskAssignor.TaskAssignment( + mkSet( + KafkaStreamsAssignment.of(processId(1), mkSet( + new KafkaStreamsAssignment.AssignedTask( + new TaskId(1, 1), KafkaStreamsAssignment.AssignedTask.Type.ACTIVE + ) + )), + KafkaStreamsAssignment.of(processId(2), mkSet( + new KafkaStreamsAssignment.AssignedTask( + new TaskId(1, 1), KafkaStreamsAssignment.AssignedTask.Type.ACTIVE + ) + )) + ) + ); + error = TaskAssignmentUtils.validateTaskAssignment(applicationState, activeTaskDuplicated); + assertThat(error, equalTo(TaskAssignor.AssignmentError.ACTIVE_TASK_ASSIGNED_MULTIPLE_TIMES)); + } + public static class TestApplicationState implements ApplicationState { private final AssignmentConfigs assignmentConfigs; @@ -292,6 +509,18 @@ public static Map.Entry mkAssignment(final As ); } + public static Map.Entry mkAssignment(final int client, + final AssignedTask... tasks) { + final ProcessId processId = processId(client); + return mkEntry( + processId, + KafkaStreamsAssignment.of( + processId, + Arrays.stream(tasks).collect(Collectors.toSet()) + ) + ); + } + public static Map.Entry mkTaskInfo(final TaskId taskId, final boolean isStateful) { return mkTaskInfo(taskId, isStateful, null); } From 4ed82520e5d9696e9578ebfa94cd8cd47440889f Mon Sep 17 00:00:00 2001 From: Antoine Pourchet Date: Tue, 11 Jun 2024 14:23:56 -0600 Subject: [PATCH 2/5] addressed comment --- .../processor/assignment/assignors/StickyTaskAssignor.java | 2 -- 1 file changed, 2 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/assignment/assignors/StickyTaskAssignor.java b/streams/src/main/java/org/apache/kafka/streams/processor/assignment/assignors/StickyTaskAssignor.java index 31cf0d13cbd59..827c9138c996a 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/assignment/assignors/StickyTaskAssignor.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/assignment/assignors/StickyTaskAssignor.java @@ -105,8 +105,6 @@ private void optimizeActive(final ApplicationState applicationState, .filter(task -> !task.isStateful()) .map(TaskInfo::id) .collect(Collectors.toSet()); - - // TODO: We need to use stateless traffic costs and non overlap costs here. final Map optimizedAssignmentsForAllTasks = TaskAssignmentUtils.optimizeRackAwareActiveTasks( applicationState, optimizedAssignmentsForStatefulTasks, new TreeSet<>(statelessTasks)); From 4ba836d638e2dcba8244f04fed488d6b55d21305 Mon Sep 17 00:00:00 2001 From: Antoine Pourchet Date: Wed, 12 Jun 2024 09:56:33 -0600 Subject: [PATCH 3/5] Squashed commit of the following: commit 9368ef81b5cc6ce149979ff916c14709d1f2e491 Author: Gantigmaa Selenge <39860586+tinaselenge@users.noreply.github.com> Date: Wed Jun 12 16:04:24 2024 +0100 KAFKA-16865: Add IncludeTopicAuthorizedOperations option for DescribeTopicPartitionsRequest (#16136) Reviewers: Mickael Maison , Chia-Ping Tsai , Calvin Liu , Andrew Schofield , Apoorv Mittal commit 46eb0814f63236fa3c3532975ba76180485933fb Author: gongxuanzhang Date: Wed Jun 12 22:23:39 2024 +0800 KAFKA-10787 Apply spotless to log4j-appender, trogdor, jmh-benchmarks, examples, shell and generator (#16296) Reviewers: Chia-Ping Tsai commit 79b9c44acdbc8c3618bc61a6268791412593bf74 Author: gongxuanzhang Date: Wed Jun 12 22:19:47 2024 +0800 KAFKA-10787 Apply spotless to connect module (#16299) Reviewers: Chia-Ping Tsai commit b5fb6543a202f826461d5074ef0ea8f6022566e0 Author: Abhijeet Kumar Date: Wed Jun 12 19:47:46 2024 +0530 KAFKA-15265: Dynamic broker configs for remote fetch/copy quotas (#16078) Reviewers: Kamal Chandraprakash, Satish Duggana commit faee6a4385c525c406606195357ca3b114c7a527 Author: Dmitry Werner Date: Wed Jun 12 15:44:11 2024 +0500 MINOR: Use predetermined dir IDs in ReplicationQuotasTest Use predetermined directory IDs instead of Uuid.randomUuid() in ReplicationQuotasTest. Reviewers: Igor Soarez commit 638844f833b165d6f9ca52c173858d26b7254fac Author: David Jacot Date: Wed Jun 12 08:29:50 2024 +0200 KAFKA-16770; [2/2] Coalesce records into bigger batches (#16215) This patch is the continuation of https://github.com/apache/kafka/pull/15964. It introduces the records coalescing to the CoordinatorRuntime. It also introduces a new configuration `group.coordinator.append.linger.ms` which allows administrators to chose the linger time or disable it with zero. The new configuration defaults to 10ms. Reviewers: Jeff Kim , Justine Olshan commit 39ffdea6d321ef3dd5e787aef1b1102c33448c0f Author: Bruno Cadonna Date: Wed Jun 12 07:51:38 2024 +0200 KAFKA-10199: Enable state updater by default (#16107) We have already enabled the state updater by default once. However, we ran into issues that forced us to disable it again. We think that we fixed those issues. So we want to enable the state updater again by default. Reviewers: Lucas Brutschy , Matthias J. Sax commit 0782232fbeb6313a316b930d12508d1d6148f3c9 Author: Antoine Pourchet Date: Tue Jun 11 22:31:43 2024 -0600 KAFKA-15045: (KIP-924 pt. 22) Add RackAwareOptimizationParams and other minor TaskAssignmentUtils changes (#16294) We now provide a way to more easily customize the rack aware optimizations that we provide by way of a configuration class called RackAwareOptimizationParams. We also simplified the APIs for the optimizeXYZ utility functions since they were mutating the inputs anyway. Reviewers: Anna Sophie Blee-Goldman commit 226ac5e8fca181dc6e1241df78da55659e5240bf Author: Murali Basani Date: Wed Jun 12 05:38:50 2024 +0200 KAFKA-16922 Adding unit tests for NewTopic (#16255) Reviewers: Chia-Ping Tsai commit 23fe71d579f84d59ebfe6d5a29e688315cec1285 Author: Abhijeet Kumar Date: Wed Jun 12 06:27:02 2024 +0530 KAFKA-15265: Integrate RLMQuotaManager for throttling copies to remote storage (#15820) - Added the integration of the quota manager to throttle copy requests to the remote storage. Reference KIP-956 - Added unit-tests for the copy throttling logic. Reviewers: Satish Duggana , Luke Chen , Kamal Chandraprakash commit 2fa2c72581de641a8d62b8124209b75f75152b78 Author: Chris Egerton Date: Tue Jun 11 23:15:07 2024 +0200 MINOR: Wait for embedded clusters to start before using them in Connect OffsetsApiIntegrationTest (#16286) Reviewers: Greg Harris --- build.gradle | 16 +- checkstyle/import-control-core.xml | 1 + .../kafka/clients/admin/KafkaAdminClient.java | 8 +- .../common/record/MemoryRecordsBuilder.java | 12 + .../clients/admin/KafkaAdminClientTest.java | 73 ++ .../kafka/clients/admin/NewTopicTest.java | 185 +++++ .../kafka/connect/connector/Connector.java | 2 +- .../org/apache/kafka/connect/data/Struct.java | 1 - .../org/apache/kafka/connect/data/Values.java | 1 + .../kafka/connect/health/AbstractState.java | 4 +- .../kafka/connect/health/ConnectorHealth.java | 4 +- .../connect/sink/ErrantRecordReporter.java | 3 +- .../storage/SimpleHeaderConverter.java | 1 + .../connect/connector/ConnectorTest.java | 6 +- .../kafka/connect/data/ConnectSchemaTest.java | 1 + .../apache/kafka/connect/data/DateTest.java | 1 + .../kafka/connect/data/SchemaBuilderTest.java | 1 + .../connect/data/SchemaProjectorTest.java | 1 + .../apache/kafka/connect/data/TimeTest.java | 1 + .../kafka/connect/data/TimestampTest.java | 1 + .../apache/kafka/connect/data/ValuesTest.java | 1 + .../connect/header/ConnectHeaderTest.java | 1 + .../connect/header/ConnectHeadersTest.java | 1 + .../kafka/connect/sink/SinkConnectorTest.java | 6 +- .../kafka/connect/sink/SinkRecordTest.java | 1 + .../connect/source/SourceConnectorTest.java | 6 +- .../connect/source/SourceRecordTest.java | 1 + .../storage/SimpleHeaderConverterTest.java | 1 + .../connect/storage/StringConverterTest.java | 1 + .../BasicAuthSecurityRestExtension.java | 4 +- .../auth/extension/JaasBasicAuthFilter.java | 20 +- .../extension/PropertyFileLoginModule.java | 1 + .../BasicAuthSecurityRestExtensionTest.java | 7 +- .../extension/JaasBasicAuthFilterTest.java | 14 +- .../connect/file/FileStreamSinkConnector.java | 1 + .../connect/file/FileStreamSinkTask.java | 1 + .../file/FileStreamSourceConnector.java | 1 + .../connect/file/FileStreamSourceTask.java | 17 +- .../file/FileStreamSinkConnectorTest.java | 9 +- .../connect/file/FileStreamSinkTaskTest.java | 2 + .../file/FileStreamSourceConnectorTest.java | 1 + .../file/FileStreamSourceTaskTest.java | 1 + ...ileStreamSinkConnectorIntegrationTest.java | 1 + ...eStreamSourceConnectorIntegrationTest.java | 1 + .../kafka/connect/json/JsonConverter.java | 25 +- .../connect/json/JsonConverterConfig.java | 2 +- .../kafka/connect/json/JsonDeserializer.java | 5 +- .../kafka/connect/json/JsonSerializer.java | 5 +- .../connect/json/JsonConverterConfigTest.java | 1 + .../kafka/connect/json/JsonConverterTest.java | 12 +- .../kafka/connect/mirror/Checkpoint.java | 9 +- .../mirror/DefaultReplicationPolicy.java | 6 +- .../kafka/connect/mirror/Heartbeat.java | 7 +- .../mirror/IdentityReplicationPolicy.java | 4 +- .../kafka/connect/mirror/MirrorClient.java | 20 +- .../connect/mirror/MirrorClientConfig.java | 6 +- .../connect/mirror/RemoteClusterUtils.java | 4 +- .../kafka/connect/mirror/SourceAndTarget.java | 1 - .../connect/mirror/MirrorClientTest.java | 10 +- .../connect/mirror/ReplicationPolicyTest.java | 2 +- .../kafka/connect/mirror/CheckpointStore.java | 1 + .../connect/mirror/ConfigPropertyFilter.java | 1 + .../mirror/DefaultConfigPropertyFilter.java | 2 +- .../connect/mirror/DefaultGroupFilter.java | 2 +- .../connect/mirror/DefaultTopicFilter.java | 2 +- .../kafka/connect/mirror/GroupFilter.java | 1 + .../mirror/MirrorCheckpointConnector.java | 1 + .../mirror/MirrorCheckpointMetrics.java | 14 +- .../connect/mirror/MirrorCheckpointTask.java | 24 +- .../mirror/MirrorCheckpointTaskConfig.java | 7 +- .../connect/mirror/MirrorConnectorConfig.java | 15 +- .../mirror/MirrorHeartbeatConnector.java | 10 +- .../connect/mirror/MirrorHeartbeatTask.java | 10 +- .../kafka/connect/mirror/MirrorHerder.java | 1 + .../kafka/connect/mirror/MirrorMaker.java | 40 +- .../connect/mirror/MirrorMakerConfig.java | 23 +- .../connect/mirror/MirrorSourceConnector.java | 71 +- .../connect/mirror/MirrorSourceTask.java | 28 +- .../mirror/MirrorSourceTaskConfig.java | 6 +- .../kafka/connect/mirror/MirrorUtils.java | 11 +- .../kafka/connect/mirror/OffsetSync.java | 3 +- .../kafka/connect/mirror/OffsetSyncStore.java | 1 + .../kafka/connect/mirror/Scheduler.java | 13 +- .../kafka/connect/mirror/TopicFilter.java | 1 + .../formatters/CheckpointFormatter.java | 4 +- .../mirror/formatters/HeartbeatFormatter.java | 4 +- .../formatters/OffsetSyncFormatter.java | 4 +- .../connect/mirror/rest/MirrorRestServer.java | 1 + .../resources/InternalMirrorResource.java | 3 +- .../connect/mirror/CheckpointStoreTest.java | 5 +- .../mirror/MirrorCheckpointConfigTest.java | 1 + .../mirror/MirrorCheckpointConnectorTest.java | 1 + .../mirror/MirrorCheckpointTaskTest.java | 15 +- .../mirror/MirrorConnectorConfigTest.java | 7 +- .../mirror/MirrorHeartBeatConnectorTest.java | 17 +- .../mirror/MirrorHeartbeatTaskTest.java | 1 + .../connect/mirror/MirrorMakerConfigTest.java | 14 +- .../mirror/MirrorSourceConfigTest.java | 1 + .../mirror/MirrorSourceConnectorTest.java | 46 +- .../mirror/MirrorSourceMetricsTest.java | 1 + .../connect/mirror/MirrorSourceTaskTest.java | 11 +- .../kafka/connect/mirror/MirrorUtilsTest.java | 1 + .../FakeForwardingAdminWithLocalMetadata.java | 1 + .../clients/admin/FakeLocalMetadataStore.java | 1 + .../DedicatedMirrorIntegrationTest.java | 1 + .../IdentityReplicationIntegrationTest.java | 5 +- .../MirrorConnectorsIntegrationBaseTest.java | 28 +- ...rConnectorsIntegrationExactlyOnceTest.java | 1 + .../MirrorConnectorsIntegrationSSLTest.java | 9 +- ...ConnectorsIntegrationTransactionsTest.java | 1 + ...hCustomForwardingAdminIntegrationTest.java | 1 + .../OffsetsApiIntegrationTest.java | 9 + .../kafka/connect/tools/MockConnector.java | 1 + .../kafka/connect/tools/MockSinkTask.java | 1 + .../kafka/connect/tools/MockSourceTask.java | 1 + .../kafka/connect/tools/SchemaSourceTask.java | 1 + .../connect/tools/VerifiableSinkTask.java | 5 +- .../connect/tools/VerifiableSourceTask.java | 12 +- .../apache/kafka/connect/transforms/Cast.java | 1 + .../connect/transforms/ExtractField.java | 2 +- .../kafka/connect/transforms/Filter.java | 4 +- .../kafka/connect/transforms/InsertField.java | 2 +- .../kafka/connect/transforms/MaskField.java | 2 +- .../kafka/connect/transforms/RegexRouter.java | 1 + .../connect/transforms/SetSchemaMetadata.java | 1 + .../transforms/predicates/HasHeaderKey.java | 6 +- .../predicates/RecordIsTombstone.java | 4 +- .../predicates/TopicNameMatches.java | 8 +- .../kafka/connect/transforms/CastTest.java | 1 + .../connect/transforms/DropHeadersTest.java | 2 +- .../connect/transforms/ExtractFieldTest.java | 3 +- .../kafka/connect/transforms/FlattenTest.java | 1 + .../connect/transforms/HeaderFromTest.java | 2 +- .../connect/transforms/HoistFieldTest.java | 1 + .../connect/transforms/InsertFieldTest.java | 1 + .../connect/transforms/InsertHeaderTest.java | 2 +- .../connect/transforms/MaskFieldTest.java | 1 + .../connect/transforms/RegexRouterTest.java | 1 + .../connect/transforms/ReplaceFieldTest.java | 1 + .../transforms/SetSchemaMetadataTest.java | 1 + .../transforms/TimestampConverterTest.java | 3 +- .../transforms/TimestampRouterTest.java | 1 + .../connect/transforms/ValueToKeyTest.java | 1 + .../field/FieldPathNotationTest.java | 1 + .../field/FieldSyntaxVersionTest.java | 1 + .../transforms/field/SingleFieldPathTest.java | 7 +- .../predicates/HasHeaderKeyTest.java | 1 + .../predicates/TopicNameMatchesTest.java | 1 + .../util/NonEmptyListValidatorTest.java | 1 + .../kafka/log/remote/RemoteLogManager.java | 40 + .../scala/kafka/server/BrokerServer.scala | 1 + .../kafka/server/DynamicBrokerConfig.scala | 55 +- .../main/scala/kafka/server/KafkaConfig.scala | 2 + .../log/remote/RemoteLogManagerTest.java | 210 +++++ .../server/DynamicBrokerConfigTest.scala | 96 +++ .../kafka/server/ReplicationQuotasTest.scala | 25 +- .../kafka/message/MessageGenerator.java | 1 + .../apache/kafka/message/CodeBufferTest.java | 2 +- .../kafka/message/MessageGeneratorTest.java | 2 +- .../kafka/message/VersionConditionalTest.java | 2 +- .../apache/kafka/message/VersionsTest.java | 2 +- .../group/GroupCoordinatorConfig.java | 12 + .../group/GroupCoordinatorService.java | 2 +- .../group/runtime/CoordinatorRuntime.java | 672 ++++++++++++---- .../group/GroupCoordinatorConfigTest.java | 3 + .../group/GroupCoordinatorServiceTest.java | 1 + .../group/runtime/CoordinatorRuntimeTest.java | 736 +++++++++++++++++- .../kafka/jmh/acl/AuthorizerBenchmark.java | 7 +- .../StandardAuthorizerUpdateBenchmark.java | 1 + .../assignor/ClientSideAssignorBenchmark.java | 1 + .../assignor/ServerSideAssignorBenchmark.java | 13 +- .../TargetAssignmentBuilderBenchmark.java | 7 +- .../kafka/jmh/cache/LRUCacheBenchmark.java | 4 +- .../jmh/common/FetchRequestBenchmark.java | 2 + .../jmh/common/FetchResponseBenchmark.java | 3 +- ...ImplicitLinkedHashCollectionBenchmark.java | 1 + .../common/ListOffsetRequestBenchmark.java | 2 + .../jmh/common/ProduceRequestBenchmark.java | 2 + .../kafka/jmh/common/TopicBenchmark.java | 1 + .../jmh/connect/JsonConverterBenchmark.java | 1 + .../jmh/connect/ReplaceFieldBenchmark.java | 1 + .../kafka/jmh/connect/ValuesBenchmark.java | 1 + .../consumer/SubscriptionStateBenchmark.java | 1 + .../ReplicaFetcherThreadBenchmark.java | 17 +- .../fetchsession/FetchSessionBenchmark.java | 3 +- .../KRaftMetadataRequestBenchmark.java | 4 +- .../metadata/MetadataRequestBenchmark.java | 7 +- ...opicsImageSingleRecordChangeBenchmark.java | 1 + .../TopicsImageSnapshotLoadBenchmark.java | 1 + .../TopicsImageZonalOutageBenchmark.java | 1 + .../PartitionMakeFollowerBenchmark.java | 7 +- .../UpdateFollowerFetchStateBenchmark.java | 7 +- .../jmh/producer/ProducerRecordBenchmark.java | 1 + .../producer/ProducerRequestBenchmark.java | 1 + .../producer/ProducerResponseBenchmark.java | 1 + .../jmh/record/BaseRecordBatchBenchmark.java | 2 + ...pressedRecordBatchValidationBenchmark.java | 1 + .../record/RecordBatchIterationBenchmark.java | 1 + ...pressedRecordBatchValidationBenchmark.java | 1 + .../kafka/jmh/server/CheckpointBench.java | 16 +- .../jmh/server/PartitionCreationBench.java | 7 +- .../storage/ProducerStateManagerBench.java | 1 + .../timeline/TimelineHashMapBenchmark.java | 1 + .../kafka/jmh/util/ByteUtilsBenchmark.java | 11 +- .../kafka/jmh/util/Crc32CBenchmark.java | 7 +- .../log4jappender/KafkaLog4jAppender.java | 3 +- .../log4jappender/KafkaLog4jAppenderTest.java | 8 +- .../log4jappender/MockKafkaLog4jAppender.java | 1 + .../kafka/server/util/timer/TimerTask.java | 4 + .../apache/kafka/shell/InteractiveShell.java | 1 + .../org/apache/kafka/shell/MetadataShell.java | 9 +- .../shell/command/CatCommandHandler.java | 6 +- .../kafka/shell/command/CdCommandHandler.java | 6 +- .../kafka/shell/command/CommandUtils.java | 1 + .../apache/kafka/shell/command/Commands.java | 6 +- .../shell/command/ExitCommandHandler.java | 6 +- .../shell/command/FindCommandHandler.java | 6 +- .../shell/command/HelpCommandHandler.java | 6 +- .../shell/command/HistoryCommandHandler.java | 6 +- .../kafka/shell/command/LsCommandHandler.java | 6 +- .../shell/command/ManCommandHandler.java | 6 +- .../shell/command/PwdCommandHandler.java | 6 +- .../shell/command/TreeCommandHandler.java | 6 +- .../shell/state/MetadataShellPublisher.java | 1 + .../shell/MetadataShellIntegrationTest.java | 16 +- .../kafka/shell/command/CommandTest.java | 4 +- .../kafka/shell/command/CommandUtilsTest.java | 6 +- .../shell/command/LsCommandHandlerTest.java | 5 +- .../kafka/shell/glob/GlobComponentTest.java | 6 +- .../kafka/shell/glob/GlobVisitorTest.java | 7 +- .../apache/kafka/streams/StreamsConfig.java | 2 +- .../assignment/TaskAssignmentUtils.java | 207 +++-- .../assignors/StickyTaskAssignor.java | 38 +- .../internals/StoreChangelogReaderTest.java | 6 +- .../assignment/TaskAssignmentUtilsTest.java | 16 +- .../org/apache/kafka/trogdor/agent/Agent.java | 17 +- .../kafka/trogdor/agent/AgentClient.java | 21 +- .../trogdor/agent/AgentRestResource.java | 3 +- .../kafka/trogdor/agent/WorkerManager.java | 3 +- .../apache/kafka/trogdor/basic/BasicNode.java | 3 +- .../kafka/trogdor/basic/BasicPlatform.java | 4 +- .../kafka/trogdor/basic/BasicTopology.java | 3 +- .../apache/kafka/trogdor/common/Platform.java | 6 +- .../kafka/trogdor/common/WorkerUtils.java | 1 + .../trogdor/coordinator/Coordinator.java | 10 +- .../coordinator/CoordinatorClient.java | 29 +- .../coordinator/CoordinatorRestResource.java | 7 +- .../trogdor/coordinator/NodeManager.java | 1 + .../trogdor/coordinator/TaskManager.java | 12 +- .../fault/DegradedNetworkFaultSpec.java | 5 +- .../fault/DegradedNetworkFaultWorker.java | 4 +- .../fault/FilesUnreadableFaultSpec.java | 5 +- .../apache/kafka/trogdor/fault/Kibosh.java | 3 +- .../trogdor/fault/KiboshFaultWorker.java | 4 +- .../fault/NetworkPartitionFaultSpec.java | 5 +- .../fault/NetworkPartitionFaultWorker.java | 4 +- .../fault/ProcessStopFaultController.java | 1 + .../trogdor/fault/ProcessStopFaultSpec.java | 5 +- .../trogdor/fault/ProcessStopFaultWorker.java | 4 +- .../kafka/trogdor/rest/CreateTaskRequest.java | 3 +- .../trogdor/rest/CreateWorkerRequest.java | 3 +- .../org/apache/kafka/trogdor/rest/Empty.java | 3 +- .../kafka/trogdor/rest/ErrorResponse.java | 3 +- .../kafka/trogdor/rest/JsonRestServer.java | 5 +- .../trogdor/rest/RestExceptionMapper.java | 6 +- .../apache/kafka/trogdor/rest/TaskDone.java | 3 +- .../kafka/trogdor/rest/TaskPending.java | 3 +- .../kafka/trogdor/rest/TaskRunning.java | 3 +- .../apache/kafka/trogdor/rest/TaskState.java | 3 +- .../kafka/trogdor/rest/TaskStopping.java | 3 +- .../apache/kafka/trogdor/rest/WorkerDone.java | 3 +- .../kafka/trogdor/rest/WorkerReceiving.java | 3 +- .../kafka/trogdor/rest/WorkerRunning.java | 3 +- .../kafka/trogdor/rest/WorkerStarting.java | 3 +- .../kafka/trogdor/rest/WorkerState.java | 5 +- .../kafka/trogdor/rest/WorkerStopping.java | 3 +- .../kafka/trogdor/task/NoOpTaskWorker.java | 4 +- .../apache/kafka/trogdor/task/TaskSpec.java | 3 +- .../workload/ConfigurableProducerSpec.java | 6 +- .../workload/ConfigurableProducerWorker.java | 8 +- .../workload/ConnectionStressSpec.java | 5 +- .../workload/ConnectionStressWorker.java | 8 +- .../workload/ConstantFlushGenerator.java | 5 +- .../workload/ConstantThroughputGenerator.java | 3 +- .../trogdor/workload/ConsumeBenchSpec.java | 12 +- .../trogdor/workload/ConsumeBenchWorker.java | 17 +- .../trogdor/workload/ExternalCommandSpec.java | 8 +- .../workload/ExternalCommandWorker.java | 17 +- .../trogdor/workload/FlushGenerator.java | 3 +- .../workload/GaussianFlushGenerator.java | 6 +- .../workload/GaussianThroughputGenerator.java | 4 +- ...sianTimestampConstantPayloadGenerator.java | 3 +- ...ussianTimestampRandomPayloadGenerator.java | 3 +- .../trogdor/workload/PartitionsSpec.java | 5 +- .../trogdor/workload/ProduceBenchSpec.java | 5 +- .../trogdor/workload/ProduceBenchWorker.java | 8 +- .../trogdor/workload/RandomComponent.java | 1 - .../trogdor/workload/RecordProcessor.java | 4 +- .../trogdor/workload/RoundTripWorker.java | 8 +- .../workload/RoundTripWorkloadSpec.java | 5 +- .../workload/SustainedConnectionSpec.java | 5 +- .../workload/SustainedConnectionWorker.java | 8 +- .../TimeIntervalTransactionsGenerator.java | 3 +- .../TimestampConstantPayloadGenerator.java | 5 +- .../TimestampRandomPayloadGenerator.java | 5 +- .../workload/TimestampRecordProcessor.java | 9 +- .../kafka/trogdor/workload/TopicsSpec.java | 5 +- .../apache/kafka/trogdor/agent/AgentTest.java | 16 +- .../trogdor/basic/BasicPlatformTest.java | 5 +- .../common/CapturingCommandRunner.java | 1 + .../kafka/trogdor/common/ExpectedTasks.java | 6 +- .../trogdor/common/JsonSerializationTest.java | 7 +- .../kafka/trogdor/common/JsonUtilTest.java | 10 +- .../trogdor/common/MiniTrogdorCluster.java | 2 +- .../trogdor/common/StringExpanderTest.java | 6 +- .../trogdor/common/StringFormatterTest.java | 4 +- .../kafka/trogdor/common/TopologyTest.java | 10 +- .../kafka/trogdor/common/WorkerUtilsTest.java | 7 +- .../coordinator/CoordinatorClientTest.java | 6 +- .../trogdor/coordinator/CoordinatorTest.java | 11 +- .../trogdor/rest/RestExceptionMapperTest.java | 12 +- .../kafka/trogdor/task/SampleTaskWorker.java | 3 +- .../kafka/trogdor/task/TaskSpecTest.java | 4 +- .../workload/ConsumeBenchSpecTest.java | 11 +- .../workload/ExternalCommandWorkerTest.java | 14 +- .../kafka/trogdor/workload/HistogramTest.java | 5 +- .../workload/PayloadGeneratorTest.java | 15 +- .../kafka/trogdor/workload/ThrottleTest.java | 10 +- ...TimeIntervalTransactionsGeneratorTest.java | 5 +- .../trogdor/workload/TopicsSpecTest.java | 11 +- 330 files changed, 3110 insertions(+), 1008 deletions(-) create mode 100644 clients/src/test/java/org/apache/kafka/clients/admin/NewTopicTest.java diff --git a/build.gradle b/build.gradle index 337cb12ebccfa..28ec61b5bfdb3 100644 --- a/build.gradle +++ b/build.gradle @@ -202,26 +202,13 @@ def determineCommitId() { } def excludedSpotlessModules = [':clients', - ':connect:api', - ':connect:basic-auth-extension', - ':connect:file', - ':connect:json', - ':connect:mirror', - ':connect:mirror-client', ':connect:runtime', - ':connect:test-plugins', - ':connect:transforms', ':core', - ':examples', - ':generator', ':group-coordinator:group-coordinator-api', // https://github.com/apache/kafka/pull/16198 ':group-coordinator', - ':jmh-benchmarks', - ':log4j-appender', ':metadata', ':raft', ':server', - ':shell', ':storage', ':storage:storage-api', // rename in settings.gradle ':streams', @@ -250,8 +237,7 @@ def excludedSpotlessModules = [':clients', ':streams:upgrade-system-tests-34', ':streams:upgrade-system-tests-35', ':streams:upgrade-system-tests-36', - ':streams:upgrade-system-tests-37', - ':trogdor'] + ':streams:upgrade-system-tests-37'] apply from: file('wrapper.gradle') diff --git a/checkstyle/import-control-core.xml b/checkstyle/import-control-core.xml index 6724ea9bf3ba0..187fc515e526a 100644 --- a/checkstyle/import-control-core.xml +++ b/checkstyle/import-control-core.xml @@ -38,6 +38,7 @@ + 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 d7d525e443171..eca6a8b45c9ed 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 @@ -2245,7 +2245,7 @@ void handleResponse(AbstractResponse abstractResponse) { continue; } - TopicDescription currentTopicDescription = getTopicDescriptionFromDescribeTopicsResponseTopic(topic, nodes); + TopicDescription currentTopicDescription = getTopicDescriptionFromDescribeTopicsResponseTopic(topic, nodes, options.includeAuthorizedOperations()); if (partiallyFinishedTopicDescription != null && partiallyFinishedTopicDescription.name().equals(topicName)) { // Add the partitions for the cursor topic of the previous batch. @@ -2408,14 +2408,16 @@ void handleFailure(Throwable throwable) { private TopicDescription getTopicDescriptionFromDescribeTopicsResponseTopic( DescribeTopicPartitionsResponseTopic topic, - Map nodes + Map nodes, + boolean includeAuthorizedOperations ) { List partitionInfos = topic.partitions(); List partitions = new ArrayList<>(partitionInfos.size()); for (DescribeTopicPartitionsResponsePartition partitionInfo : partitionInfos) { partitions.add(DescribeTopicPartitionsResponse.partitionToTopicPartitionInfo(partitionInfo, nodes)); } - return new TopicDescription(topic.name(), topic.isInternal(), partitions, validAclOperations(topic.topicAuthorizedOperations()), topic.topicId()); + Set authorisedOperations = includeAuthorizedOperations ? validAclOperations(topic.topicAuthorizedOperations()) : null; + return new TopicDescription(topic.name(), topic.isInternal(), partitions, authorisedOperations, topic.topicId()); } private TopicDescription getTopicDescriptionFromCluster(Cluster cluster, String topicName, Uuid topicId, diff --git a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecordsBuilder.java b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecordsBuilder.java index 70f279a6c29bc..b37b1f1ca6851 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecordsBuilder.java +++ b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecordsBuilder.java @@ -870,6 +870,18 @@ public boolean hasRoomFor(long timestamp, ByteBuffer key, ByteBuffer value, Head return this.writeLimit >= estimatedBytesWritten() + recordSize; } + /** + * Check if we have room for a given number of bytes. + */ + public boolean hasRoomFor(int estimatedRecordsSize) { + if (isFull()) return false; + return this.writeLimit >= estimatedBytesWritten() + estimatedRecordsSize; + } + + public int maxAllowedBytes() { + return this.writeLimit - this.batchHeaderSizeInBytes; + } + public boolean isClosed() { return builtRecords != null; } diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java b/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java index f7b27dcb868ad..31c2cc7fcf45e 100644 --- a/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java @@ -1474,6 +1474,7 @@ public void testDescribeTopicsWithDescribeTopicPartitionsApiBasic() { assertEquals(0, topicDescription.partitions().get(0).partition()); assertEquals(1, topicDescription.partitions().get(1).partition()); topicDescription = topicDescriptions.get(topicName1); + assertNull(topicDescription.authorizedOperations()); assertEquals(1, topicDescription.partitions().size()); } catch (Exception e) { fail("describe using DescribeTopics API should not fail", e); @@ -1481,6 +1482,77 @@ public void testDescribeTopicsWithDescribeTopicPartitionsApiBasic() { } } + @Test + public void testDescribeTopicPartitionsApiWithAuthorizedOps() throws ExecutionException, InterruptedException { + try (AdminClientUnitTestEnv env = mockClientEnv()) { + env.kafkaClient().setNodeApiVersions(NodeApiVersions.create()); + String topicName0 = "test-0"; + Uuid topicId = Uuid.randomUuid(); + + int authorisedOperations = Utils.to32BitField(Utils.mkSet(AclOperation.DESCRIBE.code(), AclOperation.ALTER.code())); + env.kafkaClient().prepareResponse( + prepareDescribeClusterResponse(0, + env.cluster().nodes(), + env.cluster().clusterResource().clusterId(), + 2, + authorisedOperations) + ); + + DescribeTopicPartitionsResponseData responseData = new DescribeTopicPartitionsResponseData(); + responseData.topics().add(new DescribeTopicPartitionsResponseTopic() + .setErrorCode((short) 0) + .setTopicId(topicId) + .setName(topicName0) + .setIsInternal(false) + .setTopicAuthorizedOperations(authorisedOperations)); + env.kafkaClient().prepareResponse(new DescribeTopicPartitionsResponse(responseData)); + + DescribeTopicsResult result = env.adminClient().describeTopics( + singletonList(topicName0), new DescribeTopicsOptions().includeAuthorizedOperations(true) + ); + + Map topicDescriptions = result.allTopicNames().get(); + TopicDescription topicDescription = topicDescriptions.get(topicName0); + assertEquals(new HashSet<>(asList(AclOperation.DESCRIBE, AclOperation.ALTER)), + topicDescription.authorizedOperations()); + } + } + + @Test + public void testDescribeTopicPartitionsApiWithoutAuthorizedOps() throws ExecutionException, InterruptedException { + try (AdminClientUnitTestEnv env = mockClientEnv()) { + env.kafkaClient().setNodeApiVersions(NodeApiVersions.create()); + String topicName0 = "test-0"; + Uuid topicId = Uuid.randomUuid(); + + int authorisedOperations = Utils.to32BitField(Utils.mkSet(AclOperation.DESCRIBE.code(), AclOperation.ALTER.code())); + env.kafkaClient().prepareResponse( + prepareDescribeClusterResponse(0, + env.cluster().nodes(), + env.cluster().clusterResource().clusterId(), + 2, + authorisedOperations) + ); + + DescribeTopicPartitionsResponseData responseData = new DescribeTopicPartitionsResponseData(); + responseData.topics().add(new DescribeTopicPartitionsResponseTopic() + .setErrorCode((short) 0) + .setTopicId(topicId) + .setName(topicName0) + .setIsInternal(false) + .setTopicAuthorizedOperations(authorisedOperations)); + env.kafkaClient().prepareResponse(new DescribeTopicPartitionsResponse(responseData)); + + DescribeTopicsResult result = env.adminClient().describeTopics( + singletonList(topicName0), new DescribeTopicsOptions().includeAuthorizedOperations(false) + ); + + Map topicDescriptions = result.allTopicNames().get(); + TopicDescription topicDescription = topicDescriptions.get(topicName0); + assertNull(topicDescription.authorizedOperations()); + } + } + @SuppressWarnings({"NPathComplexity", "CyclomaticComplexity"}) @Test public void testDescribeTopicsWithDescribeTopicPartitionsApiEdgeCase() { @@ -1554,6 +1626,7 @@ public void testDescribeTopicsWithDescribeTopicPartitionsApiEdgeCase() { assertEquals(2, topicDescription.partitions().size()); topicDescription = topicDescriptions.get(topicName2); assertEquals(2, topicDescription.partitions().size()); + assertNull(topicDescription.authorizedOperations()); } catch (Exception e) { fail("describe using DescribeTopics API should not fail", e); } diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/NewTopicTest.java b/clients/src/test/java/org/apache/kafka/clients/admin/NewTopicTest.java new file mode 100644 index 0000000000000..78591cf20cab3 --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/clients/admin/NewTopicTest.java @@ -0,0 +1,185 @@ +/* + * 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.clients.admin; + +import org.apache.kafka.common.message.CreateTopicsRequestData; +import org.junit.jupiter.api.Test; + +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; + +import static org.apache.kafka.common.requests.CreateTopicsRequest.NO_NUM_PARTITIONS; +import static org.apache.kafka.common.requests.CreateTopicsRequest.NO_REPLICATION_FACTOR; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertNull; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; + + +public class NewTopicTest { + + public static final String TEST_TOPIC = "testtopic"; + public static final int NUM_PARTITIONS = 3; + public static final short REPLICATION_FACTOR = 1; + public static final String CLEANUP_POLICY_CONFIG_KEY = "cleanup.policy"; + public static final String CLEANUP_POLICY_CONFIG_VALUE = "compact"; + public static final List BROKER_IDS = Arrays.asList(1, 2); + + @Test + public void testConstructorWithPartitionsAndReplicationFactor() { + NewTopic topic = new NewTopic(TEST_TOPIC, NUM_PARTITIONS, REPLICATION_FACTOR); + assertEquals(TEST_TOPIC, topic.name()); + assertEquals(NUM_PARTITIONS, topic.numPartitions()); + assertEquals(REPLICATION_FACTOR, topic.replicationFactor()); + assertNull(topic.replicasAssignments()); + } + + @Test + public void testConstructorWithOptionalValues() { + Optional numPartitions = Optional.empty(); + Optional replicationFactor = Optional.empty(); + NewTopic topic = new NewTopic(TEST_TOPIC, numPartitions, replicationFactor); + assertEquals(TEST_TOPIC, topic.name()); + assertEquals(NO_NUM_PARTITIONS, topic.numPartitions()); + assertEquals(NO_REPLICATION_FACTOR, topic.replicationFactor()); + assertNull(topic.replicasAssignments()); + } + + @Test + public void testConstructorWithReplicasAssignments() { + Map> replicasAssignments = new HashMap<>(); + replicasAssignments.put(0, BROKER_IDS); + NewTopic newTopic = new NewTopic(TEST_TOPIC, replicasAssignments); + assertEquals(TEST_TOPIC, newTopic.name()); + assertEquals(NO_NUM_PARTITIONS, newTopic.numPartitions()); + assertEquals(NO_REPLICATION_FACTOR, newTopic.replicationFactor()); + assertEquals(replicasAssignments, newTopic.replicasAssignments()); + } + + @Test + public void testConfigsNotNull() { + NewTopic newTopic = new NewTopic(TEST_TOPIC, NUM_PARTITIONS, REPLICATION_FACTOR); + Map configs = new HashMap<>(); + configs.put(CLEANUP_POLICY_CONFIG_KEY, CLEANUP_POLICY_CONFIG_VALUE); + newTopic.configs(configs); + assertEquals(configs, newTopic.configs()); + } + + @Test + public void testConfigsNull() { + NewTopic newTopic = new NewTopic(TEST_TOPIC, NUM_PARTITIONS, REPLICATION_FACTOR); + assertNull(newTopic.configs()); + } + + @Test + public void testUnmodifiableReplicasAssignments() { + Map> replicasAssignments = new HashMap<>(); + replicasAssignments.put(0, BROKER_IDS); + NewTopic newTopic = new NewTopic(TEST_TOPIC, replicasAssignments); + Map> returnedAssignments = newTopic.replicasAssignments(); + + assertThrows(UnsupportedOperationException.class, () -> + returnedAssignments.put(1, Arrays.asList(3, 4)) + ); + } + + @Test + public void testConvertToCreatableTopicWithPartitionsAndReplicationFactor() { + NewTopic newTopic = new NewTopic(TEST_TOPIC, NUM_PARTITIONS, REPLICATION_FACTOR); + CreateTopicsRequestData.CreatableTopic creatableTopic = newTopic.convertToCreatableTopic(); + + assertEquals(TEST_TOPIC, creatableTopic.name()); + assertTrue(creatableTopic.numPartitions() > 0); + assertEquals(NUM_PARTITIONS, creatableTopic.numPartitions()); + assertTrue(creatableTopic.replicationFactor() > 0); + assertEquals(REPLICATION_FACTOR, creatableTopic.replicationFactor()); + } + + @Test + public void testConvertToCreatableTopicWithReplicasAssignments() { + int partitionIndex = 0; + Map> replicasAssignments = new HashMap<>(); + replicasAssignments.put(partitionIndex, BROKER_IDS); + NewTopic topic = new NewTopic(TEST_TOPIC, replicasAssignments); + Map configs = new HashMap<>(); + configs.put(CLEANUP_POLICY_CONFIG_KEY, CLEANUP_POLICY_CONFIG_VALUE); + topic.configs(configs); + + CreateTopicsRequestData.CreatableTopic creatableTopic = topic.convertToCreatableTopic(); + + assertEquals(TEST_TOPIC, creatableTopic.name()); + assertEquals(NO_NUM_PARTITIONS, creatableTopic.numPartitions()); + assertEquals(NO_REPLICATION_FACTOR, creatableTopic.replicationFactor()); + assertNotNull(creatableTopic.assignments()); + assertEquals(1, creatableTopic.assignments().size()); + + CreateTopicsRequestData.CreatableReplicaAssignmentCollection assignmentsCollection = creatableTopic.assignments(); + + CreateTopicsRequestData.CreatableReplicaAssignment assignment = assignmentsCollection.find(partitionIndex); + assertEquals(partitionIndex, assignment.partitionIndex()); + assertEquals(BROKER_IDS, assignment.brokerIds()); + + assertNotNull(creatableTopic.configs()); + assertEquals(1, creatableTopic.configs().size()); + + CreateTopicsRequestData.CreateableTopicConfig config = creatableTopic.configs().find(CLEANUP_POLICY_CONFIG_KEY); + assertEquals(CLEANUP_POLICY_CONFIG_KEY, config.name()); + assertEquals(CLEANUP_POLICY_CONFIG_VALUE, config.value()); + } + + @Test + public void testToString() { + NewTopic topic1 = new NewTopic(TEST_TOPIC, NUM_PARTITIONS, REPLICATION_FACTOR); + String expected1 = "(name=" + TEST_TOPIC + ", numPartitions=" + NUM_PARTITIONS + + ", replicationFactor=" + REPLICATION_FACTOR + ", replicasAssignments=null, configs=null)"; + assertEquals(expected1, topic1.toString()); + + Map configs = new HashMap<>(); + configs.put(CLEANUP_POLICY_CONFIG_KEY, CLEANUP_POLICY_CONFIG_VALUE); + topic1.configs(configs); + String expected2 = "(name=" + TEST_TOPIC + ", numPartitions=" + NUM_PARTITIONS + + ", replicationFactor=" + REPLICATION_FACTOR + ", replicasAssignments=null, configs=" + + "{" + CLEANUP_POLICY_CONFIG_KEY + "=" + CLEANUP_POLICY_CONFIG_VALUE + "})"; + assertEquals(expected2, topic1.toString()); + + int partitionIndex = 0; + Map> replicasAssignments = new HashMap<>(); + replicasAssignments.put(partitionIndex, BROKER_IDS); + NewTopic topic2 = new NewTopic(TEST_TOPIC, replicasAssignments); + String expected3 = "(name=" + TEST_TOPIC + ", numPartitions=default" + + ", replicationFactor=default, replicasAssignments=" + + "{" + partitionIndex + "=" + BROKER_IDS + "}" + ", configs=null)"; + assertEquals(expected3, topic2.toString()); + } + + @Test + public void testEqualsAndHashCode() { + NewTopic topic1 = new NewTopic(TEST_TOPIC, NUM_PARTITIONS, REPLICATION_FACTOR); + NewTopic topic2 = new NewTopic(TEST_TOPIC, NUM_PARTITIONS, REPLICATION_FACTOR); + NewTopic topic3 = new NewTopic("another-topic", NUM_PARTITIONS, REPLICATION_FACTOR); + + assertEquals(topic1, topic2); + assertNotEquals(topic1, topic3); + assertEquals(topic1.hashCode(), topic2.hashCode()); + assertNotEquals(topic1.hashCode(), topic3.hashCode()); + } +} diff --git a/connect/api/src/main/java/org/apache/kafka/connect/connector/Connector.java b/connect/api/src/main/java/org/apache/kafka/connect/connector/Connector.java index 6ea43fd99106a..927e4170f7732 100644 --- a/connect/api/src/main/java/org/apache/kafka/connect/connector/Connector.java +++ b/connect/api/src/main/java/org/apache/kafka/connect/connector/Connector.java @@ -19,8 +19,8 @@ import org.apache.kafka.common.config.Config; import org.apache.kafka.common.config.ConfigDef; import org.apache.kafka.common.config.ConfigValue; -import org.apache.kafka.connect.errors.ConnectException; import org.apache.kafka.connect.components.Versioned; +import org.apache.kafka.connect.errors.ConnectException; import java.util.List; import java.util.Map; diff --git a/connect/api/src/main/java/org/apache/kafka/connect/data/Struct.java b/connect/api/src/main/java/org/apache/kafka/connect/data/Struct.java index 1835498808b7e..6e5b81ab11537 100644 --- a/connect/api/src/main/java/org/apache/kafka/connect/data/Struct.java +++ b/connect/api/src/main/java/org/apache/kafka/connect/data/Struct.java @@ -284,4 +284,3 @@ public String toString() { } } - diff --git a/connect/api/src/main/java/org/apache/kafka/connect/data/Values.java b/connect/api/src/main/java/org/apache/kafka/connect/data/Values.java index e144b7c69b6c7..a528271d1ab44 100644 --- a/connect/api/src/main/java/org/apache/kafka/connect/data/Values.java +++ b/connect/api/src/main/java/org/apache/kafka/connect/data/Values.java @@ -19,6 +19,7 @@ import org.apache.kafka.common.utils.Utils; import org.apache.kafka.connect.data.Schema.Type; import org.apache.kafka.connect.errors.DataException; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/connect/api/src/main/java/org/apache/kafka/connect/health/AbstractState.java b/connect/api/src/main/java/org/apache/kafka/connect/health/AbstractState.java index 356658f9320b1..2dac451a1cd3f 100644 --- a/connect/api/src/main/java/org/apache/kafka/connect/health/AbstractState.java +++ b/connect/api/src/main/java/org/apache/kafka/connect/health/AbstractState.java @@ -17,10 +17,10 @@ package org.apache.kafka.connect.health; -import java.util.Objects; - import org.apache.kafka.common.utils.Utils; +import java.util.Objects; + /** * Provides the current status for a connector or a task, along with an identifier for its Connect worker */ diff --git a/connect/api/src/main/java/org/apache/kafka/connect/health/ConnectorHealth.java b/connect/api/src/main/java/org/apache/kafka/connect/health/ConnectorHealth.java index 1f781574f52a9..876a0b3e3b94c 100644 --- a/connect/api/src/main/java/org/apache/kafka/connect/health/ConnectorHealth.java +++ b/connect/api/src/main/java/org/apache/kafka/connect/health/ConnectorHealth.java @@ -16,11 +16,11 @@ */ package org.apache.kafka.connect.health; +import org.apache.kafka.common.utils.Utils; + import java.util.Map; import java.util.Objects; -import org.apache.kafka.common.utils.Utils; - /** * Provides basic health information about the connector and its tasks. */ diff --git a/connect/api/src/main/java/org/apache/kafka/connect/sink/ErrantRecordReporter.java b/connect/api/src/main/java/org/apache/kafka/connect/sink/ErrantRecordReporter.java index b5496d2d84883..81b74a58f8ea2 100644 --- a/connect/api/src/main/java/org/apache/kafka/connect/sink/ErrantRecordReporter.java +++ b/connect/api/src/main/java/org/apache/kafka/connect/sink/ErrantRecordReporter.java @@ -16,9 +16,10 @@ */ package org.apache.kafka.connect.sink; -import java.util.concurrent.Future; import org.apache.kafka.connect.errors.ConnectException; +import java.util.concurrent.Future; + /** * Component that a {@link SinkTask} can use to report problematic records (and their corresponding problems) as it * writes them through {@link SinkTask#put(java.util.Collection)}. diff --git a/connect/api/src/main/java/org/apache/kafka/connect/storage/SimpleHeaderConverter.java b/connect/api/src/main/java/org/apache/kafka/connect/storage/SimpleHeaderConverter.java index 685905e8ead6d..3589beb5087f9 100644 --- a/connect/api/src/main/java/org/apache/kafka/connect/storage/SimpleHeaderConverter.java +++ b/connect/api/src/main/java/org/apache/kafka/connect/storage/SimpleHeaderConverter.java @@ -23,6 +23,7 @@ import org.apache.kafka.connect.data.SchemaAndValue; import org.apache.kafka.connect.data.Values; import org.apache.kafka.connect.errors.DataException; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/connect/api/src/test/java/org/apache/kafka/connect/connector/ConnectorTest.java b/connect/api/src/test/java/org/apache/kafka/connect/connector/ConnectorTest.java index ce0c1d427f7fc..94ff9ad09e240 100644 --- a/connect/api/src/test/java/org/apache/kafka/connect/connector/ConnectorTest.java +++ b/connect/api/src/test/java/org/apache/kafka/connect/connector/ConnectorTest.java @@ -16,14 +16,14 @@ */ package org.apache.kafka.connect.connector; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Test; - public abstract class ConnectorTest { protected ConnectorContext context; diff --git a/connect/api/src/test/java/org/apache/kafka/connect/data/ConnectSchemaTest.java b/connect/api/src/test/java/org/apache/kafka/connect/data/ConnectSchemaTest.java index 43c2342fe3b41..b4e9f81ce8163 100644 --- a/connect/api/src/test/java/org/apache/kafka/connect/data/ConnectSchemaTest.java +++ b/connect/api/src/test/java/org/apache/kafka/connect/data/ConnectSchemaTest.java @@ -17,6 +17,7 @@ package org.apache.kafka.connect.data; import org.apache.kafka.connect.errors.DataException; + import org.junit.jupiter.api.Test; import java.math.BigDecimal; diff --git a/connect/api/src/test/java/org/apache/kafka/connect/data/DateTest.java b/connect/api/src/test/java/org/apache/kafka/connect/data/DateTest.java index 2cd656c783c13..51025b44828d5 100644 --- a/connect/api/src/test/java/org/apache/kafka/connect/data/DateTest.java +++ b/connect/api/src/test/java/org/apache/kafka/connect/data/DateTest.java @@ -17,6 +17,7 @@ package org.apache.kafka.connect.data; import org.apache.kafka.connect.errors.DataException; + import org.junit.jupiter.api.Test; import java.util.Calendar; diff --git a/connect/api/src/test/java/org/apache/kafka/connect/data/SchemaBuilderTest.java b/connect/api/src/test/java/org/apache/kafka/connect/data/SchemaBuilderTest.java index ba7c574d24682..c789541ae5377 100644 --- a/connect/api/src/test/java/org/apache/kafka/connect/data/SchemaBuilderTest.java +++ b/connect/api/src/test/java/org/apache/kafka/connect/data/SchemaBuilderTest.java @@ -17,6 +17,7 @@ package org.apache.kafka.connect.data; import org.apache.kafka.connect.errors.SchemaBuilderException; + import org.junit.jupiter.api.Test; import java.nio.ByteBuffer; diff --git a/connect/api/src/test/java/org/apache/kafka/connect/data/SchemaProjectorTest.java b/connect/api/src/test/java/org/apache/kafka/connect/data/SchemaProjectorTest.java index 32e304c218ac9..4ec35d369adb9 100644 --- a/connect/api/src/test/java/org/apache/kafka/connect/data/SchemaProjectorTest.java +++ b/connect/api/src/test/java/org/apache/kafka/connect/data/SchemaProjectorTest.java @@ -19,6 +19,7 @@ import org.apache.kafka.connect.data.Schema.Type; import org.apache.kafka.connect.errors.DataException; import org.apache.kafka.connect.errors.SchemaProjectorException; + import org.junit.jupiter.api.Test; import java.math.BigDecimal; diff --git a/connect/api/src/test/java/org/apache/kafka/connect/data/TimeTest.java b/connect/api/src/test/java/org/apache/kafka/connect/data/TimeTest.java index b07ccc09145c6..58d6831542b75 100644 --- a/connect/api/src/test/java/org/apache/kafka/connect/data/TimeTest.java +++ b/connect/api/src/test/java/org/apache/kafka/connect/data/TimeTest.java @@ -17,6 +17,7 @@ package org.apache.kafka.connect.data; import org.apache.kafka.connect.errors.DataException; + import org.junit.jupiter.api.Test; import java.util.Calendar; diff --git a/connect/api/src/test/java/org/apache/kafka/connect/data/TimestampTest.java b/connect/api/src/test/java/org/apache/kafka/connect/data/TimestampTest.java index 94f67b4250453..8a7f97b49c245 100644 --- a/connect/api/src/test/java/org/apache/kafka/connect/data/TimestampTest.java +++ b/connect/api/src/test/java/org/apache/kafka/connect/data/TimestampTest.java @@ -17,6 +17,7 @@ package org.apache.kafka.connect.data; import org.apache.kafka.connect.errors.DataException; + import org.junit.jupiter.api.Test; import java.util.Calendar; diff --git a/connect/api/src/test/java/org/apache/kafka/connect/data/ValuesTest.java b/connect/api/src/test/java/org/apache/kafka/connect/data/ValuesTest.java index df3c2ade5dd83..9a96882d976e3 100644 --- a/connect/api/src/test/java/org/apache/kafka/connect/data/ValuesTest.java +++ b/connect/api/src/test/java/org/apache/kafka/connect/data/ValuesTest.java @@ -20,6 +20,7 @@ import org.apache.kafka.connect.data.Schema.Type; import org.apache.kafka.connect.data.Values.Parser; import org.apache.kafka.connect.errors.DataException; + import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; diff --git a/connect/api/src/test/java/org/apache/kafka/connect/header/ConnectHeaderTest.java b/connect/api/src/test/java/org/apache/kafka/connect/header/ConnectHeaderTest.java index 8a84d44977b43..57d7634a6fdb4 100644 --- a/connect/api/src/test/java/org/apache/kafka/connect/header/ConnectHeaderTest.java +++ b/connect/api/src/test/java/org/apache/kafka/connect/header/ConnectHeaderTest.java @@ -20,6 +20,7 @@ import org.apache.kafka.connect.data.SchemaAndValue; import org.apache.kafka.connect.data.SchemaBuilder; import org.apache.kafka.connect.data.Struct; + import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; diff --git a/connect/api/src/test/java/org/apache/kafka/connect/header/ConnectHeadersTest.java b/connect/api/src/test/java/org/apache/kafka/connect/header/ConnectHeadersTest.java index b9b9174d73da1..44073f7722927 100644 --- a/connect/api/src/test/java/org/apache/kafka/connect/header/ConnectHeadersTest.java +++ b/connect/api/src/test/java/org/apache/kafka/connect/header/ConnectHeadersTest.java @@ -28,6 +28,7 @@ import org.apache.kafka.connect.data.Values; import org.apache.kafka.connect.errors.DataException; import org.apache.kafka.connect.header.Headers.HeaderTransform; + import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; diff --git a/connect/api/src/test/java/org/apache/kafka/connect/sink/SinkConnectorTest.java b/connect/api/src/test/java/org/apache/kafka/connect/sink/SinkConnectorTest.java index 2cf22785bd90b..dc89ff59f2937 100644 --- a/connect/api/src/test/java/org/apache/kafka/connect/sink/SinkConnectorTest.java +++ b/connect/api/src/test/java/org/apache/kafka/connect/sink/SinkConnectorTest.java @@ -16,14 +16,14 @@ */ package org.apache.kafka.connect.sink; -import java.util.List; -import java.util.Map; - import org.apache.kafka.common.config.ConfigDef; import org.apache.kafka.connect.connector.ConnectorContext; import org.apache.kafka.connect.connector.ConnectorTest; import org.apache.kafka.connect.connector.Task; +import java.util.List; +import java.util.Map; + import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertSame; import static org.junit.jupiter.api.Assertions.assertTrue; diff --git a/connect/api/src/test/java/org/apache/kafka/connect/sink/SinkRecordTest.java b/connect/api/src/test/java/org/apache/kafka/connect/sink/SinkRecordTest.java index 329b28ee8b69a..099b1a81fad27 100644 --- a/connect/api/src/test/java/org/apache/kafka/connect/sink/SinkRecordTest.java +++ b/connect/api/src/test/java/org/apache/kafka/connect/sink/SinkRecordTest.java @@ -22,6 +22,7 @@ import org.apache.kafka.connect.header.ConnectHeaders; import org.apache.kafka.connect.header.Header; import org.apache.kafka.connect.header.Headers; + import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; diff --git a/connect/api/src/test/java/org/apache/kafka/connect/source/SourceConnectorTest.java b/connect/api/src/test/java/org/apache/kafka/connect/source/SourceConnectorTest.java index 3359b1a836c3c..e1a6c54ebfd77 100644 --- a/connect/api/src/test/java/org/apache/kafka/connect/source/SourceConnectorTest.java +++ b/connect/api/src/test/java/org/apache/kafka/connect/source/SourceConnectorTest.java @@ -16,15 +16,15 @@ */ package org.apache.kafka.connect.source; -import java.util.List; -import java.util.Map; - import org.apache.kafka.common.config.ConfigDef; import org.apache.kafka.connect.connector.ConnectorContext; import org.apache.kafka.connect.connector.ConnectorTest; import org.apache.kafka.connect.connector.Task; import org.apache.kafka.connect.storage.OffsetStorageReader; +import java.util.List; +import java.util.Map; + import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertSame; import static org.junit.jupiter.api.Assertions.assertTrue; diff --git a/connect/api/src/test/java/org/apache/kafka/connect/source/SourceRecordTest.java b/connect/api/src/test/java/org/apache/kafka/connect/source/SourceRecordTest.java index d098760f286db..90bd4f897df28 100644 --- a/connect/api/src/test/java/org/apache/kafka/connect/source/SourceRecordTest.java +++ b/connect/api/src/test/java/org/apache/kafka/connect/source/SourceRecordTest.java @@ -21,6 +21,7 @@ import org.apache.kafka.connect.header.ConnectHeaders; import org.apache.kafka.connect.header.Header; import org.apache.kafka.connect.header.Headers; + import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; diff --git a/connect/api/src/test/java/org/apache/kafka/connect/storage/SimpleHeaderConverterTest.java b/connect/api/src/test/java/org/apache/kafka/connect/storage/SimpleHeaderConverterTest.java index d13b53e74d0a4..14fd0d60c142b 100644 --- a/connect/api/src/test/java/org/apache/kafka/connect/storage/SimpleHeaderConverterTest.java +++ b/connect/api/src/test/java/org/apache/kafka/connect/storage/SimpleHeaderConverterTest.java @@ -20,6 +20,7 @@ import org.apache.kafka.connect.data.Schema; import org.apache.kafka.connect.data.SchemaAndValue; import org.apache.kafka.connect.data.SchemaBuilder; + import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; diff --git a/connect/api/src/test/java/org/apache/kafka/connect/storage/StringConverterTest.java b/connect/api/src/test/java/org/apache/kafka/connect/storage/StringConverterTest.java index f6e9bdbfa16bc..463125e09404f 100644 --- a/connect/api/src/test/java/org/apache/kafka/connect/storage/StringConverterTest.java +++ b/connect/api/src/test/java/org/apache/kafka/connect/storage/StringConverterTest.java @@ -20,6 +20,7 @@ import org.apache.kafka.common.utils.Utils; import org.apache.kafka.connect.data.Schema; import org.apache.kafka.connect.data.SchemaAndValue; + import org.junit.jupiter.api.Test; import java.nio.charset.StandardCharsets; diff --git a/connect/basic-auth-extension/src/main/java/org/apache/kafka/connect/rest/basic/auth/extension/BasicAuthSecurityRestExtension.java b/connect/basic-auth-extension/src/main/java/org/apache/kafka/connect/rest/basic/auth/extension/BasicAuthSecurityRestExtension.java index 8ba0215e142d6..58aac7994aefc 100644 --- a/connect/basic-auth-extension/src/main/java/org/apache/kafka/connect/rest/basic/auth/extension/BasicAuthSecurityRestExtension.java +++ b/connect/basic-auth-extension/src/main/java/org/apache/kafka/connect/rest/basic/auth/extension/BasicAuthSecurityRestExtension.java @@ -21,13 +21,15 @@ import org.apache.kafka.connect.errors.ConnectException; import org.apache.kafka.connect.rest.ConnectRestExtension; import org.apache.kafka.connect.rest.ConnectRestExtensionContext; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import javax.security.auth.login.Configuration; import java.util.Map; import java.util.function.Supplier; +import javax.security.auth.login.Configuration; + /** * Provides the ability to authenticate incoming BasicAuth credentials using the configured JAAS {@link * javax.security.auth.spi.LoginModule}. An entry with the name {@code KafkaConnect} is expected in the JAAS config file configured in the diff --git a/connect/basic-auth-extension/src/main/java/org/apache/kafka/connect/rest/basic/auth/extension/JaasBasicAuthFilter.java b/connect/basic-auth-extension/src/main/java/org/apache/kafka/connect/rest/basic/auth/extension/JaasBasicAuthFilter.java index 964bd7c397771..b090ee21d449d 100644 --- a/connect/basic-auth-extension/src/main/java/org/apache/kafka/connect/rest/basic/auth/extension/JaasBasicAuthFilter.java +++ b/connect/basic-auth-extension/src/main/java/org/apache/kafka/connect/rest/basic/auth/extension/JaasBasicAuthFilter.java @@ -17,9 +17,17 @@ package org.apache.kafka.connect.rest.basic.auth.extension; +import org.apache.kafka.common.config.ConfigException; +import org.apache.kafka.connect.errors.ConnectException; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.nio.charset.StandardCharsets; import java.security.Principal; import java.util.ArrayList; import java.util.Arrays; +import java.util.Base64; import java.util.HashSet; import java.util.List; import java.util.Set; @@ -27,22 +35,14 @@ import java.util.regex.Pattern; import javax.annotation.Priority; -import javax.security.auth.login.Configuration; -import javax.ws.rs.HttpMethod; -import org.apache.kafka.common.config.ConfigException; -import org.apache.kafka.connect.errors.ConnectException; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.nio.charset.StandardCharsets; -import java.util.Base64; - import javax.security.auth.callback.Callback; import javax.security.auth.callback.CallbackHandler; import javax.security.auth.callback.NameCallback; import javax.security.auth.callback.PasswordCallback; +import javax.security.auth.login.Configuration; import javax.security.auth.login.LoginContext; import javax.security.auth.login.LoginException; +import javax.ws.rs.HttpMethod; import javax.ws.rs.Priorities; import javax.ws.rs.container.ContainerRequestContext; import javax.ws.rs.container.ContainerRequestFilter; diff --git a/connect/basic-auth-extension/src/main/java/org/apache/kafka/connect/rest/basic/auth/extension/PropertyFileLoginModule.java b/connect/basic-auth-extension/src/main/java/org/apache/kafka/connect/rest/basic/auth/extension/PropertyFileLoginModule.java index 7803dffa9416d..65f11990346a1 100644 --- a/connect/basic-auth-extension/src/main/java/org/apache/kafka/connect/rest/basic/auth/extension/PropertyFileLoginModule.java +++ b/connect/basic-auth-extension/src/main/java/org/apache/kafka/connect/rest/basic/auth/extension/PropertyFileLoginModule.java @@ -19,6 +19,7 @@ import org.apache.kafka.common.config.ConfigException; import org.apache.kafka.common.utils.Utils; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/connect/basic-auth-extension/src/test/java/org/apache/kafka/connect/rest/basic/auth/extension/BasicAuthSecurityRestExtensionTest.java b/connect/basic-auth-extension/src/test/java/org/apache/kafka/connect/rest/basic/auth/extension/BasicAuthSecurityRestExtensionTest.java index b1b5b1ef7cdff..146bd6a2adf63 100644 --- a/connect/basic-auth-extension/src/test/java/org/apache/kafka/connect/rest/basic/auth/extension/BasicAuthSecurityRestExtensionTest.java +++ b/connect/basic-auth-extension/src/test/java/org/apache/kafka/connect/rest/basic/auth/extension/BasicAuthSecurityRestExtensionTest.java @@ -19,19 +19,20 @@ import org.apache.kafka.connect.errors.ConnectException; import org.apache.kafka.connect.rest.ConnectRestExtensionContext; + import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.mockito.ArgumentCaptor; -import javax.security.auth.login.Configuration; -import javax.ws.rs.core.Configurable; - import java.io.IOException; import java.util.Collections; import java.util.concurrent.atomic.AtomicBoolean; import java.util.function.Supplier; +import javax.security.auth.login.Configuration; +import javax.ws.rs.core.Configurable; + import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertNotEquals; import static org.junit.jupiter.api.Assertions.assertThrows; diff --git a/connect/basic-auth-extension/src/test/java/org/apache/kafka/connect/rest/basic/auth/extension/JaasBasicAuthFilterTest.java b/connect/basic-auth-extension/src/test/java/org/apache/kafka/connect/rest/basic/auth/extension/JaasBasicAuthFilterTest.java index 44d6ed8e33d44..24ecadcc0a09b 100644 --- a/connect/basic-auth-extension/src/test/java/org/apache/kafka/connect/rest/basic/auth/extension/JaasBasicAuthFilterTest.java +++ b/connect/basic-auth-extension/src/test/java/org/apache/kafka/connect/rest/basic/auth/extension/JaasBasicAuthFilterTest.java @@ -17,16 +17,10 @@ package org.apache.kafka.connect.rest.basic.auth.extension; -import javax.security.auth.callback.Callback; -import javax.security.auth.callback.CallbackHandler; -import javax.security.auth.callback.ChoiceCallback; -import javax.ws.rs.HttpMethod; -import javax.ws.rs.core.SecurityContext; -import javax.ws.rs.core.UriInfo; - import org.apache.kafka.common.security.authenticator.TestJaasConfig; import org.apache.kafka.connect.errors.ConnectException; import org.apache.kafka.test.TestUtils; + import org.junit.jupiter.api.Test; import org.mockito.ArgumentCaptor; @@ -42,8 +36,14 @@ import java.util.List; import java.util.Map; +import javax.security.auth.callback.Callback; +import javax.security.auth.callback.CallbackHandler; +import javax.security.auth.callback.ChoiceCallback; +import javax.ws.rs.HttpMethod; import javax.ws.rs.container.ContainerRequestContext; import javax.ws.rs.core.Response; +import javax.ws.rs.core.SecurityContext; +import javax.ws.rs.core.UriInfo; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertThrows; diff --git a/connect/file/src/main/java/org/apache/kafka/connect/file/FileStreamSinkConnector.java b/connect/file/src/main/java/org/apache/kafka/connect/file/FileStreamSinkConnector.java index 68ee27cb93972..a288ab18ae7e3 100644 --- a/connect/file/src/main/java/org/apache/kafka/connect/file/FileStreamSinkConnector.java +++ b/connect/file/src/main/java/org/apache/kafka/connect/file/FileStreamSinkConnector.java @@ -24,6 +24,7 @@ import org.apache.kafka.common.utils.AppInfoParser; import org.apache.kafka.connect.connector.Task; import org.apache.kafka.connect.sink.SinkConnector; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/connect/file/src/main/java/org/apache/kafka/connect/file/FileStreamSinkTask.java b/connect/file/src/main/java/org/apache/kafka/connect/file/FileStreamSinkTask.java index cb19c01e6007f..0b5f112ce3083 100644 --- a/connect/file/src/main/java/org/apache/kafka/connect/file/FileStreamSinkTask.java +++ b/connect/file/src/main/java/org/apache/kafka/connect/file/FileStreamSinkTask.java @@ -22,6 +22,7 @@ import org.apache.kafka.connect.errors.ConnectException; import org.apache.kafka.connect.sink.SinkRecord; import org.apache.kafka.connect.sink.SinkTask; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/connect/file/src/main/java/org/apache/kafka/connect/file/FileStreamSourceConnector.java b/connect/file/src/main/java/org/apache/kafka/connect/file/FileStreamSourceConnector.java index 37cdcec1b053e..f0f07a62a2f9b 100644 --- a/connect/file/src/main/java/org/apache/kafka/connect/file/FileStreamSourceConnector.java +++ b/connect/file/src/main/java/org/apache/kafka/connect/file/FileStreamSourceConnector.java @@ -25,6 +25,7 @@ import org.apache.kafka.connect.errors.ConnectException; import org.apache.kafka.connect.source.ExactlyOnceSupport; import org.apache.kafka.connect.source.SourceConnector; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/connect/file/src/main/java/org/apache/kafka/connect/file/FileStreamSourceTask.java b/connect/file/src/main/java/org/apache/kafka/connect/file/FileStreamSourceTask.java index cda58cf2d08b5..79478c57d1fa4 100644 --- a/connect/file/src/main/java/org/apache/kafka/connect/file/FileStreamSourceTask.java +++ b/connect/file/src/main/java/org/apache/kafka/connect/file/FileStreamSourceTask.java @@ -16,6 +16,15 @@ */ package org.apache.kafka.connect.file; +import org.apache.kafka.common.config.AbstractConfig; +import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.errors.ConnectException; +import org.apache.kafka.connect.source.SourceRecord; +import org.apache.kafka.connect.source.SourceTask; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import java.io.BufferedReader; import java.io.IOException; import java.io.InputStream; @@ -29,14 +38,6 @@ import java.util.List; import java.util.Map; -import org.apache.kafka.common.config.AbstractConfig; -import org.apache.kafka.connect.data.Schema; -import org.apache.kafka.connect.errors.ConnectException; -import org.apache.kafka.connect.source.SourceRecord; -import org.apache.kafka.connect.source.SourceTask; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - /** * FileStreamSourceTask reads from stdin or a file. */ diff --git a/connect/file/src/test/java/org/apache/kafka/connect/file/FileStreamSinkConnectorTest.java b/connect/file/src/test/java/org/apache/kafka/connect/file/FileStreamSinkConnectorTest.java index 0f1ab8e6e11ef..787f1fb4c9351 100644 --- a/connect/file/src/test/java/org/apache/kafka/connect/file/FileStreamSinkConnectorTest.java +++ b/connect/file/src/test/java/org/apache/kafka/connect/file/FileStreamSinkConnectorTest.java @@ -19,17 +19,18 @@ import org.apache.kafka.common.config.ConfigValue; import org.apache.kafka.connect.connector.ConnectorContext; import org.apache.kafka.connect.sink.SinkConnector; + import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; -import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertNull; -import static org.mockito.Mockito.mock; - import java.util.HashMap; import java.util.List; import java.util.Map; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNull; +import static org.mockito.Mockito.mock; + public class FileStreamSinkConnectorTest { private static final String MULTIPLE_TOPICS = "test1,test2"; diff --git a/connect/file/src/test/java/org/apache/kafka/connect/file/FileStreamSinkTaskTest.java b/connect/file/src/test/java/org/apache/kafka/connect/file/FileStreamSinkTaskTest.java index 23a28d8527c18..dde20105e3731 100644 --- a/connect/file/src/test/java/org/apache/kafka/connect/file/FileStreamSinkTaskTest.java +++ b/connect/file/src/test/java/org/apache/kafka/connect/file/FileStreamSinkTaskTest.java @@ -20,6 +20,7 @@ import org.apache.kafka.common.TopicPartition; import org.apache.kafka.connect.data.Schema; import org.apache.kafka.connect.sink.SinkRecord; + import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.io.TempDir; @@ -35,6 +36,7 @@ import java.util.Collections; import java.util.HashMap; import java.util.Map; + import static org.junit.jupiter.api.Assertions.assertEquals; public class FileStreamSinkTaskTest { diff --git a/connect/file/src/test/java/org/apache/kafka/connect/file/FileStreamSourceConnectorTest.java b/connect/file/src/test/java/org/apache/kafka/connect/file/FileStreamSourceConnectorTest.java index 41915913b03e3..e0c14a1e6cb19 100644 --- a/connect/file/src/test/java/org/apache/kafka/connect/file/FileStreamSourceConnectorTest.java +++ b/connect/file/src/test/java/org/apache/kafka/connect/file/FileStreamSourceConnectorTest.java @@ -23,6 +23,7 @@ import org.apache.kafka.connect.errors.ConnectException; import org.apache.kafka.connect.source.ConnectorTransactionBoundaries; import org.apache.kafka.connect.source.ExactlyOnceSupport; + import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; diff --git a/connect/file/src/test/java/org/apache/kafka/connect/file/FileStreamSourceTaskTest.java b/connect/file/src/test/java/org/apache/kafka/connect/file/FileStreamSourceTaskTest.java index 9f65dd95ecf07..e0e77a8433c72 100644 --- a/connect/file/src/test/java/org/apache/kafka/connect/file/FileStreamSourceTaskTest.java +++ b/connect/file/src/test/java/org/apache/kafka/connect/file/FileStreamSourceTaskTest.java @@ -19,6 +19,7 @@ import org.apache.kafka.connect.source.SourceRecord; import org.apache.kafka.connect.source.SourceTaskContext; import org.apache.kafka.connect.storage.OffsetStorageReader; + import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; diff --git a/connect/file/src/test/java/org/apache/kafka/connect/file/integration/FileStreamSinkConnectorIntegrationTest.java b/connect/file/src/test/java/org/apache/kafka/connect/file/integration/FileStreamSinkConnectorIntegrationTest.java index 433c2004710c2..f6a1719a97f89 100644 --- a/connect/file/src/test/java/org/apache/kafka/connect/file/integration/FileStreamSinkConnectorIntegrationTest.java +++ b/connect/file/src/test/java/org/apache/kafka/connect/file/integration/FileStreamSinkConnectorIntegrationTest.java @@ -20,6 +20,7 @@ import org.apache.kafka.connect.file.FileStreamSinkConnector; import org.apache.kafka.connect.util.clusters.EmbeddedConnectCluster; import org.apache.kafka.test.TestUtils; + import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Tag; diff --git a/connect/file/src/test/java/org/apache/kafka/connect/file/integration/FileStreamSourceConnectorIntegrationTest.java b/connect/file/src/test/java/org/apache/kafka/connect/file/integration/FileStreamSourceConnectorIntegrationTest.java index 698f4fcf8d366..577b07bb5bdb4 100644 --- a/connect/file/src/test/java/org/apache/kafka/connect/file/integration/FileStreamSourceConnectorIntegrationTest.java +++ b/connect/file/src/test/java/org/apache/kafka/connect/file/integration/FileStreamSourceConnectorIntegrationTest.java @@ -20,6 +20,7 @@ import org.apache.kafka.connect.file.FileStreamSourceConnector; import org.apache.kafka.connect.util.clusters.EmbeddedConnectCluster; import org.apache.kafka.test.TestUtils; + import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Tag; diff --git a/connect/json/src/main/java/org/apache/kafka/connect/json/JsonConverter.java b/connect/json/src/main/java/org/apache/kafka/connect/json/JsonConverter.java index b623aaf14e8e2..c4ec44ca157b6 100644 --- a/connect/json/src/main/java/org/apache/kafka/connect/json/JsonConverter.java +++ b/connect/json/src/main/java/org/apache/kafka/connect/json/JsonConverter.java @@ -16,11 +16,6 @@ */ package org.apache.kafka.connect.json; -import com.fasterxml.jackson.databind.DeserializationFeature; -import com.fasterxml.jackson.databind.JsonNode; -import com.fasterxml.jackson.databind.node.ArrayNode; -import com.fasterxml.jackson.databind.node.JsonNodeFactory; -import com.fasterxml.jackson.databind.node.ObjectNode; import org.apache.kafka.common.cache.Cache; import org.apache.kafka.common.cache.LRUCache; import org.apache.kafka.common.cache.SynchronizedCache; @@ -29,22 +24,28 @@ import org.apache.kafka.common.utils.AppInfoParser; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.connect.components.Versioned; -import org.apache.kafka.connect.data.SchemaBuilder; -import org.apache.kafka.connect.data.Schema; -import org.apache.kafka.connect.data.Struct; -import org.apache.kafka.connect.data.Field; import org.apache.kafka.connect.data.ConnectSchema; +import org.apache.kafka.connect.data.Date; +import org.apache.kafka.connect.data.Decimal; +import org.apache.kafka.connect.data.Field; +import org.apache.kafka.connect.data.Schema; import org.apache.kafka.connect.data.SchemaAndValue; -import org.apache.kafka.connect.data.Timestamp; +import org.apache.kafka.connect.data.SchemaBuilder; +import org.apache.kafka.connect.data.Struct; import org.apache.kafka.connect.data.Time; -import org.apache.kafka.connect.data.Decimal; -import org.apache.kafka.connect.data.Date; +import org.apache.kafka.connect.data.Timestamp; import org.apache.kafka.connect.errors.DataException; import org.apache.kafka.connect.storage.Converter; import org.apache.kafka.connect.storage.ConverterType; import org.apache.kafka.connect.storage.HeaderConverter; import org.apache.kafka.connect.storage.StringConverterConfig; +import com.fasterxml.jackson.databind.DeserializationFeature; +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.node.ArrayNode; +import com.fasterxml.jackson.databind.node.JsonNodeFactory; +import com.fasterxml.jackson.databind.node.ObjectNode; + import java.io.IOException; import java.math.BigDecimal; import java.nio.ByteBuffer; diff --git a/connect/json/src/main/java/org/apache/kafka/connect/json/JsonConverterConfig.java b/connect/json/src/main/java/org/apache/kafka/connect/json/JsonConverterConfig.java index add8bec5b33c0..2718c45497823 100644 --- a/connect/json/src/main/java/org/apache/kafka/connect/json/JsonConverterConfig.java +++ b/connect/json/src/main/java/org/apache/kafka/connect/json/JsonConverterConfig.java @@ -16,13 +16,13 @@ */ package org.apache.kafka.connect.json; -import java.util.Locale; import org.apache.kafka.common.config.ConfigDef; import org.apache.kafka.common.config.ConfigDef.Importance; import org.apache.kafka.common.config.ConfigDef.Type; import org.apache.kafka.common.config.ConfigDef.Width; import org.apache.kafka.connect.storage.ConverterConfig; +import java.util.Locale; import java.util.Map; /** diff --git a/connect/json/src/main/java/org/apache/kafka/connect/json/JsonDeserializer.java b/connect/json/src/main/java/org/apache/kafka/connect/json/JsonDeserializer.java index d63ae7808c303..16c8fb26307f2 100644 --- a/connect/json/src/main/java/org/apache/kafka/connect/json/JsonDeserializer.java +++ b/connect/json/src/main/java/org/apache/kafka/connect/json/JsonDeserializer.java @@ -16,13 +16,14 @@ */ package org.apache.kafka.connect.json; +import org.apache.kafka.common.errors.SerializationException; +import org.apache.kafka.common.serialization.Deserializer; + import com.fasterxml.jackson.core.json.JsonReadFeature; import com.fasterxml.jackson.databind.DeserializationFeature; import com.fasterxml.jackson.databind.JsonNode; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.node.JsonNodeFactory; -import org.apache.kafka.common.errors.SerializationException; -import org.apache.kafka.common.serialization.Deserializer; import java.util.Collections; import java.util.Set; diff --git a/connect/json/src/main/java/org/apache/kafka/connect/json/JsonSerializer.java b/connect/json/src/main/java/org/apache/kafka/connect/json/JsonSerializer.java index f8b13fbb6a974..87b6980c29f65 100644 --- a/connect/json/src/main/java/org/apache/kafka/connect/json/JsonSerializer.java +++ b/connect/json/src/main/java/org/apache/kafka/connect/json/JsonSerializer.java @@ -16,12 +16,13 @@ */ package org.apache.kafka.connect.json; +import org.apache.kafka.common.errors.SerializationException; +import org.apache.kafka.common.serialization.Serializer; + import com.fasterxml.jackson.databind.JsonNode; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.SerializationFeature; import com.fasterxml.jackson.databind.node.JsonNodeFactory; -import org.apache.kafka.common.errors.SerializationException; -import org.apache.kafka.common.serialization.Serializer; import java.util.Collections; import java.util.Set; diff --git a/connect/json/src/test/java/org/apache/kafka/connect/json/JsonConverterConfigTest.java b/connect/json/src/test/java/org/apache/kafka/connect/json/JsonConverterConfigTest.java index efa1f60fb4a1a..930fb3bb4b84e 100644 --- a/connect/json/src/test/java/org/apache/kafka/connect/json/JsonConverterConfigTest.java +++ b/connect/json/src/test/java/org/apache/kafka/connect/json/JsonConverterConfigTest.java @@ -18,6 +18,7 @@ import org.apache.kafka.connect.storage.ConverterConfig; import org.apache.kafka.connect.storage.ConverterType; + import org.junit.jupiter.api.Test; import java.util.HashMap; diff --git a/connect/json/src/test/java/org/apache/kafka/connect/json/JsonConverterTest.java b/connect/json/src/test/java/org/apache/kafka/connect/json/JsonConverterTest.java index e6a356e761347..d79c8527b3c21 100644 --- a/connect/json/src/test/java/org/apache/kafka/connect/json/JsonConverterTest.java +++ b/connect/json/src/test/java/org/apache/kafka/connect/json/JsonConverterTest.java @@ -16,11 +16,6 @@ */ package org.apache.kafka.connect.json; -import com.fasterxml.jackson.databind.DeserializationFeature; -import com.fasterxml.jackson.databind.JsonNode; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.fasterxml.jackson.databind.node.ArrayNode; -import com.fasterxml.jackson.databind.node.JsonNodeFactory; import org.apache.kafka.common.utils.AppInfoParser; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.connect.data.Date; @@ -32,6 +27,13 @@ import org.apache.kafka.connect.data.Time; import org.apache.kafka.connect.data.Timestamp; import org.apache.kafka.connect.errors.DataException; + +import com.fasterxml.jackson.databind.DeserializationFeature; +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.node.ArrayNode; +import com.fasterxml.jackson.databind.node.JsonNodeFactory; + import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; diff --git a/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/Checkpoint.java b/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/Checkpoint.java index 353d2eedb9592..603f09df84cad 100644 --- a/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/Checkpoint.java +++ b/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/Checkpoint.java @@ -16,17 +16,17 @@ */ package org.apache.kafka.connect.mirror; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.protocol.types.Field; import org.apache.kafka.common.protocol.types.Schema; import org.apache.kafka.common.protocol.types.Struct; import org.apache.kafka.common.protocol.types.Type; -import org.apache.kafka.clients.consumer.ConsumerRecord; -import org.apache.kafka.clients.consumer.OffsetAndMetadata; -import java.util.Map; -import java.util.HashMap; import java.nio.ByteBuffer; +import java.util.HashMap; +import java.util.Map; import java.util.Objects; /** Checkpoint records emitted from MirrorCheckpointConnector. Encodes remote consumer group state. */ @@ -195,4 +195,3 @@ public int hashCode() { return Objects.hash(consumerGroupId, topicPartition, upstreamOffset, downstreamOffset, metadata); } } - diff --git a/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/DefaultReplicationPolicy.java b/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/DefaultReplicationPolicy.java index 65a98947d391b..fa2c5a75b24b9 100644 --- a/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/DefaultReplicationPolicy.java +++ b/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/DefaultReplicationPolicy.java @@ -18,12 +18,12 @@ import org.apache.kafka.common.Configurable; -import java.util.Map; -import java.util.regex.Pattern; - import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.util.Map; +import java.util.regex.Pattern; + /** Defines remote topics like "us-west.topic1". The separator is customizable and defaults to a period. */ public class DefaultReplicationPolicy implements ReplicationPolicy, Configurable { diff --git a/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/Heartbeat.java b/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/Heartbeat.java index f25755dee9746..ab88e60439ae5 100644 --- a/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/Heartbeat.java +++ b/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/Heartbeat.java @@ -16,15 +16,15 @@ */ package org.apache.kafka.connect.mirror; +import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.common.protocol.types.Field; import org.apache.kafka.common.protocol.types.Schema; import org.apache.kafka.common.protocol.types.Struct; import org.apache.kafka.common.protocol.types.Type; -import org.apache.kafka.clients.consumer.ConsumerRecord; -import java.util.Map; -import java.util.HashMap; import java.nio.ByteBuffer; +import java.util.HashMap; +import java.util.Map; /** Heartbeat message sent from MirrorHeartbeatTask to target cluster. Heartbeats are always replicated. */ public class Heartbeat { @@ -142,4 +142,3 @@ private static Schema valueSchema(short version) { return VALUE_SCHEMA_V0; } } - diff --git a/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/IdentityReplicationPolicy.java b/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/IdentityReplicationPolicy.java index 67ef7778f230a..16a3dfa11ff02 100644 --- a/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/IdentityReplicationPolicy.java +++ b/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/IdentityReplicationPolicy.java @@ -16,11 +16,11 @@ */ package org.apache.kafka.connect.mirror; -import java.util.Map; - import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.util.Map; + /** IdentityReplicationPolicy does not rename remote topics. This is useful for migrating * from legacy MM1, or for any use-case involving one-way replication. *

diff --git a/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/MirrorClient.java b/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/MirrorClient.java index b40b6ba552f32..759f8aa85d99a 100644 --- a/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/MirrorClient.java +++ b/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/MirrorClient.java @@ -19,28 +19,28 @@ import org.apache.kafka.clients.admin.Admin; import org.apache.kafka.clients.consumer.Consumer; -import org.apache.kafka.clients.consumer.KafkaConsumer; -import org.apache.kafka.common.serialization.ByteArrayDeserializer; -import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.common.KafkaException; -import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.consumer.ConsumerRecords; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.clients.consumer.OffsetAndMetadata; +import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.protocol.types.SchemaException; +import org.apache.kafka.common.serialization.ByteArrayDeserializer; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.time.Duration; -import java.util.Set; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; -import java.util.HashMap; -import java.util.Collections; -import java.util.Collection; -import java.util.stream.Collectors; +import java.util.Set; import java.util.concurrent.ExecutionException; +import java.util.stream.Collectors; /** Interprets MM2's internal topics (checkpoints, heartbeats) on a given cluster. *

diff --git a/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/MirrorClientConfig.java b/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/MirrorClientConfig.java index 053e594fbeb1d..b21b7f7246341 100644 --- a/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/MirrorClientConfig.java +++ b/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/MirrorClientConfig.java @@ -16,18 +16,18 @@ */ package org.apache.kafka.connect.mirror; +import org.apache.kafka.clients.CommonClientConfigs; import org.apache.kafka.clients.admin.ForwardingAdmin; import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.config.AbstractConfig; import org.apache.kafka.common.config.ConfigDef; -import org.apache.kafka.common.config.ConfigDef.Type; import org.apache.kafka.common.config.ConfigDef.Importance; -import org.apache.kafka.clients.CommonClientConfigs; +import org.apache.kafka.common.config.ConfigDef.Type; import org.apache.kafka.common.security.auth.SecurityProtocol; import org.apache.kafka.common.utils.Utils; -import java.util.Map; import java.util.HashMap; +import java.util.Map; import static org.apache.kafka.common.config.ConfigDef.CaseInsensitiveValidString.in; diff --git a/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/RemoteClusterUtils.java b/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/RemoteClusterUtils.java index e5299eb41ee06..f67a81e355956 100644 --- a/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/RemoteClusterUtils.java +++ b/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/RemoteClusterUtils.java @@ -17,13 +17,13 @@ package org.apache.kafka.connect.mirror; -import org.apache.kafka.common.TopicPartition; import org.apache.kafka.clients.consumer.OffsetAndMetadata; +import org.apache.kafka.common.TopicPartition; +import java.time.Duration; import java.util.Map; import java.util.Set; import java.util.concurrent.TimeoutException; -import java.time.Duration; /** Convenience methods for multi-cluster environments. Wraps {@link MirrorClient} diff --git a/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/SourceAndTarget.java b/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/SourceAndTarget.java index 199141d4a5826..f9793aceed90f 100644 --- a/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/SourceAndTarget.java +++ b/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/SourceAndTarget.java @@ -49,4 +49,3 @@ public boolean equals(Object other) { return other != null && toString().equals(other.toString()); } } - diff --git a/connect/mirror-client/src/test/java/org/apache/kafka/connect/mirror/MirrorClientTest.java b/connect/mirror-client/src/test/java/org/apache/kafka/connect/mirror/MirrorClientTest.java index f9f20ffead324..be728a0ebe98a 100644 --- a/connect/mirror-client/src/test/java/org/apache/kafka/connect/mirror/MirrorClientTest.java +++ b/connect/mirror-client/src/test/java/org/apache/kafka/connect/mirror/MirrorClientTest.java @@ -18,17 +18,17 @@ import org.apache.kafka.common.Configurable; +import org.junit.jupiter.api.Test; + +import java.util.Arrays; import java.util.Collections; +import java.util.HashSet; import java.util.List; import java.util.Set; -import java.util.HashSet; -import java.util.Arrays; - -import org.junit.jupiter.api.Test; import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertTrue; import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; public class MirrorClientTest { diff --git a/connect/mirror-client/src/test/java/org/apache/kafka/connect/mirror/ReplicationPolicyTest.java b/connect/mirror-client/src/test/java/org/apache/kafka/connect/mirror/ReplicationPolicyTest.java index ef286680793e7..802d0b606c234 100644 --- a/connect/mirror-client/src/test/java/org/apache/kafka/connect/mirror/ReplicationPolicyTest.java +++ b/connect/mirror-client/src/test/java/org/apache/kafka/connect/mirror/ReplicationPolicyTest.java @@ -25,8 +25,8 @@ import java.util.Map; import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertTrue; import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; public class ReplicationPolicyTest { private static final DefaultReplicationPolicy DEFAULT_REPLICATION_POLICY = new DefaultReplicationPolicy(); diff --git a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/CheckpointStore.java b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/CheckpointStore.java index cbe76efecb38c..2e88977d93cea 100644 --- a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/CheckpointStore.java +++ b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/CheckpointStore.java @@ -26,6 +26,7 @@ import org.apache.kafka.connect.util.Callback; import org.apache.kafka.connect.util.KafkaBasedLog; import org.apache.kafka.connect.util.TopicAdmin; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/ConfigPropertyFilter.java b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/ConfigPropertyFilter.java index 9183ebcf8808b..bb4ace9a6a93c 100644 --- a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/ConfigPropertyFilter.java +++ b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/ConfigPropertyFilter.java @@ -19,6 +19,7 @@ import org.apache.kafka.common.Configurable; import org.apache.kafka.common.annotation.InterfaceStability; + import java.util.Map; /** Defines which topic configuration properties should be replicated. */ diff --git a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/DefaultConfigPropertyFilter.java b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/DefaultConfigPropertyFilter.java index 322bebbe1519f..a464e8f935228 100644 --- a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/DefaultConfigPropertyFilter.java +++ b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/DefaultConfigPropertyFilter.java @@ -18,8 +18,8 @@ import org.apache.kafka.common.config.AbstractConfig; import org.apache.kafka.common.config.ConfigDef; -import org.apache.kafka.common.config.ConfigDef.Type; import org.apache.kafka.common.config.ConfigDef.Importance; +import org.apache.kafka.common.config.ConfigDef.Type; import org.apache.kafka.common.utils.ConfigUtils; import java.util.Map; diff --git a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/DefaultGroupFilter.java b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/DefaultGroupFilter.java index e3eab88121de7..ccf0d3790f72a 100644 --- a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/DefaultGroupFilter.java +++ b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/DefaultGroupFilter.java @@ -18,8 +18,8 @@ import org.apache.kafka.common.config.AbstractConfig; import org.apache.kafka.common.config.ConfigDef; -import org.apache.kafka.common.config.ConfigDef.Type; import org.apache.kafka.common.config.ConfigDef.Importance; +import org.apache.kafka.common.config.ConfigDef.Type; import org.apache.kafka.common.utils.ConfigUtils; import java.util.Map; diff --git a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/DefaultTopicFilter.java b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/DefaultTopicFilter.java index 2b4637c272165..2fa4e193776ac 100644 --- a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/DefaultTopicFilter.java +++ b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/DefaultTopicFilter.java @@ -18,8 +18,8 @@ import org.apache.kafka.common.config.AbstractConfig; import org.apache.kafka.common.config.ConfigDef; -import org.apache.kafka.common.config.ConfigDef.Type; import org.apache.kafka.common.config.ConfigDef.Importance; +import org.apache.kafka.common.config.ConfigDef.Type; import org.apache.kafka.common.utils.ConfigUtils; import java.util.Map; diff --git a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/GroupFilter.java b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/GroupFilter.java index 0202dd5d2b358..d54274523a001 100644 --- a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/GroupFilter.java +++ b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/GroupFilter.java @@ -19,6 +19,7 @@ import org.apache.kafka.common.Configurable; import org.apache.kafka.common.annotation.InterfaceStability; + import java.util.Map; /** Defines which consumer groups should be replicated. */ diff --git a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorCheckpointConnector.java b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorCheckpointConnector.java index cb9158fd8abe6..04fe29bf6954b 100644 --- a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorCheckpointConnector.java +++ b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorCheckpointConnector.java @@ -27,6 +27,7 @@ import org.apache.kafka.connect.errors.ConnectException; import org.apache.kafka.connect.source.SourceConnector; import org.apache.kafka.connect.util.ConnectorUtils; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorCheckpointMetrics.java b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorCheckpointMetrics.java index 8885dcd82cc4e..8ace7d1fc3bdb 100644 --- a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorCheckpointMetrics.java +++ b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorCheckpointMetrics.java @@ -17,21 +17,21 @@ package org.apache.kafka.connect.mirror; import org.apache.kafka.common.MetricNameTemplate; +import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.metrics.MetricsReporter; import org.apache.kafka.common.metrics.Sensor; -import org.apache.kafka.common.metrics.stats.Value; -import org.apache.kafka.common.metrics.stats.Min; -import org.apache.kafka.common.metrics.stats.Max; import org.apache.kafka.common.metrics.stats.Avg; -import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.metrics.stats.Max; +import org.apache.kafka.common.metrics.stats.Min; +import org.apache.kafka.common.metrics.stats.Value; import java.util.Arrays; -import java.util.Set; -import java.util.HashSet; -import java.util.Map; import java.util.HashMap; +import java.util.HashSet; import java.util.LinkedHashMap; +import java.util.Map; +import java.util.Set; /** Metrics for replicated topic-partitions */ class MirrorCheckpointMetrics implements AutoCloseable { diff --git a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorCheckpointTask.java b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorCheckpointTask.java index 7f446efea5df3..e3bd4e41b1b44 100644 --- a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorCheckpointTask.java +++ b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorCheckpointTask.java @@ -18,33 +18,33 @@ import org.apache.kafka.clients.admin.Admin; import org.apache.kafka.clients.admin.ConsumerGroupDescription; +import org.apache.kafka.clients.consumer.OffsetAndMetadata; +import org.apache.kafka.clients.producer.RecordMetadata; import org.apache.kafka.common.ConsumerGroupState; import org.apache.kafka.common.KafkaFuture; -import org.apache.kafka.common.errors.UnknownMemberIdException; -import org.apache.kafka.connect.source.SourceTask; -import org.apache.kafka.connect.source.SourceRecord; -import org.apache.kafka.connect.data.Schema; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.errors.UnknownMemberIdException; import org.apache.kafka.common.utils.Utils; -import org.apache.kafka.clients.consumer.OffsetAndMetadata; -import org.apache.kafka.clients.producer.RecordMetadata; +import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.source.SourceRecord; +import org.apache.kafka.connect.source.SourceTask; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.time.Duration; +import java.util.ArrayList; +import java.util.Collections; import java.util.HashMap; -import java.util.Map.Entry; -import java.util.Map; import java.util.List; -import java.util.ArrayList; +import java.util.Map; +import java.util.Map.Entry; import java.util.Optional; import java.util.OptionalLong; import java.util.Set; -import java.util.Collections; +import java.util.concurrent.ExecutionException; import java.util.function.Function; import java.util.stream.Collectors; -import java.util.concurrent.ExecutionException; -import java.time.Duration; import java.util.stream.Stream; import static org.apache.kafka.connect.mirror.MirrorUtils.adminCall; diff --git a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorCheckpointTaskConfig.java b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorCheckpointTaskConfig.java index 757651383c9ef..a8db4989b297c 100644 --- a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorCheckpointTaskConfig.java +++ b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorCheckpointTaskConfig.java @@ -18,11 +18,11 @@ import org.apache.kafka.common.config.ConfigDef; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; import java.util.Map; import java.util.Set; -import java.util.List; -import java.util.HashSet; -import java.util.Collections; public class MirrorCheckpointTaskConfig extends MirrorCheckpointConfig { @@ -64,4 +64,3 @@ String entityLabel() { ConfigDef.Importance.LOW, "The index of the task"); } - diff --git a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorConnectorConfig.java b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorConnectorConfig.java index 731ef2015c453..aa626cfe0a263 100644 --- a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorConnectorConfig.java +++ b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorConnectorConfig.java @@ -16,25 +16,26 @@ */ package org.apache.kafka.connect.mirror; +import org.apache.kafka.clients.CommonClientConfigs; import org.apache.kafka.clients.admin.ForwardingAdmin; import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.config.AbstractConfig; import org.apache.kafka.common.config.ConfigDef; import org.apache.kafka.common.metrics.KafkaMetricsContext; -import org.apache.kafka.common.metrics.MetricsReporter; import org.apache.kafka.common.metrics.MetricsContext; -import org.apache.kafka.clients.CommonClientConfigs; +import org.apache.kafka.common.metrics.MetricsReporter; import org.apache.kafka.common.security.auth.SecurityProtocol; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.connect.runtime.ConnectorConfig; -import static org.apache.kafka.clients.consumer.ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG; -import static org.apache.kafka.clients.consumer.ConsumerConfig.AUTO_OFFSET_RESET_CONFIG; -import static org.apache.kafka.common.config.ConfigDef.CaseInsensitiveValidString.in; -import java.util.Map; +import java.time.Duration; import java.util.HashMap; import java.util.List; -import java.time.Duration; +import java.util.Map; + +import static org.apache.kafka.clients.consumer.ConsumerConfig.AUTO_OFFSET_RESET_CONFIG; +import static org.apache.kafka.clients.consumer.ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG; +import static org.apache.kafka.common.config.ConfigDef.CaseInsensitiveValidString.in; /** Shared config properties used by {@link MirrorSourceConnector}, {@link MirrorCheckpointConnector}, and {@link MirrorHeartbeatConnector}. *

diff --git a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorHeartbeatConnector.java b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorHeartbeatConnector.java index 6ab9fce31bee0..f9a844fecfa13 100644 --- a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorHeartbeatConnector.java +++ b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorHeartbeatConnector.java @@ -17,16 +17,16 @@ package org.apache.kafka.connect.mirror; import org.apache.kafka.clients.admin.Admin; -import org.apache.kafka.connect.connector.Task; -import org.apache.kafka.connect.errors.ConnectException; -import org.apache.kafka.connect.source.SourceConnector; import org.apache.kafka.common.config.ConfigDef; import org.apache.kafka.common.utils.AppInfoParser; import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.connect.connector.Task; +import org.apache.kafka.connect.errors.ConnectException; +import org.apache.kafka.connect.source.SourceConnector; -import java.util.Map; -import java.util.List; import java.util.Collections; +import java.util.List; +import java.util.Map; import static org.apache.kafka.connect.mirror.Heartbeat.SOURCE_CLUSTER_ALIAS_KEY; import static org.apache.kafka.connect.mirror.Heartbeat.TARGET_CLUSTER_ALIAS_KEY; diff --git a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorHeartbeatTask.java b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorHeartbeatTask.java index 6533c5162cb60..35c9c8feccb29 100644 --- a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorHeartbeatTask.java +++ b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorHeartbeatTask.java @@ -17,16 +17,16 @@ package org.apache.kafka.connect.mirror; import org.apache.kafka.clients.producer.RecordMetadata; -import org.apache.kafka.connect.source.SourceTask; -import org.apache.kafka.connect.source.SourceRecord; import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.source.SourceRecord; +import org.apache.kafka.connect.source.SourceTask; -import java.util.Map; -import java.util.List; +import java.time.Duration; import java.util.Collections; +import java.util.List; +import java.util.Map; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; -import java.time.Duration; /** Emits heartbeats. */ public class MirrorHeartbeatTask extends SourceTask { diff --git a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorHerder.java b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorHerder.java index 36bcdfac9029f..f92b666e3e6aa 100644 --- a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorHerder.java +++ b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorHerder.java @@ -25,6 +25,7 @@ import org.apache.kafka.connect.runtime.rest.RestClient; import org.apache.kafka.connect.storage.ConfigBackingStore; import org.apache.kafka.connect.storage.StatusBackingStore; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorMaker.java b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorMaker.java index 785edc44a165c..3bc7aed02b36e 100644 --- a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorMaker.java +++ b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorMaker.java @@ -16,57 +16,57 @@ */ package org.apache.kafka.connect.mirror; +import org.apache.kafka.common.utils.Exit; import org.apache.kafka.common.utils.Time; import org.apache.kafka.common.utils.Utils; -import org.apache.kafka.common.utils.Exit; +import org.apache.kafka.connect.connector.policy.AllConnectorClientConfigOverridePolicy; +import org.apache.kafka.connect.connector.policy.ConnectorClientConfigOverridePolicy; import org.apache.kafka.connect.json.JsonConverter; import org.apache.kafka.connect.json.JsonConverterConfig; import org.apache.kafka.connect.mirror.rest.MirrorRestServer; import org.apache.kafka.connect.runtime.Herder; -import org.apache.kafka.connect.runtime.isolation.Plugins; import org.apache.kafka.connect.runtime.Worker; import org.apache.kafka.connect.runtime.WorkerConfigTransformer; import org.apache.kafka.connect.runtime.distributed.DistributedConfig; +import org.apache.kafka.connect.runtime.isolation.Plugins; import org.apache.kafka.connect.runtime.rest.RestClient; import org.apache.kafka.connect.runtime.rest.entities.ConnectorStateInfo; import org.apache.kafka.connect.runtime.rest.entities.TaskInfo; -import org.apache.kafka.connect.storage.KafkaOffsetBackingStore; -import org.apache.kafka.connect.storage.StatusBackingStore; -import org.apache.kafka.connect.storage.KafkaStatusBackingStore; import org.apache.kafka.connect.storage.ConfigBackingStore; -import org.apache.kafka.connect.storage.KafkaConfigBackingStore; import org.apache.kafka.connect.storage.Converter; +import org.apache.kafka.connect.storage.KafkaConfigBackingStore; +import org.apache.kafka.connect.storage.KafkaOffsetBackingStore; +import org.apache.kafka.connect.storage.KafkaStatusBackingStore; +import org.apache.kafka.connect.storage.StatusBackingStore; import org.apache.kafka.connect.util.Callback; import org.apache.kafka.connect.util.ConnectUtils; -import org.apache.kafka.connect.connector.policy.AllConnectorClientConfigOverridePolicy; -import org.apache.kafka.connect.connector.policy.ConnectorClientConfigOverridePolicy; - import org.apache.kafka.connect.util.SharedTopicAdmin; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; +import net.sourceforge.argparse4j.ArgumentParsers; import net.sourceforge.argparse4j.impl.Arguments; -import net.sourceforge.argparse4j.inf.Namespace; import net.sourceforge.argparse4j.inf.ArgumentParser; import net.sourceforge.argparse4j.inf.ArgumentParserException; -import net.sourceforge.argparse4j.ArgumentParsers; +import net.sourceforge.argparse4j.inf.Namespace; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import java.io.File; import java.io.UnsupportedEncodingException; import java.net.URLEncoder; import java.nio.charset.StandardCharsets; +import java.util.Arrays; import java.util.Collections; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.Map; import java.util.HashMap; -import java.util.Set; import java.util.HashSet; import java.util.List; -import java.util.Arrays; +import java.util.Map; import java.util.Properties; +import java.util.Set; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.stream.Collectors; -import java.io.File; import static org.apache.kafka.clients.CommonClientConfigs.CLIENT_ID_CONFIG; diff --git a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorMakerConfig.java b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorMakerConfig.java index fd672f56a6ce8..8f9f06f058eda 100644 --- a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorMakerConfig.java +++ b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorMakerConfig.java @@ -16,30 +16,29 @@ */ package org.apache.kafka.connect.mirror; -import java.util.Arrays; -import java.util.Map.Entry; - -import org.apache.kafka.common.security.auth.SecurityProtocol; -import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.clients.CommonClientConfigs; import org.apache.kafka.common.config.AbstractConfig; import org.apache.kafka.common.config.ConfigDef; -import org.apache.kafka.common.config.ConfigDef.Type; import org.apache.kafka.common.config.ConfigDef.Importance; -import org.apache.kafka.common.config.provider.ConfigProvider; +import org.apache.kafka.common.config.ConfigDef.Type; import org.apache.kafka.common.config.ConfigTransformer; -import org.apache.kafka.clients.CommonClientConfigs; +import org.apache.kafka.common.config.provider.ConfigProvider; +import org.apache.kafka.common.security.auth.SecurityProtocol; +import org.apache.kafka.common.utils.Utils; import org.apache.kafka.connect.runtime.WorkerConfig; import org.apache.kafka.connect.runtime.distributed.DistributedConfig; import org.apache.kafka.connect.runtime.isolation.Plugins; import org.apache.kafka.connect.runtime.rest.RestServerConfig; -import java.util.Map; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; import java.util.HashMap; +import java.util.HashSet; import java.util.List; +import java.util.Map; +import java.util.Map.Entry; import java.util.Set; -import java.util.HashSet; -import java.util.ArrayList; -import java.util.Collections; import java.util.stream.Collectors; import static org.apache.kafka.common.config.ConfigDef.CaseInsensitiveValidString.in; diff --git a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorSourceConnector.java b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorSourceConnector.java index 19c84e53cc008..16fa033c370e6 100644 --- a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorSourceConnector.java +++ b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorSourceConnector.java @@ -16,62 +16,61 @@ */ package org.apache.kafka.connect.mirror; -import java.util.HashMap; -import java.util.Map.Entry; - -import java.util.concurrent.atomic.AtomicReference; import org.apache.kafka.clients.admin.Admin; +import org.apache.kafka.clients.admin.AlterConfigOp; +import org.apache.kafka.clients.admin.Config; +import org.apache.kafka.clients.admin.ConfigEntry; +import org.apache.kafka.clients.admin.CreateTopicsOptions; +import org.apache.kafka.clients.admin.NewPartitions; +import org.apache.kafka.clients.admin.NewTopic; +import org.apache.kafka.clients.admin.TopicDescription; import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.common.IsolationLevel; -import org.apache.kafka.common.config.ConfigValue; -import org.apache.kafka.common.errors.SecurityDisabledException; -import org.apache.kafka.connect.connector.Task; -import org.apache.kafka.connect.errors.ConnectException; -import org.apache.kafka.connect.source.ExactlyOnceSupport; -import org.apache.kafka.connect.source.SourceConnector; -import org.apache.kafka.common.config.ConfigDef; -import org.apache.kafka.common.config.ConfigResource; -import org.apache.kafka.common.acl.AclBinding; -import org.apache.kafka.common.acl.AclBindingFilter; +import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.acl.AccessControlEntry; import org.apache.kafka.common.acl.AccessControlEntryFilter; -import org.apache.kafka.common.acl.AclPermissionType; +import org.apache.kafka.common.acl.AclBinding; +import org.apache.kafka.common.acl.AclBindingFilter; import org.apache.kafka.common.acl.AclOperation; -import org.apache.kafka.common.resource.ResourceType; -import org.apache.kafka.common.resource.ResourcePattern; -import org.apache.kafka.common.resource.ResourcePatternFilter; -import org.apache.kafka.common.resource.PatternType; -import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.acl.AclPermissionType; +import org.apache.kafka.common.config.ConfigDef; +import org.apache.kafka.common.config.ConfigResource; +import org.apache.kafka.common.config.ConfigValue; import org.apache.kafka.common.errors.InvalidPartitionsException; +import org.apache.kafka.common.errors.SecurityDisabledException; import org.apache.kafka.common.errors.UnsupportedVersionException; +import org.apache.kafka.common.resource.PatternType; +import org.apache.kafka.common.resource.ResourcePattern; +import org.apache.kafka.common.resource.ResourcePatternFilter; +import org.apache.kafka.common.resource.ResourceType; import org.apache.kafka.common.utils.AppInfoParser; import org.apache.kafka.common.utils.Utils; -import org.apache.kafka.clients.admin.AlterConfigOp; -import org.apache.kafka.clients.admin.TopicDescription; -import org.apache.kafka.clients.admin.Config; -import org.apache.kafka.clients.admin.ConfigEntry; -import org.apache.kafka.clients.admin.NewPartitions; -import org.apache.kafka.clients.admin.NewTopic; -import org.apache.kafka.clients.admin.CreateTopicsOptions; +import org.apache.kafka.connect.connector.Task; +import org.apache.kafka.connect.errors.ConnectException; +import org.apache.kafka.connect.source.ExactlyOnceSupport; +import org.apache.kafka.connect.source.SourceConnector; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; -import java.util.Map; -import java.util.List; import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; import java.util.Objects; import java.util.Optional; import java.util.Set; -import java.util.HashSet; -import java.util.Collection; -import java.util.Collections; +import java.util.concurrent.ExecutionException; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicReference; import java.util.function.Function; import java.util.stream.Collectors; import java.util.stream.IntStream; import java.util.stream.Stream; -import java.util.concurrent.ExecutionException; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import static org.apache.kafka.connect.mirror.MirrorSourceConfig.SYNC_TOPIC_ACLS_ENABLED; import static org.apache.kafka.connect.mirror.MirrorUtils.SOURCE_CLUSTER_KEY; diff --git a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorSourceTask.java b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorSourceTask.java index e339cc1b6b05e..a875300ad717b 100644 --- a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorSourceTask.java +++ b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorSourceTask.java @@ -16,36 +16,36 @@ */ package org.apache.kafka.connect.mirror; -import org.apache.kafka.connect.data.Schema; -import org.apache.kafka.connect.source.SourceTask; -import org.apache.kafka.connect.source.SourceRecord; -import org.apache.kafka.connect.header.Headers; -import org.apache.kafka.connect.header.ConnectHeaders; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.consumer.ConsumerRecords; import org.apache.kafka.clients.consumer.KafkaConsumer; -import org.apache.kafka.clients.producer.RecordMetadata; -import org.apache.kafka.clients.producer.ProducerRecord; 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.header.Header; import org.apache.kafka.common.errors.WakeupException; -import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.header.Header; import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.header.ConnectHeaders; +import org.apache.kafka.connect.header.Headers; +import org.apache.kafka.connect.source.SourceRecord; +import org.apache.kafka.connect.source.SourceTask; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.time.Duration; +import java.util.ArrayList; +import java.util.HashMap; import java.util.Iterator; import java.util.LinkedHashMap; -import java.util.Map; -import java.util.HashMap; import java.util.List; +import java.util.Map; import java.util.Set; -import java.util.ArrayList; -import java.util.stream.Collectors; import java.util.concurrent.Semaphore; -import java.time.Duration; +import java.util.stream.Collectors; /** Replicates a set of topic-partitions. */ public class MirrorSourceTask extends SourceTask { diff --git a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorSourceTaskConfig.java b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorSourceTaskConfig.java index 3a2eacd1a9eb7..f0c562bbcbb08 100644 --- a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorSourceTaskConfig.java +++ b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorSourceTaskConfig.java @@ -16,13 +16,13 @@ */ package org.apache.kafka.connect.mirror; -import org.apache.kafka.common.config.ConfigDef; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.config.ConfigDef; +import java.util.Collections; +import java.util.List; import java.util.Map; import java.util.Set; -import java.util.List; -import java.util.Collections; import java.util.stream.Collectors; public class MirrorSourceTaskConfig extends MirrorSourceConfig { diff --git a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorUtils.java b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorUtils.java index dd52346f21a69..d8cbba184a48c 100644 --- a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorUtils.java +++ b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorUtils.java @@ -19,8 +19,9 @@ import org.apache.kafka.clients.admin.Admin; import org.apache.kafka.clients.admin.CreateTopicsOptions; import org.apache.kafka.clients.admin.NewTopic; -import org.apache.kafka.clients.producer.KafkaProducer; import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.clients.producer.KafkaProducer; +import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.errors.ClusterAuthorizationException; import org.apache.kafka.common.errors.GroupAuthorizationException; import org.apache.kafka.common.errors.InvalidConfigurationException; @@ -30,18 +31,18 @@ import org.apache.kafka.common.errors.UnsupportedVersionException; import org.apache.kafka.common.serialization.ByteArrayDeserializer; import org.apache.kafka.common.serialization.ByteArraySerializer; -import org.apache.kafka.common.TopicPartition; import org.apache.kafka.connect.errors.ConnectException; import org.apache.kafka.connect.source.SourceRecord; import org.apache.kafka.connect.util.TopicAdmin; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.util.Arrays; -import java.util.Map; -import java.util.List; -import java.util.HashMap; import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; import java.util.Objects; import java.util.concurrent.Callable; import java.util.concurrent.ExecutionException; diff --git a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/OffsetSync.java b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/OffsetSync.java index e1ecb1e1dbad0..c46aac634fba5 100644 --- a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/OffsetSync.java +++ b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/OffsetSync.java @@ -16,12 +16,12 @@ */ package org.apache.kafka.connect.mirror; +import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.protocol.types.Field; import org.apache.kafka.common.protocol.types.Schema; import org.apache.kafka.common.protocol.types.Struct; import org.apache.kafka.common.protocol.types.Type; -import org.apache.kafka.clients.consumer.ConsumerRecord; import java.nio.ByteBuffer; @@ -117,4 +117,3 @@ byte[] recordValue() { return serializeValue().array(); } } - diff --git a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/OffsetSyncStore.java b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/OffsetSyncStore.java index 16038044ddd2b..635ab7327735e 100644 --- a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/OffsetSyncStore.java +++ b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/OffsetSyncStore.java @@ -23,6 +23,7 @@ import org.apache.kafka.common.utils.Utils; import org.apache.kafka.connect.util.KafkaBasedLog; import org.apache.kafka.connect.util.TopicAdmin; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/Scheduler.java b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/Scheduler.java index 642252a3e2536..9a39242d40bdf 100644 --- a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/Scheduler.java +++ b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/Scheduler.java @@ -16,15 +16,15 @@ */ package org.apache.kafka.connect.mirror; -import java.util.concurrent.ScheduledExecutorService; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.time.Duration; +import java.util.concurrent.ExecutionException; import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; -import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeoutException; -import java.time.Duration; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; class Scheduler implements AutoCloseable { private static final Logger LOG = LoggerFactory.getLogger(Scheduler.class); @@ -112,4 +112,3 @@ private void executeThread(Task task, String description) { run(task, description); } } - diff --git a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/TopicFilter.java b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/TopicFilter.java index f13453f116850..902b09287800e 100644 --- a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/TopicFilter.java +++ b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/TopicFilter.java @@ -19,6 +19,7 @@ import org.apache.kafka.common.Configurable; import org.apache.kafka.common.annotation.InterfaceStability; + import java.util.Map; /** Defines which topics should be replicated. */ diff --git a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/formatters/CheckpointFormatter.java b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/formatters/CheckpointFormatter.java index 33fe695874240..8d2285a7832bb 100644 --- a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/formatters/CheckpointFormatter.java +++ b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/formatters/CheckpointFormatter.java @@ -16,12 +16,12 @@ */ package org.apache.kafka.connect.mirror.formatters; -import java.io.PrintStream; - import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.common.MessageFormatter; import org.apache.kafka.connect.mirror.Checkpoint; +import java.io.PrintStream; + public class CheckpointFormatter implements MessageFormatter { @Override diff --git a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/formatters/HeartbeatFormatter.java b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/formatters/HeartbeatFormatter.java index a193dbe153099..b7d4be3a35545 100644 --- a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/formatters/HeartbeatFormatter.java +++ b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/formatters/HeartbeatFormatter.java @@ -16,12 +16,12 @@ */ package org.apache.kafka.connect.mirror.formatters; -import java.io.PrintStream; - import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.common.MessageFormatter; import org.apache.kafka.connect.mirror.Heartbeat; +import java.io.PrintStream; + public class HeartbeatFormatter implements MessageFormatter { @Override diff --git a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/formatters/OffsetSyncFormatter.java b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/formatters/OffsetSyncFormatter.java index dacae600ae53d..cc5466bda965c 100644 --- a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/formatters/OffsetSyncFormatter.java +++ b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/formatters/OffsetSyncFormatter.java @@ -16,12 +16,12 @@ */ package org.apache.kafka.connect.mirror.formatters; -import java.io.PrintStream; - import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.common.MessageFormatter; import org.apache.kafka.connect.mirror.OffsetSync; +import java.io.PrintStream; + public class OffsetSyncFormatter implements MessageFormatter { @Override diff --git a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/rest/MirrorRestServer.java b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/rest/MirrorRestServer.java index 7d24a5f14db9c..f09cb12b0f060 100644 --- a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/rest/MirrorRestServer.java +++ b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/rest/MirrorRestServer.java @@ -22,6 +22,7 @@ import org.apache.kafka.connect.runtime.rest.RestClient; import org.apache.kafka.connect.runtime.rest.RestServer; import org.apache.kafka.connect.runtime.rest.RestServerConfig; + import org.glassfish.hk2.api.TypeLiteral; import org.glassfish.hk2.utilities.binding.AbstractBinder; import org.glassfish.jersey.server.ResourceConfig; diff --git a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/rest/resources/InternalMirrorResource.java b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/rest/resources/InternalMirrorResource.java index 752fb0ffce4bd..06480bcf4a5a4 100644 --- a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/rest/resources/InternalMirrorResource.java +++ b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/rest/resources/InternalMirrorResource.java @@ -22,12 +22,13 @@ import org.apache.kafka.connect.runtime.rest.RestRequestTimeout; import org.apache.kafka.connect.runtime.rest.resources.InternalClusterResource; +import java.util.Map; + import javax.inject.Inject; import javax.ws.rs.NotFoundException; import javax.ws.rs.Path; import javax.ws.rs.core.Context; import javax.ws.rs.core.UriInfo; -import java.util.Map; @Path("/{source}/{target}/connectors") public class InternalMirrorResource extends InternalClusterResource { diff --git a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/CheckpointStoreTest.java b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/CheckpointStoreTest.java index b7b3904899f1a..476fbcceaef82 100644 --- a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/CheckpointStoreTest.java +++ b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/CheckpointStoreTest.java @@ -20,6 +20,7 @@ import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.errors.TopicAuthorizationException; import org.apache.kafka.connect.util.Callback; + import org.junit.jupiter.api.Test; import java.util.Collections; @@ -28,9 +29,9 @@ import java.util.Map; import java.util.Set; -import static org.junit.jupiter.api.Assertions.assertTrue; -import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; diff --git a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorCheckpointConfigTest.java b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorCheckpointConfigTest.java index 495fd2ebe1594..edbaf0955704d 100644 --- a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorCheckpointConfigTest.java +++ b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorCheckpointConfigTest.java @@ -17,6 +17,7 @@ package org.apache.kafka.connect.mirror; import org.apache.kafka.common.config.ConfigDef; + import org.junit.jupiter.api.Test; import java.util.Arrays; diff --git a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorCheckpointConnectorTest.java b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorCheckpointConnectorTest.java index ecc9fcbc11f6a..7e5b6d83587ee 100644 --- a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorCheckpointConnectorTest.java +++ b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorCheckpointConnectorTest.java @@ -20,6 +20,7 @@ import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.connect.errors.ConnectException; + import org.junit.jupiter.api.Test; import java.util.Arrays; diff --git a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorCheckpointTaskTest.java b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorCheckpointTaskTest.java index 0afc4f74f2f97..f4cc1e4ced6a4 100644 --- a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorCheckpointTaskTest.java +++ b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorCheckpointTaskTest.java @@ -16,26 +16,25 @@ */ package org.apache.kafka.connect.mirror; +import org.apache.kafka.clients.consumer.OffsetAndMetadata; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.connect.source.SourceRecord; + +import org.junit.jupiter.api.Test; + +import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.Collections; import java.util.Optional; import java.util.OptionalLong; import java.util.concurrent.ExecutionException; -import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.clients.consumer.OffsetAndMetadata; -import org.apache.kafka.connect.source.SourceRecord; - -import org.junit.jupiter.api.Test; - import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertNotEquals; import static org.junit.jupiter.api.Assertions.assertNull; import static org.junit.jupiter.api.Assertions.assertTrue; - import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; diff --git a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorConnectorConfigTest.java b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorConnectorConfigTest.java index 3840c49114f15..817d1684b16e2 100644 --- a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorConnectorConfigTest.java +++ b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorConnectorConfigTest.java @@ -21,17 +21,18 @@ import org.apache.kafka.common.metrics.JmxReporter; import org.apache.kafka.common.security.auth.SecurityProtocol; import org.apache.kafka.test.MockMetricsReporter; + import org.junit.jupiter.api.Test; +import java.util.HashMap; import java.util.Locale; import java.util.Map; -import java.util.HashMap; import static org.apache.kafka.connect.mirror.TestUtils.makeProps; -import static org.junit.jupiter.api.Assertions.assertTrue; -import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertSame; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; public class MirrorConnectorConfigTest { diff --git a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorHeartBeatConnectorTest.java b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorHeartBeatConnectorTest.java index 0248e487c18e7..190f749d4e71b 100644 --- a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorHeartBeatConnectorTest.java +++ b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorHeartBeatConnectorTest.java @@ -16,12 +16,9 @@ */ package org.apache.kafka.connect.mirror; -import static org.apache.kafka.connect.mirror.Heartbeat.SOURCE_CLUSTER_ALIAS_KEY; -import static org.apache.kafka.connect.mirror.Heartbeat.TARGET_CLUSTER_ALIAS_KEY; -import static org.apache.kafka.connect.mirror.TestUtils.makeProps; -import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertThrows; -import static org.junit.jupiter.api.Assertions.assertTrue; +import org.apache.kafka.connect.errors.ConnectException; + +import org.junit.jupiter.api.Test; import java.util.Arrays; import java.util.Collections; @@ -30,8 +27,12 @@ import java.util.Map; import java.util.function.Function; -import org.apache.kafka.connect.errors.ConnectException; -import org.junit.jupiter.api.Test; +import static org.apache.kafka.connect.mirror.Heartbeat.SOURCE_CLUSTER_ALIAS_KEY; +import static org.apache.kafka.connect.mirror.Heartbeat.TARGET_CLUSTER_ALIAS_KEY; +import static org.apache.kafka.connect.mirror.TestUtils.makeProps; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; public class MirrorHeartBeatConnectorTest { diff --git a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorHeartbeatTaskTest.java b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorHeartbeatTaskTest.java index 39fd6dff10e30..0ffe2635d1491 100644 --- a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorHeartbeatTaskTest.java +++ b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorHeartbeatTaskTest.java @@ -17,6 +17,7 @@ package org.apache.kafka.connect.mirror; import org.apache.kafka.connect.source.SourceRecord; + import org.junit.jupiter.api.Test; import java.util.List; diff --git a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorMakerConfigTest.java b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorMakerConfigTest.java index fe6011d203368..163ebdd4b7b91 100644 --- a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorMakerConfigTest.java +++ b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorMakerConfigTest.java @@ -19,22 +19,22 @@ import org.apache.kafka.clients.CommonClientConfigs; import org.apache.kafka.clients.admin.FakeForwardingAdmin; import org.apache.kafka.clients.admin.ForwardingAdmin; +import org.apache.kafka.common.config.ConfigData; import org.apache.kafka.common.config.ConfigException; -import org.apache.kafka.common.config.types.Password; import org.apache.kafka.common.config.provider.ConfigProvider; -import org.apache.kafka.common.config.ConfigData; +import org.apache.kafka.common.config.types.Password; import org.apache.kafka.common.metrics.FakeMetricsReporter; - import org.apache.kafka.common.security.auth.SecurityProtocol; + import org.junit.jupiter.api.Test; -import java.util.Locale; -import java.util.Map; -import java.util.Set; +import java.util.Arrays; import java.util.Collections; import java.util.HashMap; -import java.util.Arrays; import java.util.List; +import java.util.Locale; +import java.util.Map; +import java.util.Set; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; diff --git a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorSourceConfigTest.java b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorSourceConfigTest.java index b703731073761..e4d6082fa7fca 100644 --- a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorSourceConfigTest.java +++ b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorSourceConfigTest.java @@ -19,6 +19,7 @@ import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.config.ConfigDef; import org.apache.kafka.common.config.ConfigException; + import org.junit.jupiter.api.Test; import java.util.Arrays; diff --git a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorSourceConnectorTest.java b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorSourceConnectorTest.java index 3aeb3837b8717..5d7a3a020f57f 100644 --- a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorSourceConnectorTest.java +++ b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorSourceConnectorTest.java @@ -16,10 +16,13 @@ */ package org.apache.kafka.connect.mirror; -import org.apache.kafka.clients.admin.AlterConfigOp; import org.apache.kafka.clients.admin.Admin; +import org.apache.kafka.clients.admin.AlterConfigOp; +import org.apache.kafka.clients.admin.Config; +import org.apache.kafka.clients.admin.ConfigEntry; import org.apache.kafka.clients.admin.DescribeAclsResult; import org.apache.kafka.clients.admin.DescribeConfigsResult; +import org.apache.kafka.clients.admin.NewTopic; import org.apache.kafka.common.KafkaFuture; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.acl.AccessControlEntry; @@ -28,22 +31,33 @@ import org.apache.kafka.common.acl.AclPermissionType; import org.apache.kafka.common.config.ConfigResource; import org.apache.kafka.common.config.ConfigValue; +import org.apache.kafka.common.errors.SecurityDisabledException; import org.apache.kafka.common.errors.TopicAuthorizationException; import org.apache.kafka.common.errors.UnsupportedVersionException; -import org.apache.kafka.common.errors.SecurityDisabledException; import org.apache.kafka.common.resource.PatternType; import org.apache.kafka.common.resource.ResourcePattern; import org.apache.kafka.common.resource.ResourceType; -import org.apache.kafka.clients.admin.Config; import org.apache.kafka.common.utils.LogCaptureAppender; import org.apache.kafka.connect.connector.ConnectorContext; -import org.apache.kafka.clients.admin.ConfigEntry; -import org.apache.kafka.clients.admin.NewTopic; - import org.apache.kafka.connect.errors.ConnectException; import org.apache.kafka.connect.source.ExactlyOnceSupport; + import org.junit.jupiter.api.Test; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.ExecutionException; +import java.util.function.Function; +import java.util.stream.Collectors; + import static org.apache.kafka.clients.admin.AdminClientTestUtils.alterConfigsResult; import static org.apache.kafka.clients.consumer.ConsumerConfig.ISOLATION_LEVEL_CONFIG; import static org.apache.kafka.connect.mirror.MirrorConnectorConfig.CONSUMER_CLIENT_PREFIX; @@ -54,13 +68,13 @@ import static org.apache.kafka.connect.mirror.MirrorUtils.SOURCE_CLUSTER_KEY; import static org.apache.kafka.connect.mirror.MirrorUtils.TOPIC_KEY; import static org.apache.kafka.connect.mirror.TestUtils.makeProps; +import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertThrows; -import static org.junit.jupiter.api.Assertions.assertTrue; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertNull; -import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; import static org.mockito.ArgumentMatchers.isA; import static org.mockito.Mockito.any; import static org.mockito.Mockito.doAnswer; @@ -75,20 +89,6 @@ import static org.mockito.Mockito.verifyNoInteractions; import static org.mockito.Mockito.when; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collection; -import java.util.Collections; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Optional; -import java.util.Set; -import java.util.concurrent.ExecutionException; -import java.util.function.Function; -import java.util.stream.Collectors; - public class MirrorSourceConnectorTest { private ConfigPropertyFilter getConfigPropertyFilter() { return prop -> true; diff --git a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorSourceMetricsTest.java b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorSourceMetricsTest.java index ae43c72dcb066..1aafdf1626417 100644 --- a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorSourceMetricsTest.java +++ b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorSourceMetricsTest.java @@ -20,6 +20,7 @@ import org.apache.kafka.common.metrics.KafkaMetric; import org.apache.kafka.common.metrics.MetricsReporter; import org.apache.kafka.connect.runtime.ConnectorConfig; + import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; diff --git a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorSourceTaskTest.java b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorSourceTaskTest.java index 5a5046fd0f6dc..8e513ffd82bc0 100644 --- a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorSourceTaskTest.java +++ b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorSourceTaskTest.java @@ -22,17 +22,17 @@ import org.apache.kafka.clients.producer.Callback; import org.apache.kafka.clients.producer.KafkaProducer; import org.apache.kafka.clients.producer.RecordMetadata; -import org.apache.kafka.common.header.Header; -import org.apache.kafka.common.header.internals.RecordHeader; -import org.apache.kafka.common.record.TimestampType; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.header.Header; import org.apache.kafka.common.header.Headers; +import org.apache.kafka.common.header.internals.RecordHeader; import org.apache.kafka.common.header.internals.RecordHeaders; +import org.apache.kafka.common.record.TimestampType; import org.apache.kafka.connect.mirror.MirrorSourceTask.PartitionState; import org.apache.kafka.connect.source.SourceRecord; - import org.apache.kafka.connect.source.SourceTaskContext; import org.apache.kafka.connect.storage.OffsetStorageReader; + import org.junit.jupiter.api.Test; import org.mockito.ArgumentCaptor; @@ -54,12 +54,11 @@ import static org.mockito.ArgumentMatchers.anyMap; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.mock; - import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.verifyNoInteractions; import static org.mockito.Mockito.verifyNoMoreInteractions; import static org.mockito.Mockito.when; -import static org.mockito.Mockito.verifyNoInteractions; public class MirrorSourceTaskTest { diff --git a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorUtilsTest.java b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorUtilsTest.java index 99cdb610759f6..e6de8a58f7b26 100644 --- a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorUtilsTest.java +++ b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorUtilsTest.java @@ -26,6 +26,7 @@ import org.apache.kafka.common.errors.TopicExistsException; import org.apache.kafka.common.errors.UnsupportedVersionException; import org.apache.kafka.connect.errors.ConnectException; + import org.junit.jupiter.api.Test; import java.util.Collections; diff --git a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/clients/admin/FakeForwardingAdminWithLocalMetadata.java b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/clients/admin/FakeForwardingAdminWithLocalMetadata.java index 3ac8a8b17f00d..1f2f56166a23f 100644 --- a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/clients/admin/FakeForwardingAdminWithLocalMetadata.java +++ b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/clients/admin/FakeForwardingAdminWithLocalMetadata.java @@ -32,6 +32,7 @@ import org.apache.kafka.common.acl.AclBinding; import org.apache.kafka.common.config.ConfigResource; import org.apache.kafka.common.errors.TopicExistsException; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/clients/admin/FakeLocalMetadataStore.java b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/clients/admin/FakeLocalMetadataStore.java index 99706d4eaeaac..1c08cbaf72ef9 100644 --- a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/clients/admin/FakeLocalMetadataStore.java +++ b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/clients/admin/FakeLocalMetadataStore.java @@ -20,6 +20,7 @@ import org.apache.kafka.clients.admin.Config; import org.apache.kafka.clients.admin.NewTopic; import org.apache.kafka.common.acl.AclBinding; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/DedicatedMirrorIntegrationTest.java b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/DedicatedMirrorIntegrationTest.java index 07bbafd751622..6e35e962ed92a 100644 --- a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/DedicatedMirrorIntegrationTest.java +++ b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/DedicatedMirrorIntegrationTest.java @@ -35,6 +35,7 @@ import org.apache.kafka.connect.util.FutureCallback; import org.apache.kafka.connect.util.clusters.EmbeddedKafkaCluster; import org.apache.kafka.test.NoRetryException; + import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Tag; diff --git a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/IdentityReplicationIntegrationTest.java b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/IdentityReplicationIntegrationTest.java index 8dc04e6074701..0a6ab4bab158c 100644 --- a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/IdentityReplicationIntegrationTest.java +++ b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/IdentityReplicationIntegrationTest.java @@ -18,11 +18,10 @@ import org.apache.kafka.connect.mirror.IdentityReplicationPolicy; -import java.util.HashMap; - +import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Tag; -import org.junit.jupiter.api.BeforeEach; +import java.util.HashMap; /** * Tests MM2 replication and failover logic for {@link IdentityReplicationPolicy}. diff --git a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/MirrorConnectorsIntegrationBaseTest.java b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/MirrorConnectorsIntegrationBaseTest.java index 4fbd282d11ca4..3f169b46920a8 100644 --- a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/MirrorConnectorsIntegrationBaseTest.java +++ b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/MirrorConnectorsIntegrationBaseTest.java @@ -31,23 +31,23 @@ 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.TopicPartition; import org.apache.kafka.common.config.ConfigResource; import org.apache.kafka.common.config.TopicConfig; import org.apache.kafka.common.utils.Exit; -import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.utils.Time; import org.apache.kafka.common.utils.Timer; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.connect.connector.Connector; +import org.apache.kafka.connect.mirror.Checkpoint; import org.apache.kafka.connect.mirror.DefaultConfigPropertyFilter; +import org.apache.kafka.connect.mirror.MirrorCheckpointConnector; import org.apache.kafka.connect.mirror.MirrorClient; import org.apache.kafka.connect.mirror.MirrorHeartbeatConnector; import org.apache.kafka.connect.mirror.MirrorMakerConfig; import org.apache.kafka.connect.mirror.MirrorSourceConnector; import org.apache.kafka.connect.mirror.MirrorUtils; import org.apache.kafka.connect.mirror.SourceAndTarget; -import org.apache.kafka.connect.mirror.Checkpoint; -import org.apache.kafka.connect.mirror.MirrorCheckpointConnector; import org.apache.kafka.connect.mirror.TestUtils; import org.apache.kafka.connect.runtime.rest.entities.ConnectorOffset; import org.apache.kafka.connect.runtime.rest.entities.ConnectorOffsets; @@ -55,14 +55,21 @@ import org.apache.kafka.connect.util.clusters.EmbeddedKafkaCluster; import org.apache.kafka.connect.util.clusters.UngracefulShutdownException; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Tag; +import org.junit.jupiter.api.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import java.time.Duration; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; -import java.util.HashSet; -import java.util.List; import java.util.Collections; import java.util.HashMap; +import java.util.HashSet; +import java.util.List; import java.util.Map; import java.util.Objects; import java.util.Properties; @@ -77,20 +84,13 @@ import java.util.function.LongUnaryOperator; import java.util.stream.Collectors; -import org.junit.jupiter.api.Tag; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import static org.apache.kafka.clients.consumer.ConsumerConfig.AUTO_OFFSET_RESET_CONFIG; import static org.apache.kafka.test.TestUtils.waitForCondition; import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertNotEquals; import static org.junit.jupiter.api.Assertions.assertFalse; -import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.assertNotEquals; import static org.junit.jupiter.api.Assertions.assertNotNull; -import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.AfterEach; -import org.junit.jupiter.api.BeforeEach; +import static org.junit.jupiter.api.Assertions.assertTrue; /** * Tests MM2 replication and failover/failback logic. diff --git a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/MirrorConnectorsIntegrationExactlyOnceTest.java b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/MirrorConnectorsIntegrationExactlyOnceTest.java index 36e7b34dd54d1..e02cc4c02b332 100644 --- a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/MirrorConnectorsIntegrationExactlyOnceTest.java +++ b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/MirrorConnectorsIntegrationExactlyOnceTest.java @@ -19,6 +19,7 @@ import org.apache.kafka.connect.connector.Connector; import org.apache.kafka.connect.mirror.MirrorSourceConnector; import org.apache.kafka.connect.runtime.distributed.DistributedConfig; + import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Tag; import org.junit.jupiter.api.Test; diff --git a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/MirrorConnectorsIntegrationSSLTest.java b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/MirrorConnectorsIntegrationSSLTest.java index 95f8a704085c3..18f734a00a2ac 100644 --- a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/MirrorConnectorsIntegrationSSLTest.java +++ b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/MirrorConnectorsIntegrationSSLTest.java @@ -16,10 +16,6 @@ */ package org.apache.kafka.connect.mirror.integration; -import java.util.Map; -import java.util.Properties; -import java.util.stream.Collectors; - import org.apache.kafka.clients.CommonClientConfigs; import org.apache.kafka.common.config.SslConfigs; import org.apache.kafka.common.config.types.Password; @@ -32,6 +28,10 @@ import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Tag; +import java.util.Map; +import java.util.Properties; +import java.util.stream.Collectors; + /** * Tests MM2 replication with SSL enabled at backup kafka cluster */ @@ -64,4 +64,3 @@ public void startClusters() throws Exception { super.startClusters(); } } - diff --git a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/MirrorConnectorsIntegrationTransactionsTest.java b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/MirrorConnectorsIntegrationTransactionsTest.java index c192d420375ce..87fc48fd3f3d2 100644 --- a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/MirrorConnectorsIntegrationTransactionsTest.java +++ b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/MirrorConnectorsIntegrationTransactionsTest.java @@ -21,6 +21,7 @@ import org.apache.kafka.clients.producer.ProducerConfig; import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.connect.util.clusters.EmbeddedConnectCluster; + import org.junit.jupiter.api.BeforeEach; import java.util.HashMap; diff --git a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/MirrorConnectorsWithCustomForwardingAdminIntegrationTest.java b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/MirrorConnectorsWithCustomForwardingAdminIntegrationTest.java index 70f1cd6f6a343..d94ce632ae10c 100644 --- a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/MirrorConnectorsWithCustomForwardingAdminIntegrationTest.java +++ b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/MirrorConnectorsWithCustomForwardingAdminIntegrationTest.java @@ -33,6 +33,7 @@ import org.apache.kafka.connect.mirror.clients.admin.FakeForwardingAdminWithLocalMetadata; import org.apache.kafka.connect.mirror.clients.admin.FakeLocalMetadataStore; import org.apache.kafka.connect.util.clusters.EmbeddedKafkaCluster; + import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Tag; diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/OffsetsApiIntegrationTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/OffsetsApiIntegrationTest.java index dc507b68df7c6..2da52cf9abd05 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/OffsetsApiIntegrationTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/OffsetsApiIntegrationTest.java @@ -143,6 +143,15 @@ private static EmbeddedConnectCluster createOrReuseConnectWithWorkerProps(Map The type of connect record. diff --git a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/predicates/RecordIsTombstone.java b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/predicates/RecordIsTombstone.java index dc1f602b4a384..848a8452b2879 100644 --- a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/predicates/RecordIsTombstone.java +++ b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/predicates/RecordIsTombstone.java @@ -16,13 +16,13 @@ */ package org.apache.kafka.connect.transforms.predicates; -import java.util.Map; - import org.apache.kafka.common.config.ConfigDef; import org.apache.kafka.common.utils.AppInfoParser; import org.apache.kafka.connect.components.Versioned; import org.apache.kafka.connect.connector.ConnectRecord; +import java.util.Map; + /** * A predicate which is true for records which are tombstones (i.e. have null value). * @param The type of connect record. diff --git a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/predicates/TopicNameMatches.java b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/predicates/TopicNameMatches.java index 7e78a69df0010..bcd519eba5a67 100644 --- a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/predicates/TopicNameMatches.java +++ b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/predicates/TopicNameMatches.java @@ -16,10 +16,6 @@ */ package org.apache.kafka.connect.transforms.predicates; -import java.util.Map; -import java.util.regex.Pattern; -import java.util.regex.PatternSyntaxException; - import org.apache.kafka.common.config.ConfigDef; import org.apache.kafka.common.config.ConfigException; import org.apache.kafka.common.utils.AppInfoParser; @@ -28,6 +24,10 @@ import org.apache.kafka.connect.transforms.util.RegexValidator; import org.apache.kafka.connect.transforms.util.SimpleConfig; +import java.util.Map; +import java.util.regex.Pattern; +import java.util.regex.PatternSyntaxException; + /** * A predicate which is true for records with a topic name that matches the configured regular expression. * @param The type of connect record. diff --git a/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/CastTest.java b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/CastTest.java index d6a6a7fb4f9ad..a70fac1d1db6f 100644 --- a/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/CastTest.java +++ b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/CastTest.java @@ -29,6 +29,7 @@ import org.apache.kafka.connect.data.Values; import org.apache.kafka.connect.errors.DataException; import org.apache.kafka.connect.source.SourceRecord; + import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.Test; diff --git a/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/DropHeadersTest.java b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/DropHeadersTest.java index 95649bd2b3d04..d164512897b64 100644 --- a/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/DropHeadersTest.java +++ b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/DropHeadersTest.java @@ -21,6 +21,7 @@ import org.apache.kafka.connect.data.Schema; import org.apache.kafka.connect.header.ConnectHeaders; import org.apache.kafka.connect.source.SourceRecord; + import org.junit.jupiter.api.Test; import java.util.HashMap; @@ -119,4 +120,3 @@ private SourceRecord sourceRecord(ConnectHeaders headers) { keySchema, key, valueSchema, value, timestamp, headers); } } - diff --git a/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/ExtractFieldTest.java b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/ExtractFieldTest.java index 9b7d53085de51..2ac2f91829bf0 100644 --- a/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/ExtractFieldTest.java +++ b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/ExtractFieldTest.java @@ -16,17 +16,18 @@ */ package org.apache.kafka.connect.transforms; -import java.util.HashMap; import org.apache.kafka.common.utils.AppInfoParser; import org.apache.kafka.connect.data.Schema; import org.apache.kafka.connect.data.SchemaBuilder; import org.apache.kafka.connect.data.Struct; import org.apache.kafka.connect.sink.SinkRecord; import org.apache.kafka.connect.transforms.field.FieldSyntaxVersion; + import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.Test; import java.util.Collections; +import java.util.HashMap; import java.util.Map; import static org.junit.jupiter.api.Assertions.assertEquals; diff --git a/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/FlattenTest.java b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/FlattenTest.java index c418ba427c91e..f771d4f0ac3e4 100644 --- a/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/FlattenTest.java +++ b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/FlattenTest.java @@ -23,6 +23,7 @@ import org.apache.kafka.connect.data.Struct; import org.apache.kafka.connect.errors.DataException; import org.apache.kafka.connect.source.SourceRecord; + import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.Test; diff --git a/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/HeaderFromTest.java b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/HeaderFromTest.java index 61e05757474b6..91efd3fb3e195 100644 --- a/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/HeaderFromTest.java +++ b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/HeaderFromTest.java @@ -25,6 +25,7 @@ import org.apache.kafka.connect.header.Header; import org.apache.kafka.connect.header.Headers; import org.apache.kafka.connect.source.SourceRecord; + import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.Arguments; import org.junit.jupiter.params.provider.MethodSource; @@ -356,4 +357,3 @@ private static void assertSameRecord(SourceRecord expected, SourceRecord xformed } } - diff --git a/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/HoistFieldTest.java b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/HoistFieldTest.java index 63f416f99f392..b72dddcdd155c 100644 --- a/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/HoistFieldTest.java +++ b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/HoistFieldTest.java @@ -20,6 +20,7 @@ import org.apache.kafka.connect.data.Schema; import org.apache.kafka.connect.data.Struct; import org.apache.kafka.connect.sink.SinkRecord; + import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.Test; diff --git a/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/InsertFieldTest.java b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/InsertFieldTest.java index 727f9d0b21719..f9754aa5a52b2 100644 --- a/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/InsertFieldTest.java +++ b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/InsertFieldTest.java @@ -23,6 +23,7 @@ import org.apache.kafka.connect.data.Timestamp; import org.apache.kafka.connect.errors.DataException; import org.apache.kafka.connect.source.SourceRecord; + import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.Test; diff --git a/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/InsertHeaderTest.java b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/InsertHeaderTest.java index 237662701f4a4..20c5b67a50a93 100644 --- a/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/InsertHeaderTest.java +++ b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/InsertHeaderTest.java @@ -22,6 +22,7 @@ import org.apache.kafka.connect.header.ConnectHeaders; import org.apache.kafka.connect.header.Headers; import org.apache.kafka.connect.source.SourceRecord; + import org.junit.jupiter.api.Test; import java.util.HashMap; @@ -123,4 +124,3 @@ public void testInsertHeaderVersionRetrievedFromAppInfoParser() { assertEquals(AppInfoParser.getVersion(), xform.version()); } } - diff --git a/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/MaskFieldTest.java b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/MaskFieldTest.java index 2f5d7feaf2188..1f5a9641d2854 100644 --- a/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/MaskFieldTest.java +++ b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/MaskFieldTest.java @@ -27,6 +27,7 @@ import org.apache.kafka.connect.data.Timestamp; import org.apache.kafka.connect.errors.DataException; import org.apache.kafka.connect.sink.SinkRecord; + import org.junit.jupiter.api.Test; import java.math.BigDecimal; diff --git a/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/RegexRouterTest.java b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/RegexRouterTest.java index 5ba2367545b00..6ec7601bb625c 100644 --- a/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/RegexRouterTest.java +++ b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/RegexRouterTest.java @@ -18,6 +18,7 @@ import org.apache.kafka.common.utils.AppInfoParser; import org.apache.kafka.connect.sink.SinkRecord; + import org.junit.jupiter.api.Test; import java.util.HashMap; diff --git a/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/ReplaceFieldTest.java b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/ReplaceFieldTest.java index 9df4013c9a4dd..53566cada7cd5 100644 --- a/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/ReplaceFieldTest.java +++ b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/ReplaceFieldTest.java @@ -21,6 +21,7 @@ import org.apache.kafka.connect.data.SchemaBuilder; import org.apache.kafka.connect.data.Struct; import org.apache.kafka.connect.sink.SinkRecord; + import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.Test; diff --git a/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/SetSchemaMetadataTest.java b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/SetSchemaMetadataTest.java index 8c0f45ce8658c..46f8be778876a 100644 --- a/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/SetSchemaMetadataTest.java +++ b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/SetSchemaMetadataTest.java @@ -23,6 +23,7 @@ import org.apache.kafka.connect.data.Struct; import org.apache.kafka.connect.errors.DataException; import org.apache.kafka.connect.sink.SinkRecord; + import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.Test; diff --git a/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/TimestampConverterTest.java b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/TimestampConverterTest.java index 8b5611e9fbcd5..fb3f3be62650a 100644 --- a/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/TimestampConverterTest.java +++ b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/TimestampConverterTest.java @@ -26,6 +26,7 @@ import org.apache.kafka.connect.data.Time; import org.apache.kafka.connect.data.Timestamp; import org.apache.kafka.connect.source.SourceRecord; + import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.Test; @@ -37,10 +38,10 @@ import java.util.TimeZone; import static org.apache.kafka.connect.transforms.util.Requirements.requireStruct; +import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertNull; import static org.junit.jupiter.api.Assertions.assertThrows; -import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; public class TimestampConverterTest { private static final TimeZone UTC = TimeZone.getTimeZone("UTC"); diff --git a/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/TimestampRouterTest.java b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/TimestampRouterTest.java index ec1e3bbe0547c..43b3b1f384ff5 100644 --- a/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/TimestampRouterTest.java +++ b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/TimestampRouterTest.java @@ -18,6 +18,7 @@ import org.apache.kafka.common.utils.AppInfoParser; import org.apache.kafka.connect.source.SourceRecord; + import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; diff --git a/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/ValueToKeyTest.java b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/ValueToKeyTest.java index 1ffebce6e3bde..a7d032009e513 100644 --- a/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/ValueToKeyTest.java +++ b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/ValueToKeyTest.java @@ -22,6 +22,7 @@ import org.apache.kafka.connect.data.Struct; import org.apache.kafka.connect.errors.DataException; import org.apache.kafka.connect.sink.SinkRecord; + import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.Test; diff --git a/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/field/FieldPathNotationTest.java b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/field/FieldPathNotationTest.java index d823bb4524435..504c41aa45c44 100644 --- a/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/field/FieldPathNotationTest.java +++ b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/field/FieldPathNotationTest.java @@ -17,6 +17,7 @@ package org.apache.kafka.connect.transforms.field; import org.apache.kafka.common.config.ConfigException; + import org.junit.jupiter.api.Test; import static org.junit.jupiter.api.Assertions.assertArrayEquals; diff --git a/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/field/FieldSyntaxVersionTest.java b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/field/FieldSyntaxVersionTest.java index a9356b5232cc2..d400141c95b71 100644 --- a/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/field/FieldSyntaxVersionTest.java +++ b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/field/FieldSyntaxVersionTest.java @@ -19,6 +19,7 @@ import org.apache.kafka.common.config.AbstractConfig; import org.apache.kafka.common.config.ConfigDef; import org.apache.kafka.common.config.ConfigException; + import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.CsvSource; diff --git a/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/field/SingleFieldPathTest.java b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/field/SingleFieldPathTest.java index f232bdbb0c867..a7292d2060923 100644 --- a/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/field/SingleFieldPathTest.java +++ b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/field/SingleFieldPathTest.java @@ -16,17 +16,18 @@ */ package org.apache.kafka.connect.transforms.field; -import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertNull; - import org.apache.kafka.connect.data.Schema; import org.apache.kafka.connect.data.SchemaBuilder; import org.apache.kafka.connect.data.Struct; + import org.junit.jupiter.api.Test; import java.util.HashMap; import java.util.Map; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNull; + class SingleFieldPathTest { @Test void shouldFindField() { diff --git a/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/predicates/HasHeaderKeyTest.java b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/predicates/HasHeaderKeyTest.java index d21c98f98c2ec..e3e3920858d27 100644 --- a/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/predicates/HasHeaderKeyTest.java +++ b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/predicates/HasHeaderKeyTest.java @@ -22,6 +22,7 @@ import org.apache.kafka.connect.header.Header; import org.apache.kafka.connect.source.SourceRecord; import org.apache.kafka.connect.transforms.util.SimpleConfig; + import org.junit.jupiter.api.Test; import java.util.Arrays; diff --git a/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/predicates/TopicNameMatchesTest.java b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/predicates/TopicNameMatchesTest.java index 0640803e6e648..3d9ac4dba9048 100644 --- a/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/predicates/TopicNameMatchesTest.java +++ b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/predicates/TopicNameMatchesTest.java @@ -20,6 +20,7 @@ import org.apache.kafka.common.config.ConfigValue; import org.apache.kafka.connect.source.SourceRecord; import org.apache.kafka.connect.transforms.util.SimpleConfig; + import org.junit.jupiter.api.Test; import java.util.Collections; diff --git a/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/util/NonEmptyListValidatorTest.java b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/util/NonEmptyListValidatorTest.java index ff3c4f720fd55..5060346a2d91b 100644 --- a/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/util/NonEmptyListValidatorTest.java +++ b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/util/NonEmptyListValidatorTest.java @@ -17,6 +17,7 @@ package org.apache.kafka.connect.transforms.util; import org.apache.kafka.common.config.ConfigException; + import org.junit.jupiter.api.Test; import java.util.Collections; diff --git a/core/src/main/java/kafka/log/remote/RemoteLogManager.java b/core/src/main/java/kafka/log/remote/RemoteLogManager.java index 43c03190767fb..c1c87d579ef4a 100644 --- a/core/src/main/java/kafka/log/remote/RemoteLogManager.java +++ b/core/src/main/java/kafka/log/remote/RemoteLogManager.java @@ -34,6 +34,7 @@ import org.apache.kafka.common.errors.RetriableException; import org.apache.kafka.common.message.FetchResponseData; import org.apache.kafka.common.metrics.Metrics; +import org.apache.kafka.common.metrics.Quota; import org.apache.kafka.common.record.FileRecords; import org.apache.kafka.common.record.MemoryRecords; import org.apache.kafka.common.record.Record; @@ -97,6 +98,7 @@ import java.nio.charset.StandardCharsets; import java.nio.file.Path; import java.security.PrivilegedAction; +import java.time.Duration; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; @@ -123,6 +125,8 @@ import java.util.concurrent.ThreadFactory; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.locks.Condition; +import java.util.concurrent.locks.ReentrantLock; import java.util.function.BiConsumer; import java.util.function.Consumer; import java.util.function.Function; @@ -160,6 +164,8 @@ public class RemoteLogManager implements Closeable { private final RemoteLogMetadataManager remoteLogMetadataManager; + private final ReentrantLock copyQuotaManagerLock = new ReentrantLock(true); + private final Condition copyQuotaManagerLockCondition = copyQuotaManagerLock.newCondition(); private final RLMQuotaManager rlmCopyQuotaManager; private final RLMQuotaManager rlmFetchQuotaManager; @@ -244,12 +250,29 @@ public void resizeCacheSize(long remoteLogIndexFileCacheSize) { indexCache.resizeCacheSize(remoteLogIndexFileCacheSize); } + public void updateCopyQuota(long quota) { + LOGGER.info("Updating remote copy quota to {} bytes per second", quota); + rlmCopyQuotaManager.updateQuota(new Quota(quota, true)); + } + + public void updateFetchQuota(long quota) { + LOGGER.info("Updating remote fetch quota to {} bytes per second", quota); + rlmFetchQuotaManager.updateQuota(new Quota(quota, true)); + } + private void removeMetrics() { metricsGroup.removeMetric(REMOTE_LOG_MANAGER_TASKS_AVG_IDLE_PERCENT_METRIC); metricsGroup.removeMetric(REMOTE_LOG_READER_FETCH_RATE_AND_TIME_METRIC); remoteStorageReaderThreadPool.removeMetrics(); } + /** + * Returns the timeout for the RLM Tasks to wait for the quota to be available + */ + Duration quotaTimeout() { + return Duration.ofSeconds(1); + } + RLMQuotaManager createRLMCopyQuotaManager() { return new RLMQuotaManager(copyQuotaManagerConfig(rlmConfig), metrics, QuotaType.RLMCopy$.MODULE$, "Tracking copy byte-rate for Remote Log Manager", time); @@ -763,6 +786,23 @@ public void copyLogSegmentsToRemote(UnifiedLog log) throws InterruptedException isCancelled(), isLeader()); return; } + + copyQuotaManagerLock.lock(); + try { + while (rlmCopyQuotaManager.isQuotaExceeded()) { + logger.debug("Quota exceeded for copying log segments, waiting for the quota to be available."); + // If the thread gets interrupted while waiting, the InterruptedException is thrown + // back to the caller. It's important to note that the task being executed is already + // cancelled before the executing thread is interrupted. The caller is responsible + // for handling the exception gracefully by checking if the task is already cancelled. + boolean ignored = copyQuotaManagerLockCondition.await(quotaTimeout().toMillis(), TimeUnit.MILLISECONDS); + } + rlmCopyQuotaManager.record(candidateLogSegment.logSegment.log().sizeInBytes()); + // Signal waiting threads to check the quota again + copyQuotaManagerLockCondition.signalAll(); + } finally { + copyQuotaManagerLock.unlock(); + } copyLogSegment(log, candidateLogSegment.logSegment, candidateLogSegment.nextSegmentOffset); } } diff --git a/core/src/main/scala/kafka/server/BrokerServer.scala b/core/src/main/scala/kafka/server/BrokerServer.scala index 31db58c077889..7e225440abf61 100644 --- a/core/src/main/scala/kafka/server/BrokerServer.scala +++ b/core/src/main/scala/kafka/server/BrokerServer.scala @@ -570,6 +570,7 @@ class BrokerServer( val serde = new CoordinatorRecordSerde val groupCoordinatorConfig = new GroupCoordinatorConfig( config.groupCoordinatorNumThreads, + config.groupCoordinatorAppendLingerMs, config.consumerGroupSessionTimeoutMs, config.consumerGroupHeartbeatIntervalMs, config.consumerGroupMaxSize, diff --git a/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala b/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala index 22576bdceb6fd..e37a615fa7f49 100755 --- a/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala +++ b/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala @@ -1166,36 +1166,57 @@ class DynamicRemoteLogConfig(server: KafkaBroker) extends BrokerReconfigurable w override def validateReconfiguration(newConfig: KafkaConfig): Unit = { newConfig.values.forEach { (k, v) => - if (reconfigurableConfigs.contains(k)) { - if (k.equals(RemoteLogManagerConfig.REMOTE_LOG_INDEX_FILE_CACHE_TOTAL_SIZE_BYTES_PROP)) { - val newValue = v.asInstanceOf[Long] - val oldValue = getValue(server.config, k) - if (newValue != oldValue && newValue <= 0) { - val errorMsg = s"Dynamic remote log manager config update validation failed for $k=$v" - throw new ConfigException(s"$errorMsg, value should be at least 1") - } + if (RemoteLogManagerConfig.REMOTE_LOG_INDEX_FILE_CACHE_TOTAL_SIZE_BYTES_PROP.equals(k) || + RemoteLogManagerConfig.REMOTE_LOG_MANAGER_COPY_MAX_BYTES_PER_SECOND_PROP.equals(k) || + RemoteLogManagerConfig.REMOTE_LOG_MANAGER_FETCH_MAX_BYTES_PER_SECOND_PROP.equals(k)) { + val newValue = v.asInstanceOf[Long] + val oldValue = getValue(server.config, k) + if (newValue != oldValue && newValue <= 0) { + val errorMsg = s"Dynamic remote log manager config update validation failed for $k=$v" + throw new ConfigException(s"$errorMsg, value should be at least 1") } } } } override def reconfigure(oldConfig: KafkaConfig, newConfig: KafkaConfig): Unit = { - val oldValue = oldConfig.getLong(RemoteLogManagerConfig.REMOTE_LOG_INDEX_FILE_CACHE_TOTAL_SIZE_BYTES_PROP) - val newValue = newConfig.getLong(RemoteLogManagerConfig.REMOTE_LOG_INDEX_FILE_CACHE_TOTAL_SIZE_BYTES_PROP) - if (oldValue != newValue) { - val remoteLogManager = server.remoteLogManagerOpt - if (remoteLogManager.nonEmpty) { + def oldLongValue(k: String): Long = oldConfig.getLong(k) + def newLongValue(k: String): Long = newConfig.getLong(k) + + def isChangedLongValue(k : String): Boolean = oldLongValue(k) != newLongValue(k) + + val remoteLogManager = server.remoteLogManagerOpt + if (remoteLogManager.nonEmpty) { + if (isChangedLongValue(RemoteLogManagerConfig.REMOTE_LOG_INDEX_FILE_CACHE_TOTAL_SIZE_BYTES_PROP)) { + val oldValue = oldLongValue(RemoteLogManagerConfig.REMOTE_LOG_INDEX_FILE_CACHE_TOTAL_SIZE_BYTES_PROP) + val newValue = newLongValue(RemoteLogManagerConfig.REMOTE_LOG_INDEX_FILE_CACHE_TOTAL_SIZE_BYTES_PROP) remoteLogManager.get.resizeCacheSize(newValue) info(s"Dynamic remote log manager config: ${RemoteLogManagerConfig.REMOTE_LOG_INDEX_FILE_CACHE_TOTAL_SIZE_BYTES_PROP} updated, " + s"old value: $oldValue, new value: $newValue") } + if (isChangedLongValue(RemoteLogManagerConfig.REMOTE_LOG_MANAGER_COPY_MAX_BYTES_PER_SECOND_PROP)) { + val oldValue = oldLongValue(RemoteLogManagerConfig.REMOTE_LOG_MANAGER_COPY_MAX_BYTES_PER_SECOND_PROP) + val newValue = newLongValue(RemoteLogManagerConfig.REMOTE_LOG_MANAGER_COPY_MAX_BYTES_PER_SECOND_PROP) + remoteLogManager.get.updateCopyQuota(newValue) + info(s"Dynamic remote log manager config: ${RemoteLogManagerConfig.REMOTE_LOG_MANAGER_COPY_MAX_BYTES_PER_SECOND_PROP} updated, " + + s"old value: $oldValue, new value: $newValue") + } + if (isChangedLongValue(RemoteLogManagerConfig.REMOTE_LOG_MANAGER_FETCH_MAX_BYTES_PER_SECOND_PROP)) { + val oldValue = oldLongValue(RemoteLogManagerConfig.REMOTE_LOG_MANAGER_FETCH_MAX_BYTES_PER_SECOND_PROP) + val newValue = newLongValue(RemoteLogManagerConfig.REMOTE_LOG_MANAGER_FETCH_MAX_BYTES_PER_SECOND_PROP) + remoteLogManager.get.updateFetchQuota(newValue) + info(s"Dynamic remote log manager config: ${RemoteLogManagerConfig.REMOTE_LOG_MANAGER_FETCH_MAX_BYTES_PER_SECOND_PROP} updated, " + + s"old value: $oldValue, new value: $newValue") + } } } private def getValue(config: KafkaConfig, name: String): Long = { name match { - case RemoteLogManagerConfig.REMOTE_LOG_INDEX_FILE_CACHE_TOTAL_SIZE_BYTES_PROP => - config.getLong(RemoteLogManagerConfig.REMOTE_LOG_INDEX_FILE_CACHE_TOTAL_SIZE_BYTES_PROP) + case RemoteLogManagerConfig.REMOTE_LOG_INDEX_FILE_CACHE_TOTAL_SIZE_BYTES_PROP | + RemoteLogManagerConfig.REMOTE_LOG_MANAGER_COPY_MAX_BYTES_PER_SECOND_PROP | + RemoteLogManagerConfig.REMOTE_LOG_MANAGER_FETCH_MAX_BYTES_PER_SECOND_PROP => + config.getLong(name) case n => throw new IllegalStateException(s"Unexpected dynamic remote log manager config $n") } } @@ -1204,6 +1225,8 @@ class DynamicRemoteLogConfig(server: KafkaBroker) extends BrokerReconfigurable w object DynamicRemoteLogConfig { val ReconfigurableConfigs = Set( RemoteLogManagerConfig.REMOTE_LOG_INDEX_FILE_CACHE_TOTAL_SIZE_BYTES_PROP, - RemoteLogManagerConfig.REMOTE_FETCH_MAX_WAIT_MS_PROP + RemoteLogManagerConfig.REMOTE_FETCH_MAX_WAIT_MS_PROP, + RemoteLogManagerConfig.REMOTE_LOG_MANAGER_COPY_MAX_BYTES_PER_SECOND_PROP, + RemoteLogManagerConfig.REMOTE_LOG_MANAGER_FETCH_MAX_BYTES_PER_SECOND_PROP ) } diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index ad8635f7ce780..db96bcb6762cb 100755 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -278,6 +278,7 @@ object KafkaConfig { .define(GroupCoordinatorConfig.GROUP_COORDINATOR_REBALANCE_PROTOCOLS_CONFIG, LIST, GroupCoordinatorConfig.GROUP_COORDINATOR_REBALANCE_PROTOCOLS_DEFAULT, ValidList.in(Utils.enumOptions(classOf[GroupType]):_*), MEDIUM, GroupCoordinatorConfig.GROUP_COORDINATOR_REBALANCE_PROTOCOLS_DOC) .define(GroupCoordinatorConfig.GROUP_COORDINATOR_NUM_THREADS_CONFIG, INT, GroupCoordinatorConfig.GROUP_COORDINATOR_NUM_THREADS_DEFAULT, atLeast(1), MEDIUM, GroupCoordinatorConfig.GROUP_COORDINATOR_NUM_THREADS_DOC) + .define(GroupCoordinatorConfig.GROUP_COORDINATOR_APPEND_LINGER_MS_CONFIG, INT, GroupCoordinatorConfig.GROUP_COORDINATOR_APPEND_LINGER_MS_DEFAULT, atLeast(0), MEDIUM, GroupCoordinatorConfig.GROUP_COORDINATOR_APPEND_LINGER_MS_DOC) // Internal configuration used by integration and system tests. .defineInternal(GroupCoordinatorConfig.NEW_GROUP_COORDINATOR_ENABLE_CONFIG, BOOLEAN, GroupCoordinatorConfig.NEW_GROUP_COORDINATOR_ENABLE_DEFAULT, null, MEDIUM, GroupCoordinatorConfig.NEW_GROUP_COORDINATOR_ENABLE_DOC) @@ -965,6 +966,7 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami val isNewGroupCoordinatorEnabled = getBoolean(GroupCoordinatorConfig.NEW_GROUP_COORDINATOR_ENABLE_CONFIG) || groupCoordinatorRebalanceProtocols.contains(GroupType.CONSUMER) val groupCoordinatorNumThreads = getInt(GroupCoordinatorConfig.GROUP_COORDINATOR_NUM_THREADS_CONFIG) + val groupCoordinatorAppendLingerMs = getInt(GroupCoordinatorConfig.GROUP_COORDINATOR_APPEND_LINGER_MS_CONFIG) /** Consumer group configs */ val consumerGroupSessionTimeoutMs = getInt(GroupCoordinatorConfig.CONSUMER_GROUP_SESSION_TIMEOUT_MS_CONFIG) diff --git a/core/src/test/java/kafka/log/remote/RemoteLogManagerTest.java b/core/src/test/java/kafka/log/remote/RemoteLogManagerTest.java index 83fc5966b49dc..4c4976f060d11 100644 --- a/core/src/test/java/kafka/log/remote/RemoteLogManagerTest.java +++ b/core/src/test/java/kafka/log/remote/RemoteLogManagerTest.java @@ -21,6 +21,7 @@ import kafka.cluster.EndPoint; import kafka.cluster.Partition; import kafka.log.UnifiedLog; +import kafka.log.remote.quota.RLMQuotaManager; import kafka.log.remote.quota.RLMQuotaManagerConfig; import kafka.server.BrokerTopicStats; import kafka.server.KafkaConfig; @@ -87,6 +88,7 @@ import org.mockito.InOrder; import org.mockito.MockedConstruction; import org.mockito.Mockito; +import org.opentest4j.AssertionFailedError; import scala.Option; import scala.collection.JavaConverters; @@ -101,6 +103,7 @@ import java.nio.ByteBuffer; import java.nio.charset.StandardCharsets; import java.nio.file.Files; +import java.time.Duration; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; @@ -146,6 +149,7 @@ import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertNull; import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTimeoutPreemptively; import static org.junit.jupiter.api.Assertions.assertTrue; import static org.junit.jupiter.api.Assertions.fail; import static org.mockito.ArgumentMatchers.any; @@ -153,6 +157,7 @@ import static org.mockito.ArgumentMatchers.anyLong; import static org.mockito.ArgumentMatchers.anyString; import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.atLeast; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.doNothing; import static org.mockito.Mockito.doThrow; @@ -187,6 +192,7 @@ public class RemoteLogManagerTest { private final RemoteStorageManager remoteStorageManager = mock(RemoteStorageManager.class); private final RemoteLogMetadataManager remoteLogMetadataManager = mock(RemoteLogMetadataManager.class); + private final RLMQuotaManager rlmCopyQuotaManager = mock(RLMQuotaManager.class); private RemoteLogManagerConfig remoteLogManagerConfig = null; private BrokerTopicStats brokerTopicStats = null; @@ -230,6 +236,12 @@ public RemoteStorageManager createRemoteStorageManager() { public RemoteLogMetadataManager createRemoteLogMetadataManager() { return remoteLogMetadataManager; } + public RLMQuotaManager createRLMCopyQuotaManager() { + return rlmCopyQuotaManager; + } + public Duration quotaTimeout() { + return Duration.ofMillis(100); + } @Override long findLogStartOffset(TopicIdPartition topicIdPartition, UnifiedLog log) { return 0L; @@ -2735,6 +2747,204 @@ public void testEpochEntriesAsByteBuffer() throws Exception { } + @ParameterizedTest + @ValueSource(booleans = {false, true}) + public void testCopyQuota(boolean quotaExceeded) throws Exception { + RemoteLogManager.RLMTask task = setupRLMTask(quotaExceeded); + + if (quotaExceeded) { + // Verify that the copy operation times out, since no segments can be copied due to quota being exceeded + assertThrows(AssertionFailedError.class, () -> assertTimeoutPreemptively(Duration.ofMillis(200), () -> task.copyLogSegmentsToRemote(mockLog))); + + // Verify the highest offset in remote storage is updated only once + ArgumentCaptor capture = ArgumentCaptor.forClass(Long.class); + verify(mockLog, times(1)).updateHighestOffsetInRemoteStorage(capture.capture()); + // Verify the highest offset in remote storage was -1L before the copy started + assertEquals(-1L, capture.getValue()); + } else { + // Verify the copy operation completes within the timeout, since it does not need to wait for quota availability + assertTimeoutPreemptively(Duration.ofMillis(100), () -> task.copyLogSegmentsToRemote(mockLog)); + + // Verify quota check was performed + verify(rlmCopyQuotaManager, times(1)).isQuotaExceeded(); + // Verify bytes to copy was recorded with the quota manager + verify(rlmCopyQuotaManager, times(1)).record(10); + + // Verify the highest offset in remote storage is updated + ArgumentCaptor capture = ArgumentCaptor.forClass(Long.class); + verify(mockLog, times(2)).updateHighestOffsetInRemoteStorage(capture.capture()); + List capturedValues = capture.getAllValues(); + // Verify the highest offset in remote storage was -1L before the copy + assertEquals(-1L, capturedValues.get(0).longValue()); + // Verify it was updated to 149L after the copy + assertEquals(149L, capturedValues.get(1).longValue()); + } + } + + @Test + public void testRLMShutdownDuringQuotaExceededScenario() throws Exception { + remoteLogManager.startup(); + setupRLMTask(true); + remoteLogManager.onLeadershipChange( + Collections.singleton(mockPartition(leaderTopicIdPartition)), Collections.emptySet(), topicIds); + // Ensure the copy operation is waiting for quota to be available + TestUtils.waitForCondition(() -> { + verify(rlmCopyQuotaManager, atLeast(1)).isQuotaExceeded(); + return true; + }, "Quota exceeded check did not happen"); + // Verify RLM is able to shut down + assertTimeoutPreemptively(Duration.ofMillis(100), () -> remoteLogManager.close()); + } + + // helper method to set up a RemoteLogManager.RLMTask for testing copy quota behaviour + private RemoteLogManager.RLMTask setupRLMTask(boolean quotaExceeded) throws RemoteStorageException, IOException { + long oldSegmentStartOffset = 0L; + long nextSegmentStartOffset = 150L; + + when(mockLog.topicPartition()).thenReturn(leaderTopicIdPartition.topicPartition()); + + // leader epoch preparation + checkpoint.write(totalEpochEntries); + LeaderEpochFileCache cache = new LeaderEpochFileCache(leaderTopicIdPartition.topicPartition(), checkpoint, scheduler); + when(mockLog.leaderEpochCache()).thenReturn(Option.apply(cache)); + when(mockLog.parentDir()).thenReturn("dir1"); + when(remoteLogMetadataManager.highestOffsetForEpoch(any(TopicIdPartition.class), anyInt())).thenReturn(Optional.of(0L)); + + // create 2 log segments, with 0 and 150 as log start offset + LogSegment oldSegment = mock(LogSegment.class); + LogSegment activeSegment = mock(LogSegment.class); + + File tempFile = TestUtils.tempFile(); + FileRecords fileRecords = mock(FileRecords.class); + when(fileRecords.file()).thenReturn(tempFile); + when(fileRecords.sizeInBytes()).thenReturn(10); + + // Set up the segment that is eligible for copy + when(oldSegment.log()).thenReturn(fileRecords); + when(oldSegment.baseOffset()).thenReturn(oldSegmentStartOffset); + when(oldSegment.readNextOffset()).thenReturn(nextSegmentStartOffset); + + // set up the active segment + when(activeSegment.baseOffset()).thenReturn(nextSegmentStartOffset); + + when(mockLog.activeSegment()).thenReturn(activeSegment); + when(mockLog.logStartOffset()).thenReturn(oldSegmentStartOffset); + when(mockLog.logSegments(anyLong(), anyLong())).thenReturn(JavaConverters.collectionAsScalaIterable(Arrays.asList(oldSegment, activeSegment))); + + File mockProducerSnapshotIndex = TestUtils.tempFile(); + ProducerStateManager mockStateManager = mock(ProducerStateManager.class); + when(mockStateManager.fetchSnapshot(anyLong())).thenReturn(Optional.of(mockProducerSnapshotIndex)); + + when(mockLog.producerStateManager()).thenReturn(mockStateManager); + when(mockLog.lastStableOffset()).thenReturn(250L); + + File tempDir = TestUtils.tempDirectory(); + OffsetIndex idx = LazyIndex.forOffset(LogFileUtils.offsetIndexFile(tempDir, oldSegmentStartOffset, ""), oldSegmentStartOffset, 1000).get(); + TimeIndex timeIdx = LazyIndex.forTime(LogFileUtils.timeIndexFile(tempDir, oldSegmentStartOffset, ""), oldSegmentStartOffset, 1500).get(); + File txnFile = UnifiedLog.transactionIndexFile(tempDir, oldSegmentStartOffset, ""); + txnFile.createNewFile(); + TransactionIndex txnIndex = new TransactionIndex(oldSegmentStartOffset, txnFile); + when(oldSegment.timeIndex()).thenReturn(timeIdx); + when(oldSegment.offsetIndex()).thenReturn(idx); + when(oldSegment.txnIndex()).thenReturn(txnIndex); + + CompletableFuture dummyFuture = new CompletableFuture<>(); + dummyFuture.complete(null); + when(remoteLogMetadataManager.addRemoteLogSegmentMetadata(any(RemoteLogSegmentMetadata.class))).thenReturn(dummyFuture); + when(remoteLogMetadataManager.updateRemoteLogSegmentMetadata(any(RemoteLogSegmentMetadataUpdate.class))).thenReturn(dummyFuture); + when(remoteStorageManager.copyLogSegmentData(any(RemoteLogSegmentMetadata.class), any(LogSegmentData.class))).thenReturn(Optional.empty()); + + when(rlmCopyQuotaManager.isQuotaExceeded()).thenReturn(quotaExceeded); + doNothing().when(rlmCopyQuotaManager).record(anyInt()); + + RemoteLogManager.RLMTask task = remoteLogManager.new RLMTask(leaderTopicIdPartition, 128); + task.convertToLeader(2); + return task; + } + + @Test + public void testCopyThrottling() throws Exception { + long oldestSegmentStartOffset = 0L; + + when(mockLog.topicPartition()).thenReturn(leaderTopicIdPartition.topicPartition()); + + // leader epoch preparation + checkpoint.write(totalEpochEntries); + LeaderEpochFileCache cache = new LeaderEpochFileCache(leaderTopicIdPartition.topicPartition(), checkpoint, scheduler); + when(mockLog.leaderEpochCache()).thenReturn(Option.apply(cache)); + when(remoteLogMetadataManager.highestOffsetForEpoch(any(TopicIdPartition.class), anyInt())).thenReturn(Optional.of(0L)); + + // create 3 log segments + LogSegment segmentToCopy = mock(LogSegment.class); + LogSegment segmentToThrottle = mock(LogSegment.class); + LogSegment activeSegment = mock(LogSegment.class); + + File tempFile = TestUtils.tempFile(); + FileRecords fileRecords = mock(FileRecords.class); + when(fileRecords.file()).thenReturn(tempFile); + when(fileRecords.sizeInBytes()).thenReturn(10); + + // set up the segment that will be copied + when(segmentToCopy.log()).thenReturn(fileRecords); + when(segmentToCopy.baseOffset()).thenReturn(oldestSegmentStartOffset); + when(segmentToCopy.readNextOffset()).thenReturn(100L); + + // set up the segment that will not be copied because of hitting quota + when(segmentToThrottle.log()).thenReturn(fileRecords); + when(segmentToThrottle.baseOffset()).thenReturn(100L); + when(segmentToThrottle.readNextOffset()).thenReturn(150L); + + // set up the active segment + when(activeSegment.log()).thenReturn(fileRecords); + when(activeSegment.baseOffset()).thenReturn(150L); + + when(mockLog.activeSegment()).thenReturn(activeSegment); + when(mockLog.logStartOffset()).thenReturn(oldestSegmentStartOffset); + when(mockLog.logSegments(anyLong(), anyLong())).thenReturn(JavaConverters.collectionAsScalaIterable(Arrays.asList(segmentToCopy, segmentToThrottle, activeSegment))); + + File mockProducerSnapshotIndex = TestUtils.tempFile(); + ProducerStateManager mockStateManager = mock(ProducerStateManager.class); + when(mockStateManager.fetchSnapshot(anyLong())).thenReturn(Optional.of(mockProducerSnapshotIndex)); + + when(mockLog.producerStateManager()).thenReturn(mockStateManager); + when(mockLog.lastStableOffset()).thenReturn(250L); + + File tempDir = TestUtils.tempDirectory(); + OffsetIndex idx = LazyIndex.forOffset(LogFileUtils.offsetIndexFile(tempDir, oldestSegmentStartOffset, ""), oldestSegmentStartOffset, 1000).get(); + TimeIndex timeIdx = LazyIndex.forTime(LogFileUtils.timeIndexFile(tempDir, oldestSegmentStartOffset, ""), oldestSegmentStartOffset, 1500).get(); + File txnFile = UnifiedLog.transactionIndexFile(tempDir, oldestSegmentStartOffset, ""); + txnFile.createNewFile(); + TransactionIndex txnIndex = new TransactionIndex(oldestSegmentStartOffset, txnFile); + when(segmentToCopy.timeIndex()).thenReturn(timeIdx); + when(segmentToCopy.offsetIndex()).thenReturn(idx); + when(segmentToCopy.txnIndex()).thenReturn(txnIndex); + + CompletableFuture dummyFuture = new CompletableFuture<>(); + dummyFuture.complete(null); + when(remoteLogMetadataManager.addRemoteLogSegmentMetadata(any(RemoteLogSegmentMetadata.class))).thenReturn(dummyFuture); + when(remoteLogMetadataManager.updateRemoteLogSegmentMetadata(any(RemoteLogSegmentMetadataUpdate.class))).thenReturn(dummyFuture); + when(remoteStorageManager.copyLogSegmentData(any(RemoteLogSegmentMetadata.class), any(LogSegmentData.class))).thenReturn(Optional.empty()); + + // After the first call, isQuotaExceeded should return true + when(rlmCopyQuotaManager.isQuotaExceeded()).thenReturn(false, true); + doNothing().when(rlmCopyQuotaManager).record(anyInt()); + + RemoteLogManager.RLMTask task = remoteLogManager.new RLMTask(leaderTopicIdPartition, 128); + task.convertToLeader(2); + + // Verify that the copy operation times out, since the second segment cannot be copied due to quota being exceeded + assertThrows(AssertionFailedError.class, () -> assertTimeoutPreemptively(Duration.ofMillis(200), () -> task.copyLogSegmentsToRemote(mockLog))); + + // Verify the highest offset in remote storage is updated corresponding to the only segment that was copied + ArgumentCaptor capture = ArgumentCaptor.forClass(Long.class); + verify(mockLog, times(2)).updateHighestOffsetInRemoteStorage(capture.capture()); + List capturedValues = capture.getAllValues(); + // Verify the highest offset in remote storage was -1L before the copy + assertEquals(-1L, capturedValues.get(0).longValue()); + // Verify it was updated to 99L after the copy + assertEquals(99L, capturedValues.get(1).longValue()); + } + private Partition mockPartition(TopicIdPartition topicIdPartition) { TopicPartition tp = topicIdPartition.topicPartition(); Partition partition = mock(Partition.class); diff --git a/core/src/test/scala/unit/kafka/server/DynamicBrokerConfigTest.scala b/core/src/test/scala/unit/kafka/server/DynamicBrokerConfigTest.scala index 2e5d77cdc6fae..3ba158891574e 100755 --- a/core/src/test/scala/unit/kafka/server/DynamicBrokerConfigTest.scala +++ b/core/src/test/scala/unit/kafka/server/DynamicBrokerConfigTest.scala @@ -850,6 +850,102 @@ class DynamicBrokerConfigTest { Mockito.verifyNoMoreInteractions(remoteLogManagerMockOpt.get) } + @Test + def testRemoteLogManagerCopyQuotaUpdates(): Unit = { + testRemoteLogManagerQuotaUpdates( + RemoteLogManagerConfig.REMOTE_LOG_MANAGER_COPY_MAX_BYTES_PER_SECOND_PROP, + RemoteLogManagerConfig.DEFAULT_REMOTE_LOG_MANAGER_COPY_MAX_BYTES_PER_SECOND, + (remoteLogManager, quota) => Mockito.verify(remoteLogManager).updateCopyQuota(quota) + ) + } + + @Test + def testRemoteLogManagerFetchQuotaUpdates(): Unit = { + testRemoteLogManagerQuotaUpdates( + RemoteLogManagerConfig.REMOTE_LOG_MANAGER_FETCH_MAX_BYTES_PER_SECOND_PROP, + RemoteLogManagerConfig.DEFAULT_REMOTE_LOG_MANAGER_FETCH_MAX_BYTES_PER_SECOND, + (remoteLogManager, quota) => Mockito.verify(remoteLogManager).updateFetchQuota(quota) + ) + } + + def testRemoteLogManagerQuotaUpdates(quotaProp: String, defaultQuota: Long, verifyMethod: (RemoteLogManager, Long) => Unit): Unit = { + val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 9092) + val config = KafkaConfig.fromProps(props) + val serverMock: KafkaServer = mock(classOf[KafkaServer]) + val remoteLogManagerMockOpt = Option(mock(classOf[RemoteLogManager])) + + Mockito.when(serverMock.config).thenReturn(config) + Mockito.when(serverMock.remoteLogManagerOpt).thenReturn(remoteLogManagerMockOpt) + + config.dynamicConfig.initialize(None, None) + config.dynamicConfig.addBrokerReconfigurable(new DynamicRemoteLogConfig(serverMock)) + + assertEquals(defaultQuota, config.getLong(quotaProp)) + + // Update default config + props.put(quotaProp, "100") + config.dynamicConfig.updateDefaultConfig(props) + assertEquals(100, config.getLong(quotaProp)) + verifyMethod(remoteLogManagerMockOpt.get, 100) + + // Update per broker config + props.put(quotaProp, "200") + config.dynamicConfig.updateBrokerConfig(0, props) + assertEquals(200, config.getLong(quotaProp)) + verifyMethod(remoteLogManagerMockOpt.get, 200) + + Mockito.verifyNoMoreInteractions(remoteLogManagerMockOpt.get) + } + + @Test + def testRemoteLogManagerMultipleConfigUpdates(): Unit = { + val indexFileCacheSizeProp = RemoteLogManagerConfig.REMOTE_LOG_INDEX_FILE_CACHE_TOTAL_SIZE_BYTES_PROP + val copyQuotaProp = RemoteLogManagerConfig.REMOTE_LOG_MANAGER_COPY_MAX_BYTES_PER_SECOND_PROP + val fetchQuotaProp = RemoteLogManagerConfig.REMOTE_LOG_MANAGER_FETCH_MAX_BYTES_PER_SECOND_PROP + + val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 9092) + val config = KafkaConfig.fromProps(props) + val serverMock: KafkaServer = mock(classOf[KafkaServer]) + val remoteLogManagerMockOpt = Option(Mockito.mock(classOf[RemoteLogManager])) + + Mockito.when(serverMock.config).thenReturn(config) + Mockito.when(serverMock.remoteLogManagerOpt).thenReturn(remoteLogManagerMockOpt) + + config.dynamicConfig.initialize(None, None) + config.dynamicConfig.addBrokerReconfigurable(new DynamicRemoteLogConfig(serverMock)) + + // Default values + assertEquals(RemoteLogManagerConfig.DEFAULT_REMOTE_LOG_INDEX_FILE_CACHE_TOTAL_SIZE_BYTES, config.getLong(indexFileCacheSizeProp)) + assertEquals(RemoteLogManagerConfig.DEFAULT_REMOTE_LOG_MANAGER_COPY_MAX_BYTES_PER_SECOND, config.getLong(copyQuotaProp)) + assertEquals(RemoteLogManagerConfig.DEFAULT_REMOTE_LOG_MANAGER_FETCH_MAX_BYTES_PER_SECOND, config.getLong(fetchQuotaProp)) + + // Update default config + props.put(indexFileCacheSizeProp, "4") + props.put(copyQuotaProp, "100") + props.put(fetchQuotaProp, "200") + config.dynamicConfig.updateDefaultConfig(props) + assertEquals(4, config.getLong(indexFileCacheSizeProp)) + assertEquals(100, config.getLong(copyQuotaProp)) + assertEquals(200, config.getLong(fetchQuotaProp)) + Mockito.verify(remoteLogManagerMockOpt.get).resizeCacheSize(4) + Mockito.verify(remoteLogManagerMockOpt.get).updateCopyQuota(100) + Mockito.verify(remoteLogManagerMockOpt.get).updateFetchQuota(200) + + // Update per broker config + props.put(indexFileCacheSizeProp, "8") + props.put(copyQuotaProp, "200") + props.put(fetchQuotaProp, "400") + config.dynamicConfig.updateBrokerConfig(0, props) + assertEquals(8, config.getLong(indexFileCacheSizeProp)) + assertEquals(200, config.getLong(copyQuotaProp)) + assertEquals(400, config.getLong(fetchQuotaProp)) + Mockito.verify(remoteLogManagerMockOpt.get).resizeCacheSize(8) + Mockito.verify(remoteLogManagerMockOpt.get).updateCopyQuota(200) + Mockito.verify(remoteLogManagerMockOpt.get).updateFetchQuota(400) + + Mockito.verifyNoMoreInteractions(remoteLogManagerMockOpt.get) + } + def verifyIncorrectLogLocalRetentionProps(logLocalRetentionMs: Long, retentionMs: Long, logLocalRetentionBytes: Long, diff --git a/core/src/test/scala/unit/kafka/server/ReplicationQuotasTest.scala b/core/src/test/scala/unit/kafka/server/ReplicationQuotasTest.scala index 1f947dd8fae4b..33e9d3d5d467f 100644 --- a/core/src/test/scala/unit/kafka/server/ReplicationQuotasTest.scala +++ b/core/src/test/scala/unit/kafka/server/ReplicationQuotasTest.scala @@ -139,19 +139,14 @@ class ReplicationQuotasTest extends QuorumTestHarness { } } //Either throttle the six leaders or the two followers - if (leaderThrottle) { - admin.incrementalAlterConfigs( - Map(new ConfigResource(TOPIC, topic) -> Seq(new AlterConfigOp(new ConfigEntry( - QuotaConfigs.LEADER_REPLICATION_THROTTLED_REPLICAS_CONFIG, "0:100,1:101,2:102,3:103,4:104,5:105"), - SET)).asJavaCollection).asJava - ).all().get() - } else { - admin.incrementalAlterConfigs( - Map(new ConfigResource(TOPIC, topic) -> Seq(new AlterConfigOp(new ConfigEntry( - QuotaConfigs.FOLLOWER_REPLICATION_THROTTLED_REPLICAS_CONFIG, "0:106,1:106,2:106,3:107,4:107,5:107"), - SET)).asJavaCollection).asJava - ).all().get() - } + val configEntry = if (leaderThrottle) + new ConfigEntry(QuotaConfigs.LEADER_REPLICATION_THROTTLED_REPLICAS_CONFIG, "0:100,1:101,2:102,3:103,4:104,5:105") + else + new ConfigEntry(QuotaConfigs.FOLLOWER_REPLICATION_THROTTLED_REPLICAS_CONFIG, "0:106,1:106,2:106,3:107,4:107,5:107") + + admin.incrementalAlterConfigs( + Map(new ConfigResource(TOPIC, topic) -> Seq(new AlterConfigOp(configEntry, SET)).asJavaCollection).asJava + ).all().get() } //Add data equally to each partition @@ -307,7 +302,9 @@ class ReplicationQuotasTest extends QuorumTestHarness { .setClusterId(controllerServer.clusterId) .setIncarnationId(Uuid.randomUuid()) .setListeners(listeners) - .setLogDirs(Collections.singletonList(Uuid.randomUuid())) + .setLogDirs(Collections.singletonList( + Uuid.fromString(s"TESTBROKER${Integer.toString(100000 + id).substring(1)}DIRAAAA") + )) .setFeatures(features) ).get() } diff --git a/generator/src/main/java/org/apache/kafka/message/MessageGenerator.java b/generator/src/main/java/org/apache/kafka/message/MessageGenerator.java index ebffb4d88d2da..8cc4b5077a6dd 100644 --- a/generator/src/main/java/org/apache/kafka/message/MessageGenerator.java +++ b/generator/src/main/java/org/apache/kafka/message/MessageGenerator.java @@ -22,6 +22,7 @@ import com.fasterxml.jackson.databind.DeserializationFeature; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.SerializationFeature; + import net.sourceforge.argparse4j.ArgumentParsers; import net.sourceforge.argparse4j.inf.ArgumentParser; import net.sourceforge.argparse4j.inf.Namespace; diff --git a/generator/src/test/java/org/apache/kafka/message/CodeBufferTest.java b/generator/src/test/java/org/apache/kafka/message/CodeBufferTest.java index 6dd2e646071c1..a4cfe65d5cfe0 100644 --- a/generator/src/test/java/org/apache/kafka/message/CodeBufferTest.java +++ b/generator/src/test/java/org/apache/kafka/message/CodeBufferTest.java @@ -24,8 +24,8 @@ import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertNotEquals; -import static org.junit.jupiter.api.Assertions.assertTrue; import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; @Timeout(120) public class CodeBufferTest { diff --git a/generator/src/test/java/org/apache/kafka/message/MessageGeneratorTest.java b/generator/src/test/java/org/apache/kafka/message/MessageGeneratorTest.java index dcd602ca1fb3c..fd25d5e774bac 100644 --- a/generator/src/test/java/org/apache/kafka/message/MessageGeneratorTest.java +++ b/generator/src/test/java/org/apache/kafka/message/MessageGeneratorTest.java @@ -20,10 +20,10 @@ import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; +import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; -import static org.junit.jupiter.api.Assertions.assertEquals; @Timeout(120) public class MessageGeneratorTest { diff --git a/generator/src/test/java/org/apache/kafka/message/VersionConditionalTest.java b/generator/src/test/java/org/apache/kafka/message/VersionConditionalTest.java index 4dbfd0e3ed21a..833bd2ec3f9e1 100644 --- a/generator/src/test/java/org/apache/kafka/message/VersionConditionalTest.java +++ b/generator/src/test/java/org/apache/kafka/message/VersionConditionalTest.java @@ -22,8 +22,8 @@ import java.io.StringWriter; -import static org.junit.jupiter.api.Assertions.assertTrue; import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; @Timeout(120) public class VersionConditionalTest { diff --git a/generator/src/test/java/org/apache/kafka/message/VersionsTest.java b/generator/src/test/java/org/apache/kafka/message/VersionsTest.java index 23e59afd68a81..a61a1390aae29 100644 --- a/generator/src/test/java/org/apache/kafka/message/VersionsTest.java +++ b/generator/src/test/java/org/apache/kafka/message/VersionsTest.java @@ -21,9 +21,9 @@ import org.junit.jupiter.api.Timeout; import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertTrue; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertNull; +import static org.junit.jupiter.api.Assertions.assertTrue; @Timeout(120) public class VersionsTest { diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorConfig.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorConfig.java index 84219aa46ffd0..cc86b6cf818e3 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorConfig.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorConfig.java @@ -57,6 +57,10 @@ public class GroupCoordinatorConfig { Arrays.stream(Group.GroupType.values()).map(Group.GroupType::toString).collect(Collectors.joining(",")) + ". " + "The " + Group.GroupType.CONSUMER + " rebalance protocol is in early access and therefore must not be used in production."; public static final List GROUP_COORDINATOR_REBALANCE_PROTOCOLS_DEFAULT = Collections.singletonList(Group.GroupType.CLASSIC.toString()); + public final static String GROUP_COORDINATOR_APPEND_LINGER_MS_CONFIG = "group.coordinator.append.linger.ms"; + public final static String GROUP_COORDINATOR_APPEND_LINGER_MS_DOC = "The duration in milliseconds that the coordinator will " + + "wait for writes to accumulate before flushing them to disk. Transactional writes are not accumulated."; + public final static int GROUP_COORDINATOR_APPEND_LINGER_MS_DEFAULT = 10; public final static String GROUP_COORDINATOR_NUM_THREADS_CONFIG = "group.coordinator.threads"; public final static String GROUP_COORDINATOR_NUM_THREADS_DOC = "The number of threads used by the group coordinator."; @@ -164,6 +168,12 @@ public class GroupCoordinatorConfig { */ public final int numThreads; + /** + * The duration in milliseconds that the coordinator will wait for writes to + * accumulate before flushing them to disk. + */ + public final int appendLingerMs; + /** * The consumer group session timeout in milliseconds. */ @@ -259,6 +269,7 @@ public class GroupCoordinatorConfig { public GroupCoordinatorConfig( int numThreads, + int appendLingerMs, int consumerGroupSessionTimeoutMs, int consumerGroupHeartbeatIntervalMs, int consumerGroupMaxSize, @@ -277,6 +288,7 @@ public GroupCoordinatorConfig( CompressionType compressionType ) { this.numThreads = numThreads; + this.appendLingerMs = appendLingerMs; this.consumerGroupSessionTimeoutMs = consumerGroupSessionTimeoutMs; this.consumerGroupHeartbeatIntervalMs = consumerGroupHeartbeatIntervalMs; this.consumerGroupMaxSize = consumerGroupMaxSize; diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorService.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorService.java index 5e4e899faa681..f92594f09d07e 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorService.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorService.java @@ -182,12 +182,12 @@ public GroupCoordinatorService build() { .withPartitionWriter(writer) .withLoader(loader) .withCoordinatorShardBuilderSupplier(supplier) - .withTime(time) .withDefaultWriteTimeOut(Duration.ofMillis(config.offsetCommitTimeoutMs)) .withCoordinatorRuntimeMetrics(coordinatorRuntimeMetrics) .withCoordinatorMetrics(groupCoordinatorMetrics) .withSerializer(new CoordinatorRecordSerde()) .withCompression(Compression.of(config.compressionType).build()) + .withAppendLingerMs(config.appendLingerMs) .build(); return new GroupCoordinatorService( diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/CoordinatorRuntime.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/CoordinatorRuntime.java index 4207c94770b96..57927867c89ef 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/CoordinatorRuntime.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/CoordinatorRuntime.java @@ -24,11 +24,13 @@ import org.apache.kafka.common.errors.RecordTooLargeException; import org.apache.kafka.common.errors.TimeoutException; import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.record.AbstractRecords; import org.apache.kafka.common.record.ControlRecordType; import org.apache.kafka.common.record.EndTransactionMarker; import org.apache.kafka.common.record.MemoryRecords; import org.apache.kafka.common.record.MemoryRecordsBuilder; import org.apache.kafka.common.record.RecordBatch; +import org.apache.kafka.common.record.SimpleRecord; import org.apache.kafka.common.record.TimestampType; import org.apache.kafka.common.requests.TransactionResult; import org.apache.kafka.common.utils.BufferSupplier; @@ -50,10 +52,12 @@ import java.nio.ByteBuffer; import java.time.Duration; +import java.util.ArrayList; import java.util.HashMap; import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.OptionalInt; import java.util.OptionalLong; import java.util.concurrent.CompletableFuture; @@ -66,8 +70,6 @@ import java.util.function.Consumer; import java.util.stream.Collectors; -import static org.apache.kafka.common.record.Record.EMPTY_HEADERS; - /** * The CoordinatorRuntime provides a framework to implement coordinators such as the group coordinator * or the transaction coordinator. @@ -115,6 +117,7 @@ public static class Builder, U> { private CoordinatorMetrics coordinatorMetrics; private Serializer serializer; private Compression compression; + private int appendLingerMs; public Builder withLogPrefix(String logPrefix) { this.logPrefix = logPrefix; @@ -181,6 +184,11 @@ public Builder withCompression(Compression compression) { return this; } + public Builder withAppendLingerMs(int appendLingerMs) { + this.appendLingerMs = appendLingerMs; + return this; + } + public CoordinatorRuntime build() { if (logPrefix == null) logPrefix = ""; @@ -206,6 +214,8 @@ public CoordinatorRuntime build() { throw new IllegalArgumentException("Serializer must be set."); if (compression == null) compression = Compression.NONE; + if (appendLingerMs < 0) + throw new IllegalArgumentException("AppendLinger must be >= 0"); return new CoordinatorRuntime<>( logPrefix, @@ -220,7 +230,8 @@ public CoordinatorRuntime build() { runtimeMetrics, coordinatorMetrics, serializer, - compression + compression, + appendLingerMs ); } } @@ -275,7 +286,7 @@ boolean canTransitionFrom(CoordinatorState state) { FAILED { @Override boolean canTransitionFrom(CoordinatorState state) { - return state == LOADING; + return state == LOADING || state == ACTIVE; } }; @@ -434,6 +445,81 @@ public int size() { } } + /** + * A simple container class to hold all the attributes + * related to a pending batch. + */ + private static class CoordinatorBatch { + /** + * The base (or first) offset of the batch. If the batch fails + * for any reason, the state machines is rolled back to it. + */ + final long baseOffset; + + /** + * The time at which the batch was created. + */ + final long appendTimeMs; + + /** + * The max batch size. + */ + final int maxBatchSize; + + /** + * The verification guard associated to the batch if it is + * transactional. + */ + final VerificationGuard verificationGuard; + + /** + * The byte buffer backing the records builder. + */ + final ByteBuffer buffer; + + /** + * The records builder. + */ + final MemoryRecordsBuilder builder; + + /** + * The timer used to enfore the append linger time if + * it is non-zero. + */ + final Optional lingerTimeoutTask; + + /** + * The list of deferred events associated with the batch. + */ + final List deferredEvents; + + /** + * The next offset. This is updated when records + * are added to the batch. + */ + long nextOffset; + + CoordinatorBatch( + long baseOffset, + long appendTimeMs, + int maxBatchSize, + VerificationGuard verificationGuard, + ByteBuffer buffer, + MemoryRecordsBuilder builder, + Optional lingerTimeoutTask + ) { + this.baseOffset = baseOffset; + this.nextOffset = baseOffset; + this.appendTimeMs = appendTimeMs; + this.maxBatchSize = maxBatchSize; + this.verificationGuard = verificationGuard; + this.buffer = buffer; + this.builder = builder; + this.lingerTimeoutTask = lingerTimeoutTask; + this.deferredEvents = new ArrayList<>(); + } + } + /** * CoordinatorContext holds all the metadata around a coordinator state machine. */ @@ -493,6 +579,11 @@ class CoordinatorContext { */ BufferSupplier bufferSupplier; + /** + * The current (or pending) batch. + */ + CoordinatorBatch currentBatch; + /** * Constructor. * @@ -547,6 +638,7 @@ private void transitionTo( .build(), tp ); + load(); break; case ACTIVE: @@ -573,6 +665,46 @@ private void transitionTo( runtimeMetrics.recordPartitionStateChange(oldState, state); } + /** + * Loads the coordinator. + */ + private void load() { + if (state != CoordinatorState.LOADING) { + throw new IllegalStateException("Coordinator must be in loading state"); + } + + loader.load(tp, coordinator).whenComplete((summary, exception) -> { + scheduleInternalOperation("CompleteLoad(tp=" + tp + ", epoch=" + epoch + ")", tp, () -> { + CoordinatorContext context = coordinators.get(tp); + if (context != null) { + if (context.state != CoordinatorState.LOADING) { + log.info("Ignored load completion from {} because context is in {} state.", + context.tp, context.state); + return; + } + try { + if (exception != null) throw exception; + context.transitionTo(CoordinatorState.ACTIVE); + if (summary != null) { + runtimeMetrics.recordPartitionLoadSensor(summary.startTimeMs(), summary.endTimeMs()); + log.info("Finished loading of metadata from {} with epoch {} in {}ms where {}ms " + + "was spent in the scheduler. Loaded {} records which total to {} bytes.", + tp, epoch, summary.endTimeMs() - summary.startTimeMs(), + summary.schedulerQueueTimeMs(), summary.numRecords(), summary.numBytes()); + } + } catch (Throwable ex) { + log.error("Failed to load metadata from {} with epoch {} due to {}.", + tp, epoch, ex.toString()); + context.transitionTo(CoordinatorState.FAILED); + } + } else { + log.debug("Failed to complete the loading of metadata for {} in epoch {} since the coordinator does not exist.", + tp, epoch); + } + }); + }); + } + /** * Unloads the coordinator. */ @@ -583,11 +715,352 @@ private void unload() { } timer.cancelAll(); deferredEventQueue.failAll(Errors.NOT_COORDINATOR.exception()); + failCurrentBatch(Errors.NOT_COORDINATOR.exception()); if (coordinator != null) { coordinator.onUnloaded(); } coordinator = null; } + + /** + * Frees the current batch. + */ + private void freeCurrentBatch() { + // Cancel the linger timeout. + currentBatch.lingerTimeoutTask.ifPresent(TimerTask::cancel); + + // Release the buffer. + bufferSupplier.release(currentBatch.buffer); + + currentBatch = null; + } + + /** + * Flushes the current (or pending) batch to the log. When the batch is written + * locally, a new snapshot is created in the snapshot registry and the events + * associated with the batch are added to the deferred event queue. + */ + private void flushCurrentBatch() { + if (currentBatch != null) { + try { + // Write the records to the log and update the last written offset. + long offset = partitionWriter.append( + tp, + currentBatch.verificationGuard, + currentBatch.builder.build() + ); + coordinator.updateLastWrittenOffset(offset); + + if (offset != currentBatch.nextOffset) { + log.error("The state machine of the coordinator {} is out of sync with the underlying log. " + + "The last written offset returned is {} while the coordinator expected {}. The coordinator " + + "will be reloaded in order to re-synchronize the state machine.", + tp, offset, currentBatch.nextOffset); + // Transition to FAILED state to unload the state machine and complete + // exceptionally all the pending operations. + transitionTo(CoordinatorState.FAILED); + // Transition to LOADING to trigger the restoration of the state. + transitionTo(CoordinatorState.LOADING); + // Thrown NotCoordinatorException to fail the operation that + // triggered the write. We use NotCoordinatorException to be + // consistent with the transition to FAILED. + throw Errors.NOT_COORDINATOR.exception(); + } + + // Add all the pending deferred events to the deferred event queue. + for (DeferredEvent event : currentBatch.deferredEvents) { + deferredEventQueue.add(offset, event); + } + + // Free up the current batch. + freeCurrentBatch(); + } catch (Throwable t) { + log.error("Writing records to {} failed due to: {}.", tp, t.getMessage()); + failCurrentBatch(t); + // We rethrow the exception for the caller to handle it too. + throw t; + } + } + } + + /** + * Flushes the current batch if it is transactional or if it has passed the append linger time. + */ + private void maybeFlushCurrentBatch(long currentTimeMs) { + if (currentBatch != null) { + if (currentBatch.builder.isTransactional() || (currentBatch.appendTimeMs - currentTimeMs) >= appendLingerMs) { + flushCurrentBatch(); + } + } + } + + /** + * Fails the current batch, reverts to the snapshot to the base/start offset of the + * batch, fails all the associated events. + */ + private void failCurrentBatch(Throwable t) { + if (currentBatch != null) { + coordinator.revertLastWrittenOffset(currentBatch.baseOffset); + for (DeferredEvent event : currentBatch.deferredEvents) { + event.complete(t); + } + freeCurrentBatch(); + } + } + + /** + * Allocates a new batch if none already exists. + */ + private void maybeAllocateNewBatch( + long producerId, + short producerEpoch, + VerificationGuard verificationGuard, + long currentTimeMs + ) { + if (currentBatch == null) { + LogConfig logConfig = partitionWriter.config(tp); + byte magic = logConfig.recordVersion().value; + int maxBatchSize = logConfig.maxMessageSize(); + long prevLastWrittenOffset = coordinator.lastWrittenOffset(); + ByteBuffer buffer = bufferSupplier.get(maxBatchSize); + + MemoryRecordsBuilder builder = new MemoryRecordsBuilder( + buffer, + magic, + compression, + TimestampType.CREATE_TIME, + 0L, + currentTimeMs, + producerId, + producerEpoch, + 0, + producerId != RecordBatch.NO_PRODUCER_ID, + false, + RecordBatch.NO_PARTITION_LEADER_EPOCH, + maxBatchSize + ); + + Optional lingerTimeoutTask = Optional.empty(); + if (appendLingerMs > 0) { + lingerTimeoutTask = Optional.of(new TimerTask(appendLingerMs) { + @Override + public void run() { + // An event to flush the batch is pushed to the front of the queue + // to ensure that the linger time is respected. + enqueueFirst(new CoordinatorInternalEvent("FlushBatch", tp, () -> { + if (this.isCancelled()) return; + withActiveContextOrThrow(tp, CoordinatorContext::flushCurrentBatch); + })); + } + }); + CoordinatorRuntime.this.timer.add(lingerTimeoutTask.get()); + } + + currentBatch = new CoordinatorBatch( + prevLastWrittenOffset, + currentTimeMs, + maxBatchSize, + verificationGuard, + buffer, + builder, + lingerTimeoutTask + ); + } + } + + /** + * Appends records to the log and replay them to the state machine. + * + * @param producerId The producer id. + * @param producerEpoch The producer epoch. + * @param verificationGuard The verification guard. + * @param records The records to append. + * @param replay A boolean indicating whether the records + * must be replayed or not. + * @param event The event that must be completed when the + * records are written. + */ + private void append( + long producerId, + short producerEpoch, + VerificationGuard verificationGuard, + List records, + boolean replay, + DeferredEvent event + ) { + if (state != CoordinatorState.ACTIVE) { + throw new IllegalStateException("Coordinator must be active to append records"); + } + + if (records.isEmpty()) { + // If the records are empty, it was a read operation after all. In this case, + // the response can be returned directly iff there are no pending write operations; + // otherwise, the read needs to wait on the last write operation to be completed. + if (currentBatch != null) { + currentBatch.deferredEvents.add(event); + } else { + OptionalLong pendingOffset = deferredEventQueue.highestPendingOffset(); + if (pendingOffset.isPresent()) { + deferredEventQueue.add(pendingOffset.getAsLong(), event); + } else { + event.complete(null); + } + } + } else { + // If the records are not empty, first, they are applied to the state machine, + // second, they are appended to the opened batch. + long currentTimeMs = time.milliseconds(); + + // If the current write operation is transactional, the current batch + // is written before proceeding with it. + if (producerId != RecordBatch.NO_PRODUCER_ID) { + // If flushing fails, we don't catch the exception in order to let + // the caller fail the current operation. + flushCurrentBatch(); + } + + // Allocate a new batch if none exists. + maybeAllocateNewBatch( + producerId, + producerEpoch, + verificationGuard, + currentTimeMs + ); + + // Prepare the records. + List recordsToAppend = new ArrayList<>(records.size()); + for (U record : records) { + recordsToAppend.add(new SimpleRecord( + currentTimeMs, + serializer.serializeKey(record), + serializer.serializeValue(record) + )); + } + + // Compute the estimated size of the records. + int estimatedSize = AbstractRecords.estimateSizeInBytes( + currentBatch.builder.magic(), + compression.type(), + recordsToAppend + ); + + // Check if the current batch has enough space. We check is before + // replaying the records in order to avoid having to revert back + // changes if the records do not fit within a batch. + if (estimatedSize > currentBatch.builder.maxAllowedBytes()) { + throw new RecordTooLargeException("Message batch size is " + estimatedSize + + " bytes in append to partition " + tp + " which exceeds the maximum " + + "configured size of " + currentBatch.maxBatchSize + "."); + } + + if (!currentBatch.builder.hasRoomFor(estimatedSize)) { + // Otherwise, we write the current batch, allocate a new one and re-verify + // whether the records fit in it. + // If flushing fails, we don't catch the exception in order to let + // the caller fail the current operation. + flushCurrentBatch(); + maybeAllocateNewBatch( + producerId, + producerEpoch, + verificationGuard, + currentTimeMs + ); + } + + // Add the event to the list of pending events associated with the batch. + currentBatch.deferredEvents.add(event); + + try { + // Apply record to the state machine. + if (replay) { + for (int i = 0; i < records.size(); i++) { + // We compute the offset of the record based on the last written offset. The + // coordinator is the single writer to the underlying partition so we can + // deduce it like this. + coordinator.replay( + currentBatch.nextOffset + i, + producerId, + producerEpoch, + records.get(i) + ); + } + } + + // Append to the batch. + for (SimpleRecord record : recordsToAppend) { + currentBatch.builder.append(record); + currentBatch.nextOffset++; + } + } catch (Throwable t) { + log.error("Replaying records to {} failed due to: {}.", tp, t.getMessage()); + // If an exception is thrown, we fail the entire batch. Exceptions should be + // really exceptional in this code path and they would usually be the results + // of bugs preventing records to be replayed. + failCurrentBatch(t); + } + + // Write the current batch if it is transactional or if the linger timeout + // has expired. + // If flushing fails, we don't catch the exception in order to let + // the caller fail the current operation. + maybeFlushCurrentBatch(currentTimeMs); + } + } + + /** + * Completes a transaction. + * + * @param producerId The producer id. + * @param producerEpoch The producer epoch. + * @param coordinatorEpoch The coordinator epoch of the transaction coordinator. + * @param result The transaction result. + * @param event The event that must be completed when the + * control record is written. + */ + private void completeTransaction( + long producerId, + short producerEpoch, + int coordinatorEpoch, + TransactionResult result, + DeferredEvent event + ) { + if (state != CoordinatorState.ACTIVE) { + throw new IllegalStateException("Coordinator must be active to complete a transaction"); + } + + // The current batch must be written before the transaction marker is written + // in order to respect the order. + flushCurrentBatch(); + + long prevLastWrittenOffset = coordinator.lastWrittenOffset(); + try { + coordinator.replayEndTransactionMarker( + producerId, + producerEpoch, + result + ); + + long offset = partitionWriter.append( + tp, + VerificationGuard.SENTINEL, + MemoryRecords.withEndTransactionMarker( + time.milliseconds(), + producerId, + producerEpoch, + new EndTransactionMarker( + result == TransactionResult.COMMIT ? ControlRecordType.COMMIT : ControlRecordType.ABORT, + coordinatorEpoch + ) + ) + ); + coordinator.updateLastWrittenOffset(offset); + + deferredEventQueue.add(offset, event); + } catch (Throwable t) { + coordinator.revertLastWrittenOffset(prevLastWrittenOffset); + event.complete(t); + } + } } class OperationTimeout extends TimerTask { @@ -781,100 +1254,20 @@ public void run() { // Execute the operation. result = op.generateRecordsAndResult(context.coordinator.coordinator()); - if (result.records().isEmpty()) { - // If the records are empty, it was a read operation after all. In this case, - // the response can be returned directly iff there are no pending write operations; - // otherwise, the read needs to wait on the last write operation to be completed. - OptionalLong pendingOffset = context.deferredEventQueue.highestPendingOffset(); - if (pendingOffset.isPresent()) { - context.deferredEventQueue.add(pendingOffset.getAsLong(), this); - } else { - complete(null); - } - } else { - // If the records are not empty, first, they are applied to the state machine, - // second, then are written to the partition/log, and finally, the response - // is put into the deferred event queue. - long prevLastWrittenOffset = context.coordinator.lastWrittenOffset(); - LogConfig logConfig = partitionWriter.config(tp); - byte magic = logConfig.recordVersion().value; - int maxBatchSize = logConfig.maxMessageSize(); - long currentTimeMs = time.milliseconds(); - ByteBuffer buffer = context.bufferSupplier.get(Math.min(MIN_BUFFER_SIZE, maxBatchSize)); - - try { - MemoryRecordsBuilder builder = new MemoryRecordsBuilder( - buffer, - magic, - compression, - TimestampType.CREATE_TIME, - 0L, - currentTimeMs, - producerId, - producerEpoch, - 0, - producerId != RecordBatch.NO_PRODUCER_ID, - false, - RecordBatch.NO_PARTITION_LEADER_EPOCH, - maxBatchSize - ); - - // Apply the records to the state machine and add them to the batch. - for (int i = 0; i < result.records().size(); i++) { - U record = result.records().get(i); - - if (result.replayRecords()) { - // We compute the offset of the record based on the last written offset. The - // coordinator is the single writer to the underlying partition so we can - // deduce it like this. - context.coordinator.replay( - prevLastWrittenOffset + i, - producerId, - producerEpoch, - record - ); - } - - byte[] keyBytes = serializer.serializeKey(record); - byte[] valBytes = serializer.serializeValue(record); - - if (builder.hasRoomFor(currentTimeMs, keyBytes, valBytes, EMPTY_HEADERS)) { - builder.append( - currentTimeMs, - keyBytes, - valBytes, - EMPTY_HEADERS - ); - } else { - throw new RecordTooLargeException("Message batch size is " + builder.estimatedSizeInBytes() + - " bytes in append to partition " + tp + " which exceeds the maximum " + - "configured size of " + maxBatchSize + "."); - } - } - - // Write the records to the log and update the last written - // offset. - long offset = partitionWriter.append( - tp, - verificationGuard, - builder.build() - ); - context.coordinator.updateLastWrittenOffset(offset); + // Append the records and replay them to the state machine. + context.append( + producerId, + producerEpoch, + verificationGuard, + result.records(), + result.replayRecords(), + this + ); - // Add the response to the deferred queue. - if (!future.isDone()) { - context.deferredEventQueue.add(offset, this); - operationTimeout = new OperationTimeout(tp, this, writeTimeout.toMillis()); - timer.add(operationTimeout); - } else { - complete(null); - } - } catch (Throwable t) { - context.coordinator.revertLastWrittenOffset(prevLastWrittenOffset); - complete(t); - } finally { - context.bufferSupplier.release(buffer); - } + // If the operation is not done, create an operation timeout. + if (!future.isDone()) { + operationTimeout = new OperationTimeout(tp, this, writeTimeout.toMillis()); + timer.add(operationTimeout); } }); } catch (Throwable t) { @@ -1142,40 +1535,17 @@ public TopicPartition key() { public void run() { try { withActiveContextOrThrow(tp, context -> { - long prevLastWrittenOffset = context.coordinator.lastWrittenOffset(); + context.completeTransaction( + producerId, + producerEpoch, + coordinatorEpoch, + result, + this + ); - try { - context.coordinator.replayEndTransactionMarker( - producerId, - producerEpoch, - result - ); - - long offset = partitionWriter.append( - tp, - VerificationGuard.SENTINEL, - MemoryRecords.withEndTransactionMarker( - time.milliseconds(), - producerId, - producerEpoch, - new EndTransactionMarker( - result == TransactionResult.COMMIT ? ControlRecordType.COMMIT : ControlRecordType.ABORT, - coordinatorEpoch - ) - ) - ); - context.coordinator.updateLastWrittenOffset(offset); - - if (!future.isDone()) { - context.deferredEventQueue.add(offset, this); - operationTimeout = new OperationTimeout(tp, this, writeTimeout.toMillis()); - timer.add(operationTimeout); - } else { - complete(null); - } - } catch (Throwable t) { - context.coordinator.revertLastWrittenOffset(prevLastWrittenOffset); - complete(t); + if (!future.isDone()) { + operationTimeout = new OperationTimeout(tp, this, writeTimeout.toMillis()); + timer.add(operationTimeout); } }); } catch (Throwable t) { @@ -1449,6 +1819,12 @@ public void onHighWatermarkUpdated( */ private final Compression compression; + /** + * The duration in milliseconds that the coordinator will wait for writes to + * accumulate before flushing them to disk. + */ + private final int appendLingerMs; + /** * Atomic boolean indicating whether the runtime is running. */ @@ -1475,7 +1851,9 @@ public void onHighWatermarkUpdated( * @param coordinatorMetrics The coordinator metrics. * @param serializer The serializer. * @param compression The compression codec. + * @param appendLingerMs The append linger time in ms. */ + @SuppressWarnings("checkstyle:ParameterNumber") private CoordinatorRuntime( String logPrefix, LogContext logContext, @@ -1489,7 +1867,8 @@ private CoordinatorRuntime( CoordinatorRuntimeMetrics runtimeMetrics, CoordinatorMetrics coordinatorMetrics, Serializer serializer, - Compression compression + Compression compression, + int appendLingerMs ) { this.logPrefix = logPrefix; this.logContext = logContext; @@ -1506,6 +1885,7 @@ private CoordinatorRuntime( this.coordinatorMetrics = coordinatorMetrics; this.serializer = serializer; this.compression = compression; + this.appendLingerMs = appendLingerMs; } /** @@ -1836,36 +2216,6 @@ public void scheduleLoadOperation( case FAILED: case INITIAL: context.transitionTo(CoordinatorState.LOADING); - loader.load(tp, context.coordinator).whenComplete((summary, exception) -> { - scheduleInternalOperation("CompleteLoad(tp=" + tp + ", epoch=" + partitionEpoch + ")", tp, () -> { - CoordinatorContext ctx = coordinators.get(tp); - if (ctx != null) { - if (ctx.state != CoordinatorState.LOADING) { - log.info("Ignored load completion from {} because context is in {} state.", - ctx.tp, ctx.state); - return; - } - try { - if (exception != null) throw exception; - ctx.transitionTo(CoordinatorState.ACTIVE); - if (summary != null) { - runtimeMetrics.recordPartitionLoadSensor(summary.startTimeMs(), summary.endTimeMs()); - log.info("Finished loading of metadata from {} with epoch {} in {}ms where {}ms " + - "was spent in the scheduler. Loaded {} records which total to {} bytes.", - tp, partitionEpoch, summary.endTimeMs() - summary.startTimeMs(), - summary.schedulerQueueTimeMs(), summary.numRecords(), summary.numBytes()); - } - } catch (Throwable ex) { - log.error("Failed to load metadata from {} with epoch {} due to {}.", - tp, partitionEpoch, ex.toString()); - ctx.transitionTo(CoordinatorState.FAILED); - } - } else { - log.debug("Failed to complete the loading of metadata for {} in epoch {} since the coordinator does not exist.", - tp, partitionEpoch); - } - }); - }); break; case LOADING: diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorConfigTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorConfigTest.java index 03306c9040743..b65ceda74ed0b 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorConfigTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorConfigTest.java @@ -30,6 +30,7 @@ public class GroupCoordinatorConfigTest { public void testConfigs() { ConsumerGroupPartitionAssignor assignor = new RangeAssignor(); GroupCoordinatorConfig config = new GroupCoordinatorConfig( + 10, 10, 30, 10, @@ -65,6 +66,7 @@ public void testConfigs() { assertEquals(24 * 60 * 60 * 1000L, config.offsetsRetentionMs); assertEquals(5000, config.offsetCommitTimeoutMs); assertEquals(CompressionType.GZIP, config.compressionType); + assertEquals(10, config.appendLingerMs); } public static GroupCoordinatorConfig createGroupCoordinatorConfig( @@ -74,6 +76,7 @@ public static GroupCoordinatorConfig createGroupCoordinatorConfig( ) { return new GroupCoordinatorConfig( 1, + 10, 45, 5, Integer.MAX_VALUE, diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorServiceTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorServiceTest.java index 6cd96458c647d..7ddb04a6d8b68 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorServiceTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorServiceTest.java @@ -118,6 +118,7 @@ private CoordinatorRuntime mockRuntime private GroupCoordinatorConfig createConfig() { return new GroupCoordinatorConfig( 1, + 10, 45, 5, Integer.MAX_VALUE, diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/runtime/CoordinatorRuntimeTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/runtime/CoordinatorRuntimeTest.java index ae1d404792406..5052881d3c0ba 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/runtime/CoordinatorRuntimeTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/runtime/CoordinatorRuntimeTest.java @@ -21,6 +21,7 @@ import org.apache.kafka.common.compress.Compression; import org.apache.kafka.common.errors.NotCoordinatorException; import org.apache.kafka.common.errors.NotEnoughReplicasException; +import org.apache.kafka.common.errors.RecordTooLargeException; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.record.AbstractRecords; @@ -60,8 +61,8 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; +import java.util.Comparator; import java.util.Deque; -import java.util.HashSet; import java.util.LinkedList; import java.util.List; import java.util.Objects; @@ -74,6 +75,7 @@ import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicInteger; import java.util.stream.Collectors; +import java.util.stream.Stream; import static org.apache.kafka.common.utils.Utils.mkSet; import static org.apache.kafka.coordinator.group.runtime.CoordinatorRuntime.CoordinatorState.ACTIVE; @@ -85,6 +87,8 @@ import static org.apache.kafka.test.TestUtils.assertFutureThrows; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNotEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertNull; import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -147,7 +151,7 @@ public void close() throws Exception {} * when poll() is called. */ private static class ManualEventProcessor implements CoordinatorEventProcessor { - private Deque queue = new LinkedList<>(); + private final Deque queue = new LinkedList<>(); @Override public void enqueueLast(CoordinatorEvent event) throws RejectedExecutionException { @@ -274,9 +278,72 @@ public long append( * A simple Coordinator implementation that stores the records into a set. */ static class MockCoordinatorShard implements CoordinatorShard { + static class RecordAndMetadata { + public final long offset; + public final long producerId; + public final short producerEpoch; + public final String record; + + public RecordAndMetadata( + long offset, + String record + ) { + this( + offset, + RecordBatch.NO_PRODUCER_ID, + RecordBatch.NO_PRODUCER_EPOCH, + record + ); + } + + public RecordAndMetadata( + long offset, + long producerId, + short producerEpoch, + String record + ) { + this.offset = offset; + this.producerId = producerId; + this.producerEpoch = producerEpoch; + this.record = record; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + RecordAndMetadata that = (RecordAndMetadata) o; + + if (offset != that.offset) return false; + if (producerId != that.producerId) return false; + if (producerEpoch != that.producerEpoch) return false; + return Objects.equals(record, that.record); + } + + @Override + public int hashCode() { + int result = (int) (offset ^ (offset >>> 32)); + result = 31 * result + (int) (producerId ^ (producerId >>> 32)); + result = 31 * result + (int) producerEpoch; + result = 31 * result + (record != null ? record.hashCode() : 0); + return result; + } + + @Override + public String toString() { + return "RecordAndMetadata(" + + "offset=" + offset + + ", producerId=" + producerId + + ", producerEpoch=" + producerEpoch + + ", record='" + record.substring(0, 10) + '\'' + + ')'; + } + } + private final SnapshotRegistry snapshotRegistry; - private final TimelineHashSet records; - private final TimelineHashMap> pendingRecords; + private final TimelineHashSet records; + private final TimelineHashMap> pendingRecords; private final CoordinatorTimer timer; MockCoordinatorShard( @@ -296,12 +363,19 @@ public void replay( short producerEpoch, String record ) throws RuntimeException { + RecordAndMetadata recordAndMetadata = new RecordAndMetadata( + offset, + producerId, + producerEpoch, + record + ); + if (producerId == RecordBatch.NO_PRODUCER_ID) { - records.add(record); + records.add(recordAndMetadata); } else { pendingRecords .computeIfAbsent(producerId, __ -> new TimelineHashSet<>(snapshotRegistry, 0)) - .add(record); + .add(recordAndMetadata); } } @@ -312,7 +386,7 @@ public void replayEndTransactionMarker( TransactionResult result ) throws RuntimeException { if (result == TransactionResult.COMMIT) { - TimelineHashSet pending = pendingRecords.remove(producerId); + TimelineHashSet pending = pendingRecords.remove(producerId); if (pending == null) return; records.addAll(pending); } else { @@ -321,13 +395,26 @@ public void replayEndTransactionMarker( } Set pendingRecords(long producerId) { - TimelineHashSet pending = pendingRecords.get(producerId); + TimelineHashSet pending = pendingRecords.get(producerId); if (pending == null) return Collections.emptySet(); - return Collections.unmodifiableSet(new HashSet<>(pending)); + return Collections.unmodifiableSet( + pending.stream().map(record -> record.record).collect(Collectors.toSet()) + ); } Set records() { - return Collections.unmodifiableSet(new HashSet<>(records)); + return Collections.unmodifiableSet( + records.stream().map(record -> record.record).collect(Collectors.toSet()) + ); + } + + List fullRecords() { + return Collections.unmodifiableList( + records + .stream() + .sorted(Comparator.comparingLong(record -> record.offset)) + .collect(Collectors.toList()) + ); } CoordinatorTimer timer() { @@ -407,10 +494,17 @@ private static MemoryRecords records( long timestamp, String... records ) { - if (records.length == 0) + return records(timestamp, Arrays.stream(records).collect(Collectors.toList())); + } + + private static MemoryRecords records( + long timestamp, + List records + ) { + if (records.isEmpty()) return MemoryRecords.EMPTY; - List simpleRecords = Arrays.stream(records).map(record -> + List simpleRecords = records.stream().map(record -> new SimpleRecord(timestamp, record.getBytes(Charset.defaultCharset())) ).collect(Collectors.toList()); @@ -447,10 +541,24 @@ private static MemoryRecords transactionalRecords( long timestamp, String... records ) { - if (records.length == 0) + return transactionalRecords( + producerId, + producerEpoch, + timestamp, + Arrays.stream(records).collect(Collectors.toList()) + ); + } + + private static MemoryRecords transactionalRecords( + long producerId, + short producerEpoch, + long timestamp, + List records + ) { + if (records.isEmpty()) return MemoryRecords.EMPTY; - List simpleRecords = Arrays.stream(records).map(record -> + List simpleRecords = records.stream().map(record -> new SimpleRecord(timestamp, record.getBytes(Charset.defaultCharset())) ).collect(Collectors.toList()); @@ -986,13 +1094,13 @@ public void testScheduleWriteOp() throws ExecutionException, InterruptedExceptio // Records have been replayed to the coordinator. assertEquals(mkSet("record1", "record2"), ctx.coordinator.coordinator().records()); // Records have been written to the log. - assertEquals(Arrays.asList( + assertEquals(Collections.singletonList( records(timer.time().milliseconds(), "record1", "record2") ), writer.entries(TP)); // Write #2. CompletableFuture write2 = runtime.scheduleWriteOperation("write#2", TP, DEFAULT_WRITE_TIMEOUT, - state -> new CoordinatorResult<>(Arrays.asList("record3"), "response2")); + state -> new CoordinatorResult<>(Collections.singletonList("record3"), "response2")); // Verify that the write is not committed yet. assertFalse(write2.isDone()); @@ -1540,7 +1648,7 @@ public void testScheduleTransactionCompletion(TransactionResult result) throws E 100L )); // Records have been written to the log. - assertEquals(Arrays.asList( + assertEquals(Collections.singletonList( transactionalRecords(100L, (short) 5, timer.time().milliseconds(), "record1", "record2") ), writer.entries(TP)); @@ -1785,7 +1893,7 @@ public void replayEndTransactionMarker( assertEquals(Arrays.asList(0L, 2L), ctx.coordinator.snapshotRegistry().epochsList()); assertEquals(mkSet("record1", "record2"), ctx.coordinator.coordinator().pendingRecords(100L)); assertEquals(Collections.emptySet(), ctx.coordinator.coordinator().records()); - assertEquals(Arrays.asList( + assertEquals(Collections.singletonList( transactionalRecords(100L, (short) 5, timer.time().milliseconds(), "record1", "record2") ), writer.entries(TP)); @@ -1807,7 +1915,7 @@ public void replayEndTransactionMarker( assertEquals(Arrays.asList(0L, 2L), ctx.coordinator.snapshotRegistry().epochsList()); assertEquals(mkSet("record1", "record2"), ctx.coordinator.coordinator().pendingRecords(100L)); assertEquals(Collections.emptySet(), ctx.coordinator.coordinator().records()); - assertEquals(Arrays.asList( + assertEquals(Collections.singletonList( transactionalRecords(100L, (short) 5, timer.time().milliseconds(), "record1", "record2") ), writer.entries(TP)); } @@ -1985,7 +2093,7 @@ public void testScheduleReadAllOp() throws ExecutionException, InterruptedExcept // Read. List>> responses = runtime.scheduleReadAllOperation( "read", - (state, offset) -> new ArrayList<>(state.records) + (state, offset) -> new ArrayList<>(state.records()) ); assertEquals( @@ -3059,6 +3167,594 @@ public void testAppendRecordBatchSize() { assertTrue(batchSize > MIN_BUFFER_SIZE && batchSize < maxBatchSize); } + @Test + public void testScheduleWriteOperationWithBatching() throws ExecutionException, InterruptedException, TimeoutException { + MockTimer timer = new MockTimer(); + MockPartitionWriter writer = new MockPartitionWriter(); + + CoordinatorRuntime runtime = + new CoordinatorRuntime.Builder() + .withTime(timer.time()) + .withTimer(timer) + .withDefaultWriteTimeOut(Duration.ofMillis(20)) + .withLoader(new MockCoordinatorLoader()) + .withEventProcessor(new DirectEventProcessor()) + .withPartitionWriter(writer) + .withCoordinatorShardBuilderSupplier(new MockCoordinatorShardBuilderSupplier()) + .withCoordinatorRuntimeMetrics(mock(GroupCoordinatorRuntimeMetrics.class)) + .withCoordinatorMetrics(mock(GroupCoordinatorMetrics.class)) + .withSerializer(new StringSerializer()) + .withAppendLingerMs(10) + .build(); + + // Schedule the loading. + runtime.scheduleLoadOperation(TP, 10); + + // Verify the initial state. + CoordinatorRuntime.CoordinatorContext ctx = runtime.contextOrThrow(TP); + assertEquals(0L, ctx.coordinator.lastWrittenOffset()); + assertEquals(0L, ctx.coordinator.lastCommittedOffset()); + assertEquals(Collections.singletonList(0L), ctx.coordinator.snapshotRegistry().epochsList()); + assertNull(ctx.currentBatch); + + // Get the max batch size. + int maxBatchSize = writer.config(TP).maxMessageSize(); + + // Create records with a quarter of the max batch size each. Keep in mind that + // each batch has a header so it is not possible to have those four records + // in one single batch. + List records = Stream.of('1', '2', '3', '4').map(c -> { + char[] payload = new char[maxBatchSize / 4]; + Arrays.fill(payload, c); + return new String(payload); + }).collect(Collectors.toList()); + + // Write #1 with two records. + CompletableFuture write1 = runtime.scheduleWriteOperation("write#1", TP, Duration.ofMillis(20), + state -> new CoordinatorResult<>(records.subList(0, 2), "response1") + ); + + // Verify that the write is not committed yet. + assertFalse(write1.isDone()); + + // A batch has been created. + assertNotNull(ctx.currentBatch); + + // Verify the state. Records are replayed but no batch written. + assertEquals(0L, ctx.coordinator.lastWrittenOffset()); + assertEquals(0L, ctx.coordinator.lastCommittedOffset()); + assertEquals(Collections.singletonList(0L), ctx.coordinator.snapshotRegistry().epochsList()); + assertEquals(Arrays.asList( + new MockCoordinatorShard.RecordAndMetadata(0, records.get(0)), + new MockCoordinatorShard.RecordAndMetadata(1, records.get(1)) + ), ctx.coordinator.coordinator().fullRecords()); + assertEquals(Collections.emptyList(), writer.entries(TP)); + + // Write #2 with one record. + CompletableFuture write2 = runtime.scheduleWriteOperation("write#2", TP, Duration.ofMillis(20), + state -> new CoordinatorResult<>(records.subList(2, 3), "response2") + ); + + // Verify that the write is not committed yet. + assertFalse(write2.isDone()); + + // Verify the state. Records are replayed but no batch written. + assertEquals(0L, ctx.coordinator.lastWrittenOffset()); + assertEquals(0L, ctx.coordinator.lastCommittedOffset()); + assertEquals(Collections.singletonList(0L), ctx.coordinator.snapshotRegistry().epochsList()); + assertEquals(Arrays.asList( + new MockCoordinatorShard.RecordAndMetadata(0, records.get(0)), + new MockCoordinatorShard.RecordAndMetadata(1, records.get(1)), + new MockCoordinatorShard.RecordAndMetadata(2, records.get(2)) + ), ctx.coordinator.coordinator().fullRecords()); + assertEquals(Collections.emptyList(), writer.entries(TP)); + + // Write #3 with one record. This one cannot go into the existing batch + // so the existing batch should be flushed and a new one should be created. + CompletableFuture write3 = runtime.scheduleWriteOperation("write#3", TP, Duration.ofMillis(20), + state -> new CoordinatorResult<>(records.subList(3, 4), "response3") + ); + + // Verify that the write is not committed yet. + assertFalse(write3.isDone()); + + // Verify the state. Records are replayed. The previous batch + // got flushed with all the records but the new one from #3. + assertEquals(3L, ctx.coordinator.lastWrittenOffset()); + assertEquals(0L, ctx.coordinator.lastCommittedOffset()); + assertEquals(Arrays.asList(0L, 3L), ctx.coordinator.snapshotRegistry().epochsList()); + assertEquals(Arrays.asList( + new MockCoordinatorShard.RecordAndMetadata(0, records.get(0)), + new MockCoordinatorShard.RecordAndMetadata(1, records.get(1)), + new MockCoordinatorShard.RecordAndMetadata(2, records.get(2)), + new MockCoordinatorShard.RecordAndMetadata(3, records.get(3)) + ), ctx.coordinator.coordinator().fullRecords()); + assertEquals(Collections.singletonList( + records(timer.time().milliseconds(), records.subList(0, 3)) + ), writer.entries(TP)); + + // Advance past the linger time. + timer.advanceClock(11); + + // Verify the state. The pending batch is flushed. + assertEquals(4L, ctx.coordinator.lastWrittenOffset()); + assertEquals(0L, ctx.coordinator.lastCommittedOffset()); + assertEquals(Arrays.asList(0L, 3L, 4L), ctx.coordinator.snapshotRegistry().epochsList()); + assertEquals(Arrays.asList( + new MockCoordinatorShard.RecordAndMetadata(0, records.get(0)), + new MockCoordinatorShard.RecordAndMetadata(1, records.get(1)), + new MockCoordinatorShard.RecordAndMetadata(2, records.get(2)), + new MockCoordinatorShard.RecordAndMetadata(3, records.get(3)) + ), ctx.coordinator.coordinator().fullRecords()); + assertEquals(Arrays.asList( + records(timer.time().milliseconds() - 11, records.subList(0, 3)), + records(timer.time().milliseconds() - 11, records.subList(3, 4)) + ), writer.entries(TP)); + + // Commit and verify that writes are completed. + writer.commit(TP); + assertTrue(write1.isDone()); + assertTrue(write2.isDone()); + assertTrue(write3.isDone()); + assertEquals("response1", write1.get(5, TimeUnit.SECONDS)); + assertEquals("response2", write2.get(5, TimeUnit.SECONDS)); + assertEquals("response3", write3.get(5, TimeUnit.SECONDS)); + } + + @Test + public void testScheduleWriteOperationWithBatchingWhenRecordsTooLarge() { + MockTimer timer = new MockTimer(); + MockPartitionWriter writer = new MockPartitionWriter(); + + CoordinatorRuntime runtime = + new CoordinatorRuntime.Builder() + .withTime(timer.time()) + .withTimer(timer) + .withDefaultWriteTimeOut(Duration.ofMillis(20)) + .withLoader(new MockCoordinatorLoader()) + .withEventProcessor(new DirectEventProcessor()) + .withPartitionWriter(writer) + .withCoordinatorShardBuilderSupplier(new MockCoordinatorShardBuilderSupplier()) + .withCoordinatorRuntimeMetrics(mock(GroupCoordinatorRuntimeMetrics.class)) + .withCoordinatorMetrics(mock(GroupCoordinatorMetrics.class)) + .withSerializer(new StringSerializer()) + .withAppendLingerMs(10) + .build(); + + // Schedule the loading. + runtime.scheduleLoadOperation(TP, 10); + + // Verify the initial state. + CoordinatorRuntime.CoordinatorContext ctx = runtime.contextOrThrow(TP); + assertEquals(0L, ctx.coordinator.lastWrittenOffset()); + assertEquals(0L, ctx.coordinator.lastCommittedOffset()); + assertEquals(Collections.singletonList(0L), ctx.coordinator.snapshotRegistry().epochsList()); + assertNull(ctx.currentBatch); + + // Get the max batch size. + int maxBatchSize = writer.config(TP).maxMessageSize(); + + // Create records with a quarter of the max batch size each. Keep in mind that + // each batch has a header so it is not possible to have those four records + // in one single batch. + List records = Stream.of('1', '2', '3', '4').map(c -> { + char[] payload = new char[maxBatchSize / 4]; + Arrays.fill(payload, c); + return new String(payload); + }).collect(Collectors.toList()); + + // Write all the records. + CompletableFuture write = runtime.scheduleWriteOperation("write#1", TP, Duration.ofMillis(20), + state -> new CoordinatorResult<>(records, "response1") + ); + + assertFutureThrows(write, RecordTooLargeException.class); + } + + @Test + public void testScheduleWriteOperationWithBatchingWhenWriteFails() { + MockTimer timer = new MockTimer(); + // The partition writer only accept no writes. + MockPartitionWriter writer = new MockPartitionWriter(0); + + CoordinatorRuntime runtime = + new CoordinatorRuntime.Builder() + .withTime(timer.time()) + .withTimer(timer) + .withDefaultWriteTimeOut(Duration.ofMillis(20)) + .withLoader(new MockCoordinatorLoader()) + .withEventProcessor(new DirectEventProcessor()) + .withPartitionWriter(writer) + .withCoordinatorShardBuilderSupplier(new MockCoordinatorShardBuilderSupplier()) + .withCoordinatorRuntimeMetrics(mock(GroupCoordinatorRuntimeMetrics.class)) + .withCoordinatorMetrics(mock(GroupCoordinatorMetrics.class)) + .withSerializer(new StringSerializer()) + .withAppendLingerMs(10) + .build(); + + // Schedule the loading. + runtime.scheduleLoadOperation(TP, 10); + + // Verify the initial state. + CoordinatorRuntime.CoordinatorContext ctx = runtime.contextOrThrow(TP); + assertEquals(0L, ctx.coordinator.lastWrittenOffset()); + assertEquals(0L, ctx.coordinator.lastCommittedOffset()); + assertEquals(Collections.singletonList(0L), ctx.coordinator.snapshotRegistry().epochsList()); + assertNull(ctx.currentBatch); + + // Get the max batch size. + int maxBatchSize = writer.config(TP).maxMessageSize(); + + // Create records with a quarter of the max batch size each. Keep in mind that + // each batch has a header so it is not possible to have those four records + // in one single batch. + List records = Stream.of('1', '2', '3', '4').map(c -> { + char[] payload = new char[maxBatchSize / 4]; + Arrays.fill(payload, c); + return new String(payload); + }).collect(Collectors.toList()); + + // Write #1. + CompletableFuture write1 = runtime.scheduleWriteOperation("write#1", TP, Duration.ofMillis(20), + state -> new CoordinatorResult<>(records.subList(0, 1), "response1")); + + // Write #2. + CompletableFuture write2 = runtime.scheduleWriteOperation("write#2", TP, Duration.ofMillis(20), + state -> new CoordinatorResult<>(records.subList(1, 2), "response2")); + + // Write #3. + CompletableFuture write3 = runtime.scheduleWriteOperation("write#3", TP, Duration.ofMillis(20), + state -> new CoordinatorResult<>(records.subList(2, 3), "response3")); + + // Verify the state. + assertEquals(0L, ctx.coordinator.lastWrittenOffset()); + assertEquals(0L, ctx.coordinator.lastCommittedOffset()); + assertEquals(Collections.singletonList(0L), ctx.coordinator.snapshotRegistry().epochsList()); + assertEquals(Arrays.asList( + new MockCoordinatorShard.RecordAndMetadata(0, records.get(0)), + new MockCoordinatorShard.RecordAndMetadata(1, records.get(1)), + new MockCoordinatorShard.RecordAndMetadata(2, records.get(2)) + ), ctx.coordinator.coordinator().fullRecords()); + assertEquals(Collections.emptyList(), writer.entries(TP)); + + // Write #4. This write cannot make it in the current batch. So the current batch + // is flushed. It will fail. So we expect all writes to fail. + CompletableFuture write4 = runtime.scheduleWriteOperation("write#4", TP, Duration.ofMillis(20), + state -> new CoordinatorResult<>(records.subList(3, 4), "response4")); + + // Verify the futures. + assertFutureThrows(write1, KafkaException.class); + assertFutureThrows(write2, KafkaException.class); + assertFutureThrows(write3, KafkaException.class); + // Write #4 is also expected to fail. + assertFutureThrows(write4, KafkaException.class); + + // Verify the state. The state should be reverted to the initial state. + assertEquals(0L, ctx.coordinator.lastWrittenOffset()); + assertEquals(0L, ctx.coordinator.lastCommittedOffset()); + assertEquals(Collections.singletonList(0L), ctx.coordinator.snapshotRegistry().epochsList()); + assertEquals(Collections.emptyList(), ctx.coordinator.coordinator().fullRecords()); + assertEquals(Collections.emptyList(), writer.entries(TP)); + } + + @Test + public void testScheduleWriteOperationWithBatchingWhenReplayFails() { + MockTimer timer = new MockTimer(); + MockPartitionWriter writer = new MockPartitionWriter(); + + CoordinatorRuntime runtime = + new CoordinatorRuntime.Builder() + .withTime(timer.time()) + .withTimer(timer) + .withDefaultWriteTimeOut(Duration.ofMillis(20)) + .withLoader(new MockCoordinatorLoader()) + .withEventProcessor(new DirectEventProcessor()) + .withPartitionWriter(writer) + .withCoordinatorShardBuilderSupplier(new MockCoordinatorShardBuilderSupplier()) + .withCoordinatorRuntimeMetrics(mock(GroupCoordinatorRuntimeMetrics.class)) + .withCoordinatorMetrics(mock(GroupCoordinatorMetrics.class)) + .withSerializer(new StringSerializer()) + .withAppendLingerMs(10) + .build(); + + // Schedule the loading. + runtime.scheduleLoadOperation(TP, 10); + + // Verify the initial state. + CoordinatorRuntime.CoordinatorContext ctx = runtime.contextOrThrow(TP); + assertEquals(0L, ctx.coordinator.lastWrittenOffset()); + assertEquals(0L, ctx.coordinator.lastCommittedOffset()); + assertEquals(Collections.singletonList(0L), ctx.coordinator.snapshotRegistry().epochsList()); + assertNull(ctx.currentBatch); + + // Override the coordinator with a coordinator that throws + // an exception when replay is called. + SnapshotRegistry snapshotRegistry = ctx.coordinator.snapshotRegistry(); + ctx.coordinator = new SnapshottableCoordinator<>( + new LogContext(), + snapshotRegistry, + new MockCoordinatorShard(snapshotRegistry, ctx.timer) { + @Override + public void replay( + long offset, + long producerId, + short producerEpoch, + String record + ) throws RuntimeException { + if (offset >= 1) { + throw new IllegalArgumentException("error"); + } + super.replay( + offset, + producerId, + producerEpoch, + record + ); + } + }, + TP + ); + + // Get the max batch size. + int maxBatchSize = writer.config(TP).maxMessageSize(); + + // Create records with a quarter of the max batch size each. + List records = Stream.of('1', '2').map(c -> { + char[] payload = new char[maxBatchSize / 4]; + Arrays.fill(payload, c); + return new String(payload); + }).collect(Collectors.toList()); + + // Write #1. + CompletableFuture write1 = runtime.scheduleWriteOperation("write#1", TP, Duration.ofMillis(20), + state -> new CoordinatorResult<>(records.subList(0, 1), "response1")); + + // Verify the state. + assertEquals(0L, ctx.coordinator.lastWrittenOffset()); + assertEquals(0L, ctx.coordinator.lastCommittedOffset()); + assertEquals(Collections.singletonList(0L), ctx.coordinator.snapshotRegistry().epochsList()); + assertEquals(Collections.singletonList( + new MockCoordinatorShard.RecordAndMetadata(0, records.get(0)) + ), ctx.coordinator.coordinator().fullRecords()); + assertEquals(Collections.emptyList(), writer.entries(TP)); + + // Write #2. It should fail. + CompletableFuture write2 = runtime.scheduleWriteOperation("write#2", TP, Duration.ofMillis(20), + state -> new CoordinatorResult<>(records.subList(1, 2), "response2")); + + // Verify the futures. + assertFutureThrows(write1, IllegalArgumentException.class); + assertFutureThrows(write2, IllegalArgumentException.class); + + // Verify the state. + assertEquals(0L, ctx.coordinator.lastWrittenOffset()); + assertEquals(0L, ctx.coordinator.lastCommittedOffset()); + assertEquals(Collections.singletonList(0L), ctx.coordinator.snapshotRegistry().epochsList()); + assertEquals(Collections.emptyList(), ctx.coordinator.coordinator().fullRecords()); + assertEquals(Collections.emptyList(), writer.entries(TP)); + } + + @Test + public void testScheduleTransactionalWriteOperationWithBatching() throws ExecutionException, InterruptedException, TimeoutException { + MockTimer timer = new MockTimer(); + MockPartitionWriter writer = new MockPartitionWriter(); + + CoordinatorRuntime runtime = + new CoordinatorRuntime.Builder() + .withTime(timer.time()) + .withTimer(timer) + .withDefaultWriteTimeOut(Duration.ofMillis(20)) + .withLoader(new MockCoordinatorLoader()) + .withEventProcessor(new DirectEventProcessor()) + .withPartitionWriter(writer) + .withCoordinatorShardBuilderSupplier(new MockCoordinatorShardBuilderSupplier()) + .withCoordinatorRuntimeMetrics(mock(GroupCoordinatorRuntimeMetrics.class)) + .withCoordinatorMetrics(mock(GroupCoordinatorMetrics.class)) + .withSerializer(new StringSerializer()) + .withAppendLingerMs(10) + .build(); + + // Schedule the loading. + runtime.scheduleLoadOperation(TP, 10); + + // Verify the initial state. + CoordinatorRuntime.CoordinatorContext ctx = runtime.contextOrThrow(TP); + assertEquals(0L, ctx.coordinator.lastWrittenOffset()); + assertEquals(0L, ctx.coordinator.lastCommittedOffset()); + assertEquals(Collections.singletonList(0L), ctx.coordinator.snapshotRegistry().epochsList()); + assertNull(ctx.currentBatch); + + // Write #1 with one record. + CompletableFuture write1 = runtime.scheduleWriteOperation("write#1", TP, Duration.ofMillis(20), + state -> new CoordinatorResult<>(Collections.singletonList("record#1"), "response1") + ); + + // Verify that the write is not committed yet. + assertFalse(write1.isDone()); + + // Verify the state. Records are replayed but no batch written. + assertEquals(0L, ctx.coordinator.lastWrittenOffset()); + assertEquals(0L, ctx.coordinator.lastCommittedOffset()); + assertEquals(Collections.singletonList(0L), ctx.coordinator.snapshotRegistry().epochsList()); + assertEquals(Collections.emptySet(), ctx.coordinator.coordinator().pendingRecords(100L)); + assertEquals(mkSet("record#1"), ctx.coordinator.coordinator().records()); + assertEquals(Collections.emptyList(), writer.entries(TP)); + + // Transactional write #2 with one record. This will flush the current batch. + CompletableFuture write2 = runtime.scheduleTransactionalWriteOperation( + "txn-write#1", + TP, + "transactional-id", + 100L, + (short) 50, + Duration.ofMillis(20), + state -> new CoordinatorResult<>(Collections.singletonList("record#2"), "response2"), + TXN_OFFSET_COMMIT_LATEST_VERSION + ); + + // Verify that the write is not committed yet. + assertFalse(write2.isDone()); + + // Verify the state. The current batch and the transactional records are + // written to the log. + assertEquals(2L, ctx.coordinator.lastWrittenOffset()); + assertEquals(0L, ctx.coordinator.lastCommittedOffset()); + assertEquals(Arrays.asList(0L, 1L, 2L), ctx.coordinator.snapshotRegistry().epochsList()); + assertEquals(mkSet("record#2"), ctx.coordinator.coordinator().pendingRecords(100L)); + assertEquals(mkSet("record#1"), ctx.coordinator.coordinator().records()); + assertEquals(Arrays.asList( + records(timer.time().milliseconds(), "record#1"), + transactionalRecords(100L, (short) 50, timer.time().milliseconds(), "record#2") + ), writer.entries(TP)); + + // Write #3 with one record. + CompletableFuture write3 = runtime.scheduleWriteOperation("write#3", TP, Duration.ofMillis(20), + state -> new CoordinatorResult<>(Collections.singletonList("record#3"), "response3") + ); + + // Verify that the write is not committed yet. + assertFalse(write3.isDone()); + + // Verify the state. + assertEquals(2L, ctx.coordinator.lastWrittenOffset()); + assertEquals(0L, ctx.coordinator.lastCommittedOffset()); + assertEquals(Arrays.asList(0L, 1L, 2L), ctx.coordinator.snapshotRegistry().epochsList()); + assertEquals(mkSet("record#2"), ctx.coordinator.coordinator().pendingRecords(100L)); + assertEquals(mkSet("record#1", "record#3"), ctx.coordinator.coordinator().records()); + assertEquals(Arrays.asList( + records(timer.time().milliseconds(), "record#1"), + transactionalRecords(100L, (short) 50, timer.time().milliseconds(), "record#2") + ), writer.entries(TP)); + + // Complete transaction #1. It will flush the current batch if any. + CompletableFuture complete1 = runtime.scheduleTransactionCompletion( + "complete#1", + TP, + 100L, + (short) 50, + 10, + TransactionResult.COMMIT, + DEFAULT_WRITE_TIMEOUT + ); + + // Verify that the completion is not committed yet. + assertFalse(complete1.isDone()); + + // Verify the state. + assertEquals(4L, ctx.coordinator.lastWrittenOffset()); + assertEquals(0L, ctx.coordinator.lastCommittedOffset()); + assertEquals(Arrays.asList(0L, 1L, 2L, 3L, 4L), ctx.coordinator.snapshotRegistry().epochsList()); + assertEquals(Collections.emptySet(), ctx.coordinator.coordinator().pendingRecords(100L)); + assertEquals(mkSet("record#1", "record#2", "record#3"), ctx.coordinator.coordinator().records()); + assertEquals(Arrays.asList( + records(timer.time().milliseconds(), "record#1"), + transactionalRecords(100L, (short) 50, timer.time().milliseconds(), "record#2"), + records(timer.time().milliseconds(), "record#3"), + endTransactionMarker(100L, (short) 50, timer.time().milliseconds(), 10, ControlRecordType.COMMIT) + ), writer.entries(TP)); + + // Commit and verify that writes are completed. + writer.commit(TP); + assertTrue(write1.isDone()); + assertTrue(write2.isDone()); + assertTrue(write3.isDone()); + assertTrue(complete1.isDone()); + assertEquals("response1", write1.get(5, TimeUnit.SECONDS)); + assertEquals("response2", write2.get(5, TimeUnit.SECONDS)); + assertEquals("response3", write3.get(5, TimeUnit.SECONDS)); + assertNull(complete1.get(5, TimeUnit.SECONDS)); + } + + @Test + public void testStateMachineIsReloadedWhenOutOfSync() { + MockTimer timer = new MockTimer(); + MockCoordinatorLoader loader = spy(new MockCoordinatorLoader()); + MockPartitionWriter writer = new MockPartitionWriter() { + @Override + public long append( + TopicPartition tp, + VerificationGuard verificationGuard, + MemoryRecords batch + ) { + // Add 1 to the returned offsets. + return super.append(tp, verificationGuard, batch) + 1; + } + }; + + CoordinatorRuntime runtime = + new CoordinatorRuntime.Builder() + .withTime(timer.time()) + .withTimer(timer) + .withDefaultWriteTimeOut(Duration.ofMillis(20)) + .withLoader(loader) + .withEventProcessor(new DirectEventProcessor()) + .withPartitionWriter(writer) + .withCoordinatorShardBuilderSupplier(new MockCoordinatorShardBuilderSupplier()) + .withCoordinatorRuntimeMetrics(mock(GroupCoordinatorRuntimeMetrics.class)) + .withCoordinatorMetrics(mock(GroupCoordinatorMetrics.class)) + .withSerializer(new StringSerializer()) + .withAppendLingerMs(10) + .build(); + + // Schedule the loading. + runtime.scheduleLoadOperation(TP, 10); + + // Verify the initial state. + CoordinatorRuntime.CoordinatorContext ctx = runtime.contextOrThrow(TP); + assertEquals(ACTIVE, ctx.state); + assertEquals(0L, ctx.coordinator.lastWrittenOffset()); + assertEquals(0L, ctx.coordinator.lastCommittedOffset()); + assertEquals(Collections.singletonList(0L), ctx.coordinator.snapshotRegistry().epochsList()); + assertNull(ctx.currentBatch); + + // Keep a reference to the current coordinator. + SnapshottableCoordinator coordinator = ctx.coordinator; + + // Get the max batch size. + int maxBatchSize = writer.config(TP).maxMessageSize(); + + // Create records with a quarter of the max batch size each. Keep in mind that + // each batch has a header so it is not possible to have those four records + // in one single batch. + List records = Stream.of('1', '2', '3', '4').map(c -> { + char[] payload = new char[maxBatchSize / 4]; + Arrays.fill(payload, c); + return new String(payload); + }).collect(Collectors.toList()); + + // Write #1. + CompletableFuture write1 = runtime.scheduleWriteOperation("write#1", TP, Duration.ofMillis(20), + state -> new CoordinatorResult<>(records.subList(0, 1), "response1")); + + // Write #2. + CompletableFuture write2 = runtime.scheduleWriteOperation("write#2", TP, Duration.ofMillis(20), + state -> new CoordinatorResult<>(records.subList(1, 2), "response2")); + + // Write #3. + CompletableFuture write3 = runtime.scheduleWriteOperation("write#3", TP, Duration.ofMillis(20), + state -> new CoordinatorResult<>(records.subList(2, 3), "response3")); + + // Write #4. This write cannot make it in the current batch. So the current batch + // is flushed. It will fail. So we expect all writes to fail. + CompletableFuture write4 = runtime.scheduleWriteOperation("write#4", TP, Duration.ofMillis(20), + state -> new CoordinatorResult<>(records.subList(3, 4), "response4")); + + // Verify the futures. + assertFutureThrows(write1, NotCoordinatorException.class); + assertFutureThrows(write2, NotCoordinatorException.class); + assertFutureThrows(write3, NotCoordinatorException.class); + // Write #4 is also expected to fail. + assertFutureThrows(write4, NotCoordinatorException.class); + + // Verify that the state machine was loaded twice. + verify(loader, times(2)).load(eq(TP), any()); + + // Verify that the state is active and that the state machine + // is actually a new one. + assertEquals(ACTIVE, ctx.state); + assertNotEquals(coordinator, ctx.coordinator); + } + private static , U> ArgumentMatcher> coordinatorMatcher( CoordinatorRuntime runtime, TopicPartition tp diff --git a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/acl/AuthorizerBenchmark.java b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/acl/AuthorizerBenchmark.java index 6b7d4285876cf..3002c579ba978 100644 --- a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/acl/AuthorizerBenchmark.java +++ b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/acl/AuthorizerBenchmark.java @@ -19,6 +19,7 @@ import kafka.security.authorizer.AclAuthorizer; import kafka.security.authorizer.AclAuthorizer.VersionedAcls; + import org.apache.kafka.common.Uuid; import org.apache.kafka.common.acl.AccessControlEntry; import org.apache.kafka.common.acl.AclBinding; @@ -35,11 +36,12 @@ import org.apache.kafka.common.resource.ResourceType; import org.apache.kafka.common.security.auth.KafkaPrincipal; import org.apache.kafka.common.security.auth.SecurityProtocol; -import org.apache.kafka.security.authorizer.AclEntry; import org.apache.kafka.metadata.authorizer.StandardAcl; import org.apache.kafka.metadata.authorizer.StandardAuthorizer; +import org.apache.kafka.security.authorizer.AclEntry; import org.apache.kafka.server.authorizer.Action; import org.apache.kafka.server.authorizer.Authorizer; + import org.openjdk.jmh.annotations.Benchmark; import org.openjdk.jmh.annotations.BenchmarkMode; import org.openjdk.jmh.annotations.Fork; @@ -53,7 +55,6 @@ import org.openjdk.jmh.annotations.State; import org.openjdk.jmh.annotations.TearDown; import org.openjdk.jmh.annotations.Warmup; -import scala.collection.JavaConverters; import java.io.IOException; import java.net.InetAddress; @@ -69,6 +70,8 @@ import java.util.concurrent.TimeUnit; import java.util.function.Supplier; +import scala.collection.JavaConverters; + @State(Scope.Benchmark) @Fork(value = 1) @Warmup(iterations = 5) diff --git a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/acl/StandardAuthorizerUpdateBenchmark.java b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/acl/StandardAuthorizerUpdateBenchmark.java index 46c41908cf793..5a38fa77fe042 100644 --- a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/acl/StandardAuthorizerUpdateBenchmark.java +++ b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/acl/StandardAuthorizerUpdateBenchmark.java @@ -25,6 +25,7 @@ import org.apache.kafka.metadata.authorizer.StandardAcl; import org.apache.kafka.metadata.authorizer.StandardAclWithId; import org.apache.kafka.metadata.authorizer.StandardAuthorizer; + import org.openjdk.jmh.annotations.Benchmark; import org.openjdk.jmh.annotations.BenchmarkMode; import org.openjdk.jmh.annotations.Fork; diff --git a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/assignor/ClientSideAssignorBenchmark.java b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/assignor/ClientSideAssignorBenchmark.java index 9cee98371a7b8..4be40e2148c3b 100644 --- a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/assignor/ClientSideAssignorBenchmark.java +++ b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/assignor/ClientSideAssignorBenchmark.java @@ -24,6 +24,7 @@ import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.coordinator.group.api.assignor.SubscriptionType; + import org.openjdk.jmh.annotations.Benchmark; import org.openjdk.jmh.annotations.BenchmarkMode; import org.openjdk.jmh.annotations.Fork; diff --git a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/assignor/ServerSideAssignorBenchmark.java b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/assignor/ServerSideAssignorBenchmark.java index 56d5142a3290f..ab16487b3785d 100644 --- a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/assignor/ServerSideAssignorBenchmark.java +++ b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/assignor/ServerSideAssignorBenchmark.java @@ -17,21 +17,20 @@ package org.apache.kafka.jmh.assignor; import org.apache.kafka.common.Uuid; -import org.apache.kafka.coordinator.group.consumer.MemberAssignmentImpl; -import org.apache.kafka.coordinator.group.consumer.GroupSpecImpl; import org.apache.kafka.coordinator.group.api.assignor.GroupAssignment; import org.apache.kafka.coordinator.group.api.assignor.MemberAssignment; -import org.apache.kafka.coordinator.group.consumer.MemberSubscriptionAndAssignmentImpl; import org.apache.kafka.coordinator.group.api.assignor.PartitionAssignor; -import org.apache.kafka.coordinator.group.assignor.RangeAssignor; import org.apache.kafka.coordinator.group.api.assignor.SubscribedTopicDescriber; import org.apache.kafka.coordinator.group.api.assignor.SubscriptionType; -import org.apache.kafka.coordinator.group.consumer.Assignment; -import org.apache.kafka.coordinator.group.consumer.TopicIds; +import org.apache.kafka.coordinator.group.assignor.RangeAssignor; import org.apache.kafka.coordinator.group.assignor.UniformAssignor; +import org.apache.kafka.coordinator.group.consumer.Assignment; +import org.apache.kafka.coordinator.group.consumer.GroupSpecImpl; +import org.apache.kafka.coordinator.group.consumer.MemberAssignmentImpl; +import org.apache.kafka.coordinator.group.consumer.MemberSubscriptionAndAssignmentImpl; import org.apache.kafka.coordinator.group.consumer.SubscribedTopicDescriberImpl; +import org.apache.kafka.coordinator.group.consumer.TopicIds; import org.apache.kafka.coordinator.group.consumer.TopicMetadata; - import org.apache.kafka.image.MetadataDelta; import org.apache.kafka.image.MetadataImage; import org.apache.kafka.image.MetadataProvenance; diff --git a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/assignor/TargetAssignmentBuilderBenchmark.java b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/assignor/TargetAssignmentBuilderBenchmark.java index 2731437771c31..0e361f1c095db 100644 --- a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/assignor/TargetAssignmentBuilderBenchmark.java +++ b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/assignor/TargetAssignmentBuilderBenchmark.java @@ -17,19 +17,18 @@ package org.apache.kafka.jmh.assignor; import org.apache.kafka.common.Uuid; -import org.apache.kafka.coordinator.group.consumer.GroupSpecImpl; import org.apache.kafka.coordinator.group.api.assignor.GroupAssignment; import org.apache.kafka.coordinator.group.api.assignor.MemberAssignment; -import org.apache.kafka.coordinator.group.consumer.MemberSubscriptionAndAssignmentImpl; import org.apache.kafka.coordinator.group.api.assignor.PartitionAssignor; -import org.apache.kafka.coordinator.group.consumer.TopicIds; import org.apache.kafka.coordinator.group.assignor.UniformAssignor; import org.apache.kafka.coordinator.group.consumer.Assignment; import org.apache.kafka.coordinator.group.consumer.ConsumerGroupMember; +import org.apache.kafka.coordinator.group.consumer.GroupSpecImpl; +import org.apache.kafka.coordinator.group.consumer.MemberSubscriptionAndAssignmentImpl; import org.apache.kafka.coordinator.group.consumer.SubscribedTopicDescriberImpl; import org.apache.kafka.coordinator.group.consumer.TargetAssignmentBuilder; +import org.apache.kafka.coordinator.group.consumer.TopicIds; import org.apache.kafka.coordinator.group.consumer.TopicMetadata; - import org.apache.kafka.image.MetadataDelta; import org.apache.kafka.image.MetadataImage; import org.apache.kafka.image.MetadataProvenance; diff --git a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/cache/LRUCacheBenchmark.java b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/cache/LRUCacheBenchmark.java index 5b2d004dcd562..209b498f92bba 100644 --- a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/cache/LRUCacheBenchmark.java +++ b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/cache/LRUCacheBenchmark.java @@ -17,8 +17,8 @@ package org.apache.kafka.jmh.cache; -import java.util.concurrent.TimeUnit; import org.apache.kafka.common.cache.LRUCache; + import org.openjdk.jmh.annotations.Benchmark; import org.openjdk.jmh.annotations.Level; import org.openjdk.jmh.annotations.OutputTimeUnit; @@ -30,6 +30,8 @@ import org.openjdk.jmh.runner.options.Options; import org.openjdk.jmh.runner.options.OptionsBuilder; +import java.util.concurrent.TimeUnit; + /** * This is a simple example of a JMH benchmark. * diff --git a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/common/FetchRequestBenchmark.java b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/common/FetchRequestBenchmark.java index f7321b5bc6582..eecb619c0e86d 100644 --- a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/common/FetchRequestBenchmark.java +++ b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/common/FetchRequestBenchmark.java @@ -18,6 +18,7 @@ package org.apache.kafka.jmh.common; import kafka.network.RequestConvertToJson; + import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.Uuid; import org.apache.kafka.common.network.Send; @@ -26,6 +27,7 @@ import org.apache.kafka.common.requests.ByteBufferChannel; import org.apache.kafka.common.requests.FetchRequest; import org.apache.kafka.common.requests.RequestHeader; + import org.openjdk.jmh.annotations.Benchmark; import org.openjdk.jmh.annotations.BenchmarkMode; import org.openjdk.jmh.annotations.Fork; diff --git a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/common/FetchResponseBenchmark.java b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/common/FetchResponseBenchmark.java index 0320310606c07..ed68202abe9c8 100644 --- a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/common/FetchResponseBenchmark.java +++ b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/common/FetchResponseBenchmark.java @@ -17,8 +17,8 @@ package org.apache.kafka.jmh.common; -import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.TopicIdPartition; +import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.Uuid; import org.apache.kafka.common.compress.Compression; import org.apache.kafka.common.message.FetchResponseData; @@ -30,6 +30,7 @@ import org.apache.kafka.common.requests.ByteBufferChannel; import org.apache.kafka.common.requests.FetchResponse; import org.apache.kafka.common.requests.ResponseHeader; + import org.openjdk.jmh.annotations.Benchmark; import org.openjdk.jmh.annotations.BenchmarkMode; import org.openjdk.jmh.annotations.Fork; diff --git a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/common/ImplicitLinkedHashCollectionBenchmark.java b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/common/ImplicitLinkedHashCollectionBenchmark.java index c79c7f8dc2668..8e814daa967f0 100644 --- a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/common/ImplicitLinkedHashCollectionBenchmark.java +++ b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/common/ImplicitLinkedHashCollectionBenchmark.java @@ -19,6 +19,7 @@ import org.apache.kafka.common.Uuid; import org.apache.kafka.common.utils.ImplicitLinkedHashCollection; + import org.openjdk.jmh.annotations.Benchmark; import org.openjdk.jmh.annotations.BenchmarkMode; import org.openjdk.jmh.annotations.Fork; diff --git a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/common/ListOffsetRequestBenchmark.java b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/common/ListOffsetRequestBenchmark.java index e6fc2dc141c17..fe791350a5dca 100644 --- a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/common/ListOffsetRequestBenchmark.java +++ b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/common/ListOffsetRequestBenchmark.java @@ -18,11 +18,13 @@ package org.apache.kafka.jmh.common; import kafka.network.RequestConvertToJson; + import org.apache.kafka.common.IsolationLevel; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.message.ListOffsetsRequestData; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.requests.ListOffsetsRequest; + import org.openjdk.jmh.annotations.Benchmark; import org.openjdk.jmh.annotations.BenchmarkMode; import org.openjdk.jmh.annotations.Fork; diff --git a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/common/ProduceRequestBenchmark.java b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/common/ProduceRequestBenchmark.java index 405458fef0274..2fcafc5ada159 100644 --- a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/common/ProduceRequestBenchmark.java +++ b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/common/ProduceRequestBenchmark.java @@ -18,9 +18,11 @@ package org.apache.kafka.jmh.common; import kafka.network.RequestConvertToJson; + import org.apache.kafka.common.message.ProduceRequestData; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.requests.ProduceRequest; + import org.openjdk.jmh.annotations.Benchmark; import org.openjdk.jmh.annotations.BenchmarkMode; import org.openjdk.jmh.annotations.Fork; diff --git a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/common/TopicBenchmark.java b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/common/TopicBenchmark.java index fde239ea1d168..0c86eaa580dfe 100644 --- a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/common/TopicBenchmark.java +++ b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/common/TopicBenchmark.java @@ -17,6 +17,7 @@ package org.apache.kafka.jmh.common; import org.apache.kafka.common.internals.Topic; + import org.openjdk.jmh.annotations.Benchmark; import org.openjdk.jmh.annotations.BenchmarkMode; import org.openjdk.jmh.annotations.Fork; diff --git a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/connect/JsonConverterBenchmark.java b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/connect/JsonConverterBenchmark.java index e24c23c673f6e..aa45a7e711b8b 100644 --- a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/connect/JsonConverterBenchmark.java +++ b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/connect/JsonConverterBenchmark.java @@ -19,6 +19,7 @@ import org.apache.kafka.connect.data.SchemaBuilder; import org.apache.kafka.connect.data.Struct; import org.apache.kafka.connect.json.JsonConverter; + import org.openjdk.jmh.annotations.Benchmark; import org.openjdk.jmh.annotations.BenchmarkMode; import org.openjdk.jmh.annotations.Fork; diff --git a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/connect/ReplaceFieldBenchmark.java b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/connect/ReplaceFieldBenchmark.java index 808441615066e..2ed64b2a98646 100644 --- a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/connect/ReplaceFieldBenchmark.java +++ b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/connect/ReplaceFieldBenchmark.java @@ -19,6 +19,7 @@ import org.apache.kafka.connect.source.SourceRecord; import org.apache.kafka.connect.transforms.ReplaceField; + import org.openjdk.jmh.annotations.Benchmark; import org.openjdk.jmh.annotations.BenchmarkMode; import org.openjdk.jmh.annotations.Fork; diff --git a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/connect/ValuesBenchmark.java b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/connect/ValuesBenchmark.java index ddcdb5e2418ff..92e37d6fba269 100644 --- a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/connect/ValuesBenchmark.java +++ b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/connect/ValuesBenchmark.java @@ -24,6 +24,7 @@ import org.apache.kafka.connect.data.Time; import org.apache.kafka.connect.data.Timestamp; import org.apache.kafka.connect.data.Values; + import org.openjdk.jmh.annotations.Benchmark; import org.openjdk.jmh.annotations.BenchmarkMode; import org.openjdk.jmh.annotations.Fork; diff --git a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/consumer/SubscriptionStateBenchmark.java b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/consumer/SubscriptionStateBenchmark.java index d88a32ba4f495..9652ce464c6d3 100644 --- a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/consumer/SubscriptionStateBenchmark.java +++ b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/consumer/SubscriptionStateBenchmark.java @@ -23,6 +23,7 @@ import org.apache.kafka.common.Node; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.utils.LogContext; + import org.openjdk.jmh.annotations.Benchmark; import org.openjdk.jmh.annotations.BenchmarkMode; import org.openjdk.jmh.annotations.Fork; 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 c1c36447c2940..5b30ed08fdd7c 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 @@ -17,14 +17,13 @@ package org.apache.kafka.jmh.fetcher; +import kafka.cluster.AlterPartitionListener; import kafka.cluster.BrokerEndPoint; import kafka.cluster.DelayedOperations; -import kafka.cluster.AlterPartitionListener; import kafka.cluster.Partition; -import org.apache.kafka.server.util.MockTime; -import org.apache.kafka.storage.internals.log.LogAppendInfo; import kafka.log.LogManager; import kafka.server.AlterPartitionManager; +import kafka.server.BrokerBlockingSender; import kafka.server.BrokerFeatures; import kafka.server.BrokerTopicStats; import kafka.server.FailedPartitions; @@ -34,7 +33,6 @@ import kafka.server.OffsetTruncationState; import kafka.server.QuotaFactory; import kafka.server.RemoteLeaderEndPoint; -import kafka.server.BrokerBlockingSender; import kafka.server.ReplicaFetcherThread; import kafka.server.ReplicaManager; import kafka.server.ReplicaQuota; @@ -46,9 +44,10 @@ import kafka.utils.Pool; import kafka.utils.TestUtils; import kafka.zk.KafkaZkClient; + import org.apache.kafka.clients.FetchSessionHandler; -import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.TopicIdPartition; +import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.Uuid; import org.apache.kafka.common.message.FetchResponseData; import org.apache.kafka.common.message.LeaderAndIsrRequestData; @@ -66,12 +65,15 @@ import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.Time; import org.apache.kafka.common.utils.Utils; -import org.apache.kafka.server.common.OffsetAndEpoch; import org.apache.kafka.server.common.MetadataVersion; +import org.apache.kafka.server.common.OffsetAndEpoch; +import org.apache.kafka.server.util.KafkaScheduler; +import org.apache.kafka.server.util.MockTime; import org.apache.kafka.storage.internals.log.CleanerConfig; +import org.apache.kafka.storage.internals.log.LogAppendInfo; import org.apache.kafka.storage.internals.log.LogConfig; import org.apache.kafka.storage.internals.log.LogDirFailureChannel; -import org.apache.kafka.server.util.KafkaScheduler; + import org.mockito.Mockito; import org.openjdk.jmh.annotations.Benchmark; import org.openjdk.jmh.annotations.BenchmarkMode; @@ -98,6 +100,7 @@ import java.util.Properties; import java.util.UUID; import java.util.concurrent.TimeUnit; + import scala.Option; import scala.collection.Iterator; import scala.collection.Map; diff --git a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/fetchsession/FetchSessionBenchmark.java b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/fetchsession/FetchSessionBenchmark.java index 0dddf2d796e44..11d584446f44d 100644 --- a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/fetchsession/FetchSessionBenchmark.java +++ b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/fetchsession/FetchSessionBenchmark.java @@ -18,8 +18,8 @@ package org.apache.kafka.jmh.fetchsession; import org.apache.kafka.clients.FetchSessionHandler; -import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.TopicIdPartition; +import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.Uuid; import org.apache.kafka.common.message.FetchResponseData; import org.apache.kafka.common.protocol.ApiKeys; @@ -27,6 +27,7 @@ import org.apache.kafka.common.requests.FetchRequest; import org.apache.kafka.common.requests.FetchResponse; import org.apache.kafka.common.utils.LogContext; + import org.openjdk.jmh.annotations.Benchmark; import org.openjdk.jmh.annotations.BenchmarkMode; import org.openjdk.jmh.annotations.Fork; diff --git a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/metadata/KRaftMetadataRequestBenchmark.java b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/metadata/KRaftMetadataRequestBenchmark.java index 19b2f25d5a4b8..6b31224cf4c9b 100644 --- a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/metadata/KRaftMetadataRequestBenchmark.java +++ b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/metadata/KRaftMetadataRequestBenchmark.java @@ -38,6 +38,7 @@ import kafka.server.builders.KafkaApisBuilder; import kafka.server.metadata.KRaftMetadataCache; import kafka.server.metadata.MockConfigRepository; + import org.apache.kafka.common.Uuid; import org.apache.kafka.common.memory.MemoryPool; import org.apache.kafka.common.message.ApiMessageType; @@ -77,7 +78,6 @@ import org.openjdk.jmh.annotations.State; import org.openjdk.jmh.annotations.TearDown; import org.openjdk.jmh.annotations.Warmup; -import scala.Option; import java.nio.ByteBuffer; import java.util.Arrays; @@ -88,6 +88,8 @@ import java.util.concurrent.TimeUnit; import java.util.stream.IntStream; +import scala.Option; + @State(Scope.Benchmark) @Fork(value = 1) @Warmup(iterations = 5) diff --git a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/metadata/MetadataRequestBenchmark.java b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/metadata/MetadataRequestBenchmark.java index 5d3591454bd50..a3202a761394a 100644 --- a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/metadata/MetadataRequestBenchmark.java +++ b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/metadata/MetadataRequestBenchmark.java @@ -22,7 +22,6 @@ import kafka.network.RequestChannel; import kafka.network.RequestConvertToJson; import kafka.server.AutoTopicCreationManager; -import kafka.server.ZkBrokerEpochManager; import kafka.server.BrokerFeatures; import kafka.server.BrokerTopicStats; import kafka.server.ClientQuotaManager; @@ -37,11 +36,13 @@ import kafka.server.ReplicationQuotaManager; import kafka.server.SimpleApiVersionManager; import kafka.server.ZkAdminManager; +import kafka.server.ZkBrokerEpochManager; import kafka.server.ZkSupport; import kafka.server.builders.KafkaApisBuilder; import kafka.server.metadata.MockConfigRepository; import kafka.server.metadata.ZkMetadataCache; import kafka.zk.KafkaZkClient; + import org.apache.kafka.common.memory.MemoryPool; import org.apache.kafka.common.message.ApiMessageType; import org.apache.kafka.common.message.UpdateMetadataRequestData.UpdateMetadataBroker; @@ -63,6 +64,7 @@ import org.apache.kafka.server.common.MetadataVersion; import org.apache.kafka.server.config.ServerConfigs; import org.apache.kafka.server.config.ZkConfigs; + import org.mockito.Mockito; import org.openjdk.jmh.annotations.Benchmark; import org.openjdk.jmh.annotations.BenchmarkMode; @@ -77,7 +79,6 @@ import org.openjdk.jmh.annotations.State; import org.openjdk.jmh.annotations.TearDown; import org.openjdk.jmh.annotations.Warmup; -import scala.Option; import java.nio.ByteBuffer; import java.util.Arrays; @@ -89,6 +90,8 @@ import java.util.concurrent.TimeUnit; import java.util.stream.IntStream; +import scala.Option; + @State(Scope.Benchmark) @Fork(value = 1) @Warmup(iterations = 5) diff --git a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/metadata/TopicsImageSingleRecordChangeBenchmark.java b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/metadata/TopicsImageSingleRecordChangeBenchmark.java index 8f88a7a1e6f52..dd20f3093284b 100644 --- a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/metadata/TopicsImageSingleRecordChangeBenchmark.java +++ b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/metadata/TopicsImageSingleRecordChangeBenchmark.java @@ -22,6 +22,7 @@ import org.apache.kafka.common.metadata.TopicRecord; import org.apache.kafka.image.TopicsDelta; import org.apache.kafka.image.TopicsImage; + import org.openjdk.jmh.annotations.Benchmark; import org.openjdk.jmh.annotations.BenchmarkMode; import org.openjdk.jmh.annotations.Fork; diff --git a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/metadata/TopicsImageSnapshotLoadBenchmark.java b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/metadata/TopicsImageSnapshotLoadBenchmark.java index 10961d9d0252b..1dc63002c829a 100644 --- a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/metadata/TopicsImageSnapshotLoadBenchmark.java +++ b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/metadata/TopicsImageSnapshotLoadBenchmark.java @@ -22,6 +22,7 @@ import org.apache.kafka.common.metadata.TopicRecord; import org.apache.kafka.image.TopicsDelta; import org.apache.kafka.image.TopicsImage; + import org.openjdk.jmh.annotations.Benchmark; import org.openjdk.jmh.annotations.BenchmarkMode; import org.openjdk.jmh.annotations.Fork; diff --git a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/metadata/TopicsImageZonalOutageBenchmark.java b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/metadata/TopicsImageZonalOutageBenchmark.java index 5890763397ac6..bccadbeebe51a 100644 --- a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/metadata/TopicsImageZonalOutageBenchmark.java +++ b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/metadata/TopicsImageZonalOutageBenchmark.java @@ -21,6 +21,7 @@ import org.apache.kafka.common.metadata.PartitionRecord; import org.apache.kafka.image.TopicsDelta; import org.apache.kafka.image.TopicsImage; + import org.openjdk.jmh.annotations.Benchmark; import org.openjdk.jmh.annotations.BenchmarkMode; import org.openjdk.jmh.annotations.Fork; diff --git a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/partition/PartitionMakeFollowerBenchmark.java b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/partition/PartitionMakeFollowerBenchmark.java index 62840ef9b86a0..7000b99e24abb 100644 --- a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/partition/PartitionMakeFollowerBenchmark.java +++ b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/partition/PartitionMakeFollowerBenchmark.java @@ -17,8 +17,8 @@ package org.apache.kafka.jmh.partition; -import kafka.cluster.DelayedOperations; import kafka.cluster.AlterPartitionListener; +import kafka.cluster.DelayedOperations; import kafka.cluster.Partition; import kafka.log.LogManager; import kafka.server.AlterPartitionManager; @@ -27,6 +27,7 @@ import kafka.server.builders.LogManagerBuilder; import kafka.server.checkpoints.OffsetCheckpoints; import kafka.server.metadata.MockConfigRepository; + import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.Uuid; import org.apache.kafka.common.compress.Compression; @@ -36,10 +37,11 @@ import org.apache.kafka.common.utils.Time; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.server.common.MetadataVersion; +import org.apache.kafka.server.util.KafkaScheduler; import org.apache.kafka.storage.internals.log.CleanerConfig; import org.apache.kafka.storage.internals.log.LogConfig; import org.apache.kafka.storage.internals.log.LogDirFailureChannel; -import org.apache.kafka.server.util.KafkaScheduler; + import org.mockito.Mockito; import org.openjdk.jmh.annotations.Benchmark; import org.openjdk.jmh.annotations.BenchmarkMode; @@ -66,6 +68,7 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; + import scala.Option; import scala.compat.java8.OptionConverters; diff --git a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/partition/UpdateFollowerFetchStateBenchmark.java b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/partition/UpdateFollowerFetchStateBenchmark.java index 686ee7acf67da..99b8347f3cfe3 100644 --- a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/partition/UpdateFollowerFetchStateBenchmark.java +++ b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/partition/UpdateFollowerFetchStateBenchmark.java @@ -17,8 +17,8 @@ package org.apache.kafka.jmh.partition; -import kafka.cluster.DelayedOperations; import kafka.cluster.AlterPartitionListener; +import kafka.cluster.DelayedOperations; import kafka.cluster.Partition; import kafka.cluster.Replica; import kafka.log.LogManager; @@ -28,16 +28,18 @@ import kafka.server.builders.LogManagerBuilder; import kafka.server.checkpoints.OffsetCheckpoints; import kafka.server.metadata.MockConfigRepository; + import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.Uuid; import org.apache.kafka.common.message.LeaderAndIsrRequestData.LeaderAndIsrPartitionState; import org.apache.kafka.common.utils.Time; import org.apache.kafka.server.common.MetadataVersion; +import org.apache.kafka.server.util.KafkaScheduler; import org.apache.kafka.storage.internals.log.CleanerConfig; import org.apache.kafka.storage.internals.log.LogConfig; import org.apache.kafka.storage.internals.log.LogDirFailureChannel; import org.apache.kafka.storage.internals.log.LogOffsetMetadata; -import org.apache.kafka.server.util.KafkaScheduler; + import org.mockito.Mockito; import org.openjdk.jmh.annotations.Benchmark; import org.openjdk.jmh.annotations.BenchmarkMode; @@ -60,6 +62,7 @@ import java.util.Properties; import java.util.UUID; import java.util.concurrent.TimeUnit; + import scala.Option; import scala.compat.java8.OptionConverters; diff --git a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/producer/ProducerRecordBenchmark.java b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/producer/ProducerRecordBenchmark.java index 83d5c2b19c9f1..6346f3d0c8853 100644 --- a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/producer/ProducerRecordBenchmark.java +++ b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/producer/ProducerRecordBenchmark.java @@ -18,6 +18,7 @@ package org.apache.kafka.jmh.producer; import org.apache.kafka.clients.producer.ProducerRecord; + import org.openjdk.jmh.annotations.Benchmark; import org.openjdk.jmh.annotations.BenchmarkMode; import org.openjdk.jmh.annotations.Fork; diff --git a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/producer/ProducerRequestBenchmark.java b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/producer/ProducerRequestBenchmark.java index 4ea35a563f321..3bde86f827e4f 100644 --- a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/producer/ProducerRequestBenchmark.java +++ b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/producer/ProducerRequestBenchmark.java @@ -25,6 +25,7 @@ import org.apache.kafka.common.record.SimpleRecord; import org.apache.kafka.common.requests.ProduceRequest; import org.apache.kafka.common.requests.ProduceResponse; + import org.openjdk.jmh.annotations.Benchmark; import org.openjdk.jmh.annotations.BenchmarkMode; import org.openjdk.jmh.annotations.Fork; diff --git a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/producer/ProducerResponseBenchmark.java b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/producer/ProducerResponseBenchmark.java index 431880a96dc18..7bcf245b95821 100644 --- a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/producer/ProducerResponseBenchmark.java +++ b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/producer/ProducerResponseBenchmark.java @@ -21,6 +21,7 @@ import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.requests.AbstractResponse; import org.apache.kafka.common.requests.ProduceResponse; + import org.openjdk.jmh.annotations.Benchmark; import org.openjdk.jmh.annotations.BenchmarkMode; import org.openjdk.jmh.annotations.Fork; diff --git a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/record/BaseRecordBatchBenchmark.java b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/record/BaseRecordBatchBenchmark.java index 1190854a339e0..44d7272068492 100644 --- a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/record/BaseRecordBatchBenchmark.java +++ b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/record/BaseRecordBatchBenchmark.java @@ -19,6 +19,7 @@ import kafka.log.UnifiedLog; import kafka.server.BrokerTopicStats; import kafka.server.RequestLocal; + import org.apache.kafka.common.compress.Compression; import org.apache.kafka.common.header.Header; import org.apache.kafka.common.record.AbstractRecords; @@ -28,6 +29,7 @@ import org.apache.kafka.common.record.RecordBatch; import org.apache.kafka.common.record.TimestampType; import org.apache.kafka.storage.internals.log.LogValidator; + import org.openjdk.jmh.annotations.Param; import org.openjdk.jmh.annotations.Scope; import org.openjdk.jmh.annotations.Setup; diff --git a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/record/CompressedRecordBatchValidationBenchmark.java b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/record/CompressedRecordBatchValidationBenchmark.java index b43d7a9271eea..a75a8a4ca2ce4 100644 --- a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/record/CompressedRecordBatchValidationBenchmark.java +++ b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/record/CompressedRecordBatchValidationBenchmark.java @@ -26,6 +26,7 @@ import org.apache.kafka.server.common.MetadataVersion; import org.apache.kafka.storage.internals.log.AppendOrigin; import org.apache.kafka.storage.internals.log.LogValidator; + import org.openjdk.jmh.annotations.Benchmark; import org.openjdk.jmh.annotations.Fork; import org.openjdk.jmh.annotations.Measurement; diff --git a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/record/RecordBatchIterationBenchmark.java b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/record/RecordBatchIterationBenchmark.java index d8a73d53d3051..1d7d155f6bb54 100644 --- a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/record/RecordBatchIterationBenchmark.java +++ b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/record/RecordBatchIterationBenchmark.java @@ -23,6 +23,7 @@ import org.apache.kafka.common.record.Record; import org.apache.kafka.common.record.RecordBatch; import org.apache.kafka.common.utils.CloseableIterator; + import org.openjdk.jmh.annotations.Benchmark; import org.openjdk.jmh.annotations.Fork; import org.openjdk.jmh.annotations.Measurement; diff --git a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/record/UncompressedRecordBatchValidationBenchmark.java b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/record/UncompressedRecordBatchValidationBenchmark.java index 66f74c7046737..fb05990b4dd37 100644 --- a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/record/UncompressedRecordBatchValidationBenchmark.java +++ b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/record/UncompressedRecordBatchValidationBenchmark.java @@ -26,6 +26,7 @@ import org.apache.kafka.server.common.MetadataVersion; import org.apache.kafka.storage.internals.log.AppendOrigin; import org.apache.kafka.storage.internals.log.LogValidator; + import org.openjdk.jmh.annotations.Benchmark; import org.openjdk.jmh.annotations.Fork; import org.openjdk.jmh.annotations.Measurement; diff --git a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/server/CheckpointBench.java b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/server/CheckpointBench.java index 059e41dadec9d..dbc9731ff07bd 100644 --- a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/server/CheckpointBench.java +++ b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/server/CheckpointBench.java @@ -22,11 +22,6 @@ import kafka.server.BrokerFeatures; import kafka.server.BrokerTopicStats; import kafka.server.KafkaConfig; -import org.apache.kafka.server.config.ServerLogConfigs; -import org.apache.kafka.server.util.MockTime; -import org.apache.kafka.storage.internals.log.CleanerConfig; -import org.apache.kafka.storage.internals.log.LogConfig; -import org.apache.kafka.storage.internals.log.LogDirFailureChannel; import kafka.server.MetadataCache; import kafka.server.QuotaFactory; import kafka.server.ReplicaManager; @@ -34,13 +29,20 @@ import kafka.server.checkpoints.OffsetCheckpoints; import kafka.server.metadata.MockConfigRepository; import kafka.utils.TestUtils; -import org.apache.kafka.common.Uuid; + import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.Uuid; import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.server.common.MetadataVersion; +import org.apache.kafka.server.config.ServerLogConfigs; import org.apache.kafka.server.util.KafkaScheduler; +import org.apache.kafka.server.util.MockTime; import org.apache.kafka.server.util.Scheduler; +import org.apache.kafka.storage.internals.log.CleanerConfig; +import org.apache.kafka.storage.internals.log.LogConfig; +import org.apache.kafka.storage.internals.log.LogDirFailureChannel; + import org.openjdk.jmh.annotations.Benchmark; import org.openjdk.jmh.annotations.Fork; import org.openjdk.jmh.annotations.Level; @@ -61,8 +63,8 @@ import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; -import scala.collection.JavaConverters; import scala.Option; +import scala.collection.JavaConverters; @Warmup(iterations = 5) @Measurement(iterations = 5) diff --git a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/server/PartitionCreationBench.java b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/server/PartitionCreationBench.java index 24261ba1071b6..f1af675c62857 100644 --- a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/server/PartitionCreationBench.java +++ b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/server/PartitionCreationBench.java @@ -32,6 +32,7 @@ import kafka.server.metadata.MockConfigRepository; import kafka.utils.TestUtils; import kafka.zk.KafkaZkClient; + import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.Uuid; import org.apache.kafka.common.message.LeaderAndIsrRequestData; @@ -39,11 +40,12 @@ import org.apache.kafka.common.utils.Time; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.server.common.MetadataVersion; +import org.apache.kafka.server.util.KafkaScheduler; +import org.apache.kafka.server.util.Scheduler; import org.apache.kafka.storage.internals.log.CleanerConfig; import org.apache.kafka.storage.internals.log.LogConfig; import org.apache.kafka.storage.internals.log.LogDirFailureChannel; -import org.apache.kafka.server.util.KafkaScheduler; -import org.apache.kafka.server.util.Scheduler; + import org.openjdk.jmh.annotations.Benchmark; import org.openjdk.jmh.annotations.BenchmarkMode; import org.openjdk.jmh.annotations.Fork; @@ -66,6 +68,7 @@ import java.util.Properties; import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; + import scala.Option; import scala.collection.JavaConverters; diff --git a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/storage/ProducerStateManagerBench.java b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/storage/ProducerStateManagerBench.java index 49744e0b29453..c534aede37816 100644 --- a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/storage/ProducerStateManagerBench.java +++ b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/storage/ProducerStateManagerBench.java @@ -22,6 +22,7 @@ import org.apache.kafka.storage.internals.log.ProducerStateEntry; import org.apache.kafka.storage.internals.log.ProducerStateManager; import org.apache.kafka.storage.internals.log.ProducerStateManagerConfig; + import org.openjdk.jmh.annotations.Benchmark; import org.openjdk.jmh.annotations.BenchmarkMode; import org.openjdk.jmh.annotations.Fork; diff --git a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/timeline/TimelineHashMapBenchmark.java b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/timeline/TimelineHashMapBenchmark.java index ae6c56ee2e8dd..1139c193e2439 100644 --- a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/timeline/TimelineHashMapBenchmark.java +++ b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/timeline/TimelineHashMapBenchmark.java @@ -20,6 +20,7 @@ import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.timeline.SnapshotRegistry; import org.apache.kafka.timeline.TimelineHashMap; + import org.openjdk.jmh.annotations.Benchmark; import org.openjdk.jmh.annotations.BenchmarkMode; import org.openjdk.jmh.annotations.Fork; diff --git a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/util/ByteUtilsBenchmark.java b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/util/ByteUtilsBenchmark.java index 88c867a953f04..e0184efcf42c7 100644 --- a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/util/ByteUtilsBenchmark.java +++ b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/util/ByteUtilsBenchmark.java @@ -17,12 +17,8 @@ package org.apache.kafka.jmh.util; -import java.io.IOException; -import java.nio.ByteBuffer; -import java.security.SecureRandom; -import java.util.concurrent.TimeUnit; - import org.apache.kafka.common.utils.ByteUtils; + import org.openjdk.jmh.annotations.Benchmark; import org.openjdk.jmh.annotations.CompilerControl; import org.openjdk.jmh.annotations.Fork; @@ -40,6 +36,11 @@ import org.openjdk.jmh.runner.options.Options; import org.openjdk.jmh.runner.options.OptionsBuilder; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.security.SecureRandom; +import java.util.concurrent.TimeUnit; + @OutputTimeUnit(TimeUnit.SECONDS) @Fork(3) @Warmup(iterations = 3, time = 1) diff --git a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/util/Crc32CBenchmark.java b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/util/Crc32CBenchmark.java index 6f60a0f43e050..d7927676ec124 100644 --- a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/util/Crc32CBenchmark.java +++ b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/util/Crc32CBenchmark.java @@ -18,9 +18,6 @@ import org.apache.kafka.common.utils.Crc32C; -import java.nio.ByteBuffer; -import java.util.SplittableRandom; -import java.util.concurrent.TimeUnit; import org.openjdk.jmh.annotations.Benchmark; import org.openjdk.jmh.annotations.Fork; import org.openjdk.jmh.annotations.Measurement; @@ -31,6 +28,10 @@ import org.openjdk.jmh.annotations.State; import org.openjdk.jmh.annotations.Warmup; +import java.nio.ByteBuffer; +import java.util.SplittableRandom; +import java.util.concurrent.TimeUnit; + @State(Scope.Benchmark) @OutputTimeUnit(TimeUnit.MICROSECONDS) @Fork(2) diff --git a/log4j-appender/src/main/java/org/apache/kafka/log4jappender/KafkaLog4jAppender.java b/log4j-appender/src/main/java/org/apache/kafka/log4jappender/KafkaLog4jAppender.java index 64e7f9846b579..f77c18e21588c 100644 --- a/log4j-appender/src/main/java/org/apache/kafka/log4jappender/KafkaLog4jAppender.java +++ b/log4j-appender/src/main/java/org/apache/kafka/log4jappender/KafkaLog4jAppender.java @@ -22,6 +22,7 @@ import org.apache.kafka.clients.producer.RecordMetadata; import org.apache.kafka.common.config.ConfigException; import org.apache.kafka.common.serialization.ByteArraySerializer; + import org.apache.log4j.AppenderSkeleton; import org.apache.log4j.helpers.LogLog; import org.apache.log4j.spi.LoggingEvent; @@ -38,12 +39,12 @@ import static org.apache.kafka.clients.producer.ProducerConfig.BATCH_SIZE_CONFIG; import static org.apache.kafka.clients.producer.ProducerConfig.COMPRESSION_TYPE_CONFIG; import static org.apache.kafka.clients.producer.ProducerConfig.DELIVERY_TIMEOUT_MS_CONFIG; +import static org.apache.kafka.clients.producer.ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG; import static org.apache.kafka.clients.producer.ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG; import static org.apache.kafka.clients.producer.ProducerConfig.LINGER_MS_CONFIG; import static org.apache.kafka.clients.producer.ProducerConfig.MAX_BLOCK_MS_CONFIG; import static org.apache.kafka.clients.producer.ProducerConfig.RETRIES_CONFIG; import static org.apache.kafka.clients.producer.ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG; -import static org.apache.kafka.clients.producer.ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG; import static org.apache.kafka.common.config.SaslConfigs.SASL_JAAS_CONFIG; import static org.apache.kafka.common.config.SaslConfigs.SASL_KERBEROS_SERVICE_NAME; import static org.apache.kafka.common.config.SaslConfigs.SASL_MECHANISM; diff --git a/log4j-appender/src/test/java/org/apache/kafka/log4jappender/KafkaLog4jAppenderTest.java b/log4j-appender/src/test/java/org/apache/kafka/log4jappender/KafkaLog4jAppenderTest.java index eb64a85bd8869..5895f3705108d 100644 --- a/log4j-appender/src/test/java/org/apache/kafka/log4jappender/KafkaLog4jAppenderTest.java +++ b/log4j-appender/src/test/java/org/apache/kafka/log4jappender/KafkaLog4jAppenderTest.java @@ -16,14 +16,12 @@ */ package org.apache.kafka.log4jappender; -import static org.hamcrest.CoreMatchers.hasItem; -import static org.hamcrest.CoreMatchers.not; -import static org.hamcrest.MatcherAssert.assertThat; import org.apache.kafka.clients.producer.MockProducer; import org.apache.kafka.clients.producer.RecordMetadata; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.config.ConfigException; import org.apache.kafka.common.config.SaslConfigs; + import org.apache.log4j.Appender; import org.apache.log4j.Logger; import org.apache.log4j.PropertyConfigurator; @@ -37,6 +35,9 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeoutException; +import static org.hamcrest.CoreMatchers.hasItem; +import static org.hamcrest.CoreMatchers.not; +import static org.hamcrest.MatcherAssert.assertThat; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertThrows; import static org.mockito.ArgumentMatchers.any; @@ -227,4 +228,3 @@ private Properties getLog4jConfig(boolean syncSend) { return props; } } - diff --git a/log4j-appender/src/test/java/org/apache/kafka/log4jappender/MockKafkaLog4jAppender.java b/log4j-appender/src/test/java/org/apache/kafka/log4jappender/MockKafkaLog4jAppender.java index b8ac3cb989a3c..981c658b9a8ae 100644 --- a/log4j-appender/src/test/java/org/apache/kafka/log4jappender/MockKafkaLog4jAppender.java +++ b/log4j-appender/src/test/java/org/apache/kafka/log4jappender/MockKafkaLog4jAppender.java @@ -20,6 +20,7 @@ import org.apache.kafka.clients.producer.Producer; import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.test.MockSerializer; + import org.apache.log4j.spi.LoggingEvent; import java.util.List; diff --git a/server-common/src/main/java/org/apache/kafka/server/util/timer/TimerTask.java b/server-common/src/main/java/org/apache/kafka/server/util/timer/TimerTask.java index ec6e8b3d783ac..ac58b68ec5ea9 100644 --- a/server-common/src/main/java/org/apache/kafka/server/util/timer/TimerTask.java +++ b/server-common/src/main/java/org/apache/kafka/server/util/timer/TimerTask.java @@ -32,6 +32,10 @@ public void cancel() { } } + public boolean isCancelled() { + return timerTaskEntry == null; + } + final void setTimerTaskEntry(TimerTaskEntry entry) { synchronized (this) { // if this timerTask is already held by an existing timer task entry, diff --git a/shell/src/main/java/org/apache/kafka/shell/InteractiveShell.java b/shell/src/main/java/org/apache/kafka/shell/InteractiveShell.java index bf786e09a58c7..c382d178da5da 100644 --- a/shell/src/main/java/org/apache/kafka/shell/InteractiveShell.java +++ b/shell/src/main/java/org/apache/kafka/shell/InteractiveShell.java @@ -20,6 +20,7 @@ import org.apache.kafka.shell.command.CommandUtils; import org.apache.kafka.shell.command.Commands; import org.apache.kafka.shell.state.MetadataShellState; + import org.jline.reader.Candidate; import org.jline.reader.Completer; import org.jline.reader.EndOfFileException; diff --git a/shell/src/main/java/org/apache/kafka/shell/MetadataShell.java b/shell/src/main/java/org/apache/kafka/shell/MetadataShell.java index 9fe12aa7fad9a..06ae528ed8867 100644 --- a/shell/src/main/java/org/apache/kafka/shell/MetadataShell.java +++ b/shell/src/main/java/org/apache/kafka/shell/MetadataShell.java @@ -20,9 +20,7 @@ import kafka.raft.KafkaRaftManager; import kafka.tools.TerseFailure; import kafka.utils.FileLock; -import net.sourceforge.argparse4j.ArgumentParsers; -import net.sourceforge.argparse4j.inf.ArgumentParser; -import net.sourceforge.argparse4j.inf.Namespace; + import org.apache.kafka.common.utils.Exit; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.image.loader.MetadataLoader; @@ -33,6 +31,11 @@ import org.apache.kafka.shell.command.Commands; import org.apache.kafka.shell.state.MetadataShellPublisher; import org.apache.kafka.shell.state.MetadataShellState; + +import net.sourceforge.argparse4j.ArgumentParsers; +import net.sourceforge.argparse4j.inf.ArgumentParser; +import net.sourceforge.argparse4j.inf.Namespace; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/shell/src/main/java/org/apache/kafka/shell/command/CatCommandHandler.java b/shell/src/main/java/org/apache/kafka/shell/command/CatCommandHandler.java index 51b2f6d3cb78d..c8e5fef7cf78e 100644 --- a/shell/src/main/java/org/apache/kafka/shell/command/CatCommandHandler.java +++ b/shell/src/main/java/org/apache/kafka/shell/command/CatCommandHandler.java @@ -17,13 +17,15 @@ package org.apache.kafka.shell.command; -import net.sourceforge.argparse4j.inf.ArgumentParser; -import net.sourceforge.argparse4j.inf.Namespace; import org.apache.kafka.image.node.MetadataNode; import org.apache.kafka.shell.InteractiveShell; import org.apache.kafka.shell.glob.GlobVisitor; import org.apache.kafka.shell.node.printer.ShellNodePrinter; import org.apache.kafka.shell.state.MetadataShellState; + +import net.sourceforge.argparse4j.inf.ArgumentParser; +import net.sourceforge.argparse4j.inf.Namespace; + import org.jline.reader.Candidate; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/shell/src/main/java/org/apache/kafka/shell/command/CdCommandHandler.java b/shell/src/main/java/org/apache/kafka/shell/command/CdCommandHandler.java index ba22c0bb8ecf6..a07e242ab0360 100644 --- a/shell/src/main/java/org/apache/kafka/shell/command/CdCommandHandler.java +++ b/shell/src/main/java/org/apache/kafka/shell/command/CdCommandHandler.java @@ -17,11 +17,13 @@ package org.apache.kafka.shell.command; -import net.sourceforge.argparse4j.inf.ArgumentParser; -import net.sourceforge.argparse4j.inf.Namespace; import org.apache.kafka.shell.InteractiveShell; import org.apache.kafka.shell.glob.GlobVisitor; import org.apache.kafka.shell.state.MetadataShellState; + +import net.sourceforge.argparse4j.inf.ArgumentParser; +import net.sourceforge.argparse4j.inf.Namespace; + import org.jline.reader.Candidate; import java.io.PrintWriter; diff --git a/shell/src/main/java/org/apache/kafka/shell/command/CommandUtils.java b/shell/src/main/java/org/apache/kafka/shell/command/CommandUtils.java index 156e65acdbd76..ce04003d82625 100644 --- a/shell/src/main/java/org/apache/kafka/shell/command/CommandUtils.java +++ b/shell/src/main/java/org/apache/kafka/shell/command/CommandUtils.java @@ -19,6 +19,7 @@ import org.apache.kafka.image.node.MetadataNode; import org.apache.kafka.shell.state.MetadataShellState; + import org.jline.reader.Candidate; import java.util.ArrayList; diff --git a/shell/src/main/java/org/apache/kafka/shell/command/Commands.java b/shell/src/main/java/org/apache/kafka/shell/command/Commands.java index ddafb5e692013..77b142c635369 100644 --- a/shell/src/main/java/org/apache/kafka/shell/command/Commands.java +++ b/shell/src/main/java/org/apache/kafka/shell/command/Commands.java @@ -17,6 +17,9 @@ package org.apache.kafka.shell.command; +import org.apache.kafka.shell.InteractiveShell; +import org.apache.kafka.shell.state.MetadataShellState; + import net.sourceforge.argparse4j.ArgumentParsers; import net.sourceforge.argparse4j.inf.ArgumentParser; import net.sourceforge.argparse4j.inf.ArgumentParserException; @@ -24,8 +27,7 @@ import net.sourceforge.argparse4j.inf.Subparser; import net.sourceforge.argparse4j.inf.Subparsers; import net.sourceforge.argparse4j.internal.HelpScreenException; -import org.apache.kafka.shell.InteractiveShell; -import org.apache.kafka.shell.state.MetadataShellState; + import org.jline.reader.Candidate; import java.io.PrintWriter; diff --git a/shell/src/main/java/org/apache/kafka/shell/command/ExitCommandHandler.java b/shell/src/main/java/org/apache/kafka/shell/command/ExitCommandHandler.java index fab54be2c98be..b369d445b9afe 100644 --- a/shell/src/main/java/org/apache/kafka/shell/command/ExitCommandHandler.java +++ b/shell/src/main/java/org/apache/kafka/shell/command/ExitCommandHandler.java @@ -17,11 +17,13 @@ package org.apache.kafka.shell.command; -import net.sourceforge.argparse4j.inf.ArgumentParser; -import net.sourceforge.argparse4j.inf.Namespace; import org.apache.kafka.common.utils.Exit; import org.apache.kafka.shell.InteractiveShell; import org.apache.kafka.shell.state.MetadataShellState; + +import net.sourceforge.argparse4j.inf.ArgumentParser; +import net.sourceforge.argparse4j.inf.Namespace; + import org.jline.reader.Candidate; import java.io.PrintWriter; diff --git a/shell/src/main/java/org/apache/kafka/shell/command/FindCommandHandler.java b/shell/src/main/java/org/apache/kafka/shell/command/FindCommandHandler.java index 133cb988d01f9..ccc52a7b43758 100644 --- a/shell/src/main/java/org/apache/kafka/shell/command/FindCommandHandler.java +++ b/shell/src/main/java/org/apache/kafka/shell/command/FindCommandHandler.java @@ -17,12 +17,14 @@ package org.apache.kafka.shell.command; -import net.sourceforge.argparse4j.inf.ArgumentParser; -import net.sourceforge.argparse4j.inf.Namespace; import org.apache.kafka.image.node.MetadataNode; import org.apache.kafka.shell.InteractiveShell; import org.apache.kafka.shell.glob.GlobVisitor; import org.apache.kafka.shell.state.MetadataShellState; + +import net.sourceforge.argparse4j.inf.ArgumentParser; +import net.sourceforge.argparse4j.inf.Namespace; + import org.jline.reader.Candidate; import java.io.PrintWriter; diff --git a/shell/src/main/java/org/apache/kafka/shell/command/HelpCommandHandler.java b/shell/src/main/java/org/apache/kafka/shell/command/HelpCommandHandler.java index e0a5aa03288b0..5d2a14ed013c0 100644 --- a/shell/src/main/java/org/apache/kafka/shell/command/HelpCommandHandler.java +++ b/shell/src/main/java/org/apache/kafka/shell/command/HelpCommandHandler.java @@ -17,10 +17,12 @@ package org.apache.kafka.shell.command; -import net.sourceforge.argparse4j.inf.ArgumentParser; -import net.sourceforge.argparse4j.inf.Namespace; import org.apache.kafka.shell.InteractiveShell; import org.apache.kafka.shell.state.MetadataShellState; + +import net.sourceforge.argparse4j.inf.ArgumentParser; +import net.sourceforge.argparse4j.inf.Namespace; + import org.jline.reader.Candidate; import java.io.PrintWriter; diff --git a/shell/src/main/java/org/apache/kafka/shell/command/HistoryCommandHandler.java b/shell/src/main/java/org/apache/kafka/shell/command/HistoryCommandHandler.java index c3b299983ce78..f233fe695ef22 100644 --- a/shell/src/main/java/org/apache/kafka/shell/command/HistoryCommandHandler.java +++ b/shell/src/main/java/org/apache/kafka/shell/command/HistoryCommandHandler.java @@ -17,10 +17,12 @@ package org.apache.kafka.shell.command; -import net.sourceforge.argparse4j.inf.ArgumentParser; -import net.sourceforge.argparse4j.inf.Namespace; import org.apache.kafka.shell.InteractiveShell; import org.apache.kafka.shell.state.MetadataShellState; + +import net.sourceforge.argparse4j.inf.ArgumentParser; +import net.sourceforge.argparse4j.inf.Namespace; + import org.jline.reader.Candidate; import java.io.PrintWriter; diff --git a/shell/src/main/java/org/apache/kafka/shell/command/LsCommandHandler.java b/shell/src/main/java/org/apache/kafka/shell/command/LsCommandHandler.java index 24840502e514d..9e81bcf8bf0a9 100644 --- a/shell/src/main/java/org/apache/kafka/shell/command/LsCommandHandler.java +++ b/shell/src/main/java/org/apache/kafka/shell/command/LsCommandHandler.java @@ -17,13 +17,15 @@ package org.apache.kafka.shell.command; -import net.sourceforge.argparse4j.inf.ArgumentParser; -import net.sourceforge.argparse4j.inf.Namespace; import org.apache.kafka.image.node.MetadataNode; import org.apache.kafka.shell.InteractiveShell; import org.apache.kafka.shell.glob.GlobVisitor; import org.apache.kafka.shell.glob.GlobVisitor.MetadataNodeInfo; import org.apache.kafka.shell.state.MetadataShellState; + +import net.sourceforge.argparse4j.inf.ArgumentParser; +import net.sourceforge.argparse4j.inf.Namespace; + import org.jline.reader.Candidate; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/shell/src/main/java/org/apache/kafka/shell/command/ManCommandHandler.java b/shell/src/main/java/org/apache/kafka/shell/command/ManCommandHandler.java index 5892cdff4c5ad..df1c20292452e 100644 --- a/shell/src/main/java/org/apache/kafka/shell/command/ManCommandHandler.java +++ b/shell/src/main/java/org/apache/kafka/shell/command/ManCommandHandler.java @@ -17,11 +17,13 @@ package org.apache.kafka.shell.command; +import org.apache.kafka.shell.InteractiveShell; +import org.apache.kafka.shell.state.MetadataShellState; + import net.sourceforge.argparse4j.ArgumentParsers; import net.sourceforge.argparse4j.inf.ArgumentParser; import net.sourceforge.argparse4j.inf.Namespace; -import org.apache.kafka.shell.InteractiveShell; -import org.apache.kafka.shell.state.MetadataShellState; + import org.jline.reader.Candidate; import java.io.PrintWriter; diff --git a/shell/src/main/java/org/apache/kafka/shell/command/PwdCommandHandler.java b/shell/src/main/java/org/apache/kafka/shell/command/PwdCommandHandler.java index 55046cf5e7243..4b09ad1fd344d 100644 --- a/shell/src/main/java/org/apache/kafka/shell/command/PwdCommandHandler.java +++ b/shell/src/main/java/org/apache/kafka/shell/command/PwdCommandHandler.java @@ -17,10 +17,12 @@ package org.apache.kafka.shell.command; -import net.sourceforge.argparse4j.inf.ArgumentParser; -import net.sourceforge.argparse4j.inf.Namespace; import org.apache.kafka.shell.InteractiveShell; import org.apache.kafka.shell.state.MetadataShellState; + +import net.sourceforge.argparse4j.inf.ArgumentParser; +import net.sourceforge.argparse4j.inf.Namespace; + import org.jline.reader.Candidate; import java.io.PrintWriter; diff --git a/shell/src/main/java/org/apache/kafka/shell/command/TreeCommandHandler.java b/shell/src/main/java/org/apache/kafka/shell/command/TreeCommandHandler.java index ee937d1c2f5bb..43d2ba2a1baa3 100644 --- a/shell/src/main/java/org/apache/kafka/shell/command/TreeCommandHandler.java +++ b/shell/src/main/java/org/apache/kafka/shell/command/TreeCommandHandler.java @@ -17,13 +17,15 @@ package org.apache.kafka.shell.command; -import net.sourceforge.argparse4j.inf.ArgumentParser; -import net.sourceforge.argparse4j.inf.Namespace; import org.apache.kafka.image.node.MetadataNode; import org.apache.kafka.shell.InteractiveShell; import org.apache.kafka.shell.glob.GlobVisitor; import org.apache.kafka.shell.node.printer.ShellNodePrinter; import org.apache.kafka.shell.state.MetadataShellState; + +import net.sourceforge.argparse4j.inf.ArgumentParser; +import net.sourceforge.argparse4j.inf.Namespace; + import org.jline.reader.Candidate; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/shell/src/main/java/org/apache/kafka/shell/state/MetadataShellPublisher.java b/shell/src/main/java/org/apache/kafka/shell/state/MetadataShellPublisher.java index ec4998edca720..1878d1068c85b 100644 --- a/shell/src/main/java/org/apache/kafka/shell/state/MetadataShellPublisher.java +++ b/shell/src/main/java/org/apache/kafka/shell/state/MetadataShellPublisher.java @@ -22,6 +22,7 @@ import org.apache.kafka.image.loader.LoaderManifest; import org.apache.kafka.image.publisher.MetadataPublisher; import org.apache.kafka.shell.node.RootShellNode; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/shell/src/test/java/org/apache/kafka/shell/MetadataShellIntegrationTest.java b/shell/src/test/java/org/apache/kafka/shell/MetadataShellIntegrationTest.java index 3ef5044a82dae..b9a2b47d09e8f 100644 --- a/shell/src/test/java/org/apache/kafka/shell/MetadataShellIntegrationTest.java +++ b/shell/src/test/java/org/apache/kafka/shell/MetadataShellIntegrationTest.java @@ -17,16 +17,11 @@ package org.apache.kafka.shell; -import java.io.File; -import java.io.IOException; -import java.nio.file.Files; -import java.nio.file.NoSuchFileException; -import java.util.Collections; -import java.util.concurrent.ExecutionException; - import kafka.utils.FileLock; + import org.apache.kafka.common.utils.Utils; import org.apache.kafka.server.fault.MockFaultHandler; + import org.junit.jupiter.api.Tag; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; @@ -35,6 +30,13 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.io.File; +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.NoSuchFileException; +import java.util.Collections; +import java.util.concurrent.ExecutionException; + import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertThrows; diff --git a/shell/src/test/java/org/apache/kafka/shell/command/CommandTest.java b/shell/src/test/java/org/apache/kafka/shell/command/CommandTest.java index 58528335c4be8..3be8036ca0d45 100644 --- a/shell/src/test/java/org/apache/kafka/shell/command/CommandTest.java +++ b/shell/src/test/java/org/apache/kafka/shell/command/CommandTest.java @@ -17,8 +17,6 @@ package org.apache.kafka.shell.command; -import static org.junit.jupiter.api.Assertions.assertEquals; - import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; @@ -26,6 +24,8 @@ import java.util.Collections; import java.util.Optional; +import static org.junit.jupiter.api.Assertions.assertEquals; + @Timeout(value = 120) public class CommandTest { @Test diff --git a/shell/src/test/java/org/apache/kafka/shell/command/CommandUtilsTest.java b/shell/src/test/java/org/apache/kafka/shell/command/CommandUtilsTest.java index 7ee36a9e91ab8..52277719c90de 100644 --- a/shell/src/test/java/org/apache/kafka/shell/command/CommandUtilsTest.java +++ b/shell/src/test/java/org/apache/kafka/shell/command/CommandUtilsTest.java @@ -17,14 +17,14 @@ package org.apache.kafka.shell.command; -import static java.util.concurrent.TimeUnit.MINUTES; -import static org.junit.jupiter.api.Assertions.assertEquals; - import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; import java.util.Arrays; +import static java.util.concurrent.TimeUnit.MINUTES; +import static org.junit.jupiter.api.Assertions.assertEquals; + @Timeout(value = 5, unit = MINUTES) public class CommandUtilsTest { @Test diff --git a/shell/src/test/java/org/apache/kafka/shell/command/LsCommandHandlerTest.java b/shell/src/test/java/org/apache/kafka/shell/command/LsCommandHandlerTest.java index 0a8864bca43cf..b7bdf2f37f059 100644 --- a/shell/src/test/java/org/apache/kafka/shell/command/LsCommandHandlerTest.java +++ b/shell/src/test/java/org/apache/kafka/shell/command/LsCommandHandlerTest.java @@ -17,8 +17,6 @@ package org.apache.kafka.shell.command; -import static org.junit.jupiter.api.Assertions.assertEquals; - import org.apache.kafka.shell.command.LsCommandHandler.ColumnSchema; import org.apache.kafka.shell.command.LsCommandHandler.TargetDirectory; @@ -33,6 +31,8 @@ import java.util.Collections; import java.util.OptionalInt; +import static org.junit.jupiter.api.Assertions.assertEquals; + @Timeout(value = 120) public class LsCommandHandlerTest { @Test @@ -95,4 +95,3 @@ public void testPrintTargets() throws Exception { } } } - diff --git a/shell/src/test/java/org/apache/kafka/shell/glob/GlobComponentTest.java b/shell/src/test/java/org/apache/kafka/shell/glob/GlobComponentTest.java index 9e935e12a9819..e86b471e30154 100644 --- a/shell/src/test/java/org/apache/kafka/shell/glob/GlobComponentTest.java +++ b/shell/src/test/java/org/apache/kafka/shell/glob/GlobComponentTest.java @@ -17,14 +17,14 @@ package org.apache.kafka.shell.glob; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; + import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertNull; import static org.junit.jupiter.api.Assertions.assertTrue; -import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.Timeout; - @Timeout(value = 120) public class GlobComponentTest { private void verifyIsLiteral(GlobComponent globComponent, String component) { diff --git a/shell/src/test/java/org/apache/kafka/shell/glob/GlobVisitorTest.java b/shell/src/test/java/org/apache/kafka/shell/glob/GlobVisitorTest.java index fb54cdf85f8fb..fd9ffd8934a94 100644 --- a/shell/src/test/java/org/apache/kafka/shell/glob/GlobVisitorTest.java +++ b/shell/src/test/java/org/apache/kafka/shell/glob/GlobVisitorTest.java @@ -17,12 +17,10 @@ package org.apache.kafka.shell.glob; -import static java.util.concurrent.TimeUnit.MINUTES; -import static org.junit.jupiter.api.Assertions.assertEquals; - import org.apache.kafka.image.node.MetadataNode; import org.apache.kafka.shell.glob.GlobVisitor.MetadataNodeInfo; import org.apache.kafka.shell.state.MetadataShellState; + import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; @@ -36,6 +34,9 @@ import java.util.Optional; import java.util.function.Consumer; +import static java.util.concurrent.TimeUnit.MINUTES; +import static org.junit.jupiter.api.Assertions.assertEquals; + @Timeout(value = 5, unit = MINUTES) public class GlobVisitorTest { static private final MetadataShellState DATA; diff --git a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java index bfea3d436806c..e77e4ca795d92 100644 --- a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java +++ b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java @@ -1273,7 +1273,7 @@ public static class InternalConfig { public static final String STATE_UPDATER_ENABLED = "__state.updater.enabled__"; public static boolean getStateUpdaterEnabled(final Map configs) { - return InternalConfig.getBoolean(configs, InternalConfig.STATE_UPDATER_ENABLED, false); + return InternalConfig.getBoolean(configs, InternalConfig.STATE_UPDATER_ENABLED, true); } // Private API to enable processing threads (i.e. polling is decoupled from processing) diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/assignment/TaskAssignmentUtils.java b/streams/src/main/java/org/apache/kafka/streams/processor/assignment/TaskAssignmentUtils.java index f67e54b2e1c4d..39a698adfa516 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/assignment/TaskAssignmentUtils.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/assignment/TaskAssignmentUtils.java @@ -27,6 +27,7 @@ import java.util.Optional; import java.util.Set; import java.util.SortedSet; +import java.util.TreeSet; import java.util.function.BiConsumer; import java.util.function.BiFunction; import java.util.function.Function; @@ -55,7 +56,103 @@ public final class TaskAssignmentUtils { private TaskAssignmentUtils() {} /** - * Return an {@code AssignmentError} for a task assignment created for an application. + * A simple config container for necessary parameters and optional overrides to apply when + * running the active or standby task rack-aware optimizations. + */ + public static class RackAwareOptimizationParams { + private final ApplicationState applicationState; + private final Optional trafficCostOverride; + private final Optional nonOverlapCostOverride; + private final Optional> tasksToOptimize; + + private RackAwareOptimizationParams(final ApplicationState applicationState, + final Optional trafficCostOverride, + final Optional nonOverlapCostOverride, + final Optional> tasksToOptimize) { + this.applicationState = applicationState; + this.trafficCostOverride = trafficCostOverride; + this.nonOverlapCostOverride = nonOverlapCostOverride; + this.tasksToOptimize = tasksToOptimize; + } + + /** + * Return a new config object with no overrides and the tasksToOptimize initialized to the set of all tasks in the given ApplicationState + */ + public static RackAwareOptimizationParams of(final ApplicationState applicationState) { + return new RackAwareOptimizationParams(applicationState, Optional.empty(), Optional.empty(), Optional.empty()); + } + + /** + * Return a new config object with the tasksToOptimize set to all stateful tasks in the given ApplicationState + */ + public RackAwareOptimizationParams forStatefulTasks() { + final SortedSet tasks = applicationState.allTasks().values() + .stream() + .filter(TaskInfo::isStateful) + .map(TaskInfo::id) + .collect(Collectors.toCollection(TreeSet::new)); + return forTasks(tasks); + } + + /** + * Return a new config object with the tasksToOptimize set to all stateless tasks in the given ApplicationState + */ + public RackAwareOptimizationParams forStatelessTasks() { + final SortedSet tasks = applicationState.allTasks().values() + .stream() + .filter(taskInfo -> !taskInfo.isStateful()) + .map(TaskInfo::id) + .collect(Collectors.toCollection(TreeSet::new)); + return forTasks(tasks); + } + + /** + * Return a new config object with the provided tasksToOptimize + */ + public RackAwareOptimizationParams forTasks(final SortedSet tasksToOptimize) { + return new RackAwareOptimizationParams( + applicationState, + trafficCostOverride, + nonOverlapCostOverride, + Optional.of(tasksToOptimize) + ); + } + + /** + * Return a new config object with the provided trafficCost override applied + */ + public RackAwareOptimizationParams withTrafficCostOverride(final int trafficCostOverride) { + return new RackAwareOptimizationParams( + applicationState, + Optional.of(trafficCostOverride), + nonOverlapCostOverride, + tasksToOptimize + ); + } + + /** + * Return a new config object with the provided nonOverlapCost override applied + */ + public RackAwareOptimizationParams withNonOverlapCostOverride(final int nonOverlapCostOverride) { + return new RackAwareOptimizationParams( + applicationState, + trafficCostOverride, + Optional.of(nonOverlapCostOverride), + tasksToOptimize + ); + } + } + + /** + * Validate the passed-in {@link TaskAssignment} and return an {@link AssignmentError} representing the + * first error detected in the assignment, or {@link AssignmentError#NONE} if the assignment passes the + * verification check. + *

+ * Note: this verification is performed automatically by the StreamsPartitionAssignor on the assignment + * returned by the TaskAssignor, and the error returned to the assignor via the {@link TaskAssignor#onAssignmentComputed} + * callback. Therefore, it is not required to call this manually from the {@link TaskAssignor#assign} method. + * However, if an invalid assignment is returned it will fail the rebalance and kill the thread, so it may be useful to + * utilize this method in an assignor to verify the assignment before returning it and fix any errors it finds. * * @param applicationState The application for which this task assignment is being assessed. * @param taskAssignment The task assignment that will be validated. @@ -153,16 +250,14 @@ public static Map identityAssignment(final Ap * If rack-aware client tags are configured, the rack-aware standby task assignor will be used * * @param applicationState the metadata and other info describing the current application state - * @param kafkaStreamsAssignments the current assignment of tasks to KafkaStreams clients - * - * @return a new map containing the mappings from KafkaStreamsAssignments updated with the default standby assignment + * @param kafkaStreamsAssignments the KafkaStreams client assignments to add standby tasks to */ - public static Map defaultStandbyTaskAssignment(final ApplicationState applicationState, - final Map kafkaStreamsAssignments) { + public static void defaultStandbyTaskAssignment(final ApplicationState applicationState, + final Map kafkaStreamsAssignments) { if (!applicationState.assignmentConfigs().rackAwareAssignmentTags().isEmpty()) { - return tagBasedStandbyTaskAssignment(applicationState, kafkaStreamsAssignments); + tagBasedStandbyTaskAssignment(applicationState, kafkaStreamsAssignments); } else { - return loadBasedStandbyTaskAssignment(applicationState, kafkaStreamsAssignments); + loadBasedStandbyTaskAssignment(applicationState, kafkaStreamsAssignments); } } @@ -185,34 +280,43 @@ public static Map defaultStandbyTaskAssignmen *

* This method optimizes cross-rack traffic for active tasks only. For standby task optimization, * use {@link #optimizeRackAwareStandbyTasks}. + *

+ * It is recommended to run this optimization before assigning any standby tasks, especially if you have configured + * your KafkaStreams clients with assignment tags via the rack.aware.assignment.tags config since this method may + * shuffle around active tasks without considering the client tags and can result in a violation of the original + * client tag assignment's constraints. * - * @param applicationState the metadata and other info describing the current application state + * @param optimizationParams optional configuration parameters to apply * @param kafkaStreamsAssignments the current assignment of tasks to KafkaStreams clients - * @param tasks the set of tasks to reassign if possible. Must already be assigned to a KafkaStreams client - * - * @return a map with the KafkaStreamsAssignments updated to minimize cross-rack traffic for active tasks */ - public static Map optimizeRackAwareActiveTasks(final ApplicationState applicationState, - final Map kafkaStreamsAssignments, - final SortedSet tasks) { - if (tasks.isEmpty()) { - return kafkaStreamsAssignments; + public static void optimizeRackAwareActiveTasks(final RackAwareOptimizationParams optimizationParams, + final Map kafkaStreamsAssignments) { + final ApplicationState applicationState = optimizationParams.applicationState; + final SortedSet activeTasksToOptimize = getTasksToOptimize(kafkaStreamsAssignments, optimizationParams, AssignedTask.Type.ACTIVE); + if (activeTasksToOptimize.isEmpty()) { + return; } - if (!canPerformRackAwareOptimization(applicationState, AssignedTask.Type.ACTIVE)) { - return kafkaStreamsAssignments; + if (!canPerformRackAwareOptimization(applicationState, optimizationParams, AssignedTask.Type.ACTIVE)) { + return; } initializeAssignmentsForAllClients(applicationState, kafkaStreamsAssignments); - final int crossRackTrafficCost = applicationState.assignmentConfigs().rackAwareTrafficCost().getAsInt(); - final int nonOverlapCost = applicationState.assignmentConfigs().rackAwareNonOverlapCost().getAsInt(); + final int crossRackTrafficCost = + optimizationParams.trafficCostOverride.orElseGet(() -> applicationState.assignmentConfigs() + .rackAwareTrafficCost() + .getAsInt()); + final int nonOverlapCost = + optimizationParams.nonOverlapCostOverride.orElseGet(() -> applicationState.assignmentConfigs() + .rackAwareNonOverlapCost() + .getAsInt()); final Map kafkaStreamsStates = applicationState.kafkaStreamsStates(false); - final List taskIds = new ArrayList<>(tasks); + final List taskIds = new ArrayList<>(activeTasksToOptimize); final Map> topicPartitionsByTaskId = applicationState.allTasks().values().stream() - .filter(taskInfo -> tasks.contains(taskInfo.id())) + .filter(taskInfo -> activeTasksToOptimize.contains(taskInfo.id())) .collect(Collectors.toMap(TaskInfo::id, TaskInfo::topicPartitions)); final List clientIds = new ArrayList<>(kafkaStreamsStates.keySet()); @@ -259,8 +363,6 @@ public static Map optimizeRackAwareActiveTask (assignment, taskId) -> assignment.removeTask(new AssignedTask(taskId, AssignedTask.Type.ACTIVE)), (assignment, taskId) -> assignment.tasks().containsKey(taskId) && assignment.tasks().get(taskId).type() == AssignedTask.Type.ACTIVE ); - - return kafkaStreamsAssignments; } /** @@ -283,31 +385,34 @@ public static Map optimizeRackAwareActiveTask * This method optimizes cross-rack traffic for standby tasks only. For active task optimization, * use {@link #optimizeRackAwareActiveTasks}. * + * @param optimizationParams optional configuration parameters to apply * @param kafkaStreamsAssignments the current assignment of tasks to KafkaStreams clients - * @param applicationState the metadata and other info describing the current application state - * - * @return a map with the KafkaStreamsAssignments updated to minimize cross-rack traffic for standby tasks */ - public static Map optimizeRackAwareStandbyTasks(final ApplicationState applicationState, - final Map kafkaStreamsAssignments) { - if (!canPerformRackAwareOptimization(applicationState, AssignedTask.Type.STANDBY)) { - return kafkaStreamsAssignments; + public static void optimizeRackAwareStandbyTasks(final RackAwareOptimizationParams optimizationParams, + final Map kafkaStreamsAssignments) { + final ApplicationState applicationState = optimizationParams.applicationState; + final SortedSet standbyTasksToOptimize = getTasksToOptimize(kafkaStreamsAssignments, optimizationParams, AssignedTask.Type.STANDBY); + if (standbyTasksToOptimize.isEmpty()) { + return; + } + + if (!canPerformRackAwareOptimization(applicationState, optimizationParams, AssignedTask.Type.STANDBY)) { + return; } initializeAssignmentsForAllClients(applicationState, kafkaStreamsAssignments); - final int crossRackTrafficCost = applicationState.assignmentConfigs().rackAwareTrafficCost().getAsInt(); - final int nonOverlapCost = applicationState.assignmentConfigs().rackAwareNonOverlapCost().getAsInt(); + final int crossRackTrafficCost = + optimizationParams.trafficCostOverride.orElseGet(() -> applicationState.assignmentConfigs() + .rackAwareTrafficCost() + .getAsInt()); + final int nonOverlapCost = + optimizationParams.nonOverlapCostOverride.orElseGet(() -> applicationState.assignmentConfigs() + .rackAwareNonOverlapCost() + .getAsInt()); final Map kafkaStreamsStates = applicationState.kafkaStreamsStates(false); - final List standbyTasksToOptimize = kafkaStreamsAssignments.values().stream() - .flatMap(r -> r.tasks().values().stream()) - .filter(task -> task.type() == AssignedTask.Type.STANDBY) - .map(AssignedTask::id) - .distinct() - .collect(Collectors.toList()); - final Map> topicPartitionsByTaskId = applicationState.allTasks().values().stream().collect(Collectors.toMap( TaskInfo::id, @@ -317,7 +422,7 @@ public static Map optimizeRackAwareStandbyTas final List clientIds = new ArrayList<>(kafkaStreamsStates.keySet()); final long initialCost = computeTotalAssignmentCost( topicPartitionsByTaskId, - standbyTasksToOptimize, + new ArrayList<>(standbyTasksToOptimize), clientIds, kafkaStreamsAssignments, kafkaStreamsStates, @@ -411,7 +516,7 @@ public static Map optimizeRackAwareStandbyTas } final long finalCost = computeTotalAssignmentCost( topicPartitionsByTaskId, - standbyTasksToOptimize, + new ArrayList<>(standbyTasksToOptimize), clientIds, kafkaStreamsAssignments, kafkaStreamsStates, @@ -424,7 +529,6 @@ public static Map optimizeRackAwareStandbyTas final long duration = System.currentTimeMillis() - startTime; LOG.info("Assignment after {} rounds and {} milliseconds for standby task optimization is {}\n with cost {}", round, duration, kafkaStreamsAssignments, finalCost); - return kafkaStreamsAssignments; } private static long computeTotalAssignmentCost(final Map> topicPartitionsByTaskId, @@ -541,6 +645,7 @@ private static int getCrossRackTrafficCost(final Set topicPa * is set. */ private static boolean canPerformRackAwareOptimization(final ApplicationState applicationState, + final RackAwareOptimizationParams optimizationParams, final AssignedTask.Type taskType) { final AssignmentConfigs assignmentConfigs = applicationState.assignmentConfigs(); final String rackAwareAssignmentStrategy = assignmentConfigs.rackAwareAssignmentStrategy(); @@ -902,6 +1007,20 @@ private static void initializeAssignmentsForAllClients(final ApplicationState ap } } + private static SortedSet getTasksToOptimize(final Map assignments, + final RackAwareOptimizationParams optimizationParams, + final AssignedTask.Type taskType) { + if (optimizationParams != null && optimizationParams.tasksToOptimize.isPresent()) { + return optimizationParams.tasksToOptimize.get(); + } + + return assignments.values().stream() + .flatMap(r -> r.tasks().values().stream()) + .filter(task -> task.type() == taskType) + .map(AssignedTask::id) + .collect(Collectors.toCollection(TreeSet::new)); + } + private static class TagStatistics { private final Map> tagKeyToValues; private final Map, Set> tagEntryToClients; diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/assignment/assignors/StickyTaskAssignor.java b/streams/src/main/java/org/apache/kafka/streams/processor/assignment/assignors/StickyTaskAssignor.java index 827c9138c996a..3d5e5b247f776 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/assignment/assignors/StickyTaskAssignor.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/assignment/assignors/StickyTaskAssignor.java @@ -29,7 +29,6 @@ import java.util.Map; import java.util.Objects; import java.util.Set; -import java.util.TreeSet; import java.util.function.Function; import java.util.stream.Collectors; import org.apache.kafka.streams.processor.TaskId; @@ -39,9 +38,11 @@ import org.apache.kafka.streams.processor.assignment.KafkaStreamsState; import org.apache.kafka.streams.processor.assignment.ProcessId; import org.apache.kafka.streams.processor.assignment.TaskAssignmentUtils; +import org.apache.kafka.streams.processor.assignment.TaskAssignmentUtils.RackAwareOptimizationParams; import org.apache.kafka.streams.processor.assignment.TaskAssignor; import org.apache.kafka.streams.processor.assignment.TaskInfo; import org.apache.kafka.streams.processor.assignment.TaskTopicPartition; +import org.apache.kafka.streams.processor.internals.assignment.RackAwareTaskAssignor; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -94,21 +95,19 @@ private void optimizeActive(final ApplicationState applicationState, final Map currentAssignments = assignmentState.newAssignments; - final Set statefulTasks = applicationState.allTasks().values().stream() - .filter(TaskInfo::isStateful) - .map(TaskInfo::id) - .collect(Collectors.toSet()); - final Map optimizedAssignmentsForStatefulTasks = TaskAssignmentUtils.optimizeRackAwareActiveTasks( - applicationState, currentAssignments, new TreeSet<>(statefulTasks)); - - final Set statelessTasks = applicationState.allTasks().values().stream() - .filter(task -> !task.isStateful()) - .map(TaskInfo::id) - .collect(Collectors.toSet()); - final Map optimizedAssignmentsForAllTasks = TaskAssignmentUtils.optimizeRackAwareActiveTasks( - applicationState, optimizedAssignmentsForStatefulTasks, new TreeSet<>(statelessTasks)); - - assignmentState.processOptimizedAssignments(optimizedAssignmentsForAllTasks); + TaskAssignmentUtils.optimizeRackAwareActiveTasks( + RackAwareOptimizationParams.of(applicationState).forStatefulTasks(), + currentAssignments + ); + + TaskAssignmentUtils.optimizeRackAwareActiveTasks( + RackAwareOptimizationParams.of(applicationState) + .forStatelessTasks() + .withTrafficCostOverride(RackAwareTaskAssignor.STATELESS_TRAFFIC_COST) + .withNonOverlapCostOverride(RackAwareTaskAssignor.STATELESS_NON_OVERLAP_COST), + currentAssignments + ); + assignmentState.processOptimizedAssignments(currentAssignments); } private void optimizeStandby(final ApplicationState applicationState, final AssignmentState assignmentState) { @@ -120,10 +119,9 @@ private void optimizeStandby(final ApplicationState applicationState, final Assi return; } - final Map currentAssignments = assignmentState.newAssignments; - final Map optimizedAssignments = TaskAssignmentUtils.optimizeRackAwareStandbyTasks( - applicationState, currentAssignments); - assignmentState.processOptimizedAssignments(optimizedAssignments); + final Map assignments = assignmentState.newAssignments; + TaskAssignmentUtils.optimizeRackAwareStandbyTasks(RackAwareOptimizationParams.of(applicationState), assignments); + assignmentState.processOptimizedAssignments(assignments); } private static void assignActive(final ApplicationState applicationState, diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StoreChangelogReaderTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StoreChangelogReaderTest.java index 763394611b96d..457508cd20e8e 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StoreChangelogReaderTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StoreChangelogReaderTest.java @@ -468,10 +468,10 @@ private void shouldPollWithRightTimeout(final Properties properties) { assertEquals(Duration.ofMillis(config.getLong(StreamsConfig.POLL_MS_CONFIG)), consumer.lastPollTimeout()); } else { if (!properties.containsKey(InternalConfig.STATE_UPDATER_ENABLED) - || !((boolean) properties.get(InternalConfig.STATE_UPDATER_ENABLED))) { - assertEquals(Duration.ZERO, consumer.lastPollTimeout()); - } else { + || (boolean) properties.get(InternalConfig.STATE_UPDATER_ENABLED)) { assertEquals(Duration.ofMillis(config.getLong(StreamsConfig.POLL_MS_CONFIG)), consumer.lastPollTimeout()); + } else { + assertEquals(Duration.ZERO, consumer.lastPollTimeout()); } } } diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/TaskAssignmentUtilsTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/TaskAssignmentUtilsTest.java index 7fdbd468cbb97..d8807ee6d660d 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/TaskAssignmentUtilsTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/TaskAssignmentUtilsTest.java @@ -54,6 +54,7 @@ import org.apache.kafka.streams.processor.assignment.ProcessId; import org.apache.kafka.streams.processor.assignment.TaskAssignmentUtils; import org.apache.kafka.streams.processor.assignment.TaskAssignor; +import org.apache.kafka.streams.processor.assignment.TaskAssignmentUtils.RackAwareOptimizationParams; import org.apache.kafka.streams.processor.assignment.TaskInfo; import org.apache.kafka.streams.processor.assignment.TaskTopicPartition; import org.junit.Rule; @@ -92,14 +93,14 @@ public void shouldOptimizeActiveTaskSimple(final String strategy) { ); TaskAssignmentUtils.optimizeRackAwareActiveTasks( - applicationState, assignments, new TreeSet<>(tasks.keySet())); + RackAwareOptimizationParams.of(applicationState), assignments); assertThat(assignments.size(), equalTo(2)); assertThat(assignments.get(processId(1)).tasks().keySet(), equalTo(mkSet(TASK_0_1))); assertThat(assignments.get(processId(2)).tasks().keySet(), equalTo(mkSet(TASK_0_0))); // Repeated to make sure nothing gets shifted around after the first round of optimization. TaskAssignmentUtils.optimizeRackAwareActiveTasks( - applicationState, assignments, new TreeSet<>(tasks.keySet())); + RackAwareOptimizationParams.of(applicationState), assignments); assertThat(assignments.size(), equalTo(2)); assertThat(assignments.get(processId(1)).tasks().keySet(), equalTo(mkSet(TASK_0_1))); assertThat(assignments.get(processId(2)).tasks().keySet(), equalTo(mkSet(TASK_0_0))); @@ -131,7 +132,7 @@ public void shouldOptimizeStandbyTasksBasic(final String strategy) { mkAssignment(AssignedTask.Type.STANDBY, 3, TASK_0_0) ); - TaskAssignmentUtils.optimizeRackAwareStandbyTasks(applicationState, assignments); + TaskAssignmentUtils.optimizeRackAwareStandbyTasks(RackAwareOptimizationParams.of(applicationState), assignments); assertThat(assignments.size(), equalTo(3)); assertThat(assignments.get(processId(1)).tasks().keySet(), equalTo(mkSet(TASK_0_0, TASK_0_1))); assertThat(assignments.get(processId(2)).tasks().keySet(), equalTo(mkSet(TASK_0_0))); @@ -259,7 +260,7 @@ public void shouldNotViolateClientTagsAssignmentDuringStandbyOptimization(final mkAssignment(4) ); - TaskAssignmentUtils.optimizeRackAwareStandbyTasks(applicationState, assignments); + TaskAssignmentUtils.optimizeRackAwareStandbyTasks(RackAwareOptimizationParams.of(applicationState), assignments); assertThat(assignments.size(), equalTo(4)); assertThat(assignments.get(processId(1)).tasks().keySet(), equalTo(mkSet(TASK_0_0, TASK_0_1))); assertThat(assignments.get(processId(2)).tasks().keySet(), equalTo(mkSet(TASK_0_0, TASK_0_1))); @@ -294,8 +295,11 @@ public void shouldOptimizeStandbyTasksWithMultipleRacks(final String strategy) { mkAssignment(AssignedTask.Type.ACTIVE, 3, TASK_0_2) ); - TaskAssignmentUtils.optimizeRackAwareActiveTasks(applicationState, assignments, - new TreeSet<>(mkSet(TASK_0_0, TASK_0_1, TASK_0_2))); + TaskAssignmentUtils.optimizeRackAwareActiveTasks( + RackAwareOptimizationParams.of(applicationState) + .forTasks(new TreeSet<>(mkSet(TASK_0_0, TASK_0_1, TASK_0_2))), + assignments + ); assertThat(assignments.size(), equalTo(3)); assertThat(assignments.get(processId(1)).tasks().keySet(), equalTo(mkSet(TASK_0_0))); assertThat(assignments.get(processId(2)).tasks().keySet(), equalTo(mkSet(TASK_0_1))); diff --git a/trogdor/src/main/java/org/apache/kafka/trogdor/agent/Agent.java b/trogdor/src/main/java/org/apache/kafka/trogdor/agent/Agent.java index c3fe33249b1f8..25199f7a9b46e 100644 --- a/trogdor/src/main/java/org/apache/kafka/trogdor/agent/Agent.java +++ b/trogdor/src/main/java/org/apache/kafka/trogdor/agent/Agent.java @@ -17,12 +17,6 @@ package org.apache.kafka.trogdor.agent; -import com.fasterxml.jackson.databind.node.LongNode; -import com.fasterxml.jackson.databind.node.ObjectNode; -import net.sourceforge.argparse4j.ArgumentParsers; -import net.sourceforge.argparse4j.inf.ArgumentParser; -import net.sourceforge.argparse4j.inf.ArgumentParserException; -import net.sourceforge.argparse4j.inf.Namespace; import org.apache.kafka.common.KafkaFuture; import org.apache.kafka.common.utils.Exit; import org.apache.kafka.common.utils.Scheduler; @@ -35,9 +29,18 @@ import org.apache.kafka.trogdor.rest.DestroyWorkerRequest; import org.apache.kafka.trogdor.rest.JsonRestServer; import org.apache.kafka.trogdor.rest.StopWorkerRequest; +import org.apache.kafka.trogdor.rest.UptimeResponse; import org.apache.kafka.trogdor.task.TaskController; import org.apache.kafka.trogdor.task.TaskSpec; -import org.apache.kafka.trogdor.rest.UptimeResponse; + +import com.fasterxml.jackson.databind.node.LongNode; +import com.fasterxml.jackson.databind.node.ObjectNode; + +import net.sourceforge.argparse4j.ArgumentParsers; +import net.sourceforge.argparse4j.inf.ArgumentParser; +import net.sourceforge.argparse4j.inf.ArgumentParserException; +import net.sourceforge.argparse4j.inf.Namespace; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/trogdor/src/main/java/org/apache/kafka/trogdor/agent/AgentClient.java b/trogdor/src/main/java/org/apache/kafka/trogdor/agent/AgentClient.java index cdeba84a36880..91dd6ee1d9787 100644 --- a/trogdor/src/main/java/org/apache/kafka/trogdor/agent/AgentClient.java +++ b/trogdor/src/main/java/org/apache/kafka/trogdor/agent/AgentClient.java @@ -17,12 +17,6 @@ package org.apache.kafka.trogdor.agent; -import com.fasterxml.jackson.core.type.TypeReference; -import net.sourceforge.argparse4j.ArgumentParsers; -import net.sourceforge.argparse4j.inf.ArgumentParser; -import net.sourceforge.argparse4j.inf.Namespace; -import net.sourceforge.argparse4j.inf.Subparser; -import net.sourceforge.argparse4j.inf.Subparsers; import org.apache.kafka.common.utils.Exit; import org.apache.kafka.trogdor.common.JsonUtil; import org.apache.kafka.trogdor.common.StringFormatter; @@ -33,14 +27,21 @@ import org.apache.kafka.trogdor.rest.JsonRestServer; import org.apache.kafka.trogdor.rest.JsonRestServer.HttpResponse; import org.apache.kafka.trogdor.rest.StopWorkerRequest; +import org.apache.kafka.trogdor.rest.UptimeResponse; import org.apache.kafka.trogdor.rest.WorkerState; import org.apache.kafka.trogdor.task.TaskSpec; -import org.apache.kafka.trogdor.rest.UptimeResponse; + +import com.fasterxml.jackson.core.type.TypeReference; + +import net.sourceforge.argparse4j.ArgumentParsers; +import net.sourceforge.argparse4j.inf.ArgumentParser; +import net.sourceforge.argparse4j.inf.Namespace; +import net.sourceforge.argparse4j.inf.Subparser; +import net.sourceforge.argparse4j.inf.Subparsers; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import javax.ws.rs.core.UriBuilder; - import java.time.OffsetDateTime; import java.time.ZoneOffset; import java.util.ArrayList; @@ -48,6 +49,8 @@ import java.util.List; import java.util.Map; +import javax.ws.rs.core.UriBuilder; + import static net.sourceforge.argparse4j.impl.Arguments.store; import static net.sourceforge.argparse4j.impl.Arguments.storeTrue; import static org.apache.kafka.trogdor.common.StringFormatter.dateString; diff --git a/trogdor/src/main/java/org/apache/kafka/trogdor/agent/AgentRestResource.java b/trogdor/src/main/java/org/apache/kafka/trogdor/agent/AgentRestResource.java index ec3df8bf84696..09f5d5bb3ebb9 100644 --- a/trogdor/src/main/java/org/apache/kafka/trogdor/agent/AgentRestResource.java +++ b/trogdor/src/main/java/org/apache/kafka/trogdor/agent/AgentRestResource.java @@ -23,6 +23,8 @@ import org.apache.kafka.trogdor.rest.StopWorkerRequest; import org.apache.kafka.trogdor.rest.UptimeResponse; +import java.util.concurrent.atomic.AtomicReference; + import javax.servlet.ServletContext; import javax.ws.rs.Consumes; import javax.ws.rs.DELETE; @@ -34,7 +36,6 @@ import javax.ws.rs.Produces; import javax.ws.rs.QueryParam; import javax.ws.rs.core.MediaType; -import java.util.concurrent.atomic.AtomicReference; /** * The REST resource for the Agent. This describes the RPCs which the agent can accept. diff --git a/trogdor/src/main/java/org/apache/kafka/trogdor/agent/WorkerManager.java b/trogdor/src/main/java/org/apache/kafka/trogdor/agent/WorkerManager.java index 4510e1bb9221e..7a955e359af55 100644 --- a/trogdor/src/main/java/org/apache/kafka/trogdor/agent/WorkerManager.java +++ b/trogdor/src/main/java/org/apache/kafka/trogdor/agent/WorkerManager.java @@ -29,11 +29,12 @@ import org.apache.kafka.trogdor.rest.WorkerDone; import org.apache.kafka.trogdor.rest.WorkerRunning; import org.apache.kafka.trogdor.rest.WorkerStarting; -import org.apache.kafka.trogdor.rest.WorkerStopping; import org.apache.kafka.trogdor.rest.WorkerState; +import org.apache.kafka.trogdor.rest.WorkerStopping; import org.apache.kafka.trogdor.task.AgentWorkerStatusTracker; import org.apache.kafka.trogdor.task.TaskSpec; import org.apache.kafka.trogdor.task.TaskWorker; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/trogdor/src/main/java/org/apache/kafka/trogdor/basic/BasicNode.java b/trogdor/src/main/java/org/apache/kafka/trogdor/basic/BasicNode.java index 232a64db80b00..2c0b7faa946e7 100644 --- a/trogdor/src/main/java/org/apache/kafka/trogdor/basic/BasicNode.java +++ b/trogdor/src/main/java/org/apache/kafka/trogdor/basic/BasicNode.java @@ -17,9 +17,10 @@ package org.apache.kafka.trogdor.basic; -import com.fasterxml.jackson.databind.JsonNode; import org.apache.kafka.trogdor.common.Node; +import com.fasterxml.jackson.databind.JsonNode; + import java.util.Collections; import java.util.HashMap; import java.util.HashSet; diff --git a/trogdor/src/main/java/org/apache/kafka/trogdor/basic/BasicPlatform.java b/trogdor/src/main/java/org/apache/kafka/trogdor/basic/BasicPlatform.java index 3552e4b30896b..1968be27096a4 100644 --- a/trogdor/src/main/java/org/apache/kafka/trogdor/basic/BasicPlatform.java +++ b/trogdor/src/main/java/org/apache/kafka/trogdor/basic/BasicPlatform.java @@ -17,12 +17,14 @@ package org.apache.kafka.trogdor.basic; -import com.fasterxml.jackson.databind.JsonNode; import org.apache.kafka.common.utils.Scheduler; import org.apache.kafka.common.utils.Shell; import org.apache.kafka.trogdor.common.Node; import org.apache.kafka.trogdor.common.Platform; import org.apache.kafka.trogdor.common.Topology; + +import com.fasterxml.jackson.databind.JsonNode; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/trogdor/src/main/java/org/apache/kafka/trogdor/basic/BasicTopology.java b/trogdor/src/main/java/org/apache/kafka/trogdor/basic/BasicTopology.java index 1f291503e9190..1c3e2ef1caf24 100644 --- a/trogdor/src/main/java/org/apache/kafka/trogdor/basic/BasicTopology.java +++ b/trogdor/src/main/java/org/apache/kafka/trogdor/basic/BasicTopology.java @@ -17,10 +17,11 @@ package org.apache.kafka.trogdor.basic; -import com.fasterxml.jackson.databind.JsonNode; import org.apache.kafka.trogdor.common.Node; import org.apache.kafka.trogdor.common.Topology; +import com.fasterxml.jackson.databind.JsonNode; + import java.util.Iterator; import java.util.NavigableMap; import java.util.TreeMap; diff --git a/trogdor/src/main/java/org/apache/kafka/trogdor/common/Platform.java b/trogdor/src/main/java/org/apache/kafka/trogdor/common/Platform.java index cb2062086a6f8..defed06c32309 100644 --- a/trogdor/src/main/java/org/apache/kafka/trogdor/common/Platform.java +++ b/trogdor/src/main/java/org/apache/kafka/trogdor/common/Platform.java @@ -17,14 +17,14 @@ package org.apache.kafka.trogdor.common; +import org.apache.kafka.common.utils.Scheduler; +import org.apache.kafka.common.utils.Utils; + import com.fasterxml.jackson.databind.JsonNode; import java.io.File; import java.io.IOException; -import org.apache.kafka.common.utils.Scheduler; -import org.apache.kafka.common.utils.Utils; - /** * Defines a cluster topology */ diff --git a/trogdor/src/main/java/org/apache/kafka/trogdor/common/WorkerUtils.java b/trogdor/src/main/java/org/apache/kafka/trogdor/common/WorkerUtils.java index cd0e09ee63822..693ae2e6d3b81 100644 --- a/trogdor/src/main/java/org/apache/kafka/trogdor/common/WorkerUtils.java +++ b/trogdor/src/main/java/org/apache/kafka/trogdor/common/WorkerUtils.java @@ -36,6 +36,7 @@ import org.apache.kafka.common.internals.KafkaFutureImpl; import org.apache.kafka.common.requests.CreateTopicsRequest; import org.apache.kafka.common.utils.Time; + import org.slf4j.Logger; import java.util.ArrayList; diff --git a/trogdor/src/main/java/org/apache/kafka/trogdor/coordinator/Coordinator.java b/trogdor/src/main/java/org/apache/kafka/trogdor/coordinator/Coordinator.java index dcb838b4e0a0b..9bf8a767cd0c5 100644 --- a/trogdor/src/main/java/org/apache/kafka/trogdor/coordinator/Coordinator.java +++ b/trogdor/src/main/java/org/apache/kafka/trogdor/coordinator/Coordinator.java @@ -17,10 +17,6 @@ package org.apache.kafka.trogdor.coordinator; -import net.sourceforge.argparse4j.ArgumentParsers; -import net.sourceforge.argparse4j.inf.ArgumentParser; -import net.sourceforge.argparse4j.inf.ArgumentParserException; -import net.sourceforge.argparse4j.inf.Namespace; import org.apache.kafka.common.utils.Exit; import org.apache.kafka.common.utils.Scheduler; import org.apache.kafka.common.utils.Time; @@ -36,6 +32,12 @@ import org.apache.kafka.trogdor.rest.TasksRequest; import org.apache.kafka.trogdor.rest.TasksResponse; import org.apache.kafka.trogdor.rest.UptimeResponse; + +import net.sourceforge.argparse4j.ArgumentParsers; +import net.sourceforge.argparse4j.inf.ArgumentParser; +import net.sourceforge.argparse4j.inf.ArgumentParserException; +import net.sourceforge.argparse4j.inf.Namespace; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/trogdor/src/main/java/org/apache/kafka/trogdor/coordinator/CoordinatorClient.java b/trogdor/src/main/java/org/apache/kafka/trogdor/coordinator/CoordinatorClient.java index 57797ec80a537..00c26adc4c4c8 100644 --- a/trogdor/src/main/java/org/apache/kafka/trogdor/coordinator/CoordinatorClient.java +++ b/trogdor/src/main/java/org/apache/kafka/trogdor/coordinator/CoordinatorClient.java @@ -17,13 +17,6 @@ package org.apache.kafka.trogdor.coordinator; -import com.fasterxml.jackson.core.type.TypeReference; -import net.sourceforge.argparse4j.ArgumentParsers; -import net.sourceforge.argparse4j.inf.ArgumentParser; -import net.sourceforge.argparse4j.inf.MutuallyExclusiveGroup; -import net.sourceforge.argparse4j.inf.Namespace; -import net.sourceforge.argparse4j.inf.Subparser; -import net.sourceforge.argparse4j.inf.Subparsers; import org.apache.kafka.common.utils.Exit; import org.apache.kafka.trogdor.common.JsonUtil; import org.apache.kafka.trogdor.common.StringFormatter; @@ -33,25 +26,32 @@ import org.apache.kafka.trogdor.rest.Empty; import org.apache.kafka.trogdor.rest.JsonRestServer; import org.apache.kafka.trogdor.rest.JsonRestServer.HttpResponse; +import org.apache.kafka.trogdor.rest.RequestConflictException; import org.apache.kafka.trogdor.rest.StopTaskRequest; import org.apache.kafka.trogdor.rest.TaskDone; import org.apache.kafka.trogdor.rest.TaskPending; import org.apache.kafka.trogdor.rest.TaskRequest; import org.apache.kafka.trogdor.rest.TaskRunning; +import org.apache.kafka.trogdor.rest.TaskState; import org.apache.kafka.trogdor.rest.TaskStateType; import org.apache.kafka.trogdor.rest.TaskStopping; import org.apache.kafka.trogdor.rest.TasksRequest; -import org.apache.kafka.trogdor.rest.TaskState; import org.apache.kafka.trogdor.rest.TasksResponse; -import org.apache.kafka.trogdor.task.TaskSpec; -import org.apache.kafka.trogdor.rest.RequestConflictException; import org.apache.kafka.trogdor.rest.UptimeResponse; +import org.apache.kafka.trogdor.task.TaskSpec; + +import com.fasterxml.jackson.core.type.TypeReference; + +import net.sourceforge.argparse4j.ArgumentParsers; +import net.sourceforge.argparse4j.inf.ArgumentParser; +import net.sourceforge.argparse4j.inf.MutuallyExclusiveGroup; +import net.sourceforge.argparse4j.inf.Namespace; +import net.sourceforge.argparse4j.inf.Subparser; +import net.sourceforge.argparse4j.inf.Subparsers; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import javax.ws.rs.NotFoundException; -import javax.ws.rs.core.UriBuilder; - import java.time.OffsetDateTime; import java.time.ZoneOffset; import java.util.ArrayList; @@ -63,6 +63,9 @@ import java.util.regex.Pattern; import java.util.regex.PatternSyntaxException; +import javax.ws.rs.NotFoundException; +import javax.ws.rs.core.UriBuilder; + import static net.sourceforge.argparse4j.impl.Arguments.append; import static net.sourceforge.argparse4j.impl.Arguments.store; import static net.sourceforge.argparse4j.impl.Arguments.storeTrue; diff --git a/trogdor/src/main/java/org/apache/kafka/trogdor/coordinator/CoordinatorRestResource.java b/trogdor/src/main/java/org/apache/kafka/trogdor/coordinator/CoordinatorRestResource.java index c44b696da0b5d..a65c4d26a73a1 100644 --- a/trogdor/src/main/java/org/apache/kafka/trogdor/coordinator/CoordinatorRestResource.java +++ b/trogdor/src/main/java/org/apache/kafka/trogdor/coordinator/CoordinatorRestResource.java @@ -29,6 +29,10 @@ import org.apache.kafka.trogdor.rest.TasksResponse; import org.apache.kafka.trogdor.rest.UptimeResponse; +import java.util.List; +import java.util.Optional; +import java.util.concurrent.atomic.AtomicReference; + import javax.servlet.ServletContext; import javax.ws.rs.Consumes; import javax.ws.rs.DELETE; @@ -43,9 +47,6 @@ import javax.ws.rs.QueryParam; import javax.ws.rs.core.MediaType; import javax.ws.rs.core.Response; -import java.util.List; -import java.util.Optional; -import java.util.concurrent.atomic.AtomicReference; /** * The REST resource for the Coordinator. This describes the RPCs which the coordinator diff --git a/trogdor/src/main/java/org/apache/kafka/trogdor/coordinator/NodeManager.java b/trogdor/src/main/java/org/apache/kafka/trogdor/coordinator/NodeManager.java index e85708a514366..577b22b8c67fe 100644 --- a/trogdor/src/main/java/org/apache/kafka/trogdor/coordinator/NodeManager.java +++ b/trogdor/src/main/java/org/apache/kafka/trogdor/coordinator/NodeManager.java @@ -56,6 +56,7 @@ import org.apache.kafka.trogdor.rest.WorkerState; import org.apache.kafka.trogdor.rest.WorkerStopping; import org.apache.kafka.trogdor.task.TaskSpec; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/trogdor/src/main/java/org/apache/kafka/trogdor/coordinator/TaskManager.java b/trogdor/src/main/java/org/apache/kafka/trogdor/coordinator/TaskManager.java index b270a377305b1..9c2a13cb68f77 100644 --- a/trogdor/src/main/java/org/apache/kafka/trogdor/coordinator/TaskManager.java +++ b/trogdor/src/main/java/org/apache/kafka/trogdor/coordinator/TaskManager.java @@ -17,11 +17,6 @@ package org.apache.kafka.trogdor.coordinator; -import com.fasterxml.jackson.core.JsonProcessingException; -import com.fasterxml.jackson.databind.JsonNode; -import com.fasterxml.jackson.databind.node.JsonNodeFactory; -import com.fasterxml.jackson.databind.node.LongNode; -import com.fasterxml.jackson.databind.node.ObjectNode; import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.errors.InvalidRequestException; import org.apache.kafka.common.utils.Scheduler; @@ -46,6 +41,13 @@ import org.apache.kafka.trogdor.rest.WorkerState; import org.apache.kafka.trogdor.task.TaskController; import org.apache.kafka.trogdor.task.TaskSpec; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.node.JsonNodeFactory; +import com.fasterxml.jackson.databind.node.LongNode; +import com.fasterxml.jackson.databind.node.ObjectNode; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/trogdor/src/main/java/org/apache/kafka/trogdor/fault/DegradedNetworkFaultSpec.java b/trogdor/src/main/java/org/apache/kafka/trogdor/fault/DegradedNetworkFaultSpec.java index 127a4da5ddb26..59a2eb00eb0e6 100644 --- a/trogdor/src/main/java/org/apache/kafka/trogdor/fault/DegradedNetworkFaultSpec.java +++ b/trogdor/src/main/java/org/apache/kafka/trogdor/fault/DegradedNetworkFaultSpec.java @@ -17,12 +17,13 @@ package org.apache.kafka.trogdor.fault; -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; import org.apache.kafka.trogdor.task.TaskController; import org.apache.kafka.trogdor.task.TaskSpec; import org.apache.kafka.trogdor.task.TaskWorker; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + import java.util.Collections; import java.util.Map; diff --git a/trogdor/src/main/java/org/apache/kafka/trogdor/fault/DegradedNetworkFaultWorker.java b/trogdor/src/main/java/org/apache/kafka/trogdor/fault/DegradedNetworkFaultWorker.java index d071d12a845e1..f9657d042e6ef 100644 --- a/trogdor/src/main/java/org/apache/kafka/trogdor/fault/DegradedNetworkFaultWorker.java +++ b/trogdor/src/main/java/org/apache/kafka/trogdor/fault/DegradedNetworkFaultWorker.java @@ -17,12 +17,14 @@ package org.apache.kafka.trogdor.fault; -import com.fasterxml.jackson.databind.node.TextNode; import org.apache.kafka.common.internals.KafkaFutureImpl; import org.apache.kafka.trogdor.common.Node; import org.apache.kafka.trogdor.common.Platform; import org.apache.kafka.trogdor.task.TaskWorker; import org.apache.kafka.trogdor.task.WorkerStatusTracker; + +import com.fasterxml.jackson.databind.node.TextNode; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/trogdor/src/main/java/org/apache/kafka/trogdor/fault/FilesUnreadableFaultSpec.java b/trogdor/src/main/java/org/apache/kafka/trogdor/fault/FilesUnreadableFaultSpec.java index f38a89b5178b1..41c63acd217bc 100644 --- a/trogdor/src/main/java/org/apache/kafka/trogdor/fault/FilesUnreadableFaultSpec.java +++ b/trogdor/src/main/java/org/apache/kafka/trogdor/fault/FilesUnreadableFaultSpec.java @@ -17,13 +17,14 @@ package org.apache.kafka.trogdor.fault; -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; import org.apache.kafka.trogdor.fault.Kibosh.KiboshFilesUnreadableFaultSpec; import org.apache.kafka.trogdor.task.TaskController; import org.apache.kafka.trogdor.task.TaskSpec; import org.apache.kafka.trogdor.task.TaskWorker; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + import java.util.HashSet; import java.util.Set; diff --git a/trogdor/src/main/java/org/apache/kafka/trogdor/fault/Kibosh.java b/trogdor/src/main/java/org/apache/kafka/trogdor/fault/Kibosh.java index 95356c5209e35..df8f5c485ecf3 100644 --- a/trogdor/src/main/java/org/apache/kafka/trogdor/fault/Kibosh.java +++ b/trogdor/src/main/java/org/apache/kafka/trogdor/fault/Kibosh.java @@ -17,11 +17,12 @@ package org.apache.kafka.trogdor.fault; +import org.apache.kafka.trogdor.common.JsonUtil; + import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonTypeInfo; -import org.apache.kafka.trogdor.common.JsonUtil; import java.io.IOException; import java.nio.file.Files; diff --git a/trogdor/src/main/java/org/apache/kafka/trogdor/fault/KiboshFaultWorker.java b/trogdor/src/main/java/org/apache/kafka/trogdor/fault/KiboshFaultWorker.java index 97934a88bbdea..4da42fd2cd268 100644 --- a/trogdor/src/main/java/org/apache/kafka/trogdor/fault/KiboshFaultWorker.java +++ b/trogdor/src/main/java/org/apache/kafka/trogdor/fault/KiboshFaultWorker.java @@ -17,12 +17,14 @@ package org.apache.kafka.trogdor.fault; -import com.fasterxml.jackson.databind.node.TextNode; import org.apache.kafka.common.internals.KafkaFutureImpl; import org.apache.kafka.trogdor.common.Platform; import org.apache.kafka.trogdor.fault.Kibosh.KiboshFaultSpec; import org.apache.kafka.trogdor.task.TaskWorker; import org.apache.kafka.trogdor.task.WorkerStatusTracker; + +import com.fasterxml.jackson.databind.node.TextNode; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/trogdor/src/main/java/org/apache/kafka/trogdor/fault/NetworkPartitionFaultSpec.java b/trogdor/src/main/java/org/apache/kafka/trogdor/fault/NetworkPartitionFaultSpec.java index 445ed92b2e63f..d49317d7d2743 100644 --- a/trogdor/src/main/java/org/apache/kafka/trogdor/fault/NetworkPartitionFaultSpec.java +++ b/trogdor/src/main/java/org/apache/kafka/trogdor/fault/NetworkPartitionFaultSpec.java @@ -17,12 +17,13 @@ package org.apache.kafka.trogdor.fault; -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; import org.apache.kafka.trogdor.task.TaskController; import org.apache.kafka.trogdor.task.TaskSpec; import org.apache.kafka.trogdor.task.TaskWorker; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + import java.util.ArrayList; import java.util.HashSet; import java.util.List; diff --git a/trogdor/src/main/java/org/apache/kafka/trogdor/fault/NetworkPartitionFaultWorker.java b/trogdor/src/main/java/org/apache/kafka/trogdor/fault/NetworkPartitionFaultWorker.java index 1b99a93d8fbd0..9cb8b5724a22d 100644 --- a/trogdor/src/main/java/org/apache/kafka/trogdor/fault/NetworkPartitionFaultWorker.java +++ b/trogdor/src/main/java/org/apache/kafka/trogdor/fault/NetworkPartitionFaultWorker.java @@ -17,13 +17,15 @@ package org.apache.kafka.trogdor.fault; -import com.fasterxml.jackson.databind.node.TextNode; import org.apache.kafka.common.internals.KafkaFutureImpl; import org.apache.kafka.trogdor.common.Node; import org.apache.kafka.trogdor.common.Platform; import org.apache.kafka.trogdor.common.Topology; import org.apache.kafka.trogdor.task.TaskWorker; import org.apache.kafka.trogdor.task.WorkerStatusTracker; + +import com.fasterxml.jackson.databind.node.TextNode; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/trogdor/src/main/java/org/apache/kafka/trogdor/fault/ProcessStopFaultController.java b/trogdor/src/main/java/org/apache/kafka/trogdor/fault/ProcessStopFaultController.java index 6ec803a3588cc..2499e903706bf 100644 --- a/trogdor/src/main/java/org/apache/kafka/trogdor/fault/ProcessStopFaultController.java +++ b/trogdor/src/main/java/org/apache/kafka/trogdor/fault/ProcessStopFaultController.java @@ -19,6 +19,7 @@ import org.apache.kafka.trogdor.common.Topology; import org.apache.kafka.trogdor.task.TaskController; + import java.util.Set; public class ProcessStopFaultController implements TaskController { diff --git a/trogdor/src/main/java/org/apache/kafka/trogdor/fault/ProcessStopFaultSpec.java b/trogdor/src/main/java/org/apache/kafka/trogdor/fault/ProcessStopFaultSpec.java index a99cb7cb2e22e..7bb00f2ee9031 100644 --- a/trogdor/src/main/java/org/apache/kafka/trogdor/fault/ProcessStopFaultSpec.java +++ b/trogdor/src/main/java/org/apache/kafka/trogdor/fault/ProcessStopFaultSpec.java @@ -17,12 +17,13 @@ package org.apache.kafka.trogdor.fault; -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; import org.apache.kafka.trogdor.task.TaskController; import org.apache.kafka.trogdor.task.TaskSpec; import org.apache.kafka.trogdor.task.TaskWorker; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + import java.util.HashSet; import java.util.List; import java.util.Set; diff --git a/trogdor/src/main/java/org/apache/kafka/trogdor/fault/ProcessStopFaultWorker.java b/trogdor/src/main/java/org/apache/kafka/trogdor/fault/ProcessStopFaultWorker.java index 5d9d68e443c08..ffab9f72f994e 100644 --- a/trogdor/src/main/java/org/apache/kafka/trogdor/fault/ProcessStopFaultWorker.java +++ b/trogdor/src/main/java/org/apache/kafka/trogdor/fault/ProcessStopFaultWorker.java @@ -17,11 +17,13 @@ package org.apache.kafka.trogdor.fault; -import com.fasterxml.jackson.databind.node.TextNode; import org.apache.kafka.common.internals.KafkaFutureImpl; import org.apache.kafka.trogdor.common.Platform; import org.apache.kafka.trogdor.task.TaskWorker; import org.apache.kafka.trogdor.task.WorkerStatusTracker; + +import com.fasterxml.jackson.databind.node.TextNode; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/trogdor/src/main/java/org/apache/kafka/trogdor/rest/CreateTaskRequest.java b/trogdor/src/main/java/org/apache/kafka/trogdor/rest/CreateTaskRequest.java index f3c216e02e0d3..8b786f17d2d29 100644 --- a/trogdor/src/main/java/org/apache/kafka/trogdor/rest/CreateTaskRequest.java +++ b/trogdor/src/main/java/org/apache/kafka/trogdor/rest/CreateTaskRequest.java @@ -17,9 +17,10 @@ package org.apache.kafka.trogdor.rest; +import org.apache.kafka.trogdor.task.TaskSpec; + import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import org.apache.kafka.trogdor.task.TaskSpec; /** * A request to the Trogdor coordinator to create a task. diff --git a/trogdor/src/main/java/org/apache/kafka/trogdor/rest/CreateWorkerRequest.java b/trogdor/src/main/java/org/apache/kafka/trogdor/rest/CreateWorkerRequest.java index 4acc943251e40..ee49a2ffebd53 100644 --- a/trogdor/src/main/java/org/apache/kafka/trogdor/rest/CreateWorkerRequest.java +++ b/trogdor/src/main/java/org/apache/kafka/trogdor/rest/CreateWorkerRequest.java @@ -17,9 +17,10 @@ package org.apache.kafka.trogdor.rest; +import org.apache.kafka.trogdor.task.TaskSpec; + import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import org.apache.kafka.trogdor.task.TaskSpec; /** * A request to the Trogdor agent to create a worker. diff --git a/trogdor/src/main/java/org/apache/kafka/trogdor/rest/Empty.java b/trogdor/src/main/java/org/apache/kafka/trogdor/rest/Empty.java index 68547930ad016..55bcd1f85da2f 100644 --- a/trogdor/src/main/java/org/apache/kafka/trogdor/rest/Empty.java +++ b/trogdor/src/main/java/org/apache/kafka/trogdor/rest/Empty.java @@ -17,9 +17,10 @@ package org.apache.kafka.trogdor.rest; -import com.fasterxml.jackson.annotation.JsonCreator; import org.apache.kafka.trogdor.common.JsonUtil; +import com.fasterxml.jackson.annotation.JsonCreator; + /** * An empty request or response. */ diff --git a/trogdor/src/main/java/org/apache/kafka/trogdor/rest/ErrorResponse.java b/trogdor/src/main/java/org/apache/kafka/trogdor/rest/ErrorResponse.java index 08bf6cdad12b2..fa8df147b8423 100644 --- a/trogdor/src/main/java/org/apache/kafka/trogdor/rest/ErrorResponse.java +++ b/trogdor/src/main/java/org/apache/kafka/trogdor/rest/ErrorResponse.java @@ -17,9 +17,10 @@ package org.apache.kafka.trogdor.rest; +import org.apache.kafka.trogdor.common.JsonUtil; + import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import org.apache.kafka.trogdor.common.JsonUtil; import java.util.Objects; diff --git a/trogdor/src/main/java/org/apache/kafka/trogdor/rest/JsonRestServer.java b/trogdor/src/main/java/org/apache/kafka/trogdor/rest/JsonRestServer.java index cc3bbc0cc6de4..44e69ee2dc77f 100644 --- a/trogdor/src/main/java/org/apache/kafka/trogdor/rest/JsonRestServer.java +++ b/trogdor/src/main/java/org/apache/kafka/trogdor/rest/JsonRestServer.java @@ -17,11 +17,12 @@ package org.apache.kafka.trogdor.rest; +import org.apache.kafka.common.utils.ThreadUtils; +import org.apache.kafka.trogdor.common.JsonUtil; + import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.jaxrs.json.JacksonJsonProvider; -import org.apache.kafka.common.utils.ThreadUtils; -import org.apache.kafka.trogdor.common.JsonUtil; import org.eclipse.jetty.server.Connector; import org.eclipse.jetty.server.CustomRequestLog; import org.eclipse.jetty.server.Handler; diff --git a/trogdor/src/main/java/org/apache/kafka/trogdor/rest/RestExceptionMapper.java b/trogdor/src/main/java/org/apache/kafka/trogdor/rest/RestExceptionMapper.java index 57c54ec04d846..2c1a046b480bf 100644 --- a/trogdor/src/main/java/org/apache/kafka/trogdor/rest/RestExceptionMapper.java +++ b/trogdor/src/main/java/org/apache/kafka/trogdor/rest/RestExceptionMapper.java @@ -16,10 +16,12 @@ */ package org.apache.kafka.trogdor.rest; -import com.fasterxml.jackson.databind.JsonMappingException; -import com.fasterxml.jackson.databind.exc.InvalidTypeIdException; import org.apache.kafka.common.errors.InvalidRequestException; import org.apache.kafka.common.errors.SerializationException; + +import com.fasterxml.jackson.databind.JsonMappingException; +import com.fasterxml.jackson.databind.exc.InvalidTypeIdException; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/trogdor/src/main/java/org/apache/kafka/trogdor/rest/TaskDone.java b/trogdor/src/main/java/org/apache/kafka/trogdor/rest/TaskDone.java index 6e9761bf5e59b..f888d46ff4d9e 100644 --- a/trogdor/src/main/java/org/apache/kafka/trogdor/rest/TaskDone.java +++ b/trogdor/src/main/java/org/apache/kafka/trogdor/rest/TaskDone.java @@ -16,10 +16,11 @@ */ package org.apache.kafka.trogdor.rest; +import org.apache.kafka.trogdor.task.TaskSpec; + import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.databind.JsonNode; -import org.apache.kafka.trogdor.task.TaskSpec; /** * The state a task is in once it's done. diff --git a/trogdor/src/main/java/org/apache/kafka/trogdor/rest/TaskPending.java b/trogdor/src/main/java/org/apache/kafka/trogdor/rest/TaskPending.java index ca1d314a53acd..3c20227d2791d 100644 --- a/trogdor/src/main/java/org/apache/kafka/trogdor/rest/TaskPending.java +++ b/trogdor/src/main/java/org/apache/kafka/trogdor/rest/TaskPending.java @@ -17,10 +17,11 @@ package org.apache.kafka.trogdor.rest; +import org.apache.kafka.trogdor.task.TaskSpec; + import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.databind.node.NullNode; -import org.apache.kafka.trogdor.task.TaskSpec; /** * The state for a task which is still pending. diff --git a/trogdor/src/main/java/org/apache/kafka/trogdor/rest/TaskRunning.java b/trogdor/src/main/java/org/apache/kafka/trogdor/rest/TaskRunning.java index 8487bc39d1691..baee886a721ac 100644 --- a/trogdor/src/main/java/org/apache/kafka/trogdor/rest/TaskRunning.java +++ b/trogdor/src/main/java/org/apache/kafka/trogdor/rest/TaskRunning.java @@ -17,10 +17,11 @@ package org.apache.kafka.trogdor.rest; +import org.apache.kafka.trogdor.task.TaskSpec; + import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.databind.JsonNode; -import org.apache.kafka.trogdor.task.TaskSpec; /** * The state for a task which is being run by the agent. diff --git a/trogdor/src/main/java/org/apache/kafka/trogdor/rest/TaskState.java b/trogdor/src/main/java/org/apache/kafka/trogdor/rest/TaskState.java index 6484d0692e3ba..6bb9e7f4f1f26 100644 --- a/trogdor/src/main/java/org/apache/kafka/trogdor/rest/TaskState.java +++ b/trogdor/src/main/java/org/apache/kafka/trogdor/rest/TaskState.java @@ -17,12 +17,13 @@ package org.apache.kafka.trogdor.rest; +import org.apache.kafka.trogdor.task.TaskSpec; + import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.fasterxml.jackson.databind.JsonNode; import com.fasterxml.jackson.databind.node.NullNode; -import org.apache.kafka.trogdor.task.TaskSpec; /** * The state which a task is in on the Coordinator. diff --git a/trogdor/src/main/java/org/apache/kafka/trogdor/rest/TaskStopping.java b/trogdor/src/main/java/org/apache/kafka/trogdor/rest/TaskStopping.java index 2b2c4c4ec0b07..a54d6cdbf58cf 100644 --- a/trogdor/src/main/java/org/apache/kafka/trogdor/rest/TaskStopping.java +++ b/trogdor/src/main/java/org/apache/kafka/trogdor/rest/TaskStopping.java @@ -17,10 +17,11 @@ package org.apache.kafka.trogdor.rest; +import org.apache.kafka.trogdor.task.TaskSpec; + import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.databind.JsonNode; -import org.apache.kafka.trogdor.task.TaskSpec; /** * The state for a task which is being stopped on the coordinator. diff --git a/trogdor/src/main/java/org/apache/kafka/trogdor/rest/WorkerDone.java b/trogdor/src/main/java/org/apache/kafka/trogdor/rest/WorkerDone.java index 5f773bba5c443..5b6f1501cc6bb 100644 --- a/trogdor/src/main/java/org/apache/kafka/trogdor/rest/WorkerDone.java +++ b/trogdor/src/main/java/org/apache/kafka/trogdor/rest/WorkerDone.java @@ -17,11 +17,12 @@ package org.apache.kafka.trogdor.rest; +import org.apache.kafka.trogdor.task.TaskSpec; + import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.databind.JsonNode; import com.fasterxml.jackson.databind.node.NullNode; -import org.apache.kafka.trogdor.task.TaskSpec; /** * The state a worker is in once it's done. diff --git a/trogdor/src/main/java/org/apache/kafka/trogdor/rest/WorkerReceiving.java b/trogdor/src/main/java/org/apache/kafka/trogdor/rest/WorkerReceiving.java index 1babcce2a5762..8196359d10b0b 100644 --- a/trogdor/src/main/java/org/apache/kafka/trogdor/rest/WorkerReceiving.java +++ b/trogdor/src/main/java/org/apache/kafka/trogdor/rest/WorkerReceiving.java @@ -17,11 +17,12 @@ package org.apache.kafka.trogdor.rest; +import org.apache.kafka.trogdor.task.TaskSpec; + import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.databind.JsonNode; import com.fasterxml.jackson.databind.node.TextNode; -import org.apache.kafka.trogdor.task.TaskSpec; /** * When we're in the process of sending a TaskSpec to the Agent, the Worker is regarded diff --git a/trogdor/src/main/java/org/apache/kafka/trogdor/rest/WorkerRunning.java b/trogdor/src/main/java/org/apache/kafka/trogdor/rest/WorkerRunning.java index 15e77528d6229..49bc0160a29bb 100644 --- a/trogdor/src/main/java/org/apache/kafka/trogdor/rest/WorkerRunning.java +++ b/trogdor/src/main/java/org/apache/kafka/trogdor/rest/WorkerRunning.java @@ -17,11 +17,12 @@ package org.apache.kafka.trogdor.rest; +import org.apache.kafka.trogdor.task.TaskSpec; + import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.databind.JsonNode; import com.fasterxml.jackson.databind.node.NullNode; -import org.apache.kafka.trogdor.task.TaskSpec; /** * The state for a task which is being run by the agent. diff --git a/trogdor/src/main/java/org/apache/kafka/trogdor/rest/WorkerStarting.java b/trogdor/src/main/java/org/apache/kafka/trogdor/rest/WorkerStarting.java index 7a06eac5b7d08..a6c58a376c9e0 100644 --- a/trogdor/src/main/java/org/apache/kafka/trogdor/rest/WorkerStarting.java +++ b/trogdor/src/main/java/org/apache/kafka/trogdor/rest/WorkerStarting.java @@ -17,11 +17,12 @@ package org.apache.kafka.trogdor.rest; +import org.apache.kafka.trogdor.task.TaskSpec; + import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.databind.JsonNode; import com.fasterxml.jackson.databind.node.TextNode; -import org.apache.kafka.trogdor.task.TaskSpec; /** * When we have just started a worker. diff --git a/trogdor/src/main/java/org/apache/kafka/trogdor/rest/WorkerState.java b/trogdor/src/main/java/org/apache/kafka/trogdor/rest/WorkerState.java index aa97d5706e0d6..d1dfe5cb60364 100644 --- a/trogdor/src/main/java/org/apache/kafka/trogdor/rest/WorkerState.java +++ b/trogdor/src/main/java/org/apache/kafka/trogdor/rest/WorkerState.java @@ -17,12 +17,13 @@ package org.apache.kafka.trogdor.rest; +import org.apache.kafka.common.KafkaException; +import org.apache.kafka.trogdor.task.TaskSpec; + import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.fasterxml.jackson.databind.JsonNode; -import org.apache.kafka.common.KafkaException; -import org.apache.kafka.trogdor.task.TaskSpec; /** * The state which a worker is in on the Agent. diff --git a/trogdor/src/main/java/org/apache/kafka/trogdor/rest/WorkerStopping.java b/trogdor/src/main/java/org/apache/kafka/trogdor/rest/WorkerStopping.java index 2942e118ac6bf..bfac86857ed06 100644 --- a/trogdor/src/main/java/org/apache/kafka/trogdor/rest/WorkerStopping.java +++ b/trogdor/src/main/java/org/apache/kafka/trogdor/rest/WorkerStopping.java @@ -17,11 +17,12 @@ package org.apache.kafka.trogdor.rest; +import org.apache.kafka.trogdor.task.TaskSpec; + import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.databind.JsonNode; import com.fasterxml.jackson.databind.node.NullNode; -import org.apache.kafka.trogdor.task.TaskSpec; /** * The state for a worker which is being stopped on the agent. diff --git a/trogdor/src/main/java/org/apache/kafka/trogdor/task/NoOpTaskWorker.java b/trogdor/src/main/java/org/apache/kafka/trogdor/task/NoOpTaskWorker.java index 77336d87059f3..c2c3c057e470f 100644 --- a/trogdor/src/main/java/org/apache/kafka/trogdor/task/NoOpTaskWorker.java +++ b/trogdor/src/main/java/org/apache/kafka/trogdor/task/NoOpTaskWorker.java @@ -17,9 +17,11 @@ package org.apache.kafka.trogdor.task; -import com.fasterxml.jackson.databind.node.TextNode; import org.apache.kafka.common.internals.KafkaFutureImpl; import org.apache.kafka.trogdor.common.Platform; + +import com.fasterxml.jackson.databind.node.TextNode; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/trogdor/src/main/java/org/apache/kafka/trogdor/task/TaskSpec.java b/trogdor/src/main/java/org/apache/kafka/trogdor/task/TaskSpec.java index 7b67c4ee78967..670030acc67b4 100644 --- a/trogdor/src/main/java/org/apache/kafka/trogdor/task/TaskSpec.java +++ b/trogdor/src/main/java/org/apache/kafka/trogdor/task/TaskSpec.java @@ -17,9 +17,10 @@ package org.apache.kafka.trogdor.task; +import org.apache.kafka.trogdor.common.JsonUtil; + import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonTypeInfo; -import org.apache.kafka.trogdor.common.JsonUtil; import java.util.Collections; import java.util.Map; diff --git a/trogdor/src/main/java/org/apache/kafka/trogdor/workload/ConfigurableProducerSpec.java b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/ConfigurableProducerSpec.java index 29d4a451f57f4..64332e13f5b4b 100644 --- a/trogdor/src/main/java/org/apache/kafka/trogdor/workload/ConfigurableProducerSpec.java +++ b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/ConfigurableProducerSpec.java @@ -17,11 +17,13 @@ package org.apache.kafka.trogdor.workload; -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; import org.apache.kafka.trogdor.task.TaskController; import org.apache.kafka.trogdor.task.TaskSpec; import org.apache.kafka.trogdor.task.TaskWorker; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + import java.util.Collections; import java.util.Map; import java.util.Optional; diff --git a/trogdor/src/main/java/org/apache/kafka/trogdor/workload/ConfigurableProducerWorker.java b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/ConfigurableProducerWorker.java index b08ef44d6388e..9fc930585d195 100644 --- a/trogdor/src/main/java/org/apache/kafka/trogdor/workload/ConfigurableProducerWorker.java +++ b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/ConfigurableProducerWorker.java @@ -17,9 +17,6 @@ package org.apache.kafka.trogdor.workload; -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import com.fasterxml.jackson.databind.node.TextNode; import org.apache.kafka.clients.admin.NewTopic; import org.apache.kafka.clients.producer.Callback; import org.apache.kafka.clients.producer.KafkaProducer; @@ -36,6 +33,11 @@ import org.apache.kafka.trogdor.common.WorkerUtils; import org.apache.kafka.trogdor.task.TaskWorker; import org.apache.kafka.trogdor.task.WorkerStatusTracker; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.databind.node.TextNode; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/trogdor/src/main/java/org/apache/kafka/trogdor/workload/ConnectionStressSpec.java b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/ConnectionStressSpec.java index 8738091b4990d..aa6cf7e5dae9c 100644 --- a/trogdor/src/main/java/org/apache/kafka/trogdor/workload/ConnectionStressSpec.java +++ b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/ConnectionStressSpec.java @@ -17,12 +17,13 @@ package org.apache.kafka.trogdor.workload; -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; import org.apache.kafka.trogdor.task.TaskController; import org.apache.kafka.trogdor.task.TaskSpec; import org.apache.kafka.trogdor.task.TaskWorker; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + import java.util.ArrayList; import java.util.Collections; import java.util.List; diff --git a/trogdor/src/main/java/org/apache/kafka/trogdor/workload/ConnectionStressWorker.java b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/ConnectionStressWorker.java index 6cd367c06d9f2..34d683af7f3c1 100644 --- a/trogdor/src/main/java/org/apache/kafka/trogdor/workload/ConnectionStressWorker.java +++ b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/ConnectionStressWorker.java @@ -17,9 +17,6 @@ package org.apache.kafka.trogdor.workload; -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import com.fasterxml.jackson.databind.JsonNode; import org.apache.kafka.clients.ApiVersions; import org.apache.kafka.clients.ClientUtils; import org.apache.kafka.clients.ManualMetadataUpdater; @@ -43,6 +40,11 @@ import org.apache.kafka.trogdor.common.WorkerUtils; import org.apache.kafka.trogdor.task.TaskWorker; import org.apache.kafka.trogdor.task.WorkerStatusTracker; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.databind.JsonNode; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/trogdor/src/main/java/org/apache/kafka/trogdor/workload/ConstantFlushGenerator.java b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/ConstantFlushGenerator.java index 9d656b219a1be..9de6dd738b957 100644 --- a/trogdor/src/main/java/org/apache/kafka/trogdor/workload/ConstantFlushGenerator.java +++ b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/ConstantFlushGenerator.java @@ -16,11 +16,12 @@ */ package org.apache.kafka.trogdor.workload; -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; import org.apache.kafka.clients.producer.KafkaProducer; import org.apache.kafka.common.errors.InterruptException; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + /** * This generator will flush the producer after a specific number of messages. This is useful to simulate a specific * number of messages in a batch regardless of the message size, since batch flushing is not exposed in the diff --git a/trogdor/src/main/java/org/apache/kafka/trogdor/workload/ConstantThroughputGenerator.java b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/ConstantThroughputGenerator.java index 9e5eeb967e96e..d46594cbe9406 100644 --- a/trogdor/src/main/java/org/apache/kafka/trogdor/workload/ConstantThroughputGenerator.java +++ b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/ConstantThroughputGenerator.java @@ -16,9 +16,10 @@ */ package org.apache.kafka.trogdor.workload; +import org.apache.kafka.common.utils.Time; + import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import org.apache.kafka.common.utils.Time; /** * This throughput generator configures constant throughput. diff --git a/trogdor/src/main/java/org/apache/kafka/trogdor/workload/ConsumeBenchSpec.java b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/ConsumeBenchSpec.java index 294b251dedd2e..3813b418fba59 100644 --- a/trogdor/src/main/java/org/apache/kafka/trogdor/workload/ConsumeBenchSpec.java +++ b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/ConsumeBenchSpec.java @@ -17,9 +17,6 @@ package org.apache.kafka.trogdor.workload; -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; - import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.config.ConfigException; import org.apache.kafka.trogdor.common.StringExpander; @@ -27,15 +24,18 @@ import org.apache.kafka.trogdor.task.TaskSpec; import org.apache.kafka.trogdor.task.TaskWorker; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + import java.util.ArrayList; import java.util.Collection; import java.util.Collections; -import java.util.List; -import java.util.Map; import java.util.HashMap; -import java.util.Set; import java.util.HashSet; +import java.util.List; +import java.util.Map; import java.util.Optional; +import java.util.Set; /** * The specification for a benchmark that consumer messages from a set of topic/partitions. diff --git a/trogdor/src/main/java/org/apache/kafka/trogdor/workload/ConsumeBenchWorker.java b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/ConsumeBenchWorker.java index 83e73db1a7b66..da2de44074e28 100644 --- a/trogdor/src/main/java/org/apache/kafka/trogdor/workload/ConsumeBenchWorker.java +++ b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/ConsumeBenchWorker.java @@ -17,9 +17,6 @@ package org.apache.kafka.trogdor.workload; -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import com.fasterxml.jackson.databind.JsonNode; import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.consumer.ConsumerRecords; @@ -34,22 +31,26 @@ import org.apache.kafka.trogdor.common.JsonUtil; import org.apache.kafka.trogdor.common.Platform; import org.apache.kafka.trogdor.common.WorkerUtils; +import org.apache.kafka.trogdor.task.TaskWorker; import org.apache.kafka.trogdor.task.WorkerStatusTracker; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.databind.JsonNode; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.kafka.trogdor.task.TaskWorker; - import java.time.Duration; import java.util.ArrayList; import java.util.Collection; +import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.Set; -import java.util.UUID; import java.util.Optional; import java.util.Properties; -import java.util.HashMap; +import java.util.Set; +import java.util.UUID; import java.util.concurrent.Callable; import java.util.concurrent.Executors; import java.util.concurrent.Future; diff --git a/trogdor/src/main/java/org/apache/kafka/trogdor/workload/ExternalCommandSpec.java b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/ExternalCommandSpec.java index f6a738d01448a..838beb62cff27 100644 --- a/trogdor/src/main/java/org/apache/kafka/trogdor/workload/ExternalCommandSpec.java +++ b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/ExternalCommandSpec.java @@ -17,14 +17,14 @@ package org.apache.kafka.trogdor.workload; +import org.apache.kafka.trogdor.task.TaskController; +import org.apache.kafka.trogdor.task.TaskSpec; +import org.apache.kafka.trogdor.task.TaskWorker; + import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.databind.JsonNode; - import com.fasterxml.jackson.databind.node.NullNode; -import org.apache.kafka.trogdor.task.TaskController; -import org.apache.kafka.trogdor.task.TaskSpec; -import org.apache.kafka.trogdor.task.TaskWorker; import java.util.ArrayList; import java.util.Collections; diff --git a/trogdor/src/main/java/org/apache/kafka/trogdor/workload/ExternalCommandWorker.java b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/ExternalCommandWorker.java index 6543e22798865..be9046235218d 100644 --- a/trogdor/src/main/java/org/apache/kafka/trogdor/workload/ExternalCommandWorker.java +++ b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/ExternalCommandWorker.java @@ -17,33 +17,34 @@ package org.apache.kafka.trogdor.workload; -import com.fasterxml.jackson.databind.JsonNode; -import com.fasterxml.jackson.databind.node.JsonNodeFactory; -import com.fasterxml.jackson.databind.node.NullNode; -import com.fasterxml.jackson.databind.node.ObjectNode; -import com.fasterxml.jackson.databind.node.TextNode; import org.apache.kafka.common.internals.KafkaFutureImpl; import org.apache.kafka.common.utils.ThreadUtils; import org.apache.kafka.trogdor.common.JsonUtil; import org.apache.kafka.trogdor.common.Platform; import org.apache.kafka.trogdor.task.TaskWorker; import org.apache.kafka.trogdor.task.WorkerStatusTracker; + +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.node.JsonNodeFactory; +import com.fasterxml.jackson.databind.node.NullNode; +import com.fasterxml.jackson.databind.node.ObjectNode; +import com.fasterxml.jackson.databind.node.TextNode; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.io.BufferedReader; import java.io.IOException; - import java.io.InputStreamReader; import java.io.OutputStreamWriter; import java.nio.charset.StandardCharsets; +import java.util.Optional; +import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.Future; -import java.util.concurrent.ExecutorService; import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; -import java.util.Optional; /** * ExternalCommandWorker starts an external process to run a Trogdor command. diff --git a/trogdor/src/main/java/org/apache/kafka/trogdor/workload/FlushGenerator.java b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/FlushGenerator.java index a0cea563524da..0dc55d45208c2 100644 --- a/trogdor/src/main/java/org/apache/kafka/trogdor/workload/FlushGenerator.java +++ b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/FlushGenerator.java @@ -17,9 +17,10 @@ package org.apache.kafka.trogdor.workload; +import org.apache.kafka.clients.producer.KafkaProducer; + import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonTypeInfo; -import org.apache.kafka.clients.producer.KafkaProducer; /** * This interface is used to facilitate flushing the KafkaProducers on a cadence specified by the user. diff --git a/trogdor/src/main/java/org/apache/kafka/trogdor/workload/GaussianFlushGenerator.java b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/GaussianFlushGenerator.java index eb6845e38046e..9e9fe4ae3cc34 100644 --- a/trogdor/src/main/java/org/apache/kafka/trogdor/workload/GaussianFlushGenerator.java +++ b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/GaussianFlushGenerator.java @@ -17,10 +17,12 @@ package org.apache.kafka.trogdor.workload; -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; import org.apache.kafka.clients.producer.KafkaProducer; import org.apache.kafka.common.errors.InterruptException; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + import java.util.Random; /** diff --git a/trogdor/src/main/java/org/apache/kafka/trogdor/workload/GaussianThroughputGenerator.java b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/GaussianThroughputGenerator.java index a77298f0be8b0..5e064f58e417a 100644 --- a/trogdor/src/main/java/org/apache/kafka/trogdor/workload/GaussianThroughputGenerator.java +++ b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/GaussianThroughputGenerator.java @@ -17,9 +17,11 @@ package org.apache.kafka.trogdor.workload; +import org.apache.kafka.common.utils.Time; + import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import org.apache.kafka.common.utils.Time; + import java.util.Random; /* diff --git a/trogdor/src/main/java/org/apache/kafka/trogdor/workload/GaussianTimestampConstantPayloadGenerator.java b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/GaussianTimestampConstantPayloadGenerator.java index 8660ed329a293..fef98d619ed86 100644 --- a/trogdor/src/main/java/org/apache/kafka/trogdor/workload/GaussianTimestampConstantPayloadGenerator.java +++ b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/GaussianTimestampConstantPayloadGenerator.java @@ -17,9 +17,10 @@ package org.apache.kafka.trogdor.workload; +import org.apache.kafka.common.utils.Time; + import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import org.apache.kafka.common.utils.Time; import java.nio.ByteBuffer; import java.nio.ByteOrder; diff --git a/trogdor/src/main/java/org/apache/kafka/trogdor/workload/GaussianTimestampRandomPayloadGenerator.java b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/GaussianTimestampRandomPayloadGenerator.java index 48261a454f32a..6b7f38047cb6c 100644 --- a/trogdor/src/main/java/org/apache/kafka/trogdor/workload/GaussianTimestampRandomPayloadGenerator.java +++ b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/GaussianTimestampRandomPayloadGenerator.java @@ -17,9 +17,10 @@ package org.apache.kafka.trogdor.workload; +import org.apache.kafka.common.utils.Time; + import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import org.apache.kafka.common.utils.Time; import java.nio.ByteBuffer; import java.nio.ByteOrder; diff --git a/trogdor/src/main/java/org/apache/kafka/trogdor/workload/PartitionsSpec.java b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/PartitionsSpec.java index c1dc7c68761e8..1bd2f788a3017 100644 --- a/trogdor/src/main/java/org/apache/kafka/trogdor/workload/PartitionsSpec.java +++ b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/PartitionsSpec.java @@ -17,11 +17,12 @@ package org.apache.kafka.trogdor.workload; -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; import org.apache.kafka.clients.admin.NewTopic; import org.apache.kafka.trogdor.rest.Message; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; diff --git a/trogdor/src/main/java/org/apache/kafka/trogdor/workload/ProduceBenchSpec.java b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/ProduceBenchSpec.java index 540ae763e6677..653fa9cddb6f4 100644 --- a/trogdor/src/main/java/org/apache/kafka/trogdor/workload/ProduceBenchSpec.java +++ b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/ProduceBenchSpec.java @@ -17,12 +17,13 @@ package org.apache.kafka.trogdor.workload; -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; import org.apache.kafka.trogdor.task.TaskController; import org.apache.kafka.trogdor.task.TaskSpec; import org.apache.kafka.trogdor.task.TaskWorker; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + import java.util.Collections; import java.util.Map; import java.util.Optional; diff --git a/trogdor/src/main/java/org/apache/kafka/trogdor/workload/ProduceBenchWorker.java b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/ProduceBenchWorker.java index d1f6ad75f6533..7cdcf2e45df4f 100644 --- a/trogdor/src/main/java/org/apache/kafka/trogdor/workload/ProduceBenchWorker.java +++ b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/ProduceBenchWorker.java @@ -17,9 +17,6 @@ package org.apache.kafka.trogdor.workload; -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import com.fasterxml.jackson.databind.node.TextNode; import org.apache.kafka.clients.admin.NewTopic; import org.apache.kafka.clients.producer.Callback; import org.apache.kafka.clients.producer.KafkaProducer; @@ -37,6 +34,11 @@ import org.apache.kafka.trogdor.task.TaskWorker; import org.apache.kafka.trogdor.task.WorkerStatusTracker; import org.apache.kafka.trogdor.workload.TransactionGenerator.TransactionAction; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.databind.node.TextNode; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/trogdor/src/main/java/org/apache/kafka/trogdor/workload/RandomComponent.java b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/RandomComponent.java index b5973a8c74443..5f5da5735dea0 100644 --- a/trogdor/src/main/java/org/apache/kafka/trogdor/workload/RandomComponent.java +++ b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/RandomComponent.java @@ -46,4 +46,3 @@ public PayloadGenerator component() { return component; } } - diff --git a/trogdor/src/main/java/org/apache/kafka/trogdor/workload/RecordProcessor.java b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/RecordProcessor.java index 451ce1271c915..39a2f8b56b623 100644 --- a/trogdor/src/main/java/org/apache/kafka/trogdor/workload/RecordProcessor.java +++ b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/RecordProcessor.java @@ -17,10 +17,11 @@ package org.apache.kafka.trogdor.workload; +import org.apache.kafka.clients.consumer.ConsumerRecords; + import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.fasterxml.jackson.databind.JsonNode; -import org.apache.kafka.clients.consumer.ConsumerRecords; /** * RecordProcessor allows for acting on data polled from ConsumeBench workloads. @@ -33,4 +34,3 @@ public interface RecordProcessor { void processRecords(ConsumerRecords consumerRecords); JsonNode processorStatus(); } - diff --git a/trogdor/src/main/java/org/apache/kafka/trogdor/workload/RoundTripWorker.java b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/RoundTripWorker.java index 241bda0c5d066..385e14162ee83 100644 --- a/trogdor/src/main/java/org/apache/kafka/trogdor/workload/RoundTripWorker.java +++ b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/RoundTripWorker.java @@ -17,9 +17,6 @@ package org.apache.kafka.trogdor.workload; -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import com.fasterxml.jackson.databind.node.TextNode; import org.apache.kafka.clients.admin.NewTopic; import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.clients.consumer.ConsumerRecord; @@ -43,6 +40,11 @@ import org.apache.kafka.trogdor.common.WorkerUtils; import org.apache.kafka.trogdor.task.TaskWorker; import org.apache.kafka.trogdor.task.WorkerStatusTracker; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.databind.node.TextNode; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/trogdor/src/main/java/org/apache/kafka/trogdor/workload/RoundTripWorkloadSpec.java b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/RoundTripWorkloadSpec.java index 296c47f23d3ef..1000fb32ead42 100644 --- a/trogdor/src/main/java/org/apache/kafka/trogdor/workload/RoundTripWorkloadSpec.java +++ b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/RoundTripWorkloadSpec.java @@ -17,12 +17,13 @@ package org.apache.kafka.trogdor.workload; -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; import org.apache.kafka.trogdor.task.TaskController; import org.apache.kafka.trogdor.task.TaskSpec; import org.apache.kafka.trogdor.task.TaskWorker; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + import java.util.Collections; import java.util.Map; diff --git a/trogdor/src/main/java/org/apache/kafka/trogdor/workload/SustainedConnectionSpec.java b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/SustainedConnectionSpec.java index 94ce71eea12ac..516de304f2f11 100644 --- a/trogdor/src/main/java/org/apache/kafka/trogdor/workload/SustainedConnectionSpec.java +++ b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/SustainedConnectionSpec.java @@ -17,12 +17,13 @@ package org.apache.kafka.trogdor.workload; -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; import org.apache.kafka.trogdor.task.TaskController; import org.apache.kafka.trogdor.task.TaskSpec; import org.apache.kafka.trogdor.task.TaskWorker; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + import java.util.Collections; import java.util.Map; diff --git a/trogdor/src/main/java/org/apache/kafka/trogdor/workload/SustainedConnectionWorker.java b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/SustainedConnectionWorker.java index aad02d8f7deb6..9ac4ac68b35e8 100644 --- a/trogdor/src/main/java/org/apache/kafka/trogdor/workload/SustainedConnectionWorker.java +++ b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/SustainedConnectionWorker.java @@ -17,9 +17,6 @@ package org.apache.kafka.trogdor.workload; -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import com.fasterxml.jackson.databind.JsonNode; import org.apache.kafka.clients.admin.Admin; import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.clients.consumer.KafkaConsumer; @@ -38,6 +35,11 @@ import org.apache.kafka.trogdor.common.WorkerUtils; import org.apache.kafka.trogdor.task.TaskWorker; import org.apache.kafka.trogdor.task.WorkerStatusTracker; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.databind.JsonNode; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/trogdor/src/main/java/org/apache/kafka/trogdor/workload/TimeIntervalTransactionsGenerator.java b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/TimeIntervalTransactionsGenerator.java index 8d5f05b1ee001..4672bcafbf219 100644 --- a/trogdor/src/main/java/org/apache/kafka/trogdor/workload/TimeIntervalTransactionsGenerator.java +++ b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/TimeIntervalTransactionsGenerator.java @@ -16,9 +16,10 @@ */ package org.apache.kafka.trogdor.workload; +import org.apache.kafka.common.utils.Time; + import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import org.apache.kafka.common.utils.Time; /** * A transactions generator where we commit a transaction every N milliseconds diff --git a/trogdor/src/main/java/org/apache/kafka/trogdor/workload/TimestampConstantPayloadGenerator.java b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/TimestampConstantPayloadGenerator.java index e9c4bc8a54068..a2f6793a3248b 100644 --- a/trogdor/src/main/java/org/apache/kafka/trogdor/workload/TimestampConstantPayloadGenerator.java +++ b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/TimestampConstantPayloadGenerator.java @@ -17,12 +17,13 @@ package org.apache.kafka.trogdor.workload; +import org.apache.kafka.common.utils.Time; + import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import org.apache.kafka.common.utils.Time; -import java.nio.ByteOrder; import java.nio.ByteBuffer; +import java.nio.ByteOrder; /** * A PayloadGenerator which generates a timestamped constant payload. diff --git a/trogdor/src/main/java/org/apache/kafka/trogdor/workload/TimestampRandomPayloadGenerator.java b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/TimestampRandomPayloadGenerator.java index d2cfdc702b500..6cefefc9b7168 100644 --- a/trogdor/src/main/java/org/apache/kafka/trogdor/workload/TimestampRandomPayloadGenerator.java +++ b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/TimestampRandomPayloadGenerator.java @@ -17,13 +17,14 @@ package org.apache.kafka.trogdor.workload; +import org.apache.kafka.common.utils.Time; + import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import org.apache.kafka.common.utils.Time; +import java.nio.ByteBuffer; import java.nio.ByteOrder; import java.util.Random; -import java.nio.ByteBuffer; /** * A PayloadGenerator which generates a timestamped uniform random payload. diff --git a/trogdor/src/main/java/org/apache/kafka/trogdor/workload/TimestampRecordProcessor.java b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/TimestampRecordProcessor.java index 035d45928398a..edbcefc4b8c34 100644 --- a/trogdor/src/main/java/org/apache/kafka/trogdor/workload/TimestampRecordProcessor.java +++ b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/TimestampRecordProcessor.java @@ -17,13 +17,14 @@ package org.apache.kafka.trogdor.workload; -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import com.fasterxml.jackson.databind.JsonNode; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.consumer.ConsumerRecords; -import org.apache.kafka.trogdor.common.JsonUtil; import org.apache.kafka.common.utils.Time; +import org.apache.kafka.trogdor.common.JsonUtil; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.databind.JsonNode; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/trogdor/src/main/java/org/apache/kafka/trogdor/workload/TopicsSpec.java b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/TopicsSpec.java index dcb8d8ad50b11..600fd46f21fcc 100644 --- a/trogdor/src/main/java/org/apache/kafka/trogdor/workload/TopicsSpec.java +++ b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/TopicsSpec.java @@ -17,11 +17,12 @@ package org.apache.kafka.trogdor.workload; +import org.apache.kafka.trogdor.common.StringExpander; +import org.apache.kafka.trogdor.rest.Message; + import com.fasterxml.jackson.annotation.JsonAnyGetter; import com.fasterxml.jackson.annotation.JsonAnySetter; import com.fasterxml.jackson.annotation.JsonCreator; -import org.apache.kafka.trogdor.common.StringExpander; -import org.apache.kafka.trogdor.rest.Message; import java.util.Collections; import java.util.HashMap; diff --git a/trogdor/src/test/java/org/apache/kafka/trogdor/agent/AgentTest.java b/trogdor/src/test/java/org/apache/kafka/trogdor/agent/AgentTest.java index 652780ca53f9e..070de473056e7 100644 --- a/trogdor/src/test/java/org/apache/kafka/trogdor/agent/AgentTest.java +++ b/trogdor/src/test/java/org/apache/kafka/trogdor/agent/AgentTest.java @@ -17,12 +17,6 @@ package org.apache.kafka.trogdor.agent; -import static java.util.Arrays.asList; -import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertNotEquals; -import static org.junit.jupiter.api.Assertions.assertThrows; - -import com.fasterxml.jackson.databind.node.TextNode; import org.apache.kafka.common.utils.MockScheduler; import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.common.utils.Scheduler; @@ -53,8 +47,12 @@ import org.apache.kafka.trogdor.task.NoOpTaskSpec; import org.apache.kafka.trogdor.task.SampleTaskSpec; import org.apache.kafka.trogdor.task.TaskSpec; + +import com.fasterxml.jackson.databind.node.TextNode; + import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import java.io.ByteArrayOutputStream; import java.io.File; @@ -66,7 +64,11 @@ import java.util.Collections; import java.util.HashMap; import java.util.TreeMap; -import org.junit.jupiter.api.Timeout; + +import static java.util.Arrays.asList; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotEquals; +import static org.junit.jupiter.api.Assertions.assertThrows; @Timeout(value = 120) public class AgentTest { diff --git a/trogdor/src/test/java/org/apache/kafka/trogdor/basic/BasicPlatformTest.java b/trogdor/src/test/java/org/apache/kafka/trogdor/basic/BasicPlatformTest.java index 7277fb6db4a4c..eaba1124fd60f 100644 --- a/trogdor/src/test/java/org/apache/kafka/trogdor/basic/BasicPlatformTest.java +++ b/trogdor/src/test/java/org/apache/kafka/trogdor/basic/BasicPlatformTest.java @@ -17,17 +17,18 @@ package org.apache.kafka.trogdor.basic; -import static org.junit.jupiter.api.Assertions.assertEquals; import org.apache.kafka.test.TestUtils; import org.apache.kafka.trogdor.common.Platform; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import java.io.File; import java.io.OutputStreamWriter; import java.nio.charset.StandardCharsets; import java.nio.file.Files; -import org.junit.jupiter.api.Timeout; + +import static org.junit.jupiter.api.Assertions.assertEquals; @Timeout(value = 120) public class BasicPlatformTest { diff --git a/trogdor/src/test/java/org/apache/kafka/trogdor/common/CapturingCommandRunner.java b/trogdor/src/test/java/org/apache/kafka/trogdor/common/CapturingCommandRunner.java index 654f14b26a03b..d05c73b7ebf00 100644 --- a/trogdor/src/test/java/org/apache/kafka/trogdor/common/CapturingCommandRunner.java +++ b/trogdor/src/test/java/org/apache/kafka/trogdor/common/CapturingCommandRunner.java @@ -18,6 +18,7 @@ package org.apache.kafka.trogdor.common; import org.apache.kafka.trogdor.basic.BasicPlatform; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/trogdor/src/test/java/org/apache/kafka/trogdor/common/ExpectedTasks.java b/trogdor/src/test/java/org/apache/kafka/trogdor/common/ExpectedTasks.java index 3fdffd5adceed..87f818018c80e 100644 --- a/trogdor/src/test/java/org/apache/kafka/trogdor/common/ExpectedTasks.java +++ b/trogdor/src/test/java/org/apache/kafka/trogdor/common/ExpectedTasks.java @@ -17,8 +17,6 @@ package org.apache.kafka.trogdor.common; -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; import org.apache.kafka.test.TestUtils; import org.apache.kafka.trogdor.agent.AgentClient; import org.apache.kafka.trogdor.coordinator.CoordinatorClient; @@ -28,6 +26,10 @@ import org.apache.kafka.trogdor.rest.TasksResponse; import org.apache.kafka.trogdor.rest.WorkerState; import org.apache.kafka.trogdor.task.TaskSpec; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/trogdor/src/test/java/org/apache/kafka/trogdor/common/JsonSerializationTest.java b/trogdor/src/test/java/org/apache/kafka/trogdor/common/JsonSerializationTest.java index 584acbae7faf9..4605e66ef1783 100644 --- a/trogdor/src/test/java/org/apache/kafka/trogdor/common/JsonSerializationTest.java +++ b/trogdor/src/test/java/org/apache/kafka/trogdor/common/JsonSerializationTest.java @@ -17,8 +17,6 @@ package org.apache.kafka.trogdor.common; -import static org.junit.jupiter.api.Assertions.assertNotNull; - import org.apache.kafka.trogdor.fault.FilesUnreadableFaultSpec; import org.apache.kafka.trogdor.fault.Kibosh; import org.apache.kafka.trogdor.fault.NetworkPartitionFaultSpec; @@ -33,13 +31,16 @@ import org.apache.kafka.trogdor.workload.RoundTripWorkloadSpec; import org.apache.kafka.trogdor.workload.TopicsSpec; +import org.junit.jupiter.api.Test; + import java.lang.reflect.Field; import java.util.Arrays; import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Optional; -import org.junit.jupiter.api.Test; + +import static org.junit.jupiter.api.Assertions.assertNotNull; public class JsonSerializationTest { @Test diff --git a/trogdor/src/test/java/org/apache/kafka/trogdor/common/JsonUtilTest.java b/trogdor/src/test/java/org/apache/kafka/trogdor/common/JsonUtilTest.java index 922222df655f3..85059bb2b4af0 100644 --- a/trogdor/src/test/java/org/apache/kafka/trogdor/common/JsonUtilTest.java +++ b/trogdor/src/test/java/org/apache/kafka/trogdor/common/JsonUtilTest.java @@ -17,13 +17,11 @@ package org.apache.kafka.trogdor.common; -import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertFalse; -import static org.junit.jupiter.api.Assertions.assertTrue; +import org.apache.kafka.test.TestUtils; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import org.apache.kafka.test.TestUtils; + import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; @@ -31,6 +29,10 @@ import java.nio.charset.StandardCharsets; import java.nio.file.Files; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; + @Timeout(value = 120) public class JsonUtilTest { diff --git a/trogdor/src/test/java/org/apache/kafka/trogdor/common/MiniTrogdorCluster.java b/trogdor/src/test/java/org/apache/kafka/trogdor/common/MiniTrogdorCluster.java index 7f5cfc998ce9f..64e8c5def1ce3 100644 --- a/trogdor/src/test/java/org/apache/kafka/trogdor/common/MiniTrogdorCluster.java +++ b/trogdor/src/test/java/org/apache/kafka/trogdor/common/MiniTrogdorCluster.java @@ -26,10 +26,10 @@ import org.apache.kafka.trogdor.basic.BasicPlatform; import org.apache.kafka.trogdor.basic.BasicTopology; import org.apache.kafka.trogdor.coordinator.Coordinator; - import org.apache.kafka.trogdor.coordinator.CoordinatorClient; import org.apache.kafka.trogdor.coordinator.CoordinatorRestResource; import org.apache.kafka.trogdor.rest.JsonRestServer; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/trogdor/src/test/java/org/apache/kafka/trogdor/common/StringExpanderTest.java b/trogdor/src/test/java/org/apache/kafka/trogdor/common/StringExpanderTest.java index 22da4987c3015..0464db269a113 100644 --- a/trogdor/src/test/java/org/apache/kafka/trogdor/common/StringExpanderTest.java +++ b/trogdor/src/test/java/org/apache/kafka/trogdor/common/StringExpanderTest.java @@ -17,14 +17,14 @@ package org.apache.kafka.trogdor.common; -import static org.junit.jupiter.api.Assertions.assertEquals; - import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import java.util.Arrays; import java.util.Collections; import java.util.HashSet; -import org.junit.jupiter.api.Timeout; + +import static org.junit.jupiter.api.Assertions.assertEquals; @Timeout(value = 120) public class StringExpanderTest { diff --git a/trogdor/src/test/java/org/apache/kafka/trogdor/common/StringFormatterTest.java b/trogdor/src/test/java/org/apache/kafka/trogdor/common/StringFormatterTest.java index f950dbcaf032a..2c78f61aac0a9 100644 --- a/trogdor/src/test/java/org/apache/kafka/trogdor/common/StringFormatterTest.java +++ b/trogdor/src/test/java/org/apache/kafka/trogdor/common/StringFormatterTest.java @@ -17,14 +17,14 @@ package org.apache.kafka.trogdor.common; -import org.junit.jupiter.api.Timeout; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import java.time.ZoneOffset; import java.util.Arrays; -import static org.apache.kafka.trogdor.common.StringFormatter.durationString; import static org.apache.kafka.trogdor.common.StringFormatter.dateString; +import static org.apache.kafka.trogdor.common.StringFormatter.durationString; import static org.junit.jupiter.api.Assertions.assertEquals; @Timeout(value = 120) diff --git a/trogdor/src/test/java/org/apache/kafka/trogdor/common/TopologyTest.java b/trogdor/src/test/java/org/apache/kafka/trogdor/common/TopologyTest.java index 7c1d695ddaa9f..3acca80baa935 100644 --- a/trogdor/src/test/java/org/apache/kafka/trogdor/common/TopologyTest.java +++ b/trogdor/src/test/java/org/apache/kafka/trogdor/common/TopologyTest.java @@ -17,22 +17,22 @@ package org.apache.kafka.trogdor.common; -import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertTrue; - import org.apache.kafka.trogdor.agent.Agent; import org.apache.kafka.trogdor.basic.BasicNode; import org.apache.kafka.trogdor.basic.BasicTopology; - import org.apache.kafka.trogdor.coordinator.Coordinator; + import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import java.util.HashMap; import java.util.HashSet; import java.util.Map; import java.util.Set; import java.util.TreeMap; -import org.junit.jupiter.api.Timeout; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; @Timeout(value = 120) public class TopologyTest { diff --git a/trogdor/src/test/java/org/apache/kafka/trogdor/common/WorkerUtilsTest.java b/trogdor/src/test/java/org/apache/kafka/trogdor/common/WorkerUtilsTest.java index e48590450d52d..a826bcf3988ae 100644 --- a/trogdor/src/test/java/org/apache/kafka/trogdor/common/WorkerUtilsTest.java +++ b/trogdor/src/test/java/org/apache/kafka/trogdor/common/WorkerUtilsTest.java @@ -17,9 +17,6 @@ package org.apache.kafka.trogdor.common; -import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertThrows; - import org.apache.kafka.clients.admin.MockAdminClient; import org.apache.kafka.clients.admin.NewTopic; import org.apache.kafka.clients.admin.TopicDescription; @@ -30,6 +27,7 @@ import org.apache.kafka.common.errors.TopicExistsException; import org.apache.kafka.common.errors.UnknownTopicOrPartitionException; import org.apache.kafka.common.utils.Utils; + import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.slf4j.Logger; @@ -45,6 +43,9 @@ import java.util.Map; import java.util.Properties; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertThrows; + public class WorkerUtilsTest { private static final Logger log = LoggerFactory.getLogger(WorkerUtilsTest.class); diff --git a/trogdor/src/test/java/org/apache/kafka/trogdor/coordinator/CoordinatorClientTest.java b/trogdor/src/test/java/org/apache/kafka/trogdor/coordinator/CoordinatorClientTest.java index d5f5d86c1ff9c..12449b185a2bd 100644 --- a/trogdor/src/test/java/org/apache/kafka/trogdor/coordinator/CoordinatorClientTest.java +++ b/trogdor/src/test/java/org/apache/kafka/trogdor/coordinator/CoordinatorClientTest.java @@ -17,17 +17,19 @@ package org.apache.kafka.trogdor.coordinator; -import com.fasterxml.jackson.databind.node.JsonNodeFactory; import org.apache.kafka.trogdor.rest.TaskDone; import org.apache.kafka.trogdor.rest.TaskPending; import org.apache.kafka.trogdor.rest.TaskRunning; import org.apache.kafka.trogdor.rest.TaskStopping; import org.apache.kafka.trogdor.task.NoOpTaskSpec; -import java.time.ZoneOffset; +import com.fasterxml.jackson.databind.node.JsonNodeFactory; + import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; +import java.time.ZoneOffset; + import static org.junit.jupiter.api.Assertions.assertEquals; @Timeout(value = 120) diff --git a/trogdor/src/test/java/org/apache/kafka/trogdor/coordinator/CoordinatorTest.java b/trogdor/src/test/java/org/apache/kafka/trogdor/coordinator/CoordinatorTest.java index 9a84f1aa2e4d4..849e2713d9e85 100644 --- a/trogdor/src/test/java/org/apache/kafka/trogdor/coordinator/CoordinatorTest.java +++ b/trogdor/src/test/java/org/apache/kafka/trogdor/coordinator/CoordinatorTest.java @@ -17,9 +17,6 @@ package org.apache.kafka.trogdor.coordinator; -import com.fasterxml.jackson.databind.node.JsonNodeFactory; -import com.fasterxml.jackson.databind.node.ObjectNode; -import com.fasterxml.jackson.databind.node.TextNode; import org.apache.kafka.common.utils.MockScheduler; import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.common.utils.Scheduler; @@ -48,19 +45,25 @@ import org.apache.kafka.trogdor.rest.WorkerRunning; import org.apache.kafka.trogdor.task.NoOpTaskSpec; import org.apache.kafka.trogdor.task.SampleTaskSpec; + +import com.fasterxml.jackson.databind.node.JsonNodeFactory; +import com.fasterxml.jackson.databind.node.ObjectNode; +import com.fasterxml.jackson.databind.node.TextNode; + import org.junit.jupiter.api.Tag; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import javax.ws.rs.NotFoundException; import java.util.ArrayList; import java.util.Arrays; import java.util.HashMap; import java.util.List; import java.util.Optional; +import javax.ws.rs.NotFoundException; + import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertNotEquals; diff --git a/trogdor/src/test/java/org/apache/kafka/trogdor/rest/RestExceptionMapperTest.java b/trogdor/src/test/java/org/apache/kafka/trogdor/rest/RestExceptionMapperTest.java index ed3d1ed34ca1c..e446fec68b701 100644 --- a/trogdor/src/test/java/org/apache/kafka/trogdor/rest/RestExceptionMapperTest.java +++ b/trogdor/src/test/java/org/apache/kafka/trogdor/rest/RestExceptionMapperTest.java @@ -16,19 +16,21 @@ */ package org.apache.kafka.trogdor.rest; -import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertThrows; +import org.apache.kafka.common.errors.InvalidRequestException; +import org.apache.kafka.common.errors.SerializationException; import com.fasterxml.jackson.core.JsonParser; import com.fasterxml.jackson.databind.JavaType; import com.fasterxml.jackson.databind.JsonMappingException; import com.fasterxml.jackson.databind.exc.InvalidTypeIdException; + +import org.junit.jupiter.api.Test; + import javax.ws.rs.NotFoundException; import javax.ws.rs.core.Response; -import org.apache.kafka.common.errors.InvalidRequestException; -import org.apache.kafka.common.errors.SerializationException; -import org.junit.jupiter.api.Test; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertThrows; public class RestExceptionMapperTest { diff --git a/trogdor/src/test/java/org/apache/kafka/trogdor/task/SampleTaskWorker.java b/trogdor/src/test/java/org/apache/kafka/trogdor/task/SampleTaskWorker.java index 9d4fa916986f5..f3bd2a22fe12b 100644 --- a/trogdor/src/test/java/org/apache/kafka/trogdor/task/SampleTaskWorker.java +++ b/trogdor/src/test/java/org/apache/kafka/trogdor/task/SampleTaskWorker.java @@ -17,11 +17,12 @@ package org.apache.kafka.trogdor.task; -import com.fasterxml.jackson.databind.node.TextNode; import org.apache.kafka.common.internals.KafkaFutureImpl; import org.apache.kafka.common.utils.ThreadUtils; import org.apache.kafka.trogdor.common.Platform; +import com.fasterxml.jackson.databind.node.TextNode; + import java.util.concurrent.Executors; import java.util.concurrent.Future; import java.util.concurrent.ScheduledExecutorService; diff --git a/trogdor/src/test/java/org/apache/kafka/trogdor/task/TaskSpecTest.java b/trogdor/src/test/java/org/apache/kafka/trogdor/task/TaskSpecTest.java index ad9189182358f..2593afa74905f 100644 --- a/trogdor/src/test/java/org/apache/kafka/trogdor/task/TaskSpecTest.java +++ b/trogdor/src/test/java/org/apache/kafka/trogdor/task/TaskSpecTest.java @@ -17,8 +17,10 @@ package org.apache.kafka.trogdor.task; -import com.fasterxml.jackson.databind.exc.InvalidTypeIdException; import org.apache.kafka.trogdor.common.JsonUtil; + +import com.fasterxml.jackson.databind.exc.InvalidTypeIdException; + import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; diff --git a/trogdor/src/test/java/org/apache/kafka/trogdor/workload/ConsumeBenchSpecTest.java b/trogdor/src/test/java/org/apache/kafka/trogdor/workload/ConsumeBenchSpecTest.java index a0e3eb0dd15d7..e07991d3bf021 100644 --- a/trogdor/src/test/java/org/apache/kafka/trogdor/workload/ConsumeBenchSpecTest.java +++ b/trogdor/src/test/java/org/apache/kafka/trogdor/workload/ConsumeBenchSpecTest.java @@ -17,22 +17,23 @@ package org.apache.kafka.trogdor.workload; -import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertThrows; - import org.apache.kafka.common.TopicPartition; + import org.junit.jupiter.api.Test; -import java.util.Arrays; 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.HashMap; import java.util.Optional; import java.util.stream.Collectors; import java.util.stream.IntStream; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertThrows; + public class ConsumeBenchSpecTest { @Test diff --git a/trogdor/src/test/java/org/apache/kafka/trogdor/workload/ExternalCommandWorkerTest.java b/trogdor/src/test/java/org/apache/kafka/trogdor/workload/ExternalCommandWorkerTest.java index b57fd59a8670c..852dd3c3b473b 100644 --- a/trogdor/src/test/java/org/apache/kafka/trogdor/workload/ExternalCommandWorkerTest.java +++ b/trogdor/src/test/java/org/apache/kafka/trogdor/workload/ExternalCommandWorkerTest.java @@ -17,16 +17,20 @@ package org.apache.kafka.trogdor.workload; -import com.fasterxml.jackson.databind.node.IntNode; -import com.fasterxml.jackson.databind.node.JsonNodeFactory; -import com.fasterxml.jackson.databind.node.ObjectNode; -import com.fasterxml.jackson.databind.node.TextNode; import org.apache.kafka.common.internals.KafkaFutureImpl; import org.apache.kafka.common.utils.OperatingSystem; import org.apache.kafka.test.TestUtils; import org.apache.kafka.trogdor.task.AgentWorkerStatusTracker; import org.apache.kafka.trogdor.task.WorkerStatusTracker; +import com.fasterxml.jackson.databind.node.IntNode; +import com.fasterxml.jackson.databind.node.JsonNodeFactory; +import com.fasterxml.jackson.databind.node.ObjectNode; +import com.fasterxml.jackson.databind.node.TextNode; + +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; + import java.io.File; import java.io.OutputStream; import java.nio.charset.StandardCharsets; @@ -34,8 +38,6 @@ import java.util.Arrays; import java.util.Optional; import java.util.concurrent.CompletableFuture; -import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.Timeout; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertTrue; diff --git a/trogdor/src/test/java/org/apache/kafka/trogdor/workload/HistogramTest.java b/trogdor/src/test/java/org/apache/kafka/trogdor/workload/HistogramTest.java index 1afaf7587d01c..7002a4106a499 100644 --- a/trogdor/src/test/java/org/apache/kafka/trogdor/workload/HistogramTest.java +++ b/trogdor/src/test/java/org/apache/kafka/trogdor/workload/HistogramTest.java @@ -17,10 +17,10 @@ package org.apache.kafka.trogdor.workload; -import static org.junit.jupiter.api.Assertions.assertEquals; - import org.junit.jupiter.api.Test; +import static org.junit.jupiter.api.Assertions.assertEquals; + public class HistogramTest { private static Histogram createHistogram(int maxValue, int... values) { Histogram histogram = new Histogram(maxValue); @@ -83,4 +83,3 @@ public void testHistogramPercentiles() { assertEquals(200, summary.percentiles().get(2).value()); } } - diff --git a/trogdor/src/test/java/org/apache/kafka/trogdor/workload/PayloadGeneratorTest.java b/trogdor/src/test/java/org/apache/kafka/trogdor/workload/PayloadGeneratorTest.java index bcb08db9eaba5..999cc51ef7656 100644 --- a/trogdor/src/test/java/org/apache/kafka/trogdor/workload/PayloadGeneratorTest.java +++ b/trogdor/src/test/java/org/apache/kafka/trogdor/workload/PayloadGeneratorTest.java @@ -17,18 +17,19 @@ package org.apache.kafka.trogdor.workload; -import static org.junit.jupiter.api.Assertions.assertArrayEquals; -import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertNull; -import static org.junit.jupiter.api.Assertions.assertThrows; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import java.nio.ByteBuffer; import java.nio.ByteOrder; -import java.util.Arrays; import java.util.ArrayList; +import java.util.Arrays; import java.util.List; -import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.Timeout; + +import static org.junit.jupiter.api.Assertions.assertArrayEquals; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNull; +import static org.junit.jupiter.api.Assertions.assertThrows; @Timeout(value = 120) public class PayloadGeneratorTest { diff --git a/trogdor/src/test/java/org/apache/kafka/trogdor/workload/ThrottleTest.java b/trogdor/src/test/java/org/apache/kafka/trogdor/workload/ThrottleTest.java index 1fef2ae4a64a5..32801680358b6 100644 --- a/trogdor/src/test/java/org/apache/kafka/trogdor/workload/ThrottleTest.java +++ b/trogdor/src/test/java/org/apache/kafka/trogdor/workload/ThrottleTest.java @@ -17,14 +17,15 @@ package org.apache.kafka.trogdor.workload; -import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertFalse; -import static org.junit.jupiter.api.Assertions.assertTrue; - import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.common.utils.Time; + import org.junit.jupiter.api.Test; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; + public class ThrottleTest { /** * ThrottleMock is a subclass of Throttle that uses a MockTime object. It calls @@ -70,4 +71,3 @@ public void testThrottle() throws Exception { assertEquals(200, time.milliseconds()); } } - diff --git a/trogdor/src/test/java/org/apache/kafka/trogdor/workload/TimeIntervalTransactionsGeneratorTest.java b/trogdor/src/test/java/org/apache/kafka/trogdor/workload/TimeIntervalTransactionsGeneratorTest.java index a5fa5903697a9..877d4e2aed7a9 100644 --- a/trogdor/src/test/java/org/apache/kafka/trogdor/workload/TimeIntervalTransactionsGeneratorTest.java +++ b/trogdor/src/test/java/org/apache/kafka/trogdor/workload/TimeIntervalTransactionsGeneratorTest.java @@ -17,11 +17,12 @@ package org.apache.kafka.trogdor.workload; -import static org.junit.jupiter.api.Assertions.assertEquals; - import org.apache.kafka.common.utils.MockTime; + import org.junit.jupiter.api.Test; +import static org.junit.jupiter.api.Assertions.assertEquals; + public class TimeIntervalTransactionsGeneratorTest { @Test public void testCommitsTransactionAfterIntervalPasses() { diff --git a/trogdor/src/test/java/org/apache/kafka/trogdor/workload/TopicsSpecTest.java b/trogdor/src/test/java/org/apache/kafka/trogdor/workload/TopicsSpecTest.java index f1c50d7a3a5a7..16d14659a4c3a 100644 --- a/trogdor/src/test/java/org/apache/kafka/trogdor/workload/TopicsSpecTest.java +++ b/trogdor/src/test/java/org/apache/kafka/trogdor/workload/TopicsSpecTest.java @@ -17,17 +17,18 @@ package org.apache.kafka.trogdor.workload; -import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertTrue; +import org.apache.kafka.trogdor.common.JsonUtil; + +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import java.util.Arrays; import java.util.HashMap; import java.util.List; import java.util.Map; -import org.apache.kafka.trogdor.common.JsonUtil; -import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.Timeout; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; @Timeout(value = 120) public class TopicsSpecTest { From fc83e492d7b1a8e57a46a3048fd60fa23e11b8cc Mon Sep 17 00:00:00 2001 From: Antoine Pourchet Date: Wed, 12 Jun 2024 10:10:29 -0600 Subject: [PATCH 4/5] part 22 fixes --- .../assignment/AssignmentConfigs.java | 24 ++----------------- .../assignment/TaskAssignmentUtils.java | 8 +++---- .../assignors/StickyTaskAssignor.java | 23 ++++++++++++++---- 3 files changed, 24 insertions(+), 31 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/assignment/AssignmentConfigs.java b/streams/src/main/java/org/apache/kafka/streams/processor/assignment/AssignmentConfigs.java index 6a7ca68a50f72..abd9d50b8f417 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/assignment/AssignmentConfigs.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/assignment/AssignmentConfigs.java @@ -20,8 +20,6 @@ import java.util.OptionalInt; import org.apache.kafka.common.config.ConfigDef; import org.apache.kafka.streams.StreamsConfig; -import org.apache.kafka.streams.processor.assignment.assignors.StickyTaskAssignor; -import org.apache.kafka.streams.processor.internals.assignment.HighAvailabilityTaskAssignor; /** * Assignment related configs for the Kafka Streams {@link TaskAssignor}. @@ -43,26 +41,8 @@ public static AssignmentConfigs of(final StreamsConfig configs) { final long probingRebalanceIntervalMs = configs.getLong(StreamsConfig.PROBING_REBALANCE_INTERVAL_MS_CONFIG); final List rackAwareAssignmentTags = configs.getList(StreamsConfig.RACK_AWARE_ASSIGNMENT_TAGS_CONFIG); final String rackAwareAssignmentStrategy = configs.getString(StreamsConfig.RACK_AWARE_ASSIGNMENT_STRATEGY_CONFIG); - Integer rackAwareTrafficCost = configs.getInt(StreamsConfig.RACK_AWARE_ASSIGNMENT_TRAFFIC_COST_CONFIG); - Integer rackAwareNonOverlapCost = configs.getInt(StreamsConfig.RACK_AWARE_ASSIGNMENT_NON_OVERLAP_COST_CONFIG); - - final String assignorClassName = configs.getString(StreamsConfig.TASK_ASSIGNOR_CLASS_CONFIG); - if (StickyTaskAssignor.class.getName().equals(assignorClassName)) { - if (rackAwareTrafficCost == null) { - rackAwareTrafficCost = StickyTaskAssignor.DEFAULT_STICKY_TRAFFIC_COST; - } - if (rackAwareNonOverlapCost == null) { - rackAwareNonOverlapCost = StickyTaskAssignor.DEFAULT_STICKY_NON_OVERLAP_COST; - } - } else if (HighAvailabilityTaskAssignor.class.getName().equals(assignorClassName)) { - // TODO KAFKA-16869: replace with the HighAvailabilityTaskAssignor class once it implements the new TaskAssignor interface - if (rackAwareTrafficCost == null) { - rackAwareTrafficCost = HighAvailabilityTaskAssignor.DEFAULT_HIGH_AVAILABILITY_TRAFFIC_COST; - } - if (rackAwareNonOverlapCost == null) { - rackAwareNonOverlapCost = HighAvailabilityTaskAssignor.DEFAULT_HIGH_AVAILABILITY_NON_OVERLAP_COST; - } - } + final Integer rackAwareTrafficCost = configs.getInt(StreamsConfig.RACK_AWARE_ASSIGNMENT_TRAFFIC_COST_CONFIG); + final Integer rackAwareNonOverlapCost = configs.getInt(StreamsConfig.RACK_AWARE_ASSIGNMENT_NON_OVERLAP_COST_CONFIG); return new AssignmentConfigs( acceptableRecoveryLag, diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/assignment/TaskAssignmentUtils.java b/streams/src/main/java/org/apache/kafka/streams/processor/assignment/TaskAssignmentUtils.java index 39a698adfa516..9af8a10cbd6d8 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/assignment/TaskAssignmentUtils.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/assignment/TaskAssignmentUtils.java @@ -59,16 +59,16 @@ private TaskAssignmentUtils() {} * A simple config container for necessary parameters and optional overrides to apply when * running the active or standby task rack-aware optimizations. */ - public static class RackAwareOptimizationParams { + public static final class RackAwareOptimizationParams { private final ApplicationState applicationState; private final Optional trafficCostOverride; private final Optional nonOverlapCostOverride; private final Optional> tasksToOptimize; private RackAwareOptimizationParams(final ApplicationState applicationState, - final Optional trafficCostOverride, - final Optional nonOverlapCostOverride, - final Optional> tasksToOptimize) { + final Optional trafficCostOverride, + final Optional nonOverlapCostOverride, + final Optional> tasksToOptimize) { this.applicationState = applicationState; this.trafficCostOverride = trafficCostOverride; this.nonOverlapCostOverride = nonOverlapCostOverride; diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/assignment/assignors/StickyTaskAssignor.java b/streams/src/main/java/org/apache/kafka/streams/processor/assignment/assignors/StickyTaskAssignor.java index 3d5e5b247f776..fe01b502a22df 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/assignment/assignors/StickyTaskAssignor.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/assignment/assignors/StickyTaskAssignor.java @@ -95,10 +95,15 @@ private void optimizeActive(final ApplicationState applicationState, final Map currentAssignments = assignmentState.newAssignments; - TaskAssignmentUtils.optimizeRackAwareActiveTasks( - RackAwareOptimizationParams.of(applicationState).forStatefulTasks(), - currentAssignments - ); + final RackAwareOptimizationParams statefulTaskParams = RackAwareOptimizationParams.of(applicationState) + .withTrafficCostOverride( + applicationState.assignmentConfigs().rackAwareTrafficCost().orElse(DEFAULT_STICKY_TRAFFIC_COST) + ) + .withNonOverlapCostOverride( + applicationState.assignmentConfigs().rackAwareNonOverlapCost().orElse(DEFAULT_STICKY_NON_OVERLAP_COST) + ) + .forStatefulTasks(); + TaskAssignmentUtils.optimizeRackAwareActiveTasks(statefulTaskParams, currentAssignments); TaskAssignmentUtils.optimizeRackAwareActiveTasks( RackAwareOptimizationParams.of(applicationState) @@ -120,7 +125,15 @@ private void optimizeStandby(final ApplicationState applicationState, final Assi } final Map assignments = assignmentState.newAssignments; - TaskAssignmentUtils.optimizeRackAwareStandbyTasks(RackAwareOptimizationParams.of(applicationState), assignments); + + final RackAwareOptimizationParams optimizationParams = RackAwareOptimizationParams.of(applicationState) + .withTrafficCostOverride( + applicationState.assignmentConfigs().rackAwareTrafficCost().orElse(DEFAULT_STICKY_TRAFFIC_COST) + ) + .withNonOverlapCostOverride( + applicationState.assignmentConfigs().rackAwareNonOverlapCost().orElse(DEFAULT_STICKY_NON_OVERLAP_COST) + ); + TaskAssignmentUtils.optimizeRackAwareStandbyTasks(optimizationParams, assignments); assignmentState.processOptimizedAssignments(assignments); } From c37fafecc7bfa47086f3aebf7338ae66ced2c6b7 Mon Sep 17 00:00:00 2001 From: Antoine Pourchet Date: Wed, 12 Jun 2024 12:01:35 -0600 Subject: [PATCH 5/5] added ProcessId.randomProcessId function and cleaned up internals --- .../assignment/KafkaStreamsAssignment.java | 16 ++++++ .../processor/assignment/ProcessId.java | 8 +++ .../internals/StreamsPartitionAssignor.java | 3 +- .../processor/internals/TaskManagerTest.java | 3 +- .../KafkaStreamsAssignmentTest.java | 49 +++++++++++++++++++ .../assignment/KafkaStreamsStateTest.java | 5 +- .../assignment/TaskAssignmentUtilsTest.java | 3 +- 7 files changed, 78 insertions(+), 9 deletions(-) create mode 100644 streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/KafkaStreamsAssignmentTest.java diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/assignment/KafkaStreamsAssignment.java b/streams/src/main/java/org/apache/kafka/streams/processor/assignment/KafkaStreamsAssignment.java index f5205c8422bdf..848219d8c72dc 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/assignment/KafkaStreamsAssignment.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/assignment/KafkaStreamsAssignment.java @@ -19,6 +19,7 @@ import static java.util.Collections.unmodifiableMap; import java.time.Instant; +import java.util.Arrays; import java.util.Map; import java.util.Optional; import java.util.Set; @@ -108,6 +109,16 @@ public Optional followupRebalanceDeadline() { return followupRebalanceDeadline; } + @Override + public String toString() { + return String.format( + "KafkaStreamsAssignment{%s, %s, %s}", + processId, + Arrays.toString(tasks.values().toArray(new AssignedTask[0])), + followupRebalanceDeadline + ); + } + public static class AssignedTask { private final TaskId id; private final Type taskType; @@ -157,5 +168,10 @@ public boolean equals(final Object obj) { final AssignedTask other = (AssignedTask) obj; return this.id.equals(other.id()) && this.taskType == other.taskType; } + + @Override + public String toString() { + return String.format("AssignedTask{%s, %s}", taskType, id); + } } } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/assignment/ProcessId.java b/streams/src/main/java/org/apache/kafka/streams/processor/assignment/ProcessId.java index 9dd4025112a62..0a3c2c2bfb4e1 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/assignment/ProcessId.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/assignment/ProcessId.java @@ -35,6 +35,14 @@ public UUID id() { return id; } + /** + * + * @return a randomly generated process id + */ + public static ProcessId randomProcessId() { + return new ProcessId(UUID.randomUUID()); + } + @Override public String toString() { return id.toString(); diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignor.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignor.java index 38b164f1969df..887ef86faf54b 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignor.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignor.java @@ -93,7 +93,6 @@ import static java.util.Collections.unmodifiableSet; import static java.util.Map.Entry.comparingByKey; -import static java.util.UUID.randomUUID; import static org.apache.kafka.common.utils.Utils.filterMap; import static org.apache.kafka.streams.processor.internals.ClientUtils.fetchCommittedOffsets; import static org.apache.kafka.streams.processor.internals.ClientUtils.fetchEndOffsetsResult; @@ -197,7 +196,7 @@ public interface UserTaskAssignmentListener { } // keep track of any future consumers in a "dummy" Client since we can't decipher their subscription - private static final ProcessId FUTURE_ID = new ProcessId(randomUUID()); + private static final ProcessId FUTURE_ID = ProcessId.randomProcessId(); protected static final Comparator PARTITION_COMPARATOR = Comparator.comparing(TopicPartition::topic).thenComparingInt(TopicPartition::partition); diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/TaskManagerTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/TaskManagerTest.java index c6f568466e596..22fd4052b34d8 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/TaskManagerTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/TaskManagerTest.java @@ -16,7 +16,6 @@ */ package org.apache.kafka.streams.processor.internals; -import java.util.UUID; import org.apache.kafka.clients.admin.Admin; import org.apache.kafka.clients.admin.DeleteRecordsResult; import org.apache.kafka.clients.admin.DeletedRecords; @@ -240,7 +239,7 @@ private TaskManager setUpTaskManager(final ProcessingMode processingMode, final TaskManager taskManager = new TaskManager( time, changeLogReader, - new ProcessId(UUID.randomUUID()), + ProcessId.randomProcessId(), "taskManagerTest", activeTaskCreator, standbyTaskCreator, diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/KafkaStreamsAssignmentTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/KafkaStreamsAssignmentTest.java new file mode 100644 index 0000000000000..ef595d142b68e --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/KafkaStreamsAssignmentTest.java @@ -0,0 +1,49 @@ +/* + * 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.streams.processor.internals.assignment; + +import static org.apache.kafka.common.utils.Utils.mkSet; +import static org.apache.kafka.streams.processor.internals.assignment.AssignmentTestUtils.TASK_0_0; +import static org.apache.kafka.streams.processor.internals.assignment.AssignmentTestUtils.TASK_0_1; +import static org.apache.kafka.streams.processor.internals.assignment.AssignmentTestUtils.TASK_0_2; +import static org.apache.kafka.streams.processor.internals.assignment.AssignmentTestUtils.processIdForInt; +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.MatcherAssert.assertThat; + +import org.apache.kafka.streams.processor.assignment.KafkaStreamsAssignment; +import org.apache.kafka.streams.processor.assignment.KafkaStreamsAssignment.AssignedTask; +import org.junit.Test; + +public class KafkaStreamsAssignmentTest { + @Test + public void shouldHaveReadableString() { + final KafkaStreamsAssignment assignment = KafkaStreamsAssignment.of( + processIdForInt(1), + mkSet( + new AssignedTask(TASK_0_0, AssignedTask.Type.ACTIVE), + new AssignedTask(TASK_0_1, AssignedTask.Type.STANDBY), + new AssignedTask(TASK_0_2, AssignedTask.Type.ACTIVE) + ) + ); + + assertThat( + assignment.toString(), + equalTo("KafkaStreamsAssignment{00000000-0000-0000-0000-000000000001, " + + "[AssignedTask{ACTIVE, 0_2}, AssignedTask{STANDBY, 0_1}, AssignedTask{ACTIVE, 0_0}], " + + "Optional.empty}")); + } +} diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/KafkaStreamsStateTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/KafkaStreamsStateTest.java index a0b0c457a3bcd..b28cd3678a10b 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/KafkaStreamsStateTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/KafkaStreamsStateTest.java @@ -31,7 +31,6 @@ import java.util.Optional; import java.util.TreeMap; import java.util.TreeSet; -import java.util.UUID; import org.apache.kafka.streams.processor.assignment.KafkaStreamsState; import org.apache.kafka.streams.processor.assignment.ProcessId; import org.junit.Test; @@ -40,7 +39,7 @@ public class KafkaStreamsStateTest { @Test public void shouldCorrectlyReturnTasksByLag() { final KafkaStreamsState state = new DefaultKafkaStreamsState( - new ProcessId(UUID.randomUUID()), + ProcessId.randomProcessId(), 10, mkMap(), mkSortedSet(NAMED_TASK_T0_0_0, NAMED_TASK_T0_0_1), @@ -71,7 +70,7 @@ public void shouldCorrectlyReturnTasksByLag() { @Test public void shouldThrowExceptionOnLagOperationsIfLagsWereNotComputed() { final KafkaStreamsState state = new DefaultKafkaStreamsState( - new ProcessId(UUID.randomUUID()), + ProcessId.randomProcessId(), 10, mkMap(), mkSortedSet(NAMED_TASK_T0_0_0, NAMED_TASK_T0_0_1), diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/TaskAssignmentUtilsTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/TaskAssignmentUtilsTest.java index d8807ee6d660d..2295a865aa744 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/TaskAssignmentUtilsTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/TaskAssignmentUtilsTest.java @@ -40,7 +40,6 @@ import java.util.Set; import java.util.TreeMap; import java.util.TreeSet; -import java.util.UUID; import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; import org.apache.kafka.common.TopicPartition; @@ -385,7 +384,7 @@ public void testValidateTaskAssignment() { ) )), KafkaStreamsAssignment.of(processId(2), mkSet()), - KafkaStreamsAssignment.of(new ProcessId(UUID.randomUUID()), mkSet()) + KafkaStreamsAssignment.of(ProcessId.randomProcessId(), mkSet()) ) ); error = TaskAssignmentUtils.validateTaskAssignment(applicationState, unknownProcessId);