github-code-scanning[bot] commented on code in PR #13197: URL: https://github.com/apache/druid/pull/13197#discussion_r1165974823
########## server/src/main/java/org/apache/druid/server/coordinator/duty/TierSegmentBalancer.java: ########## @@ -0,0 +1,293 @@ +/* + * 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.druid.server.coordinator.duty; + +import com.google.common.collect.Lists; +import org.apache.druid.client.ImmutableDruidDataSource; +import org.apache.druid.java.util.emitter.EmittingLogger; +import org.apache.druid.server.coordinator.BalancerSegmentHolder; +import org.apache.druid.server.coordinator.BalancerStrategy; +import org.apache.druid.server.coordinator.CoordinatorDynamicConfig; +import org.apache.druid.server.coordinator.DruidCoordinatorRuntimeParams; +import org.apache.druid.server.coordinator.SegmentLoader; +import org.apache.druid.server.coordinator.ServerHolder; +import org.apache.druid.server.coordinator.stats.CoordinatorRunStats; +import org.apache.druid.server.coordinator.stats.Dimension; +import org.apache.druid.server.coordinator.stats.RowKey; +import org.apache.druid.server.coordinator.stats.Stats; +import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.SegmentId; + +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; + +/** + * Balances segments within the servers of a tier using the balancer strategy. + * Segments are prioritized for move in the following order: + * <ul> + * <li>Segments loaded on decommissioning servers</li> + * <li>Segments loading on active servers</li> + * <li>Segments loaded on active servers</li> + * </ul> + */ +public class TierSegmentBalancer +{ + private static final EmittingLogger log = new EmittingLogger(TierSegmentBalancer.class); + + private final String tier; + private final DruidCoordinatorRuntimeParams params; + private final SegmentLoader loader; + + private final BalancerStrategy strategy; + private final CoordinatorDynamicConfig dynamicConfig; + private final CoordinatorRunStats runStats; + + private final Set<ServerHolder> allServers; + private final List<ServerHolder> activeServers; + private final List<ServerHolder> decommissioningServers; + private final int totalMaxSegmentsToMove; + + public TierSegmentBalancer( + String tier, + Set<ServerHolder> servers, + SegmentLoader loader, + DruidCoordinatorRuntimeParams params + ) + { + this.tier = tier; + this.params = params; + this.loader = loader; + + this.strategy = params.getBalancerStrategy(); + this.dynamicConfig = params.getCoordinatorDynamicConfig(); + this.totalMaxSegmentsToMove = dynamicConfig.getMaxSegmentsToMove(); + this.runStats = loader.getStats(); + + Map<Boolean, List<ServerHolder>> partitions = + servers.stream().collect(Collectors.partitioningBy(ServerHolder::isDecommissioning)); + decommissioningServers = partitions.get(true); + activeServers = partitions.get(false); + this.allServers = servers; + } + + void run() + { + if (activeServers.isEmpty() || (activeServers.size() <= 1 && decommissioningServers.isEmpty())) { + log.warn( + "Skipping balance for tier [%s] with [%d] active servers and [%d] decomissioning servers.", + tier, activeServers.size(), decommissioningServers.size() + ); + return; + } + + log.info( + "Balancing segments in tier [%s] with [%d] active servers and [%d] decommissioning servers.", + tier, activeServers.size(), decommissioningServers.size() + ); + + // Move segments from decommissioning to active servers + int maxDecommPercentToMove = dynamicConfig.getDecommissioningMaxPercentOfMaxSegmentsToMove(); + int maxDecommSegmentsToMove = (int) Math.ceil(totalMaxSegmentsToMove * (maxDecommPercentToMove / 100.0)); + int movedDecommSegments = moveSegmentsFromTo(decommissioningServers, activeServers, maxDecommSegmentsToMove); + log.info( + "Moved [%d] segments out of max [%d (%d%%)] from decommissioning to active servers.", + movedDecommSegments, maxDecommSegmentsToMove, maxDecommPercentToMove + ); + + // Move segments across active servers + int maxGeneralSegmentsToMove = totalMaxSegmentsToMove - movedDecommSegments; + int movedGeneralSegments = moveSegmentsFromTo(activeServers, activeServers, maxGeneralSegmentsToMove); + log.info( + "Moved [%d] segments out of max [%d] between active servers.", + movedGeneralSegments, maxGeneralSegmentsToMove + ); + + if (dynamicConfig.emitBalancingStats()) { + strategy.emitStats(tier, loader.getStats(), Lists.newArrayList(allServers)); + } + } + + private int moveSegmentsFromTo( + List<ServerHolder> sourceServers, + List<ServerHolder> destServers, + int maxSegmentsToMove + ) + { + if (maxSegmentsToMove <= 0 || sourceServers.isEmpty() || destServers.isEmpty()) { + return 0; + } + + Iterator<BalancerSegmentHolder> pickedSegments + = pickSegmentsFrom(sourceServers, maxSegmentsToMove, true); + int movedCount = moveSegmentsTo(destServers, pickedSegments, maxSegmentsToMove); + + maxSegmentsToMove -= movedCount; + pickedSegments = pickSegmentsFrom(sourceServers, maxSegmentsToMove, false); + movedCount += moveSegmentsTo(destServers, pickedSegments, maxSegmentsToMove); + + return movedCount; + } + + private Iterator<BalancerSegmentHolder> pickSegmentsFrom( + List<ServerHolder> sourceServers, + int maxSegmentsToPick, + boolean pickLoadingSegments + ) + { + if (maxSegmentsToPick <= 0 || sourceServers.isEmpty()) { + return Collections.emptyIterator(); + } else if (dynamicConfig.useBatchedSegmentSampler()) { + return strategy.pickSegmentsToMove( + sourceServers, + params.getBroadcastDatasources(), + maxSegmentsToPick, + pickLoadingSegments + ); + } else { + if (pickLoadingSegments) { + return Collections.emptyIterator(); + } else { + return strategy.pickSegmentsToMove( + sourceServers, + params.getBroadcastDatasources(), + dynamicConfig.getPercentOfSegmentsToConsiderPerMove() Review Comment: ## Deprecated method or constructor invocation Invoking [CoordinatorDynamicConfig.getPercentOfSegmentsToConsiderPerMove](1) should be avoided because it has been deprecated. [Show more details](https://github.com/apache/druid/security/code-scanning/4811) ########## server/src/main/java/org/apache/druid/server/coordinator/duty/TierSegmentBalancer.java: ########## @@ -0,0 +1,293 @@ +/* + * 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.druid.server.coordinator.duty; + +import com.google.common.collect.Lists; +import org.apache.druid.client.ImmutableDruidDataSource; +import org.apache.druid.java.util.emitter.EmittingLogger; +import org.apache.druid.server.coordinator.BalancerSegmentHolder; +import org.apache.druid.server.coordinator.BalancerStrategy; +import org.apache.druid.server.coordinator.CoordinatorDynamicConfig; +import org.apache.druid.server.coordinator.DruidCoordinatorRuntimeParams; +import org.apache.druid.server.coordinator.SegmentLoader; +import org.apache.druid.server.coordinator.ServerHolder; +import org.apache.druid.server.coordinator.stats.CoordinatorRunStats; +import org.apache.druid.server.coordinator.stats.Dimension; +import org.apache.druid.server.coordinator.stats.RowKey; +import org.apache.druid.server.coordinator.stats.Stats; +import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.SegmentId; + +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; + +/** + * Balances segments within the servers of a tier using the balancer strategy. + * Segments are prioritized for move in the following order: + * <ul> + * <li>Segments loaded on decommissioning servers</li> + * <li>Segments loading on active servers</li> + * <li>Segments loaded on active servers</li> + * </ul> + */ +public class TierSegmentBalancer +{ + private static final EmittingLogger log = new EmittingLogger(TierSegmentBalancer.class); + + private final String tier; + private final DruidCoordinatorRuntimeParams params; + private final SegmentLoader loader; + + private final BalancerStrategy strategy; + private final CoordinatorDynamicConfig dynamicConfig; + private final CoordinatorRunStats runStats; + + private final Set<ServerHolder> allServers; + private final List<ServerHolder> activeServers; + private final List<ServerHolder> decommissioningServers; + private final int totalMaxSegmentsToMove; + + public TierSegmentBalancer( + String tier, + Set<ServerHolder> servers, + SegmentLoader loader, + DruidCoordinatorRuntimeParams params + ) + { + this.tier = tier; + this.params = params; + this.loader = loader; + + this.strategy = params.getBalancerStrategy(); + this.dynamicConfig = params.getCoordinatorDynamicConfig(); + this.totalMaxSegmentsToMove = dynamicConfig.getMaxSegmentsToMove(); + this.runStats = loader.getStats(); + + Map<Boolean, List<ServerHolder>> partitions = + servers.stream().collect(Collectors.partitioningBy(ServerHolder::isDecommissioning)); + decommissioningServers = partitions.get(true); + activeServers = partitions.get(false); + this.allServers = servers; + } + + void run() + { + if (activeServers.isEmpty() || (activeServers.size() <= 1 && decommissioningServers.isEmpty())) { + log.warn( + "Skipping balance for tier [%s] with [%d] active servers and [%d] decomissioning servers.", + tier, activeServers.size(), decommissioningServers.size() + ); + return; + } + + log.info( + "Balancing segments in tier [%s] with [%d] active servers and [%d] decommissioning servers.", + tier, activeServers.size(), decommissioningServers.size() + ); + + // Move segments from decommissioning to active servers + int maxDecommPercentToMove = dynamicConfig.getDecommissioningMaxPercentOfMaxSegmentsToMove(); + int maxDecommSegmentsToMove = (int) Math.ceil(totalMaxSegmentsToMove * (maxDecommPercentToMove / 100.0)); + int movedDecommSegments = moveSegmentsFromTo(decommissioningServers, activeServers, maxDecommSegmentsToMove); + log.info( + "Moved [%d] segments out of max [%d (%d%%)] from decommissioning to active servers.", + movedDecommSegments, maxDecommSegmentsToMove, maxDecommPercentToMove + ); + + // Move segments across active servers + int maxGeneralSegmentsToMove = totalMaxSegmentsToMove - movedDecommSegments; + int movedGeneralSegments = moveSegmentsFromTo(activeServers, activeServers, maxGeneralSegmentsToMove); + log.info( + "Moved [%d] segments out of max [%d] between active servers.", + movedGeneralSegments, maxGeneralSegmentsToMove + ); + + if (dynamicConfig.emitBalancingStats()) { + strategy.emitStats(tier, loader.getStats(), Lists.newArrayList(allServers)); + } + } + + private int moveSegmentsFromTo( + List<ServerHolder> sourceServers, + List<ServerHolder> destServers, + int maxSegmentsToMove + ) + { + if (maxSegmentsToMove <= 0 || sourceServers.isEmpty() || destServers.isEmpty()) { + return 0; + } + + Iterator<BalancerSegmentHolder> pickedSegments + = pickSegmentsFrom(sourceServers, maxSegmentsToMove, true); + int movedCount = moveSegmentsTo(destServers, pickedSegments, maxSegmentsToMove); + + maxSegmentsToMove -= movedCount; + pickedSegments = pickSegmentsFrom(sourceServers, maxSegmentsToMove, false); + movedCount += moveSegmentsTo(destServers, pickedSegments, maxSegmentsToMove); + + return movedCount; + } + + private Iterator<BalancerSegmentHolder> pickSegmentsFrom( + List<ServerHolder> sourceServers, + int maxSegmentsToPick, + boolean pickLoadingSegments + ) + { + if (maxSegmentsToPick <= 0 || sourceServers.isEmpty()) { + return Collections.emptyIterator(); + } else if (dynamicConfig.useBatchedSegmentSampler()) { + return strategy.pickSegmentsToMove( + sourceServers, + params.getBroadcastDatasources(), + maxSegmentsToPick, + pickLoadingSegments + ); + } else { + if (pickLoadingSegments) { + return Collections.emptyIterator(); + } else { + return strategy.pickSegmentsToMove( + sourceServers, + params.getBroadcastDatasources(), + dynamicConfig.getPercentOfSegmentsToConsiderPerMove() + ); Review Comment: ## Deprecated method or constructor invocation Invoking [BalancerStrategy.pickSegmentsToMove](1) should be avoided because it has been deprecated. [Show more details](https://github.com/apache/druid/security/code-scanning/4810) ########## server/src/test/java/org/apache/druid/server/coordinator/CuratorDruidCoordinatorTest.java: ########## @@ -497,4 +466,27 @@ .size(0) .build(); } + + private SegmentLoader createSegmentLoader( + SegmentLoadQueueManager loadQueueManager, + DruidCoordinatorRuntimeParams params, + String... tiersEligibleForReplication + ) + { + final CoordinatorDynamicConfig dynamicConfig = params.getCoordinatorDynamicConfig(); + ReplicationThrottler throttler = new ReplicationThrottler( + new HashSet<>(Arrays.asList(tiersEligibleForReplication)), + dynamicConfig.getReplicationThrottleLimit(), + dynamicConfig.getReplicantLifetime(), + dynamicConfig.getMaxNonPrimaryReplicantsToLoad() Review Comment: ## Deprecated method or constructor invocation Invoking [CoordinatorDynamicConfig.getMaxNonPrimaryReplicantsToLoad](1) should be avoided because it has been deprecated. [Show more details](https://github.com/apache/druid/security/code-scanning/4813) ########## server/src/test/java/org/apache/druid/server/coordinator/loadqueue/HttpLoadQueuePeonTest.java: ########## @@ -0,0 +1,382 @@ +/* + * 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.druid.server.coordinator.loadqueue; + +import com.fasterxml.jackson.core.type.TypeReference; +import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.ListenableFuture; +import org.apache.druid.java.util.common.RE; +import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.java.util.http.client.HttpClient; +import org.apache.druid.java.util.http.client.Request; +import org.apache.druid.java.util.http.client.response.HttpResponseHandler; +import org.apache.druid.server.ServerTestHelper; +import org.apache.druid.server.coordination.DataSegmentChangeCallback; +import org.apache.druid.server.coordination.DataSegmentChangeHandler; +import org.apache.druid.server.coordination.DataSegmentChangeRequest; +import org.apache.druid.server.coordination.SegmentLoadDropHandler; +import org.apache.druid.server.coordinator.CreateDataSegments; +import org.apache.druid.server.coordinator.TestDruidCoordinatorConfig; +import org.apache.druid.server.coordinator.simulate.BlockingExecutorService; +import org.apache.druid.server.coordinator.simulate.WrappingScheduledExecutorService; +import org.apache.druid.timeline.DataSegment; +import org.jboss.netty.buffer.ChannelBuffers; +import org.jboss.netty.handler.codec.http.DefaultHttpResponse; +import org.jboss.netty.handler.codec.http.HttpResponse; +import org.jboss.netty.handler.codec.http.HttpResponseStatus; +import org.jboss.netty.handler.codec.http.HttpVersion; +import org.joda.time.Duration; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.io.ByteArrayInputStream; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.function.Consumer; +import java.util.stream.Collectors; + +/** + * + */ +public class HttpLoadQueuePeonTest +{ + private final List<DataSegment> segments = + CreateDataSegments.ofDatasource("test") + .forIntervals(1, Granularities.DAY) + .startingAt("2022-01-01") + .withNumPartitions(4) + .eachOfSizeInMb(100); + + private TestHttpClient httpClient; + private HttpLoadQueuePeon httpLoadQueuePeon; + private BlockingExecutorService processingExecutor; + private BlockingExecutorService callbackExecutor; + + private final List<DataSegment> processedSegments = new ArrayList<>(); + + @Before + public void setUp() + { + httpClient = new TestHttpClient(); + processingExecutor = new BlockingExecutorService("HttpLoadQueuePeonTest-%s"); + callbackExecutor = new BlockingExecutorService("HttpLoadQueuePeonTest-cb"); + processedSegments.clear(); + + httpLoadQueuePeon = new HttpLoadQueuePeon( + "http://dummy:4000", + ServerTestHelper.MAPPER, + httpClient, + new TestDruidCoordinatorConfig.Builder() + .withHttpLoadQueuePeonBatchSize(10) + .build(), + new WrappingScheduledExecutorService("HttpLoadQueuePeonTest-%s", processingExecutor, true), + callbackExecutor + ); + httpLoadQueuePeon.start(); + } + + @After + public void tearDown() + { + httpLoadQueuePeon.stop(); + } + + @Test + public void testSimple() + { + httpLoadQueuePeon + .dropSegment(segments.get(0), markSegmentProcessed(segments.get(0))); + httpLoadQueuePeon + .loadSegment(segments.get(1), SegmentAction.LOAD, markSegmentProcessed(segments.get(1))); + httpLoadQueuePeon + .loadSegment(segments.get(2), SegmentAction.REPLICATE, markSegmentProcessed(segments.get(2))); + httpLoadQueuePeon + .loadSegment(segments.get(3), SegmentAction.MOVE_TO, markSegmentProcessed(segments.get(3))); + + // Send requests to server + processingExecutor.finishAllPendingTasks(); + Assert.assertEquals(segments, httpClient.segmentsSentToServer); + + // Verify that all callbacks are executed + callbackExecutor.finishAllPendingTasks(); + Assert.assertEquals(segments, processedSegments); + } + + @Test + public void testLoadDropAfterStop() + { + // Verify that requests sent after stopping the peon fail immediately + httpLoadQueuePeon.stop(); + + final Set<DataSegment> failedSegments = new HashSet<>(); + final DataSegment segment1 = segments.get(0); + httpLoadQueuePeon.dropSegment(segment1, success -> { + if (!success) { + failedSegments.add(segment1); + } + }); + final DataSegment segment2 = segments.get(1); + httpLoadQueuePeon.loadSegment(segment2, SegmentAction.MOVE_TO, success -> { + if (!success) { + failedSegments.add(segment2); + } + }); + + Assert.assertTrue(failedSegments.contains(segment1)); + Assert.assertTrue(failedSegments.contains(segment2)); + } + + @Test + public void testPriorityOfSegmentAction() + { + // Shuffle the segments for the same day + final List<DataSegment> segmentsDay1 = new ArrayList<>(segments); + Collections.shuffle(segmentsDay1); + + // Assign segments to the actions in their order of priority + // Order: drop, priorityLoad, load, move + final List<QueueAction> actions = Arrays.asList( + QueueAction.of(segmentsDay1.get(0), s -> httpLoadQueuePeon.dropSegment(s, null)), + QueueAction.of(segmentsDay1.get(1), s -> httpLoadQueuePeon.loadSegment(s, SegmentAction.LOAD, null)), + QueueAction.of(segmentsDay1.get(2), s -> httpLoadQueuePeon.loadSegment(s, SegmentAction.REPLICATE, null)), + QueueAction.of(segmentsDay1.get(3), s -> httpLoadQueuePeon.loadSegment(s, SegmentAction.MOVE_TO, null)) + ); + + // Queue the actions on the peon in a random order + Collections.shuffle(actions); + actions.forEach(QueueAction::invoke); + + // Send one batch of requests to the server + processingExecutor.finishAllPendingTasks(); + + // Verify that all segments are sent to the server in the expected order + Assert.assertEquals(segmentsDay1, httpClient.segmentsSentToServer); + } + + @Test + public void testPriorityOfSegmentInterval() + { + // Create 8 segments (4 x 2days) and shuffle them + final List<DataSegment> segmentsDay1 = new ArrayList<>(segments); + Collections.shuffle(segmentsDay1); + + final List<DataSegment> segmentsDay2 = new ArrayList<>( + CreateDataSegments.ofDatasource("test") + .forIntervals(1, Granularities.DAY) + .startingAt("2022-01-02") + .withNumPartitions(4) + .eachOfSizeInMb(100) + ); + Collections.shuffle(segmentsDay2); + + // Assign segments to the actions in their order of priority + // Priority order: action (drop, priorityLoad, etc), then interval (new then old) + List<QueueAction> actions = Arrays.asList( + QueueAction.of(segmentsDay2.get(0), s -> httpLoadQueuePeon.dropSegment(s, null)), + QueueAction.of(segmentsDay1.get(0), s -> httpLoadQueuePeon.dropSegment(s, null)), + QueueAction.of(segmentsDay2.get(1), s -> httpLoadQueuePeon.loadSegment(s, SegmentAction.LOAD, null)), + QueueAction.of(segmentsDay1.get(1), s -> httpLoadQueuePeon.loadSegment(s, SegmentAction.LOAD, null)), + QueueAction.of(segmentsDay2.get(2), s -> httpLoadQueuePeon.loadSegment(s, SegmentAction.REPLICATE, null)), + QueueAction.of(segmentsDay1.get(2), s -> httpLoadQueuePeon.loadSegment(s, SegmentAction.REPLICATE, null)), + QueueAction.of(segmentsDay2.get(3), s -> httpLoadQueuePeon.loadSegment(s, SegmentAction.MOVE_TO, null)), + QueueAction.of(segmentsDay1.get(3), s -> httpLoadQueuePeon.loadSegment(s, SegmentAction.MOVE_TO, null)) + ); + final List<DataSegment> expectedSegmentOrder = + actions.stream().map(a -> a.segment).collect(Collectors.toList()); + + // Queue the actions on the peon in a random order + Collections.shuffle(actions); + actions.forEach(QueueAction::invoke); + + // Send one batch of requests to the server + processingExecutor.finishNextPendingTask(); + + // Verify that all segments are sent to the server in the expected order + Assert.assertEquals(expectedSegmentOrder, httpClient.segmentsSentToServer); + } + + @Test + public void testCancelLoad() + { + final DataSegment segment = segments.get(0); + httpLoadQueuePeon.loadSegment(segment, SegmentAction.REPLICATE, markSegmentProcessed(segment)); + Assert.assertEquals(1, httpLoadQueuePeon.getNumberOfSegmentsToLoad()); + + boolean cancelled = httpLoadQueuePeon.cancelOperation(segment); + Assert.assertTrue(cancelled); + Assert.assertEquals(0, httpLoadQueuePeon.getNumberOfSegmentsToLoad()); + + Assert.assertTrue(processedSegments.isEmpty()); + } + + @Test + public void testCancelDrop() + { + final DataSegment segment = segments.get(0); + httpLoadQueuePeon.dropSegment(segment, markSegmentProcessed(segment)); + Assert.assertEquals(1, httpLoadQueuePeon.getSegmentsToDrop().size()); + + boolean cancelled = httpLoadQueuePeon.cancelOperation(segment); + Assert.assertTrue(cancelled); + Assert.assertTrue(httpLoadQueuePeon.getSegmentsToDrop().isEmpty()); + + Assert.assertTrue(processedSegments.isEmpty()); + } + + @Test + public void testCannotCancelRequestSentToServer() + { + final DataSegment segment = segments.get(0); + httpLoadQueuePeon.loadSegment(segment, SegmentAction.REPLICATE, markSegmentProcessed(segment)); + Assert.assertTrue(httpLoadQueuePeon.getSegmentsToLoad().contains(segment)); + + // Send the request to the server + processingExecutor.finishNextPendingTask(); + Assert.assertTrue(httpClient.segmentsSentToServer.contains(segment)); + + // Segment is still in queue but operation cannot be cancelled + Assert.assertTrue(httpLoadQueuePeon.getSegmentsToLoad().contains(segment)); + boolean cancelled = httpLoadQueuePeon.cancelOperation(segment); + Assert.assertFalse(cancelled); + + // Handle response from server + processingExecutor.finishNextPendingTask(); + + // Segment has been removed from queue + Assert.assertTrue(httpLoadQueuePeon.getSegmentsToLoad().isEmpty()); + cancelled = httpLoadQueuePeon.cancelOperation(segment); + Assert.assertFalse(cancelled); + + // Execute callbacks and verify segment is fully processed + callbackExecutor.finishAllPendingTasks(); + Assert.assertTrue(processedSegments.contains(segment)); + } + + @Test + public void testCannotCancelOperationMultipleTimes() + { + final DataSegment segment = segments.get(0); + httpLoadQueuePeon.loadSegment(segment, SegmentAction.REPLICATE, markSegmentProcessed(segment)); + Assert.assertTrue(httpLoadQueuePeon.getSegmentsToLoad().contains(segment)); + + Assert.assertTrue(httpLoadQueuePeon.cancelOperation(segment)); + Assert.assertFalse(httpLoadQueuePeon.cancelOperation(segment)); + } + + private LoadPeonCallback markSegmentProcessed(DataSegment segment) + { + return success -> processedSegments.add(segment); + } + + private static class TestHttpClient implements HttpClient, DataSegmentChangeHandler + { + private final List<DataSegment> segmentsSentToServer = new ArrayList<>(); + + @Override + public <Intermediate, Final> ListenableFuture<Final> go( + Request request, + HttpResponseHandler<Intermediate, Final> httpResponseHandler + ) + { + throw new UnsupportedOperationException("Not Implemented."); + } + + @Override + public <Intermediate, Final> ListenableFuture<Final> go( + Request request, + HttpResponseHandler<Intermediate, Final> httpResponseHandler, + Duration duration + ) + { + HttpResponse httpResponse = new DefaultHttpResponse(HttpVersion.HTTP_1_1, HttpResponseStatus.OK); + httpResponse.setContent(ChannelBuffers.buffer(0)); + httpResponseHandler.handleResponse(httpResponse, null); + try { + List<DataSegmentChangeRequest> changeRequests = ServerTestHelper.MAPPER.readValue( + request.getContent().array(), new TypeReference<List<DataSegmentChangeRequest>>() + { + } + ); + + List<SegmentLoadDropHandler.DataSegmentChangeRequestAndStatus> statuses = new ArrayList<>(changeRequests.size()); + for (DataSegmentChangeRequest cr : changeRequests) { + cr.go(this, null); + statuses.add(new SegmentLoadDropHandler.DataSegmentChangeRequestAndStatus( + cr, + SegmentLoadDropHandler.Status.SUCCESS + )); + } + return (ListenableFuture) Futures.immediateFuture( + new ByteArrayInputStream( + ServerTestHelper.MAPPER + .writerWithType(HttpLoadQueuePeon.RESPONSE_ENTITY_TYPE_REF) Review Comment: ## Deprecated method or constructor invocation Invoking [ObjectMapper.writerWithType](1) should be avoided because it has been deprecated. [Show more details](https://github.com/apache/druid/security/code-scanning/4814) ########## server/src/main/java/org/apache/druid/server/coordinator/duty/TierSegmentBalancer.java: ########## @@ -0,0 +1,293 @@ +/* + * 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.druid.server.coordinator.duty; + +import com.google.common.collect.Lists; +import org.apache.druid.client.ImmutableDruidDataSource; +import org.apache.druid.java.util.emitter.EmittingLogger; +import org.apache.druid.server.coordinator.BalancerSegmentHolder; +import org.apache.druid.server.coordinator.BalancerStrategy; +import org.apache.druid.server.coordinator.CoordinatorDynamicConfig; +import org.apache.druid.server.coordinator.DruidCoordinatorRuntimeParams; +import org.apache.druid.server.coordinator.SegmentLoader; +import org.apache.druid.server.coordinator.ServerHolder; +import org.apache.druid.server.coordinator.stats.CoordinatorRunStats; +import org.apache.druid.server.coordinator.stats.Dimension; +import org.apache.druid.server.coordinator.stats.RowKey; +import org.apache.druid.server.coordinator.stats.Stats; +import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.SegmentId; + +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; + +/** + * Balances segments within the servers of a tier using the balancer strategy. + * Segments are prioritized for move in the following order: + * <ul> + * <li>Segments loaded on decommissioning servers</li> + * <li>Segments loading on active servers</li> + * <li>Segments loaded on active servers</li> + * </ul> + */ +public class TierSegmentBalancer +{ + private static final EmittingLogger log = new EmittingLogger(TierSegmentBalancer.class); + + private final String tier; + private final DruidCoordinatorRuntimeParams params; + private final SegmentLoader loader; + + private final BalancerStrategy strategy; + private final CoordinatorDynamicConfig dynamicConfig; + private final CoordinatorRunStats runStats; + + private final Set<ServerHolder> allServers; + private final List<ServerHolder> activeServers; + private final List<ServerHolder> decommissioningServers; + private final int totalMaxSegmentsToMove; + + public TierSegmentBalancer( + String tier, + Set<ServerHolder> servers, + SegmentLoader loader, + DruidCoordinatorRuntimeParams params + ) + { + this.tier = tier; + this.params = params; + this.loader = loader; + + this.strategy = params.getBalancerStrategy(); + this.dynamicConfig = params.getCoordinatorDynamicConfig(); + this.totalMaxSegmentsToMove = dynamicConfig.getMaxSegmentsToMove(); + this.runStats = loader.getStats(); + + Map<Boolean, List<ServerHolder>> partitions = + servers.stream().collect(Collectors.partitioningBy(ServerHolder::isDecommissioning)); + decommissioningServers = partitions.get(true); + activeServers = partitions.get(false); + this.allServers = servers; + } + + void run() + { + if (activeServers.isEmpty() || (activeServers.size() <= 1 && decommissioningServers.isEmpty())) { + log.warn( + "Skipping balance for tier [%s] with [%d] active servers and [%d] decomissioning servers.", + tier, activeServers.size(), decommissioningServers.size() + ); + return; + } + + log.info( + "Balancing segments in tier [%s] with [%d] active servers and [%d] decommissioning servers.", + tier, activeServers.size(), decommissioningServers.size() + ); + + // Move segments from decommissioning to active servers + int maxDecommPercentToMove = dynamicConfig.getDecommissioningMaxPercentOfMaxSegmentsToMove(); Review Comment: ## Deprecated method or constructor invocation Invoking [CoordinatorDynamicConfig.getDecommissioningMaxPercentOfMaxSegmentsToMove](1) should be avoided because it has been deprecated. [Show more details](https://github.com/apache/druid/security/code-scanning/4808) ########## server/src/main/java/org/apache/druid/server/coordinator/duty/TierSegmentBalancer.java: ########## @@ -0,0 +1,293 @@ +/* + * 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.druid.server.coordinator.duty; + +import com.google.common.collect.Lists; +import org.apache.druid.client.ImmutableDruidDataSource; +import org.apache.druid.java.util.emitter.EmittingLogger; +import org.apache.druid.server.coordinator.BalancerSegmentHolder; +import org.apache.druid.server.coordinator.BalancerStrategy; +import org.apache.druid.server.coordinator.CoordinatorDynamicConfig; +import org.apache.druid.server.coordinator.DruidCoordinatorRuntimeParams; +import org.apache.druid.server.coordinator.SegmentLoader; +import org.apache.druid.server.coordinator.ServerHolder; +import org.apache.druid.server.coordinator.stats.CoordinatorRunStats; +import org.apache.druid.server.coordinator.stats.Dimension; +import org.apache.druid.server.coordinator.stats.RowKey; +import org.apache.druid.server.coordinator.stats.Stats; +import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.SegmentId; + +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; + +/** + * Balances segments within the servers of a tier using the balancer strategy. + * Segments are prioritized for move in the following order: + * <ul> + * <li>Segments loaded on decommissioning servers</li> + * <li>Segments loading on active servers</li> + * <li>Segments loaded on active servers</li> + * </ul> + */ +public class TierSegmentBalancer +{ + private static final EmittingLogger log = new EmittingLogger(TierSegmentBalancer.class); + + private final String tier; + private final DruidCoordinatorRuntimeParams params; + private final SegmentLoader loader; + + private final BalancerStrategy strategy; + private final CoordinatorDynamicConfig dynamicConfig; + private final CoordinatorRunStats runStats; + + private final Set<ServerHolder> allServers; + private final List<ServerHolder> activeServers; + private final List<ServerHolder> decommissioningServers; + private final int totalMaxSegmentsToMove; + + public TierSegmentBalancer( + String tier, + Set<ServerHolder> servers, + SegmentLoader loader, + DruidCoordinatorRuntimeParams params + ) + { + this.tier = tier; + this.params = params; + this.loader = loader; + + this.strategy = params.getBalancerStrategy(); + this.dynamicConfig = params.getCoordinatorDynamicConfig(); + this.totalMaxSegmentsToMove = dynamicConfig.getMaxSegmentsToMove(); + this.runStats = loader.getStats(); + + Map<Boolean, List<ServerHolder>> partitions = + servers.stream().collect(Collectors.partitioningBy(ServerHolder::isDecommissioning)); + decommissioningServers = partitions.get(true); + activeServers = partitions.get(false); + this.allServers = servers; + } + + void run() + { + if (activeServers.isEmpty() || (activeServers.size() <= 1 && decommissioningServers.isEmpty())) { + log.warn( + "Skipping balance for tier [%s] with [%d] active servers and [%d] decomissioning servers.", + tier, activeServers.size(), decommissioningServers.size() + ); + return; + } + + log.info( + "Balancing segments in tier [%s] with [%d] active servers and [%d] decommissioning servers.", + tier, activeServers.size(), decommissioningServers.size() + ); + + // Move segments from decommissioning to active servers + int maxDecommPercentToMove = dynamicConfig.getDecommissioningMaxPercentOfMaxSegmentsToMove(); + int maxDecommSegmentsToMove = (int) Math.ceil(totalMaxSegmentsToMove * (maxDecommPercentToMove / 100.0)); + int movedDecommSegments = moveSegmentsFromTo(decommissioningServers, activeServers, maxDecommSegmentsToMove); + log.info( + "Moved [%d] segments out of max [%d (%d%%)] from decommissioning to active servers.", + movedDecommSegments, maxDecommSegmentsToMove, maxDecommPercentToMove + ); + + // Move segments across active servers + int maxGeneralSegmentsToMove = totalMaxSegmentsToMove - movedDecommSegments; + int movedGeneralSegments = moveSegmentsFromTo(activeServers, activeServers, maxGeneralSegmentsToMove); + log.info( + "Moved [%d] segments out of max [%d] between active servers.", + movedGeneralSegments, maxGeneralSegmentsToMove + ); + + if (dynamicConfig.emitBalancingStats()) { + strategy.emitStats(tier, loader.getStats(), Lists.newArrayList(allServers)); + } + } + + private int moveSegmentsFromTo( + List<ServerHolder> sourceServers, + List<ServerHolder> destServers, + int maxSegmentsToMove + ) + { + if (maxSegmentsToMove <= 0 || sourceServers.isEmpty() || destServers.isEmpty()) { + return 0; + } + + Iterator<BalancerSegmentHolder> pickedSegments + = pickSegmentsFrom(sourceServers, maxSegmentsToMove, true); + int movedCount = moveSegmentsTo(destServers, pickedSegments, maxSegmentsToMove); + + maxSegmentsToMove -= movedCount; + pickedSegments = pickSegmentsFrom(sourceServers, maxSegmentsToMove, false); + movedCount += moveSegmentsTo(destServers, pickedSegments, maxSegmentsToMove); + + return movedCount; + } + + private Iterator<BalancerSegmentHolder> pickSegmentsFrom( + List<ServerHolder> sourceServers, + int maxSegmentsToPick, + boolean pickLoadingSegments + ) + { + if (maxSegmentsToPick <= 0 || sourceServers.isEmpty()) { + return Collections.emptyIterator(); + } else if (dynamicConfig.useBatchedSegmentSampler()) { Review Comment: ## Deprecated method or constructor invocation Invoking [CoordinatorDynamicConfig.useBatchedSegmentSampler](1) should be avoided because it has been deprecated. [Show more details](https://github.com/apache/druid/security/code-scanning/4809) -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected] --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
