/* * Copyright 2017 LinkedIn Corp. Licensed under the BSD 2-Clause License (the "License"). See License in the project root for license information. */ package com.linkedin.kafka.cruisecontrol.executor; import com.codahale.metrics.MetricRegistry; import com.linkedin.kafka.cruisecontrol.KafkaCruiseControlUtils; import com.linkedin.kafka.cruisecontrol.common.MetadataClient; import com.linkedin.kafka.cruisecontrol.common.TestConstants; import com.linkedin.kafka.cruisecontrol.config.BrokerCapacityConfigFileResolver; import com.linkedin.kafka.cruisecontrol.config.KafkaCruiseControlConfig; import com.linkedin.kafka.cruisecontrol.config.constants.AnalyzerConfig; import com.linkedin.kafka.cruisecontrol.config.constants.ExecutorConfig; import com.linkedin.kafka.cruisecontrol.config.constants.MonitorConfig; import com.linkedin.kafka.cruisecontrol.exception.OngoingExecutionException; import com.linkedin.kafka.cruisecontrol.metricsreporter.utils.CCKafkaClientsIntegrationTestHarness; import com.linkedin.kafka.cruisecontrol.model.ReplicaPlacementInfo; import com.linkedin.kafka.cruisecontrol.detector.AnomalyDetector; import com.linkedin.kafka.cruisecontrol.monitor.LoadMonitor; import com.linkedin.kafka.cruisecontrol.monitor.sampling.NoopSampler; import com.linkedin.kafka.cruisecontrol.monitor.task.LoadMonitorTaskRunner; import com.linkedin.kafka.cruisecontrol.servlet.UserTaskManager; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Objects; import java.util.Properties; import java.util.Random; import java.util.concurrent.ExecutionException; import kafka.zk.KafkaZkClient; import org.apache.kafka.clients.admin.AdminClient; import org.apache.kafka.clients.admin.AdminClientConfig; import org.apache.kafka.clients.admin.NewTopic; import org.apache.kafka.clients.admin.TopicDescription; import org.apache.kafka.clients.producer.Producer; import org.apache.kafka.clients.producer.ProducerConfig; import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.common.Cluster; import org.apache.kafka.common.Node; import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.common.utils.SystemTime; import org.apache.kafka.common.utils.Time; import org.easymock.Capture; import org.easymock.CaptureType; import org.easymock.EasyMock; import org.junit.After; import org.junit.Before; import org.junit.Test; import static com.linkedin.kafka.cruisecontrol.KafkaCruiseControlUnitTestUtils.waitUntilTrue; import static com.linkedin.kafka.cruisecontrol.common.TestConstants.DEFAULT_BROKER_CAPACITY_CONFIG_FILE; import static com.linkedin.kafka.cruisecontrol.common.TestConstants.TOPIC0; import static com.linkedin.kafka.cruisecontrol.common.TestConstants.TOPIC1; import static com.linkedin.kafka.cruisecontrol.common.TestConstants.TOPIC2; import static com.linkedin.kafka.cruisecontrol.common.TestConstants.TOPIC3; import static com.linkedin.kafka.cruisecontrol.monitor.sampling.MetricSampler.SamplingMode.ALL; import static com.linkedin.kafka.cruisecontrol.monitor.sampling.MetricSampler.SamplingMode.BROKER_METRICS_ONLY; import static org.easymock.EasyMock.expectLastCall; import static org.easymock.EasyMock.isA; import static org.junit.Assert.assertTrue; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertThrows; import static org.junit.Assert.assertEquals; public class ExecutorTest extends CCKafkaClientsIntegrationTestHarness { private static final int PARTITION = 0; private static final TopicPartition TP0 = new TopicPartition(TOPIC0, PARTITION); private static final TopicPartition TP1 = new TopicPartition(TOPIC1, PARTITION); private static final TopicPartition TP2 = new TopicPartition(TOPIC2, PARTITION); private static final TopicPartition TP3 = new TopicPartition(TOPIC3, PARTITION); private static final String RANDOM_UUID = "random_uuid"; private static final long REMOVAL_HISTORY_RETENTION_TIME_MS = 43200000L; private static final long DEMOTION_HISTORY_RETENTION_TIME_MS = 86400000L; private static final long PRODUCE_SIZE_IN_BYTES = 10000L; private static final long EXECUTION_DEADLINE_MS = 30000L; private static final long EXECUTION_SHORT_CHECK_MS = 10L; private static final long EXECUTION_REGULAR_CHECK_MS = 100L; private static final Random RANDOM = new Random(0xDEADBEEF); private static final int MOCK_BROKER_ID_TO_DROP = 1; @Override public int clusterSize() { return 2; } @Before public void setUp() { super.setUp(); } @After public void tearDown() { super.tearDown(); } @Test public void testReplicaReassignment() throws InterruptedException, OngoingExecutionException { KafkaZkClient kafkaZkClient = KafkaCruiseControlUtils.createKafkaZkClient(zookeeper().connectionString(), "ExecutorTestMetricGroup", "ReplicaReassignment", false); try { List<ExecutionProposal> proposalsToExecute = new ArrayList<>(); List<ExecutionProposal> proposalsToCheck = new ArrayList<>(); populateProposals(proposalsToExecute, proposalsToCheck, 0); executeAndVerifyProposals(kafkaZkClient, proposalsToExecute, proposalsToCheck, false, null, false); } finally { KafkaCruiseControlUtils.closeKafkaZkClientWithTimeout(kafkaZkClient); } } @Test public void testReplicaReassignmentProgressWithThrottle() throws InterruptedException, OngoingExecutionException { KafkaZkClient kafkaZkClient = KafkaCruiseControlUtils.createKafkaZkClient(zookeeper().connectionString(), "ExecutorTestMetricGroup", "ReplicaReassignmentProgressWithThrottle", false); try { List<ExecutionProposal> proposalsToExecute = new ArrayList<>(); List<ExecutionProposal> proposalsToCheck = new ArrayList<>(); populateProposals(proposalsToExecute, proposalsToCheck, PRODUCE_SIZE_IN_BYTES); // Throttle rate is set to the half of the produce size. executeAndVerifyProposals(kafkaZkClient, proposalsToExecute, proposalsToCheck, false, PRODUCE_SIZE_IN_BYTES / 2, true); } finally { KafkaCruiseControlUtils.closeKafkaZkClientWithTimeout(kafkaZkClient); } } @Test public void testBrokerDiesBeforeMovingPartition() throws Exception { KafkaZkClient kafkaZkClient = KafkaCruiseControlUtils.createKafkaZkClient(zookeeper().connectionString(), "ExecutorTestMetricGroup", "BrokerDiesBeforeMovingPartition", false); try { Map<String, TopicDescription> topicDescriptions = createTopics((int) PRODUCE_SIZE_IN_BYTES); // initialLeader0 will be alive after killing a broker in cluster. int initialLeader0 = topicDescriptions.get(TOPIC0).partitions().get(0).leader().id(); int initialLeader1 = topicDescriptions.get(TOPIC1).partitions().get(0).leader().id(); // Kill broker before starting the reassignment. _brokers.get(initialLeader0 == 0 ? 1 : 0).shutdown(); ExecutionProposal proposal0 = new ExecutionProposal(TP0, PRODUCE_SIZE_IN_BYTES, new ReplicaPlacementInfo(initialLeader0), Collections.singletonList(new ReplicaPlacementInfo(initialLeader0)), Collections.singletonList(new ReplicaPlacementInfo(initialLeader0 == 0 ? 1 : 0))); ExecutionProposal proposal1 = new ExecutionProposal(TP1, 0, new ReplicaPlacementInfo(initialLeader1), Arrays.asList(new ReplicaPlacementInfo(initialLeader1), new ReplicaPlacementInfo(initialLeader1 == 0 ? 1 : 0)), Arrays.asList(new ReplicaPlacementInfo(initialLeader1 == 0 ? 1 : 0), new ReplicaPlacementInfo(initialLeader1))); Collection<ExecutionProposal> proposalsToExecute = Arrays.asList(proposal0, proposal1); executeAndVerifyProposals(kafkaZkClient, proposalsToExecute, Collections.emptyList(), true, null, false); // We are not doing the rollback. assertEquals(Collections.singletonList(initialLeader0 == 0 ? 1 : 0), ExecutorUtils.newAssignmentForPartition(kafkaZkClient, TP0)); // The leadership should be on the alive broker. assertEquals(initialLeader0, kafkaZkClient.getLeaderForPartition(TP0).get()); assertEquals(initialLeader0, kafkaZkClient.getLeaderForPartition(TP1).get()); } finally { KafkaCruiseControlUtils.closeKafkaZkClientWithTimeout(kafkaZkClient); } } @Test public void testExecutionKnobs() { KafkaCruiseControlConfig config = new KafkaCruiseControlConfig(getExecutorProperties()); Executor executor = new Executor(config, null, new MetricRegistry(), EasyMock.mock(MetadataClient.class), DEMOTION_HISTORY_RETENTION_TIME_MS, REMOVAL_HISTORY_RETENTION_TIME_MS, null, null, null); // Verify correctness of set/get requested execution progress check interval. long defaultExecutionProgressCheckIntervalMs = config.getLong(ExecutorConfig.EXECUTION_PROGRESS_CHECK_INTERVAL_MS_CONFIG); assertEquals(defaultExecutionProgressCheckIntervalMs, executor.executionProgressCheckIntervalMs()); executor.setRequestedExecutionProgressCheckIntervalMs(Executor.MIN_EXECUTION_PROGRESS_CHECK_INTERVAL_MS); assertEquals(Executor.MIN_EXECUTION_PROGRESS_CHECK_INTERVAL_MS, executor.executionProgressCheckIntervalMs()); assertThrows(IllegalArgumentException.class, () -> executor.setRequestedExecutionProgressCheckIntervalMs(Executor.MIN_EXECUTION_PROGRESS_CHECK_INTERVAL_MS - 1)); // Verify correctness of add/drop recently removed/demoted brokers. assertFalse(executor.dropRecentlyRemovedBrokers(Collections.emptySet())); assertFalse(executor.dropRecentlyDemotedBrokers(Collections.emptySet())); executor.addRecentlyRemovedBrokers(Collections.singleton(MOCK_BROKER_ID_TO_DROP)); executor.addRecentlyDemotedBrokers(Collections.singleton(MOCK_BROKER_ID_TO_DROP)); assertTrue(executor.dropRecentlyRemovedBrokers(Collections.singleton(MOCK_BROKER_ID_TO_DROP))); assertTrue(executor.dropRecentlyDemotedBrokers(Collections.singleton(MOCK_BROKER_ID_TO_DROP))); } @Test public void testTimeoutAndForceExecutionStop() throws InterruptedException, OngoingExecutionException { createTopics(0); // The proposal tries to move the leader. We fake the replica list to be unchanged so there is no replica // movement, but only leader movement. ExecutionProposal proposal = new ExecutionProposal(TP1, 0, new ReplicaPlacementInfo(1), Arrays.asList(new ReplicaPlacementInfo(0), new ReplicaPlacementInfo(1)), Arrays.asList(new ReplicaPlacementInfo(0), new ReplicaPlacementInfo(1))); KafkaCruiseControlConfig configs = new KafkaCruiseControlConfig(getExecutorProperties()); Time time = new MockTime(); MetadataClient mockMetadataClient = EasyMock.mock(MetadataClient.class); // Fake the metadata to never change so the leader movement will timeout. Node node0 = new Node(0, "host0", 100); Node node1 = new Node(1, "host1", 100); Node[] replicas = new Node[2]; replicas[0] = node0; replicas[1] = node1; PartitionInfo partitionInfo = new PartitionInfo(TP1.topic(), TP1.partition(), node1, replicas, replicas); Cluster cluster = new Cluster("id", Arrays.asList(node0, node1), Collections.singleton(partitionInfo), Collections.emptySet(), Collections.emptySet()); MetadataClient.ClusterAndGeneration clusterAndGeneration = new MetadataClient.ClusterAndGeneration(cluster, 0); EasyMock.expect(mockMetadataClient.refreshMetadata()).andReturn(clusterAndGeneration).anyTimes(); EasyMock.expect(mockMetadataClient.cluster()).andReturn(clusterAndGeneration.cluster()).anyTimes(); LoadMonitor mockLoadMonitor = getMockLoadMonitor(); AnomalyDetector mockAnomalyDetector = getMockAnomalyDetector(RANDOM_UUID); UserTaskManager.UserTaskInfo mockUserTaskInfo = getMockUserTaskInfo(); // This tests runs two consecutive executions. First one completes w/o error, but the second one with error. UserTaskManager mockUserTaskManager = getMockUserTaskManager(RANDOM_UUID, mockUserTaskInfo, Arrays.asList(false, true)); EasyMock.replay(mockMetadataClient, mockLoadMonitor, mockAnomalyDetector, mockUserTaskInfo, mockUserTaskManager); Collection<ExecutionProposal> proposalsToExecute = Collections.singletonList(proposal); Executor executor = new Executor(configs, time, new MetricRegistry(), mockMetadataClient, DEMOTION_HISTORY_RETENTION_TIME_MS, REMOVAL_HISTORY_RETENTION_TIME_MS, null, mockUserTaskManager, mockAnomalyDetector); executor.setExecutionMode(false); executor.executeProposals(proposalsToExecute, Collections.emptySet(), null, mockLoadMonitor, null, null, null, null, null, null, true, RANDOM_UUID, ExecutorTest.class::getSimpleName); waitUntilTrue(() -> (executor.state().state() == ExecutorState.State.LEADER_MOVEMENT_TASK_IN_PROGRESS && !executor.inExecutionTasks().isEmpty()), "Leader movement task did not start within the time limit", EXECUTION_DEADLINE_MS, EXECUTION_SHORT_CHECK_MS); // Sleep over ExecutorConfig#DEFAULT_LEADER_MOVEMENT_TIMEOUT_MS with some margin for inter-thread synchronization. time.sleep(ExecutorConfig.DEFAULT_LEADER_MOVEMENT_TIMEOUT_MS + 1L); // The execution should finish. waitUntilTrue(() -> (!executor.hasOngoingExecution() && executor.state().state() == ExecutorState.State.NO_TASK_IN_PROGRESS), "Proposal execution did not finish within the time limit", EXECUTION_DEADLINE_MS, EXECUTION_REGULAR_CHECK_MS); // The proposal tries to move replicas. proposal = new ExecutionProposal(TP1, 0, new ReplicaPlacementInfo(1), Arrays.asList(new ReplicaPlacementInfo(0), new ReplicaPlacementInfo(1)), Arrays.asList(new ReplicaPlacementInfo(1), new ReplicaPlacementInfo(0))); proposalsToExecute = Collections.singletonList(proposal); executor.executeProposals(proposalsToExecute, Collections.emptySet(), null, mockLoadMonitor, null, null, null, null, null, null, true, RANDOM_UUID, ExecutorTest.class::getSimpleName); waitUntilTrue(() -> (executor.state().state() == ExecutorState.State.INTER_BROKER_REPLICA_MOVEMENT_TASK_IN_PROGRESS), "Inter-broker replica movement task did not start within the time limit", EXECUTION_DEADLINE_MS, EXECUTION_SHORT_CHECK_MS); // Force execution to stop. executor.userTriggeredStopExecution(true); // The execution should finish. waitUntilTrue(() -> (!executor.hasOngoingExecution() && executor.state().state() == ExecutorState.State.NO_TASK_IN_PROGRESS), "Proposal execution did not finish within the time limit", EXECUTION_DEADLINE_MS, EXECUTION_REGULAR_CHECK_MS); EasyMock.verify(mockMetadataClient, mockLoadMonitor, mockAnomalyDetector, mockUserTaskInfo, mockUserTaskManager); } /** * Proposal#1: [TPO] move from original broker to the other one -- e.g. 0 -> 1 * Proposal#2: [TP1] change order and leader -- e.g. [0, 1] -> [1, 0] */ private void populateProposals(List<ExecutionProposal> proposalToExecute, List<ExecutionProposal> proposalToVerify, long topicSize) throws InterruptedException { Map<String, TopicDescription> topicDescriptions = createTopics((int) topicSize); int initialLeader0 = topicDescriptions.get(TOPIC0).partitions().get(0).leader().id(); int initialLeader1 = topicDescriptions.get(TOPIC1).partitions().get(0).leader().id(); // Valid proposals ExecutionProposal proposal0 = new ExecutionProposal(TP0, topicSize, new ReplicaPlacementInfo(initialLeader0), Collections.singletonList(new ReplicaPlacementInfo(initialLeader0)), Collections.singletonList(new ReplicaPlacementInfo(initialLeader0 == 0 ? 1 : 0))); ExecutionProposal proposal1 = new ExecutionProposal(TP1, 0, new ReplicaPlacementInfo(initialLeader1), Arrays.asList(new ReplicaPlacementInfo(initialLeader1), new ReplicaPlacementInfo(initialLeader1 == 0 ? 1 : 0)), Arrays.asList(new ReplicaPlacementInfo(initialLeader1 == 0 ? 1 : 0), new ReplicaPlacementInfo(initialLeader1))); // Invalid proposals, the targeting topics of these proposals does not exist. ExecutionProposal proposal2 = new ExecutionProposal(TP2, 0, new ReplicaPlacementInfo(initialLeader0), Collections.singletonList(new ReplicaPlacementInfo(initialLeader0)), Collections.singletonList(new ReplicaPlacementInfo(initialLeader0 == 0 ? 1 : 0))); ExecutionProposal proposal3 = new ExecutionProposal(TP3, 0, new ReplicaPlacementInfo(initialLeader1), Arrays.asList(new ReplicaPlacementInfo(initialLeader1), new ReplicaPlacementInfo(initialLeader1 == 0 ? 1 : 0)), Arrays.asList(new ReplicaPlacementInfo(initialLeader1 == 0 ? 1 : 0), new ReplicaPlacementInfo(initialLeader1))); proposalToExecute.addAll(Arrays.asList(proposal0, proposal1, proposal2, proposal3)); proposalToVerify.addAll(Arrays.asList(proposal0, proposal1)); } /** * Creates {@link com.linkedin.kafka.cruisecontrol.common.TestConstants#TOPIC0 topic0} with replication factor 1 and * {@link com.linkedin.kafka.cruisecontrol.common.TestConstants#TOPIC1 topic1} with replication factor 2. Waits until * both brokers in the mock cluster receive the metadata about created topics. * * @param produceSizeInBytes Size of random data in bytes to produce to * {@link com.linkedin.kafka.cruisecontrol.common.TestConstants#TOPIC0 topic0}. * @return A map from topic names to their description. */ private Map<String, TopicDescription> createTopics(int produceSizeInBytes) throws InterruptedException { AdminClient adminClient = KafkaCruiseControlUtils.createAdminClient(Collections.singletonMap( AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, broker(0).plaintextAddr())); try { adminClient.createTopics(Arrays.asList(new NewTopic(TOPIC0, 1, (short) 1), new NewTopic(TOPIC1, 1, (short) 2))); } finally { KafkaCruiseControlUtils.closeAdminClientWithTimeout(adminClient); } // We need to use the admin clients to query the metadata from two different brokers to make sure that // both brokers have the latest metadata. Otherwise the Executor may get confused when it does not // see expected topics in the metadata. Map<String, TopicDescription> topicDescriptions0 = null; Map<String, TopicDescription> topicDescriptions1 = null; do { AdminClient adminClient0 = KafkaCruiseControlUtils.createAdminClient(Collections.singletonMap( AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, broker(0).plaintextAddr())); AdminClient adminClient1 = KafkaCruiseControlUtils.createAdminClient(Collections.singletonMap( AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, broker(1).plaintextAddr())); try { topicDescriptions0 = adminClient0.describeTopics(Arrays.asList(TOPIC0, TOPIC1)).all().get(); topicDescriptions1 = adminClient1.describeTopics(Arrays.asList(TOPIC0, TOPIC1)).all().get(); try { Thread.sleep(100); } catch (InterruptedException e) { e.printStackTrace(); } } catch (ExecutionException ee) { // Let it go. } finally { KafkaCruiseControlUtils.closeAdminClientWithTimeout(adminClient0); KafkaCruiseControlUtils.closeAdminClientWithTimeout(adminClient1); } } while (topicDescriptions0 == null || topicDescriptions0.size() < 2 || topicDescriptions1 == null || topicDescriptions1.size() < 2); produceRandomDataToTopic(TOPIC0, produceSizeInBytes); return topicDescriptions0; } private void produceRandomDataToTopic(String topic, int produceSize) { if (produceSize > 0) { Properties props = new Properties(); props.setProperty(ProducerConfig.ACKS_CONFIG, "-1"); try (Producer<String, String> producer = createProducer(props)) { byte[] randomRecords = new byte[produceSize]; RANDOM.nextBytes(randomRecords); producer.send(new ProducerRecord<>(topic, Arrays.toString(randomRecords))); } } } private static UserTaskManager getMockUserTaskManager(String uuid, UserTaskManager.UserTaskInfo userTaskInfo, List<Boolean> completeWithError) { UserTaskManager mockUserTaskManager = EasyMock.mock(UserTaskManager.class); // Handle the case that the execution started, but did not finish. if (completeWithError.isEmpty()) { EasyMock.expect(mockUserTaskManager.markTaskExecutionBegan(uuid)).andReturn(userTaskInfo).once(); } else { // Return as many times as the executions to ensure that the same test can run multiple executions. for (boolean completeStatus : completeWithError) { EasyMock.expect(mockUserTaskManager.markTaskExecutionBegan(uuid)).andReturn(userTaskInfo).once(); mockUserTaskManager.markTaskExecutionFinished(uuid, completeStatus); } } return mockUserTaskManager; } private static LoadMonitor getMockLoadMonitor() { LoadMonitor mockLoadMonitor = EasyMock.mock(LoadMonitor.class); EasyMock.expect(mockLoadMonitor.taskRunnerState()) .andReturn(LoadMonitorTaskRunner.LoadMonitorTaskRunnerState.RUNNING) .anyTimes(); EasyMock.expect(mockLoadMonitor.samplingMode()).andReturn(ALL).anyTimes(); mockLoadMonitor.pauseMetricSampling(isA(String.class), EasyMock.anyBoolean()); expectLastCall().anyTimes(); mockLoadMonitor.setSamplingMode(BROKER_METRICS_ONLY); expectLastCall().anyTimes(); mockLoadMonitor.resumeMetricSampling(isA(String.class)); expectLastCall().anyTimes(); mockLoadMonitor.setSamplingMode(ALL); expectLastCall().anyTimes(); return mockLoadMonitor; } private static AnomalyDetector getMockAnomalyDetector(String anomalyId) { AnomalyDetector mockAnomalyDetector = EasyMock.mock(AnomalyDetector.class); mockAnomalyDetector.maybeClearOngoingAnomalyDetectionTimeMs(); expectLastCall().anyTimes(); mockAnomalyDetector.resetHasUnfixableGoals(); expectLastCall().anyTimes(); mockAnomalyDetector.markSelfHealingFinished(anomalyId); expectLastCall().anyTimes(); return mockAnomalyDetector; } private static UserTaskManager.UserTaskInfo getMockUserTaskInfo() { UserTaskManager.UserTaskInfo mockUserTaskInfo = EasyMock.mock(UserTaskManager.UserTaskInfo.class); // Run it any times to enable consecutive executions in tests. EasyMock.expect(mockUserTaskInfo.requestUrl()).andReturn("mock-request").anyTimes(); return mockUserTaskInfo; } private void executeAndVerifyProposals(KafkaZkClient kafkaZkClient, Collection<ExecutionProposal> proposalsToExecute, Collection<ExecutionProposal> proposalsToCheck, boolean completeWithError, Long replicationThrottle, boolean verifyProgress) throws OngoingExecutionException { KafkaCruiseControlConfig configs = new KafkaCruiseControlConfig(getExecutorProperties()); UserTaskManager.UserTaskInfo mockUserTaskInfo = getMockUserTaskInfo(); UserTaskManager mockUserTaskManager = getMockUserTaskManager(RANDOM_UUID, mockUserTaskInfo, Collections.singletonList(completeWithError)); ExecutorNotifier mockExecutorNotifier = EasyMock.mock(ExecutorNotifier.class); LoadMonitor mockLoadMonitor = getMockLoadMonitor(); Capture<String> captureMessage = Capture.newInstance(CaptureType.FIRST); AnomalyDetector mockAnomalyDetector = getMockAnomalyDetector(RANDOM_UUID); if (completeWithError) { mockExecutorNotifier.sendAlert(EasyMock.capture(captureMessage)); } else { mockExecutorNotifier.sendNotification(EasyMock.capture(captureMessage)); } EasyMock.replay(mockUserTaskInfo, mockUserTaskManager, mockExecutorNotifier, mockLoadMonitor, mockAnomalyDetector); Executor executor = new Executor(configs, new SystemTime(), new MetricRegistry(), null, DEMOTION_HISTORY_RETENTION_TIME_MS, REMOVAL_HISTORY_RETENTION_TIME_MS, mockExecutorNotifier, mockUserTaskManager, mockAnomalyDetector); executor.setExecutionMode(false); Map<TopicPartition, Integer> replicationFactors = new HashMap<>(proposalsToCheck.size()); for (ExecutionProposal proposal : proposalsToCheck) { TopicPartition tp = new TopicPartition(proposal.topic(), proposal.partitionId()); replicationFactors.put(tp, proposal.oldReplicas().size()); } executor.executeProposals(proposalsToExecute, Collections.emptySet(), null, mockLoadMonitor, null, null, null, null, null, replicationThrottle, true, RANDOM_UUID, ExecutorTest.class::getSimpleName); if (verifyProgress) { waitUntilTrue(() -> ExecutorUtils.partitionsBeingReassigned(kafkaZkClient).contains(TP0), "Failed to verify the start of a partition reassignment", EXECUTION_DEADLINE_MS, EXECUTION_SHORT_CHECK_MS); } waitUntilTrue(() -> (!executor.hasOngoingExecution() && executor.state().state() == ExecutorState.State.NO_TASK_IN_PROGRESS), "Proposal execution did not finish within the time limit", EXECUTION_DEADLINE_MS, EXECUTION_REGULAR_CHECK_MS); // Check notification is sent after execution has finished. String notification = captureMessage.getValue(); assertTrue(notification.contains(RANDOM_UUID)); for (ExecutionProposal proposal : proposalsToCheck) { TopicPartition tp = new TopicPartition(proposal.topic(), proposal.partitionId()); int expectedReplicationFactor = replicationFactors.get(tp); assertEquals("Replication factor for partition " + tp + " should be " + expectedReplicationFactor, expectedReplicationFactor, kafkaZkClient.getReplicasForPartition(tp).size()); if (proposal.hasReplicaAction()) { for (ReplicaPlacementInfo r : proposal.newReplicas()) { assertTrue("The partition should have moved for " + tp, kafkaZkClient.getReplicasForPartition(tp).contains(r.brokerId())); } } assertEquals("The leader should have moved for " + tp, proposal.newLeader().brokerId(), kafkaZkClient.getLeaderForPartition(tp).get()); } EasyMock.verify(mockUserTaskInfo, mockUserTaskManager, mockExecutorNotifier, mockLoadMonitor, mockAnomalyDetector); } private Properties getExecutorProperties() { Properties props = new Properties(); String capacityConfigFile = Objects.requireNonNull( this.getClass().getClassLoader().getResource(DEFAULT_BROKER_CAPACITY_CONFIG_FILE)).getFile(); props.setProperty(BrokerCapacityConfigFileResolver.CAPACITY_CONFIG_FILE, capacityConfigFile); props.setProperty(MonitorConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers()); props.setProperty(MonitorConfig.METRIC_SAMPLER_CLASS_CONFIG, NoopSampler.class.getName()); props.setProperty(ExecutorConfig.ZOOKEEPER_CONNECT_CONFIG, zookeeper().connectionString()); props.setProperty(ExecutorConfig.NUM_CONCURRENT_PARTITION_MOVEMENTS_PER_BROKER_CONFIG, "10"); props.setProperty(ExecutorConfig.EXECUTION_PROGRESS_CHECK_INTERVAL_MS_CONFIG, "400"); props.setProperty(AnalyzerConfig.DEFAULT_GOALS_CONFIG, TestConstants.DEFAULT_GOALS_VALUES); return props; } }