You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@druid.apache.org by "kfaraz (via GitHub)" <gi...@apache.org> on 2023/02/16 16:45:16 UTC

[GitHub] [druid] kfaraz opened a new pull request, #13197: Segment loading: Allow cancellation and prioritization of load queue items

kfaraz opened a new pull request, #13197:
URL: https://github.com/apache/druid/pull/13197

   Fixes multiple items in #12881
   
   ### Description
   
   This PR lays the ground work to allow load queue to safely have an unlimited number of items, and thus
   eventually phase out `maxSegmentsInNodeLoadingQueue` and `replicationThrottleLimit`.
   
   Load queue is already allowed to have unlimited items (by setting `maxSegmentsInNodeLoadingQueue = 0`)
   but this leads to
   - each coordinator run taking a very long time
   - poor assignments which take several more runs to be rectified
   
   ### Changes
   
   #### Classes to review
   - `SegmentLoader`
   - `LoadRule`, `BroadcastDistributionRule`
   - `SegmentStateManager`
   - `LoadQueuePeon`: http and curator
   - `SegmentHolder`
   - `LoadRuleTest`
   
   #### Behavioral changes
   
   Change | Motivation
   --------|-------------
   Both loaded and loading items count towards replication. | Allow coordinator to take corrective action of removing superfluous replicas without waiting for them to be fully loaded.
   Load, drop or move operations can be cancelled. | - Allow move of loading items from queue of one server to another. <br> - Allow coordinator to take corrective actions quickly.
   During tier shift, always maintain the currently configured level of replication, no matter which tier it happens to be on. | - Queue drop of unneeded segments as soon as possible, thus allowing faster decommission of servers and freeing up disk space to load new segments.<br>- Always maintain target level of replication, thus ensuring that segment read concurrency does not suffer
   LoadQueuePeon prioritizes segment actions as DROP > LOAD > REPLICATE > MOVE (i.e. balancing). | - Allow prioritization of items, which becomes important if load queue size is unlimited. <br> - Avoid considering balancing items in load queue as over-replicated.
   `replicationThrottleLimit` does not act on a tier if the segment is not loaded on that tier at all | Throttling first replica on a tier undermines the purpose of tiering. Tiering is not meant for fault tolerance, rather serving different query needs. Thus segments should be available on target tiers as soon as possible.
   ⚠️ `maxNonPrimaryReplicantsToLoad` does not act on first replica in any tier | This was done keeping in line with the changes to `replicationThrottleLimit` but it should probably be reverted to prevent unexpected behaviour for clusters using this config. cc: @capistrant 
   `maxSegmentsInNodeLoadingQueue` acts on the number of items assigned to the load queue in the current run rather than the number of items present in the queue at a given time. | Currently, if the configured load queue size is large enough to allow load of some segments while a coordinator run is in progress, the load queue limit is violated as there is always some room in the queue. This causes coordinator runs to get stuck cycling through all the segments in spite of a limited load queue.
   
   #### Structural changes
   
   Change | Motivation
   --------|-----------
   Add `SegmentLoader` which handles all the load, move and drop operations. The lifecycle of the loader is tied to a single coordinator run. In the changed code, it is instantiated once in every run of `RunRules` and `BalanceSegments`. | - Allow reuse of logic for loading, balancing and broadcasting. <br> - Single place to maintain state of a single run thus allowing better metrics and logging.
   Load rules just specify their desired state and leave the actual decision making to the `SegmentLoader`. | Simpler logic for load rules.
   Add `SegmentStateManager` that maintains state across coordinator runs and interacts with the load queues. | - Single place to interact with load queue and maintain state of all in-flight segments <br>- Allow reporting of metrics from queue callbacks. <br>- Prevent callbacks from holding references to items from the previous coordinator run.
   
   #### New metrics
   - `segment/cancelLoad/count`
   - `segment/cancelDrop/count`
   - `segment/broadcastLoad/count`
   - `segment/broadcastDrop/count`
   - some more being added
   
   ### Further work
   - Fix timeout behaviour and corresponding tests
   - Fix balancing strategy to ensure that unlimited load queues do not cause coordinator runs to take forever.
   - Allow balancing strategy to pick moving segments (this has been allowed in this PR but not enabled in any of the existing strategies)
   - Add metrics to identify time spent by items in load queue
   
   <hr>
   
   This PR has:
   - [ ] been self-reviewed.
      - [ ] using the [concurrency checklist](https://github.com/apache/druid/blob/master/dev/code-review/concurrency.md) (Remove this item if the PR doesn't have any relation to concurrency.)
   - [ ] added documentation for new or modified features or behaviors.
   - [x] added Javadocs for most classes and all non-trivial methods. Linked related entities via Javadoc links.
   - [ ] added or updated version, license, or notice information in [licenses.yaml](https://github.com/apache/druid/blob/master/dev/license.md)
   - [ ] added comments explaining the "why" and the intent of the code wherever would not be obvious for an unfamiliar reader.
   - [x] added unit tests or modified existing tests to cover new code paths, ensuring the threshold for [code coverage](https://github.com/apache/druid/blob/master/dev/code-review/code-coverage.md) is met.
   - [ ] added integration tests.
   - [ ] been tested in a test Druid cluster.
   


-- 
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: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] kfaraz commented on a diff in pull request #13197: Segment loading: Allow cancellation and prioritization of load queue items

Posted by GitBox <gi...@apache.org>.
kfaraz commented on code in PR #13197:
URL: https://github.com/apache/druid/pull/13197#discussion_r1009010534


##########
server/src/main/java/org/apache/druid/server/coordinator/SegmentAction.java:
##########
@@ -0,0 +1,33 @@
+/*
+ * 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;
+
+/**
+ * Represents actions that can be performed on a server for a single segment.
+ * <p>
+ * The different action types can be used to prioritize items in a LoadQueuePeon.
+ */
+public enum SegmentAction
+{
+  DROP,
+  LOAD_AS_PRIMARY,

Review Comment:
   Renamed to `LOAD`, `PRIORITY_LOAD`.



-- 
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: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] imply-cheddar commented on a diff in pull request #13197: Segment loading: Allow cancellation and prioritization of load queue items

Posted by GitBox <gi...@apache.org>.
imply-cheddar commented on code in PR #13197:
URL: https://github.com/apache/druid/pull/13197#discussion_r994014871


##########
server/src/main/java/org/apache/druid/server/coordinator/BalancerStrategy.java:
##########
@@ -46,13 +46,18 @@
   ServerHolder findNewSegmentHomeBalancer(DataSegment proposalSegment, List<ServerHolder> serverHolders);
 
   /**
-   * Find the best server on which to place a {@link DataSegment} replica according to the balancing strategy
+   * Finds the best servers on which to place a replica of the {@code proposalSegment}
+   * according to the balancing strategy.

Review Comment:
   This comment makes the method seem like it's finding homes for replicas.  But the name of the methods makes me think it's finding homes for new segments.  Or maybe "new" is trying to modify "home" instead of segment, English is hard.  
   
   Either way, I think that we would do better to have a much more clear distinction in language between a replica (i.e. the assignment of a segment to a node when the data is already available in the tier) versus a "new segment" (i.e. the assignment of a segment to a node when the data is currently unavailable in the tier).



##########
server/src/main/java/org/apache/druid/server/coordinator/CuratorLoadQueuePeon.java:
##########
@@ -181,10 +183,10 @@ public int getNumberOfSegmentsInQueue()
   }
 
   @Override
-  public void loadSegment(final DataSegment segment, @Nullable final LoadPeonCallback callback)
+  public void loadSegment(final DataSegment segment, SegmentAction action, @Nullable final LoadPeonCallback callback)
   {
-    SegmentHolder segmentHolder = new SegmentHolder(segment, Action.LOAD, Collections.singletonList(callback));
-    final SegmentHolder existingHolder = segmentsToLoad.putIfAbsent(segment, segmentHolder);
+    QueuedSegment segmentHolder = new QueuedSegment(segment, action, callback);

Review Comment:
   this variable is now not a great name?
   
   That said, I would've expected the queue itself to return a `QueuedSegment` rather than something creating one of those before it's actually queued...



##########
server/src/main/java/org/apache/druid/server/coordinator/DruidCluster.java:
##########
@@ -138,7 +138,7 @@ public Set<ServerHolder> getBrokers()
     return brokers;
   }
 
-  public Iterable<String> getTierNames()
+  public Set<String> getTierNames()

Review Comment:
   Remind me if `DruidCluster` is mutable or not?  I ask because the switch of this response can enable people to think that they got a set, so they can mutate it.  Or otherwise if the `historicals.keySet()` is changing, they could see those changes when they didn't expect to.  The old signature would've likely caused used to read the `Iterable` into something else "effecting" immutable usage.  I'm wondering how much risk there is of that sort of thing going on?
   
   I.e. how beneficial is it really to return the Set from here instead of having the call sites build their own Set if they want a Set?



##########
server/src/main/java/org/apache/druid/server/coordinator/SegmentLoader.java:
##########
@@ -0,0 +1,472 @@
+/*
+ * 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;
+
+import com.google.common.collect.Sets;
+import org.apache.druid.java.util.emitter.EmittingLogger;
+import org.apache.druid.timeline.DataSegment;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.EnumMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.stream.Collectors;
+
+/**
+ * Used by the coordinator in each run for segment loading, dropping, balancing
+ * and broadcasting.
+ * <p>
+ * An instance of this class is freshly created for each coordinator run.
+ */
+public class SegmentLoader
+{
+  private static final EmittingLogger log = new EmittingLogger(SegmentLoader.class);
+
+  private final SegmentStateManager stateManager;
+  private final DruidCluster cluster;
+  private final CoordinatorStats stats = new CoordinatorStats();
+  private final SegmentReplicantLookup replicantLookup;
+  private final BalancerStrategy strategy;
+  private final int maxLoadQueueSize;
+
+  public SegmentLoader(SegmentStateManager stateManager, DruidCoordinatorRuntimeParams runParams)
+  {
+    this.stateManager = stateManager;
+    this.strategy = runParams.getBalancerStrategy();
+    this.cluster = runParams.getDruidCluster();
+    this.replicantLookup = runParams.getSegmentReplicantLookup();
+    this.maxLoadQueueSize = runParams.getCoordinatorDynamicConfig()
+                                     .getMaxSegmentsInNodeLoadingQueue();
+  }
+
+  public CoordinatorStats getStats()
+  {
+    return stats;
+  }
+
+  /**
+   * Moves the given segment between two servers of the same tier.
+   * <p>
+   * See if we can move balancing here.
+   */
+  public boolean moveSegment(DataSegment segment, ServerHolder fromServer, ServerHolder toServer)
+  {
+    if (!fromServer.getServer().getTier().equals(toServer.getServer().getTier())) {
+      return false;
+    }
+
+    // fromServer must be loading or serving the segment
+    // and toServer must be able to load it
+    final SegmentState stateOnSrc = fromServer.getSegmentState(segment);
+    if ((stateOnSrc != SegmentState.LOADING && stateOnSrc != SegmentState.LOADED)
+        || !canLoadSegment(toServer, segment)) {
+      return false;
+    }
+
+    final boolean cancelSuccess = stateOnSrc == SegmentState.LOADING
+                                  && stateManager.cancelOperation(SegmentState.LOADING, segment, fromServer);
+
+    if (cancelSuccess) {
+      int loadedCountOnTier = replicantLookup
+          .getLoadedReplicants(segment.getId(), toServer.getServer().getTier());
+      stateManager.loadSegment(segment, toServer, loadedCountOnTier < 1);
+    } else {
+      return stateManager.moveSegment(segment, fromServer, toServer);
+    }
+
+    return true;
+  }
+
+  /**
+   * Queues load or drop of replicas of the given segment to achieve the
+   * target replication level in all the tiers.
+   */
+  public void updateReplicas(DataSegment segment, Map<String, Integer> tierToReplicaCount)
+  {
+    // Handle every target tier
+    tierToReplicaCount.forEach((tier, numReplicas) -> {
+      updateReplicasOnTier(segment, tier, tierToReplicaCount.get(tier));
+      stats.addToTieredStat(CoordinatorStats.REQUIRED_CAPACITY, tier, segment.getSize() * numReplicas);
+    });
+
+    // Find the minimum number of segments required for fault tolerance
+    final int totalTargetReplicas = tierToReplicaCount.values().stream()
+                                                      .reduce(0, Integer::sum);
+    final int minLoadedSegments = totalTargetReplicas > 1 ? 2 : 1;
+
+    // Drop segment from unneeded tiers if requirement is met across target tiers
+    int loadedTargetReplicas = 0;
+    final Set<String> targetTiers = tierToReplicaCount.keySet();
+    for (String tier : targetTiers) {
+      loadedTargetReplicas += replicantLookup.getLoadedReplicants(segment.getId(), tier);
+    }
+    if (loadedTargetReplicas < minLoadedSegments) {
+      return;
+    }

Review Comment:
   I'm perhaps missing something here, but the logic seems to be saying "if the replication factor is set higher than 1, make sure that there are 2 replicas and no more", which seems... weird.  We should have the target number that the user told us to have?



##########
server/src/main/java/org/apache/druid/server/coordinator/CuratorLoadQueuePeon.java:
##########
@@ -63,7 +56,7 @@
  * of the same or different methods.
  */
 @Deprecated
-public class CuratorLoadQueuePeon extends LoadQueuePeon
+public class CuratorLoadQueuePeon implements LoadQueuePeon

Review Comment:
   I *think* the change is trying to simplify the management of the queues.  In which case, I *think* the abstraction you want is a `LoadQueue` that is given segment load/drop requests and then can be read from "in order".  That could then be used by either Peon to do what it needs to that.
   
   Additionally though, the CuratorLoadQueuePeon is effectively broken at this point anyway because it puts all of the znodes on ZK as quickly as possible and it's probably too expensive to really fix (we should just use http and ignore zk for this), so I don't see a reason to fix it.  So, another approach is to consider the ZK based stuff dead and only improve on the http stuff.
   
   We should likely queue up the death of the zk-based stuff too.



##########
server/src/main/java/org/apache/druid/server/coordinator/SegmentLoader.java:
##########
@@ -0,0 +1,472 @@
+/*
+ * 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;
+
+import com.google.common.collect.Sets;
+import org.apache.druid.java.util.emitter.EmittingLogger;
+import org.apache.druid.timeline.DataSegment;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.EnumMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.stream.Collectors;
+
+/**
+ * Used by the coordinator in each run for segment loading, dropping, balancing
+ * and broadcasting.
+ * <p>
+ * An instance of this class is freshly created for each coordinator run.
+ */
+public class SegmentLoader
+{
+  private static final EmittingLogger log = new EmittingLogger(SegmentLoader.class);
+
+  private final SegmentStateManager stateManager;
+  private final DruidCluster cluster;
+  private final CoordinatorStats stats = new CoordinatorStats();
+  private final SegmentReplicantLookup replicantLookup;
+  private final BalancerStrategy strategy;
+  private final int maxLoadQueueSize;
+
+  public SegmentLoader(SegmentStateManager stateManager, DruidCoordinatorRuntimeParams runParams)
+  {
+    this.stateManager = stateManager;
+    this.strategy = runParams.getBalancerStrategy();
+    this.cluster = runParams.getDruidCluster();
+    this.replicantLookup = runParams.getSegmentReplicantLookup();
+    this.maxLoadQueueSize = runParams.getCoordinatorDynamicConfig()
+                                     .getMaxSegmentsInNodeLoadingQueue();
+  }
+
+  public CoordinatorStats getStats()
+  {
+    return stats;
+  }
+
+  /**
+   * Moves the given segment between two servers of the same tier.
+   * <p>
+   * See if we can move balancing here.
+   */
+  public boolean moveSegment(DataSegment segment, ServerHolder fromServer, ServerHolder toServer)
+  {
+    if (!fromServer.getServer().getTier().equals(toServer.getServer().getTier())) {
+      return false;
+    }
+
+    // fromServer must be loading or serving the segment
+    // and toServer must be able to load it
+    final SegmentState stateOnSrc = fromServer.getSegmentState(segment);
+    if ((stateOnSrc != SegmentState.LOADING && stateOnSrc != SegmentState.LOADED)
+        || !canLoadSegment(toServer, segment)) {
+      return false;
+    }
+
+    final boolean cancelSuccess = stateOnSrc == SegmentState.LOADING
+                                  && stateManager.cancelOperation(SegmentState.LOADING, segment, fromServer);
+
+    if (cancelSuccess) {
+      int loadedCountOnTier = replicantLookup
+          .getLoadedReplicants(segment.getId(), toServer.getServer().getTier());
+      stateManager.loadSegment(segment, toServer, loadedCountOnTier < 1);
+    } else {
+      return stateManager.moveSegment(segment, fromServer, toServer);
+    }
+
+    return true;
+  }
+
+  /**
+   * Queues load or drop of replicas of the given segment to achieve the
+   * target replication level in all the tiers.
+   */
+  public void updateReplicas(DataSegment segment, Map<String, Integer> tierToReplicaCount)
+  {
+    // Handle every target tier
+    tierToReplicaCount.forEach((tier, numReplicas) -> {
+      updateReplicasOnTier(segment, tier, tierToReplicaCount.get(tier));
+      stats.addToTieredStat(CoordinatorStats.REQUIRED_CAPACITY, tier, segment.getSize() * numReplicas);
+    });
+
+    // Find the minimum number of segments required for fault tolerance
+    final int totalTargetReplicas = tierToReplicaCount.values().stream()
+                                                      .reduce(0, Integer::sum);
+    final int minLoadedSegments = totalTargetReplicas > 1 ? 2 : 1;
+
+    // Drop segment from unneeded tiers if requirement is met across target tiers
+    int loadedTargetReplicas = 0;
+    final Set<String> targetTiers = tierToReplicaCount.keySet();
+    for (String tier : targetTiers) {
+      loadedTargetReplicas += replicantLookup.getLoadedReplicants(segment.getId(), tier);
+    }
+    if (loadedTargetReplicas < minLoadedSegments) {
+      return;
+    }
+
+    final Set<String> dropTiers = Sets.newHashSet(cluster.getTierNames());
+    dropTiers.removeAll(targetTiers);
+    for (String dropTier : dropTiers) {
+      updateReplicasOnTier(segment, dropTier, 0);
+    }
+  }
+
+  /**
+   * Broadcasts the given segment to all servers that are broadcast targets and
+   * queues a drop of the segment from decommissioning servers.
+   */
+  public void broadcastSegment(DataSegment segment)
+  {
+    int assignedCount = 0;
+    int droppedCount = 0;
+    for (ServerHolder server : cluster.getAllServers()) {
+      // Ignore servers which are not broadcast targets
+      if (!server.getServer().getType().isSegmentBroadcastTarget()) {
+        continue;
+      }
+
+      if (server.isDecommissioning()) {
+        droppedCount += dropBroadcastSegment(segment, server) ? 1 : 0;
+      } else {
+        assignedCount += loadBroadcastSegment(segment, server) ? 1 : 0;
+      }
+    }
+
+    if (assignedCount > 0) {
+      stats.addToDataSourceStat(CoordinatorStats.BROADCAST_LOADS, segment.getDataSource(), assignedCount);
+    }
+    if (droppedCount > 0) {
+      stats.addToDataSourceStat(CoordinatorStats.BROADCAST_DROPS, segment.getDataSource(), droppedCount);
+    }
+  }
+
+  /**
+   * Marks the given segment as unused.
+   */
+  public void deleteSegment(DataSegment segment)
+  {
+    stateManager.deleteSegment(segment);
+    stats.addToGlobalStat(CoordinatorStats.DELETED_SEGMENTS, 1);
+  }
+
+  /**
+   * Checks if the server can load the given segment.
+   * <p>
+   * A load is possible only if the server meets all of the following criteria:
+   * <ul>
+   *   <li>is not already serving or loading the segment</li>
+   *   <li>is not being decommissioned</li>
+   *   <li>has not already exceeded the load queue limit in this run</li>
+   *   <li>has available disk space</li>
+   * </ul>
+   */
+  public boolean canLoadSegment(ServerHolder server, DataSegment segment)
+  {
+    return server.canLoadSegment(segment)

Review Comment:
   I find myself wondering why the ServerHolder doesn't know about its own loadQueue?  Seems like this entire thing can be delegated.



##########
server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java:
##########
@@ -255,6 +253,8 @@ public DruidCoordinator(
     this.coordLeaderSelector = coordLeaderSelector;
     this.objectMapper = objectMapper;
     this.compactSegments = initializeCompactSegmentsDuty();
+    this.segmentStateManager =
+        new SegmentStateManager(serverInventoryView, segmentsMetadataManager, taskMaster.isHttpLoading());

Review Comment:
   Mixing `new` with DI always creates sadness.  This should be DI'd not new'd



##########
server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java:
##########
@@ -986,6 +887,7 @@ public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params)
 
       stopPeonsForDisappearedServers(currentServers);
 
+      segmentStateManager.prepareForRun(params);

Review Comment:
   this call is a bit scary to me.  It means that the `segmentStateManager` might be used in a context where it wasn't initialized "properly" (through a bug or whatever).  Perhaps consider the `Coordinator` having a factory for `SegmentStateManager` objects which are build with params, those objects then will only exist fully initialized.  Alternatively, the params could be passed in as part of the arguments to the various methods on `SegmentStateManager` instead.



##########
server/src/main/java/org/apache/druid/server/coordinator/HttpLoadQueuePeon.java:
##########
@@ -366,8 +372,9 @@ public void stop()
   }
 
   @Override
-  public void loadSegment(DataSegment segment, LoadPeonCallback callback)
+  public void loadSegment(DataSegment segment, SegmentAction action, LoadPeonCallback callback)
   {
+    Preconditions.checkArgument(action != SegmentAction.DROP);

Review Comment:
   If this exception actually gets thrown, what's that going to do to the code cycle?  Will it effectively cause a deadlock in the coordinator because every run will cause the exception to get thrown and it never makes progress?  Perhaps it would be better to check, log and return with an immediate failure on the callback or something than to throw an exception and potentially halt progress?



##########
server/src/main/java/org/apache/druid/server/coordinator/duty/EmitClusterStatsAndMetrics.java:
##########
@@ -43,41 +42,31 @@ public class EmitClusterStatsAndMetrics implements CoordinatorDuty
 {
   private static final Logger log = new Logger(EmitClusterStatsAndMetrics.class);
 
-  public static final String TOTAL_CAPACITY = "totalCapacity";
   public static final String TOTAL_HISTORICAL_COUNT = "totalHistoricalCount";
   public static final String MAX_REPLICATION_FACTOR = "maxReplicationFactor";
 
   private final DruidCoordinator coordinator;
   private final String groupName;
   private final boolean isContainCompactSegmentDuty;
+  private final ServiceEmitter emitter;
 
-  public EmitClusterStatsAndMetrics(DruidCoordinator coordinator, String groupName, boolean isContainCompactSegmentDuty)
+  public EmitClusterStatsAndMetrics(
+      DruidCoordinator coordinator,
+      String groupName,
+      boolean isContainCompactSegmentDuty,
+      ServiceEmitter emitter

Review Comment:
   The old code passed the emitter in on the actual emit call instead of on the constructor here.  Technically speaking, the `ServiceEmitter` carries with it a number of dimensions that it automatically adds to metrics, so having it at the call-site instead of the constructor allows for the metrics to be emitted with different sets of dimensions that this class doesn't know about.  I do now know if we were actually taking advantage of that in the previous code, but if we were then this refactor could be breaking things.  Have you validated that we aren't taking advantage of that?



##########
server/src/main/java/org/apache/druid/server/coordinator/HttpLoadQueuePeon.java:
##########
@@ -470,134 +486,98 @@ public Set<DataSegment> getSegmentsMarkedToDrop()
     return Collections.unmodifiableSet(segmentsMarkedToDrop);
   }
 
-  private abstract class SegmentHolder
+  /**
+   * A request is considered to have timed out if the time elapsed since it was
+   * first sent to the server is greater than the configured load timeout.
+   *
+   * @see DruidCoordinatorConfig#getLoadTimeoutDelay()
+   */
+  private boolean hasRequestTimedOut(QueuedSegment holder)
   {
-    private final DataSegment segment;
-    private final DataSegmentChangeRequest changeRequest;
-    private final List<LoadPeonCallback> callbacks = new ArrayList<>();
-
-    // Time when this request was sent to target server the first time.
-    private volatile long scheduleTime = -1;
-
-    private SegmentHolder(
-        DataSegment segment,
-        DataSegmentChangeRequest changeRequest,
-        LoadPeonCallback callback
-    )
-    {
-      this.segment = segment;
-      this.changeRequest = changeRequest;
-
-      if (callback != null) {
-        this.callbacks.add(callback);
-      }
-    }
-
-    public void addCallback(LoadPeonCallback newCallback)
-    {
-      synchronized (callbacks) {
-        if (newCallback != null) {
-          callbacks.add(newCallback);
-        }
-      }
-    }
-
-    public DataSegment getSegment()
-    {
-      return segment;
-    }
-
-    public DataSegmentChangeRequest getChangeRequest()
-    {
-      return changeRequest;
-    }
-
-    public boolean hasTimedOut()
-    {
-      if (scheduleTime < 0) {
-        scheduleTime = System.currentTimeMillis();
-        return false;
-      } else if (System.currentTimeMillis() - scheduleTime > config.getLoadTimeoutDelay().getMillis()) {
-        return true;
-      } else {
-        return false;
-      }
-    }
+    return System.currentTimeMillis() - holder.getFirstRequestTimeMillis()

Review Comment:
   I fear that this logic is broken.  The call to `getFirstRequestTimeMillis()` ends up actually setting the current timestamp.  But, I don't know why this check is only actually done once the request is sent to the downstream server.  Rather than relying on such side-effects we would be better off having an explicit lifecycle to the assignment and asking the holder itself how long it has been queued at the server.  I.e. `holder.getTimeElapsedSinceFirstRequestToServer()` or something like that, which can return `null` if there hasn't been any request to the server (and thus should never be timed out)



##########
server/src/main/java/org/apache/druid/server/coordinator/RandomBalancerStrategy.java:
##########
@@ -26,24 +26,20 @@
 import java.util.Iterator;
 import java.util.List;
 import java.util.NavigableSet;
-import java.util.concurrent.ThreadLocalRandom;
 import java.util.stream.Collectors;
 
 public class RandomBalancerStrategy implements BalancerStrategy
 {
   @Override
-  public ServerHolder findNewSegmentHomeReplicator(DataSegment proposalSegment, List<ServerHolder> serverHolders)
+  public Iterator<ServerHolder> findNewSegmentHomeReplicator(DataSegment proposalSegment, List<ServerHolder> serverHolders)

Review Comment:
   Sometimes returning an `Iterator<>` is right, this signature feels like we might as well just return the `List<>` instead?



##########
server/src/main/java/org/apache/druid/server/coordinator/SegmentLoader.java:
##########
@@ -0,0 +1,472 @@
+/*
+ * 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;
+
+import com.google.common.collect.Sets;
+import org.apache.druid.java.util.emitter.EmittingLogger;
+import org.apache.druid.timeline.DataSegment;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.EnumMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.stream.Collectors;
+
+/**
+ * Used by the coordinator in each run for segment loading, dropping, balancing
+ * and broadcasting.
+ * <p>
+ * An instance of this class is freshly created for each coordinator run.
+ */
+public class SegmentLoader
+{
+  private static final EmittingLogger log = new EmittingLogger(SegmentLoader.class);
+
+  private final SegmentStateManager stateManager;
+  private final DruidCluster cluster;
+  private final CoordinatorStats stats = new CoordinatorStats();
+  private final SegmentReplicantLookup replicantLookup;
+  private final BalancerStrategy strategy;
+  private final int maxLoadQueueSize;
+
+  public SegmentLoader(SegmentStateManager stateManager, DruidCoordinatorRuntimeParams runParams)
+  {
+    this.stateManager = stateManager;
+    this.strategy = runParams.getBalancerStrategy();
+    this.cluster = runParams.getDruidCluster();
+    this.replicantLookup = runParams.getSegmentReplicantLookup();
+    this.maxLoadQueueSize = runParams.getCoordinatorDynamicConfig()
+                                     .getMaxSegmentsInNodeLoadingQueue();
+  }
+
+  public CoordinatorStats getStats()
+  {
+    return stats;
+  }
+
+  /**
+   * Moves the given segment between two servers of the same tier.
+   * <p>
+   * See if we can move balancing here.
+   */
+  public boolean moveSegment(DataSegment segment, ServerHolder fromServer, ServerHolder toServer)
+  {
+    if (!fromServer.getServer().getTier().equals(toServer.getServer().getTier())) {
+      return false;
+    }
+
+    // fromServer must be loading or serving the segment
+    // and toServer must be able to load it
+    final SegmentState stateOnSrc = fromServer.getSegmentState(segment);
+    if ((stateOnSrc != SegmentState.LOADING && stateOnSrc != SegmentState.LOADED)
+        || !canLoadSegment(toServer, segment)) {
+      return false;
+    }
+
+    final boolean cancelSuccess = stateOnSrc == SegmentState.LOADING
+                                  && stateManager.cancelOperation(SegmentState.LOADING, segment, fromServer);
+
+    if (cancelSuccess) {
+      int loadedCountOnTier = replicantLookup
+          .getLoadedReplicants(segment.getId(), toServer.getServer().getTier());
+      stateManager.loadSegment(segment, toServer, loadedCountOnTier < 1);
+    } else {
+      return stateManager.moveSegment(segment, fromServer, toServer);
+    }
+
+    return true;
+  }
+
+  /**
+   * Queues load or drop of replicas of the given segment to achieve the
+   * target replication level in all the tiers.
+   */
+  public void updateReplicas(DataSegment segment, Map<String, Integer> tierToReplicaCount)
+  {
+    // Handle every target tier
+    tierToReplicaCount.forEach((tier, numReplicas) -> {
+      updateReplicasOnTier(segment, tier, tierToReplicaCount.get(tier));
+      stats.addToTieredStat(CoordinatorStats.REQUIRED_CAPACITY, tier, segment.getSize() * numReplicas);
+    });
+
+    // Find the minimum number of segments required for fault tolerance
+    final int totalTargetReplicas = tierToReplicaCount.values().stream()
+                                                      .reduce(0, Integer::sum);
+    final int minLoadedSegments = totalTargetReplicas > 1 ? 2 : 1;
+
+    // Drop segment from unneeded tiers if requirement is met across target tiers
+    int loadedTargetReplicas = 0;
+    final Set<String> targetTiers = tierToReplicaCount.keySet();
+    for (String tier : targetTiers) {
+      loadedTargetReplicas += replicantLookup.getLoadedReplicants(segment.getId(), tier);
+    }
+    if (loadedTargetReplicas < minLoadedSegments) {
+      return;
+    }
+
+    final Set<String> dropTiers = Sets.newHashSet(cluster.getTierNames());
+    dropTiers.removeAll(targetTiers);
+    for (String dropTier : dropTiers) {
+      updateReplicasOnTier(segment, dropTier, 0);
+    }
+  }
+
+  /**
+   * Broadcasts the given segment to all servers that are broadcast targets and
+   * queues a drop of the segment from decommissioning servers.
+   */
+  public void broadcastSegment(DataSegment segment)
+  {
+    int assignedCount = 0;
+    int droppedCount = 0;
+    for (ServerHolder server : cluster.getAllServers()) {
+      // Ignore servers which are not broadcast targets
+      if (!server.getServer().getType().isSegmentBroadcastTarget()) {
+        continue;
+      }
+
+      if (server.isDecommissioning()) {
+        droppedCount += dropBroadcastSegment(segment, server) ? 1 : 0;
+      } else {
+        assignedCount += loadBroadcastSegment(segment, server) ? 1 : 0;
+      }
+    }
+
+    if (assignedCount > 0) {
+      stats.addToDataSourceStat(CoordinatorStats.BROADCAST_LOADS, segment.getDataSource(), assignedCount);
+    }
+    if (droppedCount > 0) {
+      stats.addToDataSourceStat(CoordinatorStats.BROADCAST_DROPS, segment.getDataSource(), droppedCount);
+    }
+  }
+
+  /**
+   * Marks the given segment as unused.
+   */
+  public void deleteSegment(DataSegment segment)
+  {
+    stateManager.deleteSegment(segment);
+    stats.addToGlobalStat(CoordinatorStats.DELETED_SEGMENTS, 1);
+  }
+
+  /**
+   * Checks if the server can load the given segment.
+   * <p>
+   * A load is possible only if the server meets all of the following criteria:
+   * <ul>
+   *   <li>is not already serving or loading the segment</li>
+   *   <li>is not being decommissioned</li>
+   *   <li>has not already exceeded the load queue limit in this run</li>
+   *   <li>has available disk space</li>
+   * </ul>
+   */
+  public boolean canLoadSegment(ServerHolder server, DataSegment segment)
+  {
+    return server.canLoadSegment(segment)
+           && (maxLoadQueueSize == 0 || maxLoadQueueSize > server.getSegmentsQueuedForLoad());
+  }
+
+  /**
+   * Loads the broadcast segment if it is not loaded on the given server.
+   * Returns true only if the segment was successfully queued for load on the server.
+   */
+  private boolean loadBroadcastSegment(DataSegment segment, ServerHolder server)
+  {
+    final SegmentState state = server.getSegmentState(segment);
+    if (state == SegmentState.LOADED || state == SegmentState.LOADING) {
+      return false;
+    }
+
+    // Cancel drop if it is in progress
+    boolean dropCancelled = stateManager.cancelOperation(SegmentState.DROPPING, segment, server);
+    if (dropCancelled) {
+      return false;
+    }
+
+    if (canLoadSegment(server, segment)
+        && stateManager.loadSegment(segment, server, true)) {
+      return true;
+    } else {
+      log.makeAlert("Failed to broadcast segment for [%s]", segment.getDataSource())

Review Comment:
   Be careful with messages in this code.  "Failed to broadcast segment" is ambiguous about whether there was an issue with the server actually downloading the segment (definitely not the case given the code here) versus an issue with the coordinator believing that it is safe to assign the segment (much more likely).  We should be very explicit about what it is that is happening and, also, what we might expect the end user to do about it.



##########
server/src/main/java/org/apache/druid/server/coordinator/HttpLoadQueuePeon.java:
##########
@@ -65,7 +66,7 @@
 
 /**
  */
-public class HttpLoadQueuePeon extends LoadQueuePeon
+public class HttpLoadQueuePeon implements LoadQueuePeon

Review Comment:
   Fwiw, I feel like the rename of `SegmentHolder` to `QueuedSegment` is a bit gratuitous. It's generating a lot of changes and I'm not sure it's really that much more descriptive.  Sometimes renames are helpful, but this one I'm honestly wondering if the scales aren't tipped towards it just making it harder to sift out the signal from the noise in the review.



-- 
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: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] kfaraz commented on a diff in pull request #13197: Segment loading: Allow cancellation and prioritization of load queue items

Posted by GitBox <gi...@apache.org>.
kfaraz commented on code in PR #13197:
URL: https://github.com/apache/druid/pull/13197#discussion_r994052485


##########
server/src/main/java/org/apache/druid/server/coordinator/SegmentLoader.java:
##########
@@ -0,0 +1,472 @@
+/*
+ * 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;
+
+import com.google.common.collect.Sets;
+import org.apache.druid.java.util.emitter.EmittingLogger;
+import org.apache.druid.timeline.DataSegment;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.EnumMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.stream.Collectors;
+
+/**
+ * Used by the coordinator in each run for segment loading, dropping, balancing
+ * and broadcasting.
+ * <p>
+ * An instance of this class is freshly created for each coordinator run.
+ */
+public class SegmentLoader
+{
+  private static final EmittingLogger log = new EmittingLogger(SegmentLoader.class);
+
+  private final SegmentStateManager stateManager;
+  private final DruidCluster cluster;
+  private final CoordinatorStats stats = new CoordinatorStats();
+  private final SegmentReplicantLookup replicantLookup;
+  private final BalancerStrategy strategy;
+  private final int maxLoadQueueSize;
+
+  public SegmentLoader(SegmentStateManager stateManager, DruidCoordinatorRuntimeParams runParams)
+  {
+    this.stateManager = stateManager;
+    this.strategy = runParams.getBalancerStrategy();
+    this.cluster = runParams.getDruidCluster();
+    this.replicantLookup = runParams.getSegmentReplicantLookup();
+    this.maxLoadQueueSize = runParams.getCoordinatorDynamicConfig()
+                                     .getMaxSegmentsInNodeLoadingQueue();
+  }
+
+  public CoordinatorStats getStats()
+  {
+    return stats;
+  }
+
+  /**
+   * Moves the given segment between two servers of the same tier.
+   * <p>
+   * See if we can move balancing here.
+   */
+  public boolean moveSegment(DataSegment segment, ServerHolder fromServer, ServerHolder toServer)
+  {
+    if (!fromServer.getServer().getTier().equals(toServer.getServer().getTier())) {
+      return false;
+    }
+
+    // fromServer must be loading or serving the segment
+    // and toServer must be able to load it
+    final SegmentState stateOnSrc = fromServer.getSegmentState(segment);
+    if ((stateOnSrc != SegmentState.LOADING && stateOnSrc != SegmentState.LOADED)
+        || !canLoadSegment(toServer, segment)) {
+      return false;
+    }
+
+    final boolean cancelSuccess = stateOnSrc == SegmentState.LOADING
+                                  && stateManager.cancelOperation(SegmentState.LOADING, segment, fromServer);
+
+    if (cancelSuccess) {
+      int loadedCountOnTier = replicantLookup
+          .getLoadedReplicants(segment.getId(), toServer.getServer().getTier());
+      stateManager.loadSegment(segment, toServer, loadedCountOnTier < 1);
+    } else {
+      return stateManager.moveSegment(segment, fromServer, toServer);
+    }
+
+    return true;
+  }
+
+  /**
+   * Queues load or drop of replicas of the given segment to achieve the
+   * target replication level in all the tiers.
+   */
+  public void updateReplicas(DataSegment segment, Map<String, Integer> tierToReplicaCount)
+  {
+    // Handle every target tier
+    tierToReplicaCount.forEach((tier, numReplicas) -> {
+      updateReplicasOnTier(segment, tier, tierToReplicaCount.get(tier));
+      stats.addToTieredStat(CoordinatorStats.REQUIRED_CAPACITY, tier, segment.getSize() * numReplicas);
+    });
+
+    // Find the minimum number of segments required for fault tolerance
+    final int totalTargetReplicas = tierToReplicaCount.values().stream()
+                                                      .reduce(0, Integer::sum);
+    final int minLoadedSegments = totalTargetReplicas > 1 ? 2 : 1;
+
+    // Drop segment from unneeded tiers if requirement is met across target tiers
+    int loadedTargetReplicas = 0;
+    final Set<String> targetTiers = tierToReplicaCount.keySet();
+    for (String tier : targetTiers) {
+      loadedTargetReplicas += replicantLookup.getLoadedReplicants(segment.getId(), tier);
+    }
+    if (loadedTargetReplicas < minLoadedSegments) {
+      return;
+    }

Review Comment:
   This is the check for queueing up drops in unwanted tiers. If there is even the minimum level of fault tolerance (2 replicas) in the wanted tiers, we can proceed with the drops on unwanted tiers while full replication continues on the wanted tiers.
   
   The assignment to new target tiers is already done by this point.



-- 
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: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] kfaraz commented on a diff in pull request #13197: Segment loading: Allow cancellation and prioritization of load queue items

Posted by GitBox <gi...@apache.org>.
kfaraz commented on code in PR #13197:
URL: https://github.com/apache/druid/pull/13197#discussion_r994083255


##########
server/src/main/java/org/apache/druid/server/coordinator/duty/EmitClusterStatsAndMetrics.java:
##########
@@ -43,41 +42,31 @@ public class EmitClusterStatsAndMetrics implements CoordinatorDuty
 {
   private static final Logger log = new Logger(EmitClusterStatsAndMetrics.class);
 
-  public static final String TOTAL_CAPACITY = "totalCapacity";
   public static final String TOTAL_HISTORICAL_COUNT = "totalHistoricalCount";
   public static final String MAX_REPLICATION_FACTOR = "maxReplicationFactor";
 
   private final DruidCoordinator coordinator;
   private final String groupName;
   private final boolean isContainCompactSegmentDuty;
+  private final ServiceEmitter emitter;
 
-  public EmitClusterStatsAndMetrics(DruidCoordinator coordinator, String groupName, boolean isContainCompactSegmentDuty)
+  public EmitClusterStatsAndMetrics(
+      DruidCoordinator coordinator,
+      String groupName,
+      boolean isContainCompactSegmentDuty,
+      ServiceEmitter emitter

Review Comment:
   There is a single DI'd instance of `ServiceEmitter` that gets passed around. The emitter itself and the dimension map that it carries are immutable, so we should be good there.
   
   Unfortunately, we didn't have any existing test that verified (all) the dimensions that were being emitted in a metric event. Such tests passing would have helped us validate this fully. But at least looking at the code, we shouldn't be breaking anything.
   
   I will be doing some further cluster testing today to further verify the emitted metric events. We should be able to add some sim tests for this too.



-- 
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: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] kfaraz commented on a diff in pull request #13197: Segment loading: Allow cancellation and prioritization of load queue items

Posted by GitBox <gi...@apache.org>.
kfaraz commented on code in PR #13197:
URL: https://github.com/apache/druid/pull/13197#discussion_r994061396


##########
server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java:
##########
@@ -986,6 +887,7 @@ public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params)
 
       stopPeonsForDisappearedServers(currentServers);
 
+      segmentStateManager.prepareForRun(params);

Review Comment:
   Thanks for the suggestion! I was not happy with this myself.



-- 
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: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] kfaraz commented on a diff in pull request #13197: Segment loading: Allow cancellation and prioritization of load queue items

Posted by GitBox <gi...@apache.org>.
kfaraz commented on code in PR #13197:
URL: https://github.com/apache/druid/pull/13197#discussion_r996555534


##########
server/src/main/java/org/apache/druid/server/coordinator/HttpLoadQueuePeon.java:
##########
@@ -470,134 +486,98 @@ public Set<DataSegment> getSegmentsMarkedToDrop()
     return Collections.unmodifiableSet(segmentsMarkedToDrop);
   }
 
-  private abstract class SegmentHolder
+  /**
+   * A request is considered to have timed out if the time elapsed since it was
+   * first sent to the server is greater than the configured load timeout.
+   *
+   * @see DruidCoordinatorConfig#getLoadTimeoutDelay()
+   */
+  private boolean hasRequestTimedOut(QueuedSegment holder)
   {
-    private final DataSegment segment;
-    private final DataSegmentChangeRequest changeRequest;
-    private final List<LoadPeonCallback> callbacks = new ArrayList<>();
-
-    // Time when this request was sent to target server the first time.
-    private volatile long scheduleTime = -1;
-
-    private SegmentHolder(
-        DataSegment segment,
-        DataSegmentChangeRequest changeRequest,
-        LoadPeonCallback callback
-    )
-    {
-      this.segment = segment;
-      this.changeRequest = changeRequest;
-
-      if (callback != null) {
-        this.callbacks.add(callback);
-      }
-    }
-
-    public void addCallback(LoadPeonCallback newCallback)
-    {
-      synchronized (callbacks) {
-        if (newCallback != null) {
-          callbacks.add(newCallback);
-        }
-      }
-    }
-
-    public DataSegment getSegment()
-    {
-      return segment;
-    }
-
-    public DataSegmentChangeRequest getChangeRequest()
-    {
-      return changeRequest;
-    }
-
-    public boolean hasTimedOut()
-    {
-      if (scheduleTime < 0) {
-        scheduleTime = System.currentTimeMillis();
-        return false;
-      } else if (System.currentTimeMillis() - scheduleTime > config.getLoadTimeoutDelay().getMillis()) {
-        return true;
-      } else {
-        return false;
-      }
-    }
+    return System.currentTimeMillis() - holder.getFirstRequestTimeMillis()

Review Comment:
   Fixed.



##########
server/src/main/java/org/apache/druid/server/coordinator/HttpLoadQueuePeon.java:
##########
@@ -366,8 +372,9 @@ public void stop()
   }
 
   @Override
-  public void loadSegment(DataSegment segment, LoadPeonCallback callback)
+  public void loadSegment(DataSegment segment, SegmentAction action, LoadPeonCallback callback)
   {
+    Preconditions.checkArgument(action != SegmentAction.DROP);

Review Comment:
   Fixed.



-- 
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: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] kfaraz commented on a diff in pull request #13197: Segment loading: Allow cancellation and prioritization of load queue items

Posted by "kfaraz (via GitHub)" <gi...@apache.org>.
kfaraz commented on code in PR #13197:
URL: https://github.com/apache/druid/pull/13197#discussion_r1232981566


##########
server/src/main/java/org/apache/druid/server/coordinator/balancer/BalancerStrategy.java:
##########
@@ -36,77 +39,65 @@
  */
 public interface BalancerStrategy
 {
+
   /**
-   * Find the best server to move a {@link DataSegment} to according the balancing strategy.
-   * @param proposalSegment segment to move
-   * @param serverHolders servers to consider as move destinations
+   * Finds the best server to move a segment to according to the balancing strategy.
+   *
+   * @param proposalSegment    segment to move
+   * @param sourceServer       Server the segment is currently placed on.
+   * @param destinationServers servers to consider as move destinations
    * @return The server to move to, or null if no move should be made or no server is suitable
    */
   @Nullable
-  ServerHolder findNewSegmentHomeBalancer(DataSegment proposalSegment, List<ServerHolder> serverHolders);
+  ServerHolder findDestinationServerToMoveSegment(
+      DataSegment proposalSegment,
+      ServerHolder sourceServer,
+      List<ServerHolder> destinationServers
+  );
 
   /**
-   * Finds the best servers on which to place a replica of the {@code proposalSegment}
-   * according to the balancing strategy.
+   * Finds the best servers on which to place the {@code proposalSegment}.
+   * This method can be used both for placing the first copy of a segment
+   * in the tier or a replica of the segment.
    *
-   * @param proposalSegment segment to replicate
-   * @param serverHolders   servers to consider as replica holders
-   * @return Iterator over the best servers (in order) on which the replica(s)
+   * @param proposalSegment segment to place on servers
+   * @param serverHolders   servers to consider as segment homes
+   * @return Iterator over the best servers (in order) on which the segment
    * can be placed.
    */
-  Iterator<ServerHolder> findNewSegmentHomeReplicator(
+  Iterator<ServerHolder> findServersToLoadSegment(
       DataSegment proposalSegment,
       List<ServerHolder> serverHolders
   );
 
   /**
-   * Pick the best segments to move from one of the supplied set of servers according to the balancing strategy.
+   * Picks segments from the given set of servers based on the balancing strategy.
+   * Default behaviour is to pick segments using reservoir sampling.
    *
-   * @param serverHolders set of historicals to consider for moving segments
-   * @param broadcastDatasources Datasources that contain segments which were loaded via broadcast rules.
-   *                             Balancing strategies should avoid rebalancing segments for such datasources, since
-   *                             they should be loaded on all servers anyway.
-   *                             NOTE: this should really be handled on a per-segment basis, to properly support
-   *                                   the interval or period-based broadcast rules. For simplicity of the initial
-   *                                   implementation, only forever broadcast rules are supported.
-   * @param reservoirSize the reservoir size maintained by the Reservoir Sampling algorithm.
-   * @return Iterator for set of {@link BalancerSegmentHolder} containing segment to move and server they currently
-   * reside on, or empty if there are no segments to pick from (i. e. all provided serverHolders are empty).
+   * @param serverHolders        Set of historicals to consider for picking segments
+   * @param broadcastDatasources Segments belonging to these datasources will not
+   *                             be picked for balancing, since they should be
+   *                             loaded on all servers anyway.
+   * @param maxSegmentsToPick    Maximum number of segments to pick
+   * @param pickLoadingSegments  If true, picks only segments currently being
+   *                             loaded on a server. If false, picks segments
+   *                             already loaded on a server.
+   * @return Iterator over {@link BalancerSegmentHolder}s, each of which contains
+   * a segment picked for moving and the server currently serving/loading it.
    */
   default Iterator<BalancerSegmentHolder> pickSegmentsToMove(
       List<ServerHolder> serverHolders,
       Set<String> broadcastDatasources,
-      int reservoirSize
+      int maxSegmentsToPick,
+      boolean pickLoadingSegments

Review Comment:
   Updated:
   1. Removed these methods from `BalancerStrategy`
   2. Added 2 methods to `ReservoirSegmentSampler`, `pickMovableLoadingSegment` and `pickMovableLoadedSegments`, which are used directly by the coordinator duty



-- 
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: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] github-code-scanning[bot] commented on a diff in pull request #13197: Segment loading: Allow cancellation and prioritization of load queue items

Posted by GitBox <gi...@apache.org>.
github-code-scanning[bot] commented on code in PR #13197:
URL: https://github.com/apache/druid/pull/13197#discussion_r1066812010


##########
server/src/test/java/org/apache/druid/server/coordinator/BalanceSegmentsTest.java:
##########
@@ -538,69 +536,61 @@
     mockDruidServer(druidServer3, "3", "normal", 0L, 100L, Collections.emptyList());
     mockDruidServer(druidServer4, "4", "normal", 0L, 100L, Collections.emptyList());
 
-    // Mock stuff that the coordinator needs
-    mockCoordinator(coordinator);
+    DruidCoordinatorRuntimeParams params = defaultRuntimeParamsBuilder(servers).build();
 
-    DruidCoordinatorRuntimeParams params = defaultRuntimeParamsBuilder(druidServers, peons).build();
-
-    params = new BalanceSegmentsTester(coordinator).run(params);
+    params = new BalanceSegments(stateManager).run(params);
     Assert.assertTrue(params.getCoordinatorStats().getTieredStat("movedCount", "normal") > 0);
   }
 
-  /**
-   * Testing that the dynamic coordinator config value, percentOfSegmentsToConsiderPerMove, is honored when calling
-   * out to pickSegmentToMove. This config limits the number of segments that are considered when looking for a segment
-   * to move.
-   */
   @Test
-  public void testThatDynamicConfigIsHonoredWhenPickingSegmentToMove()
+  public void testThatMaxSegmentsToMoveIsHonored()
   {
     mockDruidServer(druidServer1, "1", "normal", 50L, 100L, Arrays.asList(segment1, segment2));
     mockDruidServer(druidServer2, "2", "normal", 30L, 100L, Arrays.asList(segment3, segment4));
     mockDruidServer(druidServer3, "3", "normal", 0L, 100L, Collections.emptyList());
 
     EasyMock.replay(druidServer4);
 
-    mockCoordinator(coordinator);
-
     BalancerStrategy strategy = EasyMock.createMock(BalancerStrategy.class);
 
     // Move from non-decomissioning servers
+    final ServerHolder holder2 = new ServerHolder(druidServer2, peon2);
+
     EasyMock.expect(
         strategy.pickSegmentsToMove(
             ImmutableList.of(
                 new ServerHolder(druidServer3, peon3, false),
-                new ServerHolder(druidServer2, peon2, false),
+                holder2,
                 new ServerHolder(druidServer1, peon1, false)
             ),
             broadcastDatasources,
-            40.0
+            1
         )
     )
-            .andReturn(ImmutableList.of(new BalancerSegmentHolder(druidServer2, segment3)).iterator());
+            .andReturn(ImmutableList.of(new BalancerSegmentHolder(holder2, segment3)).iterator());
 
     EasyMock.expect(strategy.findNewSegmentHomeBalancer(EasyMock.anyObject(), EasyMock.anyObject()))
             .andReturn(new ServerHolder(druidServer3, peon3))
             .anyTimes();
     EasyMock.replay(strategy);
 
     DruidCoordinatorRuntimeParams params = defaultRuntimeParamsBuilder(
-        ImmutableList.of(druidServer1, druidServer2, druidServer3),
-        ImmutableList.of(peon1, peon2, peon3),
-        ImmutableList.of(false, false, false)
+        new ServerHolder(druidServer1, peon1),
+        new ServerHolder(druidServer2, peon2),
+        new ServerHolder(druidServer3, peon3)
     )
         .withDynamicConfigs(
             CoordinatorDynamicConfig.builder()
                                     .withMaxSegmentsToMove(1)
-                                    .withUseBatchedSegmentSampler(false)
+                                    .withUseBatchedSegmentSampler(true)

Review Comment:
   ## Deprecated method or constructor invocation
   
   Invoking [Builder.withUseBatchedSegmentSampler](1) should be avoided because it has been deprecated.
   
   [Show more details](https://github.com/apache/druid/security/code-scanning/3652)



-- 
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: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] github-code-scanning[bot] commented on a diff in pull request #13197: Segment loading: Allow cancellation and prioritization of load queue items

Posted by "github-code-scanning[bot] (via GitHub)" <gi...@apache.org>.
github-code-scanning[bot] commented on code in PR #13197:
URL: https://github.com/apache/druid/pull/13197#discussion_r1167364695


##########
server/src/main/java/org/apache/druid/server/coordinator/balancer/TierSegmentBalancer.java:
##########
@@ -0,0 +1,291 @@
+/*
+ * 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.balancer;
+
+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.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;
+  }
+
+  public 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/4819)



##########
server/src/main/java/org/apache/druid/server/coordinator/balancer/TierSegmentBalancer.java:
##########
@@ -0,0 +1,291 @@
+/*
+ * 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.balancer;
+
+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.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;
+  }
+
+  public 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/4821)



##########
server/src/main/java/org/apache/druid/server/coordinator/balancer/TierSegmentBalancer.java:
##########
@@ -0,0 +1,291 @@
+/*
+ * 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.balancer;
+
+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.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;
+  }
+
+  public 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/4818)



##########
server/src/main/java/org/apache/druid/server/coordinator/balancer/TierSegmentBalancer.java:
##########
@@ -0,0 +1,291 @@
+/*
+ * 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.balancer;
+
+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.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;
+  }
+
+  public 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/4820)



-- 
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: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] kfaraz commented on a diff in pull request #13197: Segment loading: Allow cancellation and prioritization of load queue items

Posted by GitBox <gi...@apache.org>.
kfaraz commented on code in PR #13197:
URL: https://github.com/apache/druid/pull/13197#discussion_r993141986


##########
server/src/main/java/org/apache/druid/server/coordinator/SegmentStateManager.java:
##########
@@ -0,0 +1,235 @@
+/*
+ * 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;
+
+import org.apache.druid.client.ServerInventoryView;
+import org.apache.druid.java.util.emitter.EmittingLogger;
+import org.apache.druid.metadata.SegmentsMetadataManager;
+import org.apache.druid.timeline.DataSegment;
+
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * Manages state of segments being loaded.
+ */
+public class SegmentStateManager

Review Comment:
   Rename to `LoadQueueManager` as this class really acts as an interface between the duties and the load queues.



-- 
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: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] kfaraz commented on a diff in pull request #13197: Segment loading: Allow cancellation and prioritization of load queue items

Posted by GitBox <gi...@apache.org>.
kfaraz commented on code in PR #13197:
URL: https://github.com/apache/druid/pull/13197#discussion_r993657773


##########
server/src/main/java/org/apache/druid/server/coordinator/TierLoadingState.java:
##########
@@ -0,0 +1,72 @@
+/*
+ * 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;
+
+import org.apache.druid.timeline.SegmentId;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * Contains segments being loaded or replicated in a tier.
+ */
+public class TierLoadingState
+{
+  private int lifetime;

Review Comment:
   That would be the ideal behaviour but as it turns out, this is the existing implementation. I have tried to keep it unchanged for now.
   
   Let's say, in a coordinator run, we queue some segments to move within a tier. All the segments queued would start with the same lifecycle. Only when all these queued segments have been processed will we consider this tier for moving again. So, at it turns out, in `BalanceSegments.currentlyMovingSegments` in the existing code, all segment holders for a given tier will always have the same lifecycle.
   Same principle applies to replication and `ReplicationThrottleLimit.currentlyReplicating`.
   
   This behaviour will be revisited once we make further changes for relaxing replication and queue size throttling.



-- 
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: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] kfaraz commented on a diff in pull request #13197: Segment loading: Allow cancellation and prioritization of load queue items

Posted by GitBox <gi...@apache.org>.
kfaraz commented on code in PR #13197:
URL: https://github.com/apache/druid/pull/13197#discussion_r996555730


##########
server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java:
##########
@@ -986,6 +887,7 @@ public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params)
 
       stopPeonsForDisappearedServers(currentServers);
 
+      segmentStateManager.prepareForRun(params);

Review Comment:
   Fixed this to ensure that `SegmentStateManager` can never be used with improper initialization.



-- 
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: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] kfaraz commented on a diff in pull request #13197: Segment loading: Allow cancellation and prioritization of load queue items

Posted by GitBox <gi...@apache.org>.
kfaraz commented on code in PR #13197:
URL: https://github.com/apache/druid/pull/13197#discussion_r994062342


##########
server/src/main/java/org/apache/druid/server/coordinator/SegmentLoader.java:
##########
@@ -0,0 +1,472 @@
+/*
+ * 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;
+
+import com.google.common.collect.Sets;
+import org.apache.druid.java.util.emitter.EmittingLogger;
+import org.apache.druid.timeline.DataSegment;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.EnumMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.stream.Collectors;
+
+/**
+ * Used by the coordinator in each run for segment loading, dropping, balancing
+ * and broadcasting.
+ * <p>
+ * An instance of this class is freshly created for each coordinator run.
+ */
+public class SegmentLoader
+{
+  private static final EmittingLogger log = new EmittingLogger(SegmentLoader.class);
+
+  private final SegmentStateManager stateManager;
+  private final DruidCluster cluster;
+  private final CoordinatorStats stats = new CoordinatorStats();
+  private final SegmentReplicantLookup replicantLookup;
+  private final BalancerStrategy strategy;
+  private final int maxLoadQueueSize;
+
+  public SegmentLoader(SegmentStateManager stateManager, DruidCoordinatorRuntimeParams runParams)
+  {
+    this.stateManager = stateManager;
+    this.strategy = runParams.getBalancerStrategy();
+    this.cluster = runParams.getDruidCluster();
+    this.replicantLookup = runParams.getSegmentReplicantLookup();
+    this.maxLoadQueueSize = runParams.getCoordinatorDynamicConfig()
+                                     .getMaxSegmentsInNodeLoadingQueue();
+  }
+
+  public CoordinatorStats getStats()
+  {
+    return stats;
+  }
+
+  /**
+   * Moves the given segment between two servers of the same tier.
+   * <p>
+   * See if we can move balancing here.
+   */
+  public boolean moveSegment(DataSegment segment, ServerHolder fromServer, ServerHolder toServer)
+  {
+    if (!fromServer.getServer().getTier().equals(toServer.getServer().getTier())) {
+      return false;
+    }
+
+    // fromServer must be loading or serving the segment
+    // and toServer must be able to load it
+    final SegmentState stateOnSrc = fromServer.getSegmentState(segment);
+    if ((stateOnSrc != SegmentState.LOADING && stateOnSrc != SegmentState.LOADED)
+        || !canLoadSegment(toServer, segment)) {
+      return false;
+    }
+
+    final boolean cancelSuccess = stateOnSrc == SegmentState.LOADING
+                                  && stateManager.cancelOperation(SegmentState.LOADING, segment, fromServer);
+
+    if (cancelSuccess) {
+      int loadedCountOnTier = replicantLookup
+          .getLoadedReplicants(segment.getId(), toServer.getServer().getTier());
+      stateManager.loadSegment(segment, toServer, loadedCountOnTier < 1);
+    } else {
+      return stateManager.moveSegment(segment, fromServer, toServer);
+    }
+
+    return true;
+  }
+
+  /**
+   * Queues load or drop of replicas of the given segment to achieve the
+   * target replication level in all the tiers.
+   */
+  public void updateReplicas(DataSegment segment, Map<String, Integer> tierToReplicaCount)
+  {
+    // Handle every target tier
+    tierToReplicaCount.forEach((tier, numReplicas) -> {
+      updateReplicasOnTier(segment, tier, tierToReplicaCount.get(tier));
+      stats.addToTieredStat(CoordinatorStats.REQUIRED_CAPACITY, tier, segment.getSize() * numReplicas);
+    });
+
+    // Find the minimum number of segments required for fault tolerance
+    final int totalTargetReplicas = tierToReplicaCount.values().stream()
+                                                      .reduce(0, Integer::sum);
+    final int minLoadedSegments = totalTargetReplicas > 1 ? 2 : 1;
+
+    // Drop segment from unneeded tiers if requirement is met across target tiers
+    int loadedTargetReplicas = 0;
+    final Set<String> targetTiers = tierToReplicaCount.keySet();
+    for (String tier : targetTiers) {
+      loadedTargetReplicas += replicantLookup.getLoadedReplicants(segment.getId(), tier);
+    }
+    if (loadedTargetReplicas < minLoadedSegments) {
+      return;
+    }
+
+    final Set<String> dropTiers = Sets.newHashSet(cluster.getTierNames());
+    dropTiers.removeAll(targetTiers);
+    for (String dropTier : dropTiers) {
+      updateReplicasOnTier(segment, dropTier, 0);
+    }
+  }
+
+  /**
+   * Broadcasts the given segment to all servers that are broadcast targets and
+   * queues a drop of the segment from decommissioning servers.
+   */
+  public void broadcastSegment(DataSegment segment)
+  {
+    int assignedCount = 0;
+    int droppedCount = 0;
+    for (ServerHolder server : cluster.getAllServers()) {
+      // Ignore servers which are not broadcast targets
+      if (!server.getServer().getType().isSegmentBroadcastTarget()) {
+        continue;
+      }
+
+      if (server.isDecommissioning()) {
+        droppedCount += dropBroadcastSegment(segment, server) ? 1 : 0;
+      } else {
+        assignedCount += loadBroadcastSegment(segment, server) ? 1 : 0;
+      }
+    }
+
+    if (assignedCount > 0) {
+      stats.addToDataSourceStat(CoordinatorStats.BROADCAST_LOADS, segment.getDataSource(), assignedCount);
+    }
+    if (droppedCount > 0) {
+      stats.addToDataSourceStat(CoordinatorStats.BROADCAST_DROPS, segment.getDataSource(), droppedCount);
+    }
+  }
+
+  /**
+   * Marks the given segment as unused.
+   */
+  public void deleteSegment(DataSegment segment)
+  {
+    stateManager.deleteSegment(segment);
+    stats.addToGlobalStat(CoordinatorStats.DELETED_SEGMENTS, 1);
+  }
+
+  /**
+   * Checks if the server can load the given segment.
+   * <p>
+   * A load is possible only if the server meets all of the following criteria:
+   * <ul>
+   *   <li>is not already serving or loading the segment</li>
+   *   <li>is not being decommissioned</li>
+   *   <li>has not already exceeded the load queue limit in this run</li>
+   *   <li>has available disk space</li>
+   * </ul>
+   */
+  public boolean canLoadSegment(ServerHolder server, DataSegment segment)
+  {
+    return server.canLoadSegment(segment)
+           && (maxLoadQueueSize == 0 || maxLoadQueueSize > server.getSegmentsQueuedForLoad());
+  }
+
+  /**
+   * Loads the broadcast segment if it is not loaded on the given server.
+   * Returns true only if the segment was successfully queued for load on the server.
+   */
+  private boolean loadBroadcastSegment(DataSegment segment, ServerHolder server)
+  {
+    final SegmentState state = server.getSegmentState(segment);
+    if (state == SegmentState.LOADED || state == SegmentState.LOADING) {
+      return false;
+    }
+
+    // Cancel drop if it is in progress
+    boolean dropCancelled = stateManager.cancelOperation(SegmentState.DROPPING, segment, server);
+    if (dropCancelled) {
+      return false;
+    }
+
+    if (canLoadSegment(server, segment)
+        && stateManager.loadSegment(segment, server, true)) {
+      return true;
+    } else {
+      log.makeAlert("Failed to broadcast segment for [%s]", segment.getDataSource())

Review Comment:
   Yes, I am going through all the log messages to make sure they are less ambiguous.



-- 
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: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] github-code-scanning[bot] commented on a diff in pull request #13197: Segment loading: Allow cancellation and prioritization of load queue items

Posted by "github-code-scanning[bot] (via GitHub)" <gi...@apache.org>.
github-code-scanning[bot] commented on code in PR #13197:
URL: https://github.com/apache/druid/pull/13197#discussion_r1163048337


##########
server/src/test/java/org/apache/druid/server/coordinator/rules/LoadRuleTest.java:
##########
@@ -117,397 +102,290 @@
   @Before
   public void setUp()
   {
-    EmittingLogger.registerEmitter(EMITTER);
-    EMITTER.start();
-    throttler = EasyMock.createMock(ReplicationThrottler.class);
-
     exec = MoreExecutors.listeningDecorator(Execs.multiThreaded(1, "LoadRuleTest-%d"));
     balancerStrategy = new CostBalancerStrategyFactory().createBalancerStrategy(exec);
     cachingCostBalancerStrategy = new CachingCostBalancerStrategy(ClusterCostCache.builder().build(), exec);
 
     mockBalancerStrategy = EasyMock.createMock(BalancerStrategy.class);
+    stateManager = new SegmentStateManager(null, null, null);
   }
 
   @After
-  public void tearDown() throws Exception
+  public void tearDown()
   {
     exec.shutdown();
-    EMITTER.close();
   }
 
   @Test
   public void testLoad()
   {
-    EasyMock.expect(throttler.canCreateReplicant(EasyMock.anyString())).andReturn(true).anyTimes();
-
     final LoadQueuePeon mockPeon = createEmptyPeon();
-    mockPeon.loadSegment(EasyMock.anyObject(), EasyMock.anyObject());
+    mockPeon.loadSegment(EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyObject());
     EasyMock.expectLastCall().atLeastOnce();
 
-    LoadRule rule = createLoadRule(ImmutableMap.of(
-        "hot", 1,
-        DruidServer.DEFAULT_TIER, 2
-    ));
-
-    final DataSegment segment = createDataSegment("foo");
-
-    throttler.registerReplicantCreation(DruidServer.DEFAULT_TIER, segment.getId(), "hostNorm");
-    EasyMock.expectLastCall().once();
-
     if (!useRoundRobinAssignment) {
       EasyMock.expect(mockBalancerStrategy.findNewSegmentHomeReplicator(EasyMock.anyObject(), EasyMock.anyObject()))
               .andDelegateTo(balancerStrategy)
-              .times(3);
+              .times(2);
     }
-
-    EasyMock.replay(throttler, mockPeon, mockBalancerStrategy);
+    EasyMock.replay(mockPeon, mockBalancerStrategy);
 
     DruidCluster druidCluster = DruidClusterBuilder
         .newBuilder()
-        .addTier(
-            "hot",
-            new ServerHolder(
-                new DruidServer("serverHot", "hostHot", null, 1000, ServerType.HISTORICAL, "hot", 1)
-                    .toImmutableDruidServer(),
-                mockPeon
-            )
-        )
-        .addTier(
-            DruidServer.DEFAULT_TIER,
-            new ServerHolder(
-                new DruidServer(
-                    "serverNorm",
-                    "hostNorm",
-                    null,
-                    1000,
-                    ServerType.HISTORICAL,
-                    DruidServer.DEFAULT_TIER,
-                    0
-                ).toImmutableDruidServer(),
-                mockPeon
-            )
-        )
+        .addTier(Tier.T1, createServerHolder(Tier.T1, mockPeon, false))
+        .addTier(Tier.T2, createServerHolder(Tier.T2, mockPeon, false))
         .build();
 
-    CoordinatorStats stats = rule.run(null, makeCoordinatorRuntimeParams(druidCluster, segment), segment);
+    final DataSegment segment = createDataSegment(DS_WIKI);
+    LoadRule rule = loadForever(ImmutableMap.of(Tier.T1, 1, Tier.T2, 2));
+    CoordinatorRunStats stats = runRuleAndGetStats(rule, segment, druidCluster);
 
-    Assert.assertEquals(1L, stats.getTieredStat(LoadRule.ASSIGNED_COUNT, "hot"));
-    Assert.assertEquals(1L, stats.getTieredStat(LoadRule.ASSIGNED_COUNT, DruidServer.DEFAULT_TIER));
+    Assert.assertEquals(1L, stats.getSegmentStat(Stats.Segments.ASSIGNED, Tier.T1, DS_WIKI));
+    Assert.assertEquals(1L, stats.getSegmentStat(Stats.Segments.ASSIGNED, Tier.T2, DS_WIKI));
 
-    EasyMock.verify(throttler, mockPeon, mockBalancerStrategy);
+    EasyMock.verify(mockPeon, mockBalancerStrategy);
+  }
+
+  private CoordinatorRunStats runRuleAndGetStats(
+      LoadRule rule,
+      DataSegment segment,
+      DruidCluster cluster
+  )
+  {
+    return runRuleAndGetStats(rule, segment, makeCoordinatorRuntimeParams(cluster, segment));
+  }
+
+  private CoordinatorRunStats runRuleAndGetStats(
+      LoadRule rule,
+      DataSegment segment,
+      DruidCoordinatorRuntimeParams params
+  )
+  {
+    final CoordinatorDynamicConfig dynamicConfig = params.getCoordinatorDynamicConfig();
+    ReplicationThrottler throttler = new ReplicationThrottler(
+        Sets.newHashSet(params.getDruidCluster().getTierNames()),
+        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/4775)



##########
server/src/main/java/org/apache/druid/server/coordinator/duty/RunRules.java:
##########
@@ -163,9 +144,74 @@
          .emit();
     }
 
+    segmentLoader.makeAlerts();
+
+    final CoordinatorRunStats stats = segmentLoader.getStats();
+    stats.forEachRowKey(
+        (dimensionValues, statValues) ->
+            log.info("Stats for dimensions [%s]: [%s]", dimensionValues, statValues)
+    );
+
     return params.buildFromExisting()
                  .withCoordinatorStats(stats)
                  .withBroadcastDatasources(broadcastDatasources)
+                 .withReplicationManager(replicationThrottler)
                  .build();
   }
+
+  /**
+   * Reduces the lifetimes of segments currently being replicated in all the tiers.
+   * Returns the set of tiers that are currently replicatinng some segments and
+   * won't be eligible for assigning more replicas in this run.
+   */
+  private Set<String> reduceLifetimesAndGetBusyTiers(CoordinatorDynamicConfig dynamicConfig)
+  {
+    final Set<String> busyTiers = new HashSet<>();
+    stateManager.reduceLifetimesOfReplicatingSegments().forEach((tier, replicatingState) -> {
+      int numReplicatingSegments = replicatingState.getNumProcessingSegments();
+      if (numReplicatingSegments <= 0) {
+        return;
+      }
+
+      busyTiers.add(tier);
+      log.info(
+          "Skipping replication on tier [%s] as is it still has"
+          + " [%d] segments in queue with lifetime [%d / %d].",
+          tier,
+          numReplicatingSegments,
+          replicatingState.getMinLifetime(),
+          dynamicConfig.getReplicantLifetime()
+      );
+
+      // Create alerts for stuck tiers
+      if (replicatingState.getMinLifetime() <= 0) {
+        log.makeAlert(
+            "Replication queue for tier [%s] has [%d] segments stuck.",
+            tier,
+            replicatingState.getNumExpiredSegments()
+        ).addData("segments", replicatingState.getExpiredSegments()).emit();
+      }
+    });
+
+    return busyTiers;
+  }
+
+  private ReplicationThrottler createReplicationThrottler(
+      Set<String> busyTiers,
+      DruidCluster cluster,
+      CoordinatorDynamicConfig dynamicConfig
+  )
+  {
+    // Tiers that already have some replication in progress are not eligible for
+    // replication in this coordinator run
+    final Set<String> tiersEligibleForReplication = Sets.newHashSet(cluster.getTierNames());
+    tiersEligibleForReplication.removeAll(busyTiers);
+
+    return new ReplicationThrottler(
+        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/4771)



##########
server/src/main/java/org/apache/druid/server/coordinator/CoordinatorDynamicConfig.java:
##########
@@ -713,75 +691,74 @@
       return this;
     }
 
+    /**
+     * Builds a CoordinatoryDynamicConfig using either the configured values, or
+     * the default value if not configured.
+     */
     public CoordinatorDynamicConfig build()
     {
       return new CoordinatorDynamicConfig(
-          leadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments == null
-          ? DEFAULT_LEADING_TIME_MILLIS_BEFORE_CAN_MARK_AS_UNUSED_OVERSHADOWED_SEGMENTS
-          : leadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments,
-          mergeBytesLimit == null ? DEFAULT_MERGE_BYTES_LIMIT : mergeBytesLimit,
-          mergeSegmentsLimit == null ? DEFAULT_MERGE_SEGMENTS_LIMIT : mergeSegmentsLimit,
-          maxSegmentsToMove == null ? DEFAULT_MAX_SEGMENTS_TO_MOVE : maxSegmentsToMove,
-          percentOfSegmentsToConsiderPerMove == null ? DEFAULT_PERCENT_OF_SEGMENTS_TO_CONSIDER_PER_MOVE
-                                                     : percentOfSegmentsToConsiderPerMove,
-          useBatchedSegmentSampler == null ? DEFAULT_USE_BATCHED_SEGMENT_SAMPLER : useBatchedSegmentSampler,
-          replicantLifetime == null ? DEFAULT_REPLICANT_LIFETIME : replicantLifetime,
-          replicationThrottleLimit == null ? DEFAULT_REPLICATION_THROTTLE_LIMIT : replicationThrottleLimit,
-          balancerComputeThreads == null ? DEFAULT_BALANCER_COMPUTE_THREADS : balancerComputeThreads,
-          emitBalancingStats == null ? DEFAULT_EMIT_BALANCING_STATS : emitBalancingStats,
+          valueOrDefault(
+              leadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments,
+              Defaults.LEADING_MILLIS_BEFORE_MARK_UNUSED
+          ),
+          valueOrDefault(mergeBytesLimit, Defaults.MERGE_BYTES_LIMIT),
+          valueOrDefault(mergeSegmentsLimit, Defaults.MERGE_SEGMENTS_LIMIT),
+          valueOrDefault(maxSegmentsToMove, Defaults.MAX_SEGMENTS_TO_MOVE),
+          valueOrDefault(percentOfSegmentsToConsiderPerMove, Defaults.PERCENT_OF_SEGMENTS_TO_CONSIDER_PER_MOVE),
+          valueOrDefault(useBatchedSegmentSampler, Defaults.USE_BATCHED_SEGMENT_SAMPLER),
+          valueOrDefault(replicantLifetime, Defaults.REPLICANT_LIFETIME),
+          valueOrDefault(replicationThrottleLimit, Defaults.REPLICATION_THROTTLE_LIMIT),
+          valueOrDefault(balancerComputeThreads, Defaults.BALANCER_COMPUTE_THREADS),
+          valueOrDefault(emitBalancingStats, Defaults.EMIT_BALANCING_STATS),
           specificDataSourcesToKillUnusedSegmentsIn,
           dataSourcesToNotKillStalePendingSegmentsIn,
-          maxSegmentsInNodeLoadingQueue == null
-          ? DEFAULT_MAX_SEGMENTS_IN_NODE_LOADING_QUEUE
-          : maxSegmentsInNodeLoadingQueue,
+          valueOrDefault(maxSegmentsInNodeLoadingQueue, Defaults.MAX_SEGMENTS_IN_NODE_LOADING_QUEUE),
           decommissioningNodes,
-          decommissioningMaxPercentOfMaxSegmentsToMove == null
-          ? DEFAULT_DECOMMISSIONING_MAX_SEGMENTS_TO_MOVE_PERCENT
-          : decommissioningMaxPercentOfMaxSegmentsToMove,
-          pauseCoordination == null ? DEFAULT_PAUSE_COORDINATION : pauseCoordination,
-          replicateAfterLoadTimeout == null ? DEFAULT_REPLICATE_AFTER_LOAD_TIMEOUT : replicateAfterLoadTimeout,
-          maxNonPrimaryReplicantsToLoad == null ? DEFAULT_MAX_NON_PRIMARY_REPLICANTS_TO_LOAD
-                                                : maxNonPrimaryReplicantsToLoad,
-          useRoundRobinSegmentAssignment == null ? DEFAULT_USE_ROUND_ROBIN_ASSIGNMENT : useRoundRobinSegmentAssignment
+          valueOrDefault(
+              decommissioningMaxPercentOfMaxSegmentsToMove,
+              Defaults.DECOMMISSIONING_MAX_SEGMENTS_TO_MOVE_PERCENT
+          ),
+          valueOrDefault(pauseCoordination, Defaults.PAUSE_COORDINATION),
+          valueOrDefault(replicateAfterLoadTimeout, Defaults.REPLICATE_AFTER_LOAD_TIMEOUT),
+          valueOrDefault(maxNonPrimaryReplicantsToLoad, Defaults.MAX_NON_PRIMARY_REPLICANTS_TO_LOAD),
+          valueOrDefault(useRoundRobinSegmentAssignment, Defaults.USE_ROUND_ROBIN_ASSIGNMENT)
       );
     }
 
+    private static <T> T valueOrDefault(@Nullable T value, @NotNull T defaultValue)
+    {
+      return value == null ? defaultValue : value;
+    }
+
     public CoordinatorDynamicConfig build(CoordinatorDynamicConfig defaults)
     {
       return new CoordinatorDynamicConfig(
-          leadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments == null
-          ? defaults.getLeadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments()
-          : leadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments,
-          mergeBytesLimit == null ? defaults.getMergeBytesLimit() : mergeBytesLimit,
-          mergeSegmentsLimit == null ? defaults.getMergeSegmentsLimit() : mergeSegmentsLimit,
-          maxSegmentsToMove == null ? defaults.getMaxSegmentsToMove() : maxSegmentsToMove,
-          percentOfSegmentsToConsiderPerMove == null
-          ? defaults.getPercentOfSegmentsToConsiderPerMove()
-          : percentOfSegmentsToConsiderPerMove,
-          useBatchedSegmentSampler == null ? defaults.useBatchedSegmentSampler() : useBatchedSegmentSampler,
-          replicantLifetime == null ? defaults.getReplicantLifetime() : replicantLifetime,
-          replicationThrottleLimit == null ? defaults.getReplicationThrottleLimit() : replicationThrottleLimit,
-          balancerComputeThreads == null ? defaults.getBalancerComputeThreads() : balancerComputeThreads,
-          emitBalancingStats == null ? defaults.emitBalancingStats() : emitBalancingStats,
-          specificDataSourcesToKillUnusedSegmentsIn == null
-          ? defaults.getSpecificDataSourcesToKillUnusedSegmentsIn()
-          : specificDataSourcesToKillUnusedSegmentsIn,
-          dataSourcesToNotKillStalePendingSegmentsIn == null
-          ? defaults.getDataSourcesToNotKillStalePendingSegmentsIn()
-          : dataSourcesToNotKillStalePendingSegmentsIn,
-          maxSegmentsInNodeLoadingQueue == null
-          ? defaults.getMaxSegmentsInNodeLoadingQueue()
-          : maxSegmentsInNodeLoadingQueue,
-          decommissioningNodes == null ? defaults.getDecommissioningNodes() : decommissioningNodes,
-          decommissioningMaxPercentOfMaxSegmentsToMove == null
-          ? defaults.getDecommissioningMaxPercentOfMaxSegmentsToMove()
-          : decommissioningMaxPercentOfMaxSegmentsToMove,
-          pauseCoordination == null ? defaults.getPauseCoordination() : pauseCoordination,
-          replicateAfterLoadTimeout == null ? defaults.getReplicateAfterLoadTimeout() : replicateAfterLoadTimeout,
-          maxNonPrimaryReplicantsToLoad == null
-          ? defaults.getMaxNonPrimaryReplicantsToLoad()
-          : maxNonPrimaryReplicantsToLoad,
-          useRoundRobinSegmentAssignment == null ? defaults.isUseRoundRobinSegmentAssignment() : useRoundRobinSegmentAssignment
+          valueOrDefault(
+              leadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments,
+              defaults.getLeadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments()
+          ),
+          valueOrDefault(mergeBytesLimit, defaults.getMergeBytesLimit()),
+          valueOrDefault(mergeSegmentsLimit, defaults.getMergeSegmentsLimit()),
+          valueOrDefault(maxSegmentsToMove, defaults.getMaxSegmentsToMove()),
+          valueOrDefault(percentOfSegmentsToConsiderPerMove, defaults.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/4766)



##########
server/src/test/java/org/apache/druid/server/coordinator/CuratorDruidCoordinatorTest.java:
##########
@@ -497,4 +463,27 @@
                       .size(0)
                       .build();
   }
+
+  private SegmentLoader createSegmentLoader(
+      SegmentStateManager stateManager,
+      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/4773)



##########
server/src/main/java/org/apache/druid/server/coordinator/duty/BalanceSegments.java:
##########
@@ -20,292 +20,326 @@
 package org.apache.druid.server.coordinator.duty;
 
 import com.google.common.collect.Lists;
-import org.apache.druid.client.ImmutableDruidServer;
-import org.apache.druid.java.util.common.Pair;
+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.CoordinatorStats;
-import org.apache.druid.server.coordinator.DruidCoordinator;
+import org.apache.druid.server.coordinator.DruidCluster;
 import org.apache.druid.server.coordinator.DruidCoordinatorRuntimeParams;
-import org.apache.druid.server.coordinator.LoadPeonCallback;
-import org.apache.druid.server.coordinator.LoadQueuePeon;
+import org.apache.druid.server.coordinator.SegmentLoader;
+import org.apache.druid.server.coordinator.SegmentStateManager;
 import org.apache.druid.server.coordinator.ServerHolder;
+import org.apache.druid.server.coordinator.stats.CoordinatorRunStats;
+import org.apache.druid.server.coordinator.stats.Stats;
 import org.apache.druid.timeline.DataSegment;
 import org.apache.druid.timeline.SegmentId;
 
-import java.util.HashMap;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
-import java.util.NavigableSet;
 import java.util.SortedSet;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
 import java.util.stream.Collectors;
 
 /**
+ *
  */
 public class BalanceSegments implements CoordinatorDuty
 {
   protected static final EmittingLogger log = new EmittingLogger(BalanceSegments.class);
+  private final SegmentStateManager stateManager;
 
-  protected final DruidCoordinator coordinator;
-
-  protected final Map<String, ConcurrentHashMap<SegmentId, BalancerSegmentHolder>> currentlyMovingSegments =
-      new HashMap<>();
-
-  public BalanceSegments(DruidCoordinator coordinator)
+  public BalanceSegments(SegmentStateManager stateManager)
   {
-    this.coordinator = coordinator;
+    this.stateManager = stateManager;
   }
 
-  protected void reduceLifetimes(String tier)
+  /**
+   * Reduces the lifetimes of segments currently being moved in all the tiers.
+   * Raises alerts for segments stuck in the queue.
+   */
+  private void reduceLifetimesAndAlert()
   {
-    for (BalancerSegmentHolder holder : currentlyMovingSegments.get(tier).values()) {
-      holder.reduceLifetime();
-      if (holder.getLifetime() <= 0) {
-        log.makeAlert("[%s]: Balancer move segments queue has a segment stuck", tier)
-           .addData("segment", holder.getSegment().getId())
-           .addData("server", holder.getFromServer().getMetadata())
-           .emit();
+    stateManager.reduceLifetimesOfMovingSegments().forEach((tier, movingState) -> {
+      int numMovingSegments = movingState.getNumProcessingSegments();
+      if (numMovingSegments <= 0) {
+        return;
       }
-    }
+
+      // Create alerts for stuck tiers
+      if (movingState.getMinLifetime() <= 0) {
+        log.makeAlert(
+            "Balancing queue for tier [%s] has [%d] segments stuck.",
+            tier,
+            movingState.getNumExpiredSegments()
+        ).addData("segments", movingState.getExpiredSegments()).emit();
+      }
+    });
   }
 
   @Override
   public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params)
   {
-    final CoordinatorStats stats = new CoordinatorStats();
-    params.getDruidCluster().getHistoricals().forEach((String tier, NavigableSet<ServerHolder> servers) -> {
-      balanceTier(params, tier, servers, stats);
-    });
+    reduceLifetimesAndAlert();
+
+    if (params.getUsedSegments().isEmpty()) {
+      log.info("Skipping balance as there are no used segments.");
+      return params;
+    }
+
+    final DruidCluster cluster = params.getDruidCluster();
+    final int maxSegmentsToMove = params.getCoordinatorDynamicConfig().getMaxSegmentsToMove();
+    if (maxSegmentsToMove <= 0) {
+      log.info("Skipping balance as maxSegmentsToMove is [%d].", maxSegmentsToMove);
+      return params;
+    } else {
+      int maxLifetime = params.getCoordinatorDynamicConfig().getReplicantLifetime();
+      log.info(
+          "Balancing segments in tiers [%s] with maxSegmentsToMove=[%d], maxLifetime=[%d].",
+          cluster.getTierNames(), maxSegmentsToMove, maxLifetime
+      );
+    }
+
+    final CoordinatorRunStats stats = new CoordinatorRunStats();
+    final SegmentLoader loader = new SegmentLoader(
+        stateManager,
+        params.getDruidCluster(),
+        params.getSegmentReplicantLookup(),
+        params.getReplicationManager(),
+        params.getBalancerStrategy(),
+        params.getCoordinatorDynamicConfig().isUseRoundRobinSegmentAssignment()
+    );
+
+    cluster.getHistoricals().forEach(
+        (tier, servers) -> balanceTier(params, tier, servers, stats, loader)
+    );
+
+    loader.makeAlerts();
+    stats.accumulate(loader.getStats());
+
+    stats.forEachRowKey(
+        (dimensionValues, statValues) ->
+            log.info("Stats for dimensions [%s] are [%s]", dimensionValues, statValues)
+    );
+
     return params.buildFromExisting().withCoordinatorStats(stats).build();
   }
 
+  /**
+   * Moves as many segments as possible from decommissioning to active servers,
+   * then moves segments amongst active servers.
+   */
   private void balanceTier(
       DruidCoordinatorRuntimeParams params,
       String tier,
       SortedSet<ServerHolder> servers,
-      CoordinatorStats stats
+      CoordinatorRunStats stats,
+      SegmentLoader loader
   )
   {
-
-    log.info("Balancing segments in tier [%s]", tier);
-    if (params.getUsedSegments().size() == 0) {
-      log.info("Metadata segments are not available. Cannot balance.");
-      // suppress emit zero stats
-      return;
-    }
-    currentlyMovingSegments.computeIfAbsent(tier, t -> new ConcurrentHashMap<>());
-
-    if (!currentlyMovingSegments.get(tier).isEmpty()) {
-      reduceLifetimes(tier);
-      log.info(
-          "[%s]: Still waiting on %,d segments to be moved. Skipping balance.",
-          tier,
-          currentlyMovingSegments.get(tier).size()
-      );
-      // suppress emit zero stats
-      return;
-    }
-
-    /*
-      Take as many segments from decommissioning servers as decommissioningMaxPercentOfMaxSegmentsToMove allows and find
-      the best location for them on active servers. After that, balance segments within active servers pool.
-     */
     Map<Boolean, List<ServerHolder>> partitions =
         servers.stream().collect(Collectors.partitioningBy(ServerHolder::isDecommissioning));
     final List<ServerHolder> decommissioningServers = partitions.get(true);
     final List<ServerHolder> activeServers = partitions.get(false);
+
     log.info(
-        "Found %d active servers, %d decommissioning servers",
-        activeServers.size(),
-        decommissioningServers.size()
+        "Balancing segments in tier [%s] with [%d] activeServers and [%d] decommissioning servers.",
+        tier, activeServers.size(), decommissioningServers.size()
     );
 
     if ((decommissioningServers.isEmpty() && activeServers.size() <= 1) || activeServers.isEmpty()) {
-      log.warn("[%s]: insufficient active servers. Cannot balance.", tier);
-      // suppress emit zero stats
+      log.warn("Skipping balance for tier [%s] as there are [%d] active servers.", tier, activeServers.size());
       return;
     }
 
-    int numSegments = 0;
-    for (ServerHolder sourceHolder : servers) {
-      numSegments += sourceHolder.getServer().getNumSegments();
-    }
-
-    if (numSegments == 0) {
-      log.info("No segments found. Cannot balance.");
-      // suppress emit zero stats
+    final int maxSegmentsToMove = params.getCoordinatorDynamicConfig().getMaxSegmentsToMove();
+    if (maxSegmentsToMove <= 0) {
       return;
     }
 
-    final int maxSegmentsToMove = Math.min(params.getCoordinatorDynamicConfig().getMaxSegmentsToMove(), numSegments);
+    // Move segments from decommissioning to active servers
+    int maxDecommPercentToMove = params.getCoordinatorDynamicConfig()
+                                       .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/4770)



##########
server/src/test/java/org/apache/druid/server/coordinator/BalanceSegmentsTest.java:
##########
@@ -626,189 +422,115 @@
         .withBroadcastDatasources(broadcastDatasources)
         .build();
 
-    params = new BalanceSegmentsTester(coordinator).run(params);
-    Assert.assertEquals(2L, params.getCoordinatorStats().getTieredStat("movedCount", "normal"));
+    CoordinatorRunStats stats = runBalancer(params);
+    Assert.assertEquals(2L, stats.getTieredStat(Stats.Segments.MOVED, "normal"));
   }
 
-  private DruidCoordinatorRuntimeParams.Builder defaultRuntimeParamsBuilder(
-      List<ImmutableDruidServer> druidServers,
-      List<LoadQueuePeon> peons
-  )
+  private CoordinatorRunStats runBalancer(DruidCoordinatorRuntimeParams params)
   {
-    return defaultRuntimeParamsBuilder(
-        druidServers,
-        peons,
-        druidServers.stream().map(s -> false).collect(Collectors.toList())
-    );
+    params = new BalanceSegments(stateManager).run(params);
+    if (params == null) {
+      Assert.fail("BalanceSegments duty returned null params");
+      return new CoordinatorRunStats();
+    } else {
+      return params.getCoordinatorStats();
+    }
   }
 
   private DruidCoordinatorRuntimeParams.Builder defaultRuntimeParamsBuilder(
-      List<ImmutableDruidServer> druidServers,
-      List<LoadQueuePeon> peons,
-      List<Boolean> decommissioning
+      ServerHolder... servers
   )
   {
     return CoordinatorRuntimeParamsTestHelpers
         .newBuilder()
-        .withDruidCluster(
-            DruidClusterBuilder
-                .newBuilder()
-                .addTier(
-                    "normal",
-                    IntStream
-                        .range(0, druidServers.size())
-                        .mapToObj(i -> new ServerHolder(druidServers.get(i), peons.get(i), decommissioning.get(i)))
-                        .toArray(ServerHolder[]::new)
-                )
-                .build()
-        )
-        .withLoadManagementPeons(
-            IntStream
-                .range(0, peons.size())
-                .boxed()
-                .collect(Collectors.toMap(i -> String.valueOf(i + 1), peons::get))
-        )
-        .withUsedSegmentsInTest(segments)
-        .withDynamicConfigs(CoordinatorDynamicConfig.builder().withMaxSegmentsToMove(MAX_SEGMENTS_TO_MOVE).build())
+        .withDruidCluster(DruidClusterBuilder.newBuilder().addTier("normal", servers).build())
+        .withUsedSegmentsInTest(allSegments)
         .withBroadcastDatasources(broadcastDatasources)
-        .withBalancerStrategy(balancerStrategy);
+        .withBalancerStrategy(balancerStrategy)
+        .withReplicationManager(createReplicationThrottler());
   }
 
-  private static void mockDruidServer(
-      ImmutableDruidServer druidServer,
-      String name,
-      String tier,
-      long currentSize,
-      long maxSize,
-      List<DataSegment> segments
-  )
+  private ServerHolder createHolder(DruidServer server, DataSegment... loadedSegments)
   {
-    EasyMock.expect(druidServer.getName()).andReturn(name).anyTimes();
-    EasyMock.expect(druidServer.getTier()).andReturn(tier).anyTimes();
-    EasyMock.expect(druidServer.getCurrSize()).andReturn(currentSize).atLeastOnce();
-    EasyMock.expect(druidServer.getMaxSize()).andReturn(maxSize).atLeastOnce();
-    ImmutableDruidServerTests.expectSegments(druidServer, segments);
-    EasyMock.expect(druidServer.getHost()).andReturn(name).anyTimes();
-    EasyMock.expect(druidServer.getType()).andReturn(ServerType.HISTORICAL).anyTimes();
-    if (!segments.isEmpty()) {
-      segments.forEach(
-          s -> EasyMock.expect(druidServer.getSegment(s.getId())).andReturn(s).anyTimes()
-      );
-    }
-    EasyMock.expect(druidServer.getSegment(EasyMock.anyObject())).andReturn(null).anyTimes();
-    EasyMock.replay(druidServer);
+    return createHolder(server, false, loadedSegments);
   }
 
-  private static void mockCoordinator(DruidCoordinator coordinator)
+  private ServerHolder createHolder(DruidServer server, boolean isDecommissioning, DataSegment... loadedSegments)
   {
-    coordinator.moveSegment(
-        EasyMock.anyObject(),
-        EasyMock.anyObject(),
-        EasyMock.anyObject(),
-        EasyMock.anyObject(),
-        EasyMock.anyObject()
-    );
-    EasyMock.expectLastCall().anyTimes();
-    EasyMock.replay(coordinator);
+    return createHolder(server, 0, isDecommissioning, loadedSegments);
   }
 
-  private static class PredefinedPickOrderBalancerStrategy implements BalancerStrategy
+  private ServerHolder createHolder(
+      DruidServer server,
+      int maxSegmentsInLoadQueue,
+      boolean isDecommissioning,
+      DataSegment... loadedSegments
+  )
   {
-    private final BalancerStrategy delegate;
-    private final List<BalancerSegmentHolder> pickOrder;
-    private final AtomicInteger pickCounter = new AtomicInteger(0);
-
-    PredefinedPickOrderBalancerStrategy(
-        BalancerStrategy delegate,
-        List<BalancerSegmentHolder> pickOrder
-    )
-    {
-      this.delegate = delegate;
-      this.pickOrder = pickOrder;
-    }
-
-    @Override
-    public ServerHolder findNewSegmentHomeBalancer(DataSegment proposalSegment, List<ServerHolder> serverHolders)
-    {
-      return delegate.findNewSegmentHomeBalancer(proposalSegment, serverHolders);
-    }
-
-    @Override
-    public ServerHolder findNewSegmentHomeReplicator(DataSegment proposalSegment, List<ServerHolder> serverHolders)
-    {
-      return delegate.findNewSegmentHomeReplicator(proposalSegment, serverHolders);
-    }
-
-    @Override
-    public Iterator<BalancerSegmentHolder> pickSegmentsToMove(
-        List<ServerHolder> serverHolders,
-        Set<String> broadcastDatasources,
-        int numberOfSegments
-    )
-    {
-      return pickOrder.iterator();
-    }
-
-    @Override
-    public Iterator<BalancerSegmentHolder> pickSegmentsToMove(
-        List<ServerHolder> serverHolders,
-        Set<String> broadcastDatasources,
-        double percentOfSegmentsToConsider
-    )
-    {
-      return pickOrder.iterator();
+    for (DataSegment segment : loadedSegments) {
+      server.addDataSegment(segment);
     }
 
-    @Override
-    public void emitStats(String tier, CoordinatorStats stats, List<ServerHolder> serverHolderList)
-    {
-      delegate.emitStats(tier, stats, serverHolderList);
-    }
+    return new ServerHolder(
+        server.toImmutableDruidServer(),
+        new LoadQueuePeonTester(),
+        isDecommissioning,
+        maxSegmentsInLoadQueue
+    );
   }
 
-  private DruidCoordinatorRuntimeParams setupParamsForDecommissioningMaxPercentOfMaxSegmentsToMove(int percent)
+  private IExpectationSetters<Iterator<BalancerSegmentHolder>> expectPickLoadingSegmentsAndReturnEmpty(
+      BalancerStrategy strategy
+  )
   {
-    mockDruidServer(druidServer1, "1", "normal", 30L, 100L, Arrays.asList(segment1, segment3));
-    mockDruidServer(druidServer2, "2", "normal", 30L, 100L, Arrays.asList(segment2, segment3));
-    mockDruidServer(druidServer3, "3", "normal", 0L, 100L, Collections.emptyList());
-
-    EasyMock.replay(druidServer4);
-
-    mockCoordinator(coordinator);
+    return EasyMock.expect(
+        strategy.pickSegmentsToMove(
+            EasyMock.anyObject(),
+            EasyMock.anyObject(),
+            EasyMock.anyInt(),
+            EasyMock.eq(true)
+        )
+    ).andReturn(Collections.emptyIterator());
+  }
 
-    // either decommissioning servers list or acitve ones (ie servers list is [2] or [1, 3])
-    BalancerStrategy strategy = EasyMock.createMock(BalancerStrategy.class);
-    EasyMock.expect(
+  private IExpectationSetters<Iterator<BalancerSegmentHolder>> expectPickLoadedSegmentsAndReturn(
+      BalancerStrategy strategy,
+      BalancerSegmentHolder... pickedLoadedSegments
+  )
+  {
+    return EasyMock.expect(
         strategy.pickSegmentsToMove(
-            ImmutableList.of(
-                new ServerHolder(druidServer2, peon2, true)
-            ),
-            broadcastDatasources,
-            100.0
+            EasyMock.anyObject(),
+            EasyMock.anyObject(),
+            EasyMock.anyInt(),
+            EasyMock.eq(false)
         )
-    ).andReturn(
-        ImmutableList.of(new BalancerSegmentHolder(druidServer2, segment2)).iterator()
+    ).andReturn(Arrays.asList(pickedLoadedSegments).iterator());
+  }
+
+  private ReplicationThrottler createReplicationThrottler()
+  {
+    CoordinatorDynamicConfig dynamicConfig = CoordinatorDynamicConfig.builder().build();
+    return new ReplicationThrottler(
+        Collections.singleton("normal"),
+        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/4772)



##########
server/src/main/java/org/apache/druid/server/coordinator/CoordinatorDynamicConfig.java:
##########
@@ -713,75 +691,74 @@
       return this;
     }
 
+    /**
+     * Builds a CoordinatoryDynamicConfig using either the configured values, or
+     * the default value if not configured.
+     */
     public CoordinatorDynamicConfig build()
     {
       return new CoordinatorDynamicConfig(
-          leadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments == null
-          ? DEFAULT_LEADING_TIME_MILLIS_BEFORE_CAN_MARK_AS_UNUSED_OVERSHADOWED_SEGMENTS
-          : leadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments,
-          mergeBytesLimit == null ? DEFAULT_MERGE_BYTES_LIMIT : mergeBytesLimit,
-          mergeSegmentsLimit == null ? DEFAULT_MERGE_SEGMENTS_LIMIT : mergeSegmentsLimit,
-          maxSegmentsToMove == null ? DEFAULT_MAX_SEGMENTS_TO_MOVE : maxSegmentsToMove,
-          percentOfSegmentsToConsiderPerMove == null ? DEFAULT_PERCENT_OF_SEGMENTS_TO_CONSIDER_PER_MOVE
-                                                     : percentOfSegmentsToConsiderPerMove,
-          useBatchedSegmentSampler == null ? DEFAULT_USE_BATCHED_SEGMENT_SAMPLER : useBatchedSegmentSampler,
-          replicantLifetime == null ? DEFAULT_REPLICANT_LIFETIME : replicantLifetime,
-          replicationThrottleLimit == null ? DEFAULT_REPLICATION_THROTTLE_LIMIT : replicationThrottleLimit,
-          balancerComputeThreads == null ? DEFAULT_BALANCER_COMPUTE_THREADS : balancerComputeThreads,
-          emitBalancingStats == null ? DEFAULT_EMIT_BALANCING_STATS : emitBalancingStats,
+          valueOrDefault(
+              leadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments,
+              Defaults.LEADING_MILLIS_BEFORE_MARK_UNUSED
+          ),
+          valueOrDefault(mergeBytesLimit, Defaults.MERGE_BYTES_LIMIT),
+          valueOrDefault(mergeSegmentsLimit, Defaults.MERGE_SEGMENTS_LIMIT),
+          valueOrDefault(maxSegmentsToMove, Defaults.MAX_SEGMENTS_TO_MOVE),
+          valueOrDefault(percentOfSegmentsToConsiderPerMove, Defaults.PERCENT_OF_SEGMENTS_TO_CONSIDER_PER_MOVE),
+          valueOrDefault(useBatchedSegmentSampler, Defaults.USE_BATCHED_SEGMENT_SAMPLER),
+          valueOrDefault(replicantLifetime, Defaults.REPLICANT_LIFETIME),
+          valueOrDefault(replicationThrottleLimit, Defaults.REPLICATION_THROTTLE_LIMIT),
+          valueOrDefault(balancerComputeThreads, Defaults.BALANCER_COMPUTE_THREADS),
+          valueOrDefault(emitBalancingStats, Defaults.EMIT_BALANCING_STATS),
           specificDataSourcesToKillUnusedSegmentsIn,
           dataSourcesToNotKillStalePendingSegmentsIn,
-          maxSegmentsInNodeLoadingQueue == null
-          ? DEFAULT_MAX_SEGMENTS_IN_NODE_LOADING_QUEUE
-          : maxSegmentsInNodeLoadingQueue,
+          valueOrDefault(maxSegmentsInNodeLoadingQueue, Defaults.MAX_SEGMENTS_IN_NODE_LOADING_QUEUE),
           decommissioningNodes,
-          decommissioningMaxPercentOfMaxSegmentsToMove == null
-          ? DEFAULT_DECOMMISSIONING_MAX_SEGMENTS_TO_MOVE_PERCENT
-          : decommissioningMaxPercentOfMaxSegmentsToMove,
-          pauseCoordination == null ? DEFAULT_PAUSE_COORDINATION : pauseCoordination,
-          replicateAfterLoadTimeout == null ? DEFAULT_REPLICATE_AFTER_LOAD_TIMEOUT : replicateAfterLoadTimeout,
-          maxNonPrimaryReplicantsToLoad == null ? DEFAULT_MAX_NON_PRIMARY_REPLICANTS_TO_LOAD
-                                                : maxNonPrimaryReplicantsToLoad,
-          useRoundRobinSegmentAssignment == null ? DEFAULT_USE_ROUND_ROBIN_ASSIGNMENT : useRoundRobinSegmentAssignment
+          valueOrDefault(
+              decommissioningMaxPercentOfMaxSegmentsToMove,
+              Defaults.DECOMMISSIONING_MAX_SEGMENTS_TO_MOVE_PERCENT
+          ),
+          valueOrDefault(pauseCoordination, Defaults.PAUSE_COORDINATION),
+          valueOrDefault(replicateAfterLoadTimeout, Defaults.REPLICATE_AFTER_LOAD_TIMEOUT),
+          valueOrDefault(maxNonPrimaryReplicantsToLoad, Defaults.MAX_NON_PRIMARY_REPLICANTS_TO_LOAD),
+          valueOrDefault(useRoundRobinSegmentAssignment, Defaults.USE_ROUND_ROBIN_ASSIGNMENT)
       );
     }
 
+    private static <T> T valueOrDefault(@Nullable T value, @NotNull T defaultValue)
+    {
+      return value == null ? defaultValue : value;
+    }
+
     public CoordinatorDynamicConfig build(CoordinatorDynamicConfig defaults)
     {
       return new CoordinatorDynamicConfig(
-          leadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments == null
-          ? defaults.getLeadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments()
-          : leadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments,
-          mergeBytesLimit == null ? defaults.getMergeBytesLimit() : mergeBytesLimit,
-          mergeSegmentsLimit == null ? defaults.getMergeSegmentsLimit() : mergeSegmentsLimit,
-          maxSegmentsToMove == null ? defaults.getMaxSegmentsToMove() : maxSegmentsToMove,
-          percentOfSegmentsToConsiderPerMove == null
-          ? defaults.getPercentOfSegmentsToConsiderPerMove()
-          : percentOfSegmentsToConsiderPerMove,
-          useBatchedSegmentSampler == null ? defaults.useBatchedSegmentSampler() : useBatchedSegmentSampler,
-          replicantLifetime == null ? defaults.getReplicantLifetime() : replicantLifetime,
-          replicationThrottleLimit == null ? defaults.getReplicationThrottleLimit() : replicationThrottleLimit,
-          balancerComputeThreads == null ? defaults.getBalancerComputeThreads() : balancerComputeThreads,
-          emitBalancingStats == null ? defaults.emitBalancingStats() : emitBalancingStats,
-          specificDataSourcesToKillUnusedSegmentsIn == null
-          ? defaults.getSpecificDataSourcesToKillUnusedSegmentsIn()
-          : specificDataSourcesToKillUnusedSegmentsIn,
-          dataSourcesToNotKillStalePendingSegmentsIn == null
-          ? defaults.getDataSourcesToNotKillStalePendingSegmentsIn()
-          : dataSourcesToNotKillStalePendingSegmentsIn,
-          maxSegmentsInNodeLoadingQueue == null
-          ? defaults.getMaxSegmentsInNodeLoadingQueue()
-          : maxSegmentsInNodeLoadingQueue,
-          decommissioningNodes == null ? defaults.getDecommissioningNodes() : decommissioningNodes,
-          decommissioningMaxPercentOfMaxSegmentsToMove == null
-          ? defaults.getDecommissioningMaxPercentOfMaxSegmentsToMove()
-          : decommissioningMaxPercentOfMaxSegmentsToMove,
-          pauseCoordination == null ? defaults.getPauseCoordination() : pauseCoordination,
-          replicateAfterLoadTimeout == null ? defaults.getReplicateAfterLoadTimeout() : replicateAfterLoadTimeout,
-          maxNonPrimaryReplicantsToLoad == null
-          ? defaults.getMaxNonPrimaryReplicantsToLoad()
-          : maxNonPrimaryReplicantsToLoad,
-          useRoundRobinSegmentAssignment == null ? defaults.isUseRoundRobinSegmentAssignment() : useRoundRobinSegmentAssignment
+          valueOrDefault(
+              leadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments,
+              defaults.getLeadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments()
+          ),
+          valueOrDefault(mergeBytesLimit, defaults.getMergeBytesLimit()),
+          valueOrDefault(mergeSegmentsLimit, defaults.getMergeSegmentsLimit()),
+          valueOrDefault(maxSegmentsToMove, defaults.getMaxSegmentsToMove()),
+          valueOrDefault(percentOfSegmentsToConsiderPerMove, defaults.getPercentOfSegmentsToConsiderPerMove()),
+          valueOrDefault(useBatchedSegmentSampler, defaults.useBatchedSegmentSampler()),
+          valueOrDefault(replicantLifetime, defaults.getReplicantLifetime()),
+          valueOrDefault(replicationThrottleLimit, defaults.getReplicationThrottleLimit()),
+          valueOrDefault(balancerComputeThreads, defaults.getBalancerComputeThreads()),
+          valueOrDefault(emitBalancingStats, defaults.emitBalancingStats()),
+          valueOrDefault(specificDataSourcesToKillUnusedSegmentsIn, defaults.getSpecificDataSourcesToKillUnusedSegmentsIn()),
+          valueOrDefault(dataSourcesToNotKillStalePendingSegmentsIn, defaults.getDataSourcesToNotKillStalePendingSegmentsIn()),
+          valueOrDefault(maxSegmentsInNodeLoadingQueue, defaults.getMaxSegmentsInNodeLoadingQueue()),
+          valueOrDefault(decommissioningNodes, defaults.getDecommissioningNodes()),
+          valueOrDefault(
+              decommissioningMaxPercentOfMaxSegmentsToMove,
+              defaults.getDecommissioningMaxPercentOfMaxSegmentsToMove()
+          ),
+          valueOrDefault(pauseCoordination, defaults.getPauseCoordination()),
+          valueOrDefault(replicateAfterLoadTimeout, defaults.getReplicateAfterLoadTimeout()),
+          valueOrDefault(maxNonPrimaryReplicantsToLoad, defaults.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/4769)



##########
server/src/main/java/org/apache/druid/server/coordinator/CoordinatorDynamicConfig.java:
##########
@@ -713,75 +691,74 @@
       return this;
     }
 
+    /**
+     * Builds a CoordinatoryDynamicConfig using either the configured values, or
+     * the default value if not configured.
+     */
     public CoordinatorDynamicConfig build()
     {
       return new CoordinatorDynamicConfig(
-          leadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments == null
-          ? DEFAULT_LEADING_TIME_MILLIS_BEFORE_CAN_MARK_AS_UNUSED_OVERSHADOWED_SEGMENTS
-          : leadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments,
-          mergeBytesLimit == null ? DEFAULT_MERGE_BYTES_LIMIT : mergeBytesLimit,
-          mergeSegmentsLimit == null ? DEFAULT_MERGE_SEGMENTS_LIMIT : mergeSegmentsLimit,
-          maxSegmentsToMove == null ? DEFAULT_MAX_SEGMENTS_TO_MOVE : maxSegmentsToMove,
-          percentOfSegmentsToConsiderPerMove == null ? DEFAULT_PERCENT_OF_SEGMENTS_TO_CONSIDER_PER_MOVE
-                                                     : percentOfSegmentsToConsiderPerMove,
-          useBatchedSegmentSampler == null ? DEFAULT_USE_BATCHED_SEGMENT_SAMPLER : useBatchedSegmentSampler,
-          replicantLifetime == null ? DEFAULT_REPLICANT_LIFETIME : replicantLifetime,
-          replicationThrottleLimit == null ? DEFAULT_REPLICATION_THROTTLE_LIMIT : replicationThrottleLimit,
-          balancerComputeThreads == null ? DEFAULT_BALANCER_COMPUTE_THREADS : balancerComputeThreads,
-          emitBalancingStats == null ? DEFAULT_EMIT_BALANCING_STATS : emitBalancingStats,
+          valueOrDefault(
+              leadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments,
+              Defaults.LEADING_MILLIS_BEFORE_MARK_UNUSED
+          ),
+          valueOrDefault(mergeBytesLimit, Defaults.MERGE_BYTES_LIMIT),
+          valueOrDefault(mergeSegmentsLimit, Defaults.MERGE_SEGMENTS_LIMIT),
+          valueOrDefault(maxSegmentsToMove, Defaults.MAX_SEGMENTS_TO_MOVE),
+          valueOrDefault(percentOfSegmentsToConsiderPerMove, Defaults.PERCENT_OF_SEGMENTS_TO_CONSIDER_PER_MOVE),
+          valueOrDefault(useBatchedSegmentSampler, Defaults.USE_BATCHED_SEGMENT_SAMPLER),
+          valueOrDefault(replicantLifetime, Defaults.REPLICANT_LIFETIME),
+          valueOrDefault(replicationThrottleLimit, Defaults.REPLICATION_THROTTLE_LIMIT),
+          valueOrDefault(balancerComputeThreads, Defaults.BALANCER_COMPUTE_THREADS),
+          valueOrDefault(emitBalancingStats, Defaults.EMIT_BALANCING_STATS),
           specificDataSourcesToKillUnusedSegmentsIn,
           dataSourcesToNotKillStalePendingSegmentsIn,
-          maxSegmentsInNodeLoadingQueue == null
-          ? DEFAULT_MAX_SEGMENTS_IN_NODE_LOADING_QUEUE
-          : maxSegmentsInNodeLoadingQueue,
+          valueOrDefault(maxSegmentsInNodeLoadingQueue, Defaults.MAX_SEGMENTS_IN_NODE_LOADING_QUEUE),
           decommissioningNodes,
-          decommissioningMaxPercentOfMaxSegmentsToMove == null
-          ? DEFAULT_DECOMMISSIONING_MAX_SEGMENTS_TO_MOVE_PERCENT
-          : decommissioningMaxPercentOfMaxSegmentsToMove,
-          pauseCoordination == null ? DEFAULT_PAUSE_COORDINATION : pauseCoordination,
-          replicateAfterLoadTimeout == null ? DEFAULT_REPLICATE_AFTER_LOAD_TIMEOUT : replicateAfterLoadTimeout,
-          maxNonPrimaryReplicantsToLoad == null ? DEFAULT_MAX_NON_PRIMARY_REPLICANTS_TO_LOAD
-                                                : maxNonPrimaryReplicantsToLoad,
-          useRoundRobinSegmentAssignment == null ? DEFAULT_USE_ROUND_ROBIN_ASSIGNMENT : useRoundRobinSegmentAssignment
+          valueOrDefault(
+              decommissioningMaxPercentOfMaxSegmentsToMove,
+              Defaults.DECOMMISSIONING_MAX_SEGMENTS_TO_MOVE_PERCENT
+          ),
+          valueOrDefault(pauseCoordination, Defaults.PAUSE_COORDINATION),
+          valueOrDefault(replicateAfterLoadTimeout, Defaults.REPLICATE_AFTER_LOAD_TIMEOUT),
+          valueOrDefault(maxNonPrimaryReplicantsToLoad, Defaults.MAX_NON_PRIMARY_REPLICANTS_TO_LOAD),
+          valueOrDefault(useRoundRobinSegmentAssignment, Defaults.USE_ROUND_ROBIN_ASSIGNMENT)
       );
     }
 
+    private static <T> T valueOrDefault(@Nullable T value, @NotNull T defaultValue)
+    {
+      return value == null ? defaultValue : value;
+    }
+
     public CoordinatorDynamicConfig build(CoordinatorDynamicConfig defaults)
     {
       return new CoordinatorDynamicConfig(
-          leadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments == null
-          ? defaults.getLeadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments()
-          : leadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments,
-          mergeBytesLimit == null ? defaults.getMergeBytesLimit() : mergeBytesLimit,
-          mergeSegmentsLimit == null ? defaults.getMergeSegmentsLimit() : mergeSegmentsLimit,
-          maxSegmentsToMove == null ? defaults.getMaxSegmentsToMove() : maxSegmentsToMove,
-          percentOfSegmentsToConsiderPerMove == null
-          ? defaults.getPercentOfSegmentsToConsiderPerMove()
-          : percentOfSegmentsToConsiderPerMove,
-          useBatchedSegmentSampler == null ? defaults.useBatchedSegmentSampler() : useBatchedSegmentSampler,
-          replicantLifetime == null ? defaults.getReplicantLifetime() : replicantLifetime,
-          replicationThrottleLimit == null ? defaults.getReplicationThrottleLimit() : replicationThrottleLimit,
-          balancerComputeThreads == null ? defaults.getBalancerComputeThreads() : balancerComputeThreads,
-          emitBalancingStats == null ? defaults.emitBalancingStats() : emitBalancingStats,
-          specificDataSourcesToKillUnusedSegmentsIn == null
-          ? defaults.getSpecificDataSourcesToKillUnusedSegmentsIn()
-          : specificDataSourcesToKillUnusedSegmentsIn,
-          dataSourcesToNotKillStalePendingSegmentsIn == null
-          ? defaults.getDataSourcesToNotKillStalePendingSegmentsIn()
-          : dataSourcesToNotKillStalePendingSegmentsIn,
-          maxSegmentsInNodeLoadingQueue == null
-          ? defaults.getMaxSegmentsInNodeLoadingQueue()
-          : maxSegmentsInNodeLoadingQueue,
-          decommissioningNodes == null ? defaults.getDecommissioningNodes() : decommissioningNodes,
-          decommissioningMaxPercentOfMaxSegmentsToMove == null
-          ? defaults.getDecommissioningMaxPercentOfMaxSegmentsToMove()
-          : decommissioningMaxPercentOfMaxSegmentsToMove,
-          pauseCoordination == null ? defaults.getPauseCoordination() : pauseCoordination,
-          replicateAfterLoadTimeout == null ? defaults.getReplicateAfterLoadTimeout() : replicateAfterLoadTimeout,
-          maxNonPrimaryReplicantsToLoad == null
-          ? defaults.getMaxNonPrimaryReplicantsToLoad()
-          : maxNonPrimaryReplicantsToLoad,
-          useRoundRobinSegmentAssignment == null ? defaults.isUseRoundRobinSegmentAssignment() : useRoundRobinSegmentAssignment
+          valueOrDefault(
+              leadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments,
+              defaults.getLeadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments()
+          ),
+          valueOrDefault(mergeBytesLimit, defaults.getMergeBytesLimit()),
+          valueOrDefault(mergeSegmentsLimit, defaults.getMergeSegmentsLimit()),
+          valueOrDefault(maxSegmentsToMove, defaults.getMaxSegmentsToMove()),
+          valueOrDefault(percentOfSegmentsToConsiderPerMove, defaults.getPercentOfSegmentsToConsiderPerMove()),
+          valueOrDefault(useBatchedSegmentSampler, defaults.useBatchedSegmentSampler()),
+          valueOrDefault(replicantLifetime, defaults.getReplicantLifetime()),
+          valueOrDefault(replicationThrottleLimit, defaults.getReplicationThrottleLimit()),
+          valueOrDefault(balancerComputeThreads, defaults.getBalancerComputeThreads()),
+          valueOrDefault(emitBalancingStats, defaults.emitBalancingStats()),
+          valueOrDefault(specificDataSourcesToKillUnusedSegmentsIn, defaults.getSpecificDataSourcesToKillUnusedSegmentsIn()),
+          valueOrDefault(dataSourcesToNotKillStalePendingSegmentsIn, defaults.getDataSourcesToNotKillStalePendingSegmentsIn()),
+          valueOrDefault(maxSegmentsInNodeLoadingQueue, defaults.getMaxSegmentsInNodeLoadingQueue()),
+          valueOrDefault(decommissioningNodes, defaults.getDecommissioningNodes()),
+          valueOrDefault(
+              decommissioningMaxPercentOfMaxSegmentsToMove,
+              defaults.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/4768)



##########
server/src/main/java/org/apache/druid/server/coordinator/CoordinatorDynamicConfig.java:
##########
@@ -713,75 +691,74 @@
       return this;
     }
 
+    /**
+     * Builds a CoordinatoryDynamicConfig using either the configured values, or
+     * the default value if not configured.
+     */
     public CoordinatorDynamicConfig build()
     {
       return new CoordinatorDynamicConfig(
-          leadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments == null
-          ? DEFAULT_LEADING_TIME_MILLIS_BEFORE_CAN_MARK_AS_UNUSED_OVERSHADOWED_SEGMENTS
-          : leadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments,
-          mergeBytesLimit == null ? DEFAULT_MERGE_BYTES_LIMIT : mergeBytesLimit,
-          mergeSegmentsLimit == null ? DEFAULT_MERGE_SEGMENTS_LIMIT : mergeSegmentsLimit,
-          maxSegmentsToMove == null ? DEFAULT_MAX_SEGMENTS_TO_MOVE : maxSegmentsToMove,
-          percentOfSegmentsToConsiderPerMove == null ? DEFAULT_PERCENT_OF_SEGMENTS_TO_CONSIDER_PER_MOVE
-                                                     : percentOfSegmentsToConsiderPerMove,
-          useBatchedSegmentSampler == null ? DEFAULT_USE_BATCHED_SEGMENT_SAMPLER : useBatchedSegmentSampler,
-          replicantLifetime == null ? DEFAULT_REPLICANT_LIFETIME : replicantLifetime,
-          replicationThrottleLimit == null ? DEFAULT_REPLICATION_THROTTLE_LIMIT : replicationThrottleLimit,
-          balancerComputeThreads == null ? DEFAULT_BALANCER_COMPUTE_THREADS : balancerComputeThreads,
-          emitBalancingStats == null ? DEFAULT_EMIT_BALANCING_STATS : emitBalancingStats,
+          valueOrDefault(
+              leadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments,
+              Defaults.LEADING_MILLIS_BEFORE_MARK_UNUSED
+          ),
+          valueOrDefault(mergeBytesLimit, Defaults.MERGE_BYTES_LIMIT),
+          valueOrDefault(mergeSegmentsLimit, Defaults.MERGE_SEGMENTS_LIMIT),
+          valueOrDefault(maxSegmentsToMove, Defaults.MAX_SEGMENTS_TO_MOVE),
+          valueOrDefault(percentOfSegmentsToConsiderPerMove, Defaults.PERCENT_OF_SEGMENTS_TO_CONSIDER_PER_MOVE),
+          valueOrDefault(useBatchedSegmentSampler, Defaults.USE_BATCHED_SEGMENT_SAMPLER),
+          valueOrDefault(replicantLifetime, Defaults.REPLICANT_LIFETIME),
+          valueOrDefault(replicationThrottleLimit, Defaults.REPLICATION_THROTTLE_LIMIT),
+          valueOrDefault(balancerComputeThreads, Defaults.BALANCER_COMPUTE_THREADS),
+          valueOrDefault(emitBalancingStats, Defaults.EMIT_BALANCING_STATS),
           specificDataSourcesToKillUnusedSegmentsIn,
           dataSourcesToNotKillStalePendingSegmentsIn,
-          maxSegmentsInNodeLoadingQueue == null
-          ? DEFAULT_MAX_SEGMENTS_IN_NODE_LOADING_QUEUE
-          : maxSegmentsInNodeLoadingQueue,
+          valueOrDefault(maxSegmentsInNodeLoadingQueue, Defaults.MAX_SEGMENTS_IN_NODE_LOADING_QUEUE),
           decommissioningNodes,
-          decommissioningMaxPercentOfMaxSegmentsToMove == null
-          ? DEFAULT_DECOMMISSIONING_MAX_SEGMENTS_TO_MOVE_PERCENT
-          : decommissioningMaxPercentOfMaxSegmentsToMove,
-          pauseCoordination == null ? DEFAULT_PAUSE_COORDINATION : pauseCoordination,
-          replicateAfterLoadTimeout == null ? DEFAULT_REPLICATE_AFTER_LOAD_TIMEOUT : replicateAfterLoadTimeout,
-          maxNonPrimaryReplicantsToLoad == null ? DEFAULT_MAX_NON_PRIMARY_REPLICANTS_TO_LOAD
-                                                : maxNonPrimaryReplicantsToLoad,
-          useRoundRobinSegmentAssignment == null ? DEFAULT_USE_ROUND_ROBIN_ASSIGNMENT : useRoundRobinSegmentAssignment
+          valueOrDefault(
+              decommissioningMaxPercentOfMaxSegmentsToMove,
+              Defaults.DECOMMISSIONING_MAX_SEGMENTS_TO_MOVE_PERCENT
+          ),
+          valueOrDefault(pauseCoordination, Defaults.PAUSE_COORDINATION),
+          valueOrDefault(replicateAfterLoadTimeout, Defaults.REPLICATE_AFTER_LOAD_TIMEOUT),
+          valueOrDefault(maxNonPrimaryReplicantsToLoad, Defaults.MAX_NON_PRIMARY_REPLICANTS_TO_LOAD),
+          valueOrDefault(useRoundRobinSegmentAssignment, Defaults.USE_ROUND_ROBIN_ASSIGNMENT)
       );
     }
 
+    private static <T> T valueOrDefault(@Nullable T value, @NotNull T defaultValue)
+    {
+      return value == null ? defaultValue : value;
+    }
+
     public CoordinatorDynamicConfig build(CoordinatorDynamicConfig defaults)
     {
       return new CoordinatorDynamicConfig(
-          leadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments == null
-          ? defaults.getLeadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments()
-          : leadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments,
-          mergeBytesLimit == null ? defaults.getMergeBytesLimit() : mergeBytesLimit,
-          mergeSegmentsLimit == null ? defaults.getMergeSegmentsLimit() : mergeSegmentsLimit,
-          maxSegmentsToMove == null ? defaults.getMaxSegmentsToMove() : maxSegmentsToMove,
-          percentOfSegmentsToConsiderPerMove == null
-          ? defaults.getPercentOfSegmentsToConsiderPerMove()
-          : percentOfSegmentsToConsiderPerMove,
-          useBatchedSegmentSampler == null ? defaults.useBatchedSegmentSampler() : useBatchedSegmentSampler,
-          replicantLifetime == null ? defaults.getReplicantLifetime() : replicantLifetime,
-          replicationThrottleLimit == null ? defaults.getReplicationThrottleLimit() : replicationThrottleLimit,
-          balancerComputeThreads == null ? defaults.getBalancerComputeThreads() : balancerComputeThreads,
-          emitBalancingStats == null ? defaults.emitBalancingStats() : emitBalancingStats,
-          specificDataSourcesToKillUnusedSegmentsIn == null
-          ? defaults.getSpecificDataSourcesToKillUnusedSegmentsIn()
-          : specificDataSourcesToKillUnusedSegmentsIn,
-          dataSourcesToNotKillStalePendingSegmentsIn == null
-          ? defaults.getDataSourcesToNotKillStalePendingSegmentsIn()
-          : dataSourcesToNotKillStalePendingSegmentsIn,
-          maxSegmentsInNodeLoadingQueue == null
-          ? defaults.getMaxSegmentsInNodeLoadingQueue()
-          : maxSegmentsInNodeLoadingQueue,
-          decommissioningNodes == null ? defaults.getDecommissioningNodes() : decommissioningNodes,
-          decommissioningMaxPercentOfMaxSegmentsToMove == null
-          ? defaults.getDecommissioningMaxPercentOfMaxSegmentsToMove()
-          : decommissioningMaxPercentOfMaxSegmentsToMove,
-          pauseCoordination == null ? defaults.getPauseCoordination() : pauseCoordination,
-          replicateAfterLoadTimeout == null ? defaults.getReplicateAfterLoadTimeout() : replicateAfterLoadTimeout,
-          maxNonPrimaryReplicantsToLoad == null
-          ? defaults.getMaxNonPrimaryReplicantsToLoad()
-          : maxNonPrimaryReplicantsToLoad,
-          useRoundRobinSegmentAssignment == null ? defaults.isUseRoundRobinSegmentAssignment() : useRoundRobinSegmentAssignment
+          valueOrDefault(
+              leadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments,
+              defaults.getLeadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments()
+          ),
+          valueOrDefault(mergeBytesLimit, defaults.getMergeBytesLimit()),
+          valueOrDefault(mergeSegmentsLimit, defaults.getMergeSegmentsLimit()),
+          valueOrDefault(maxSegmentsToMove, defaults.getMaxSegmentsToMove()),
+          valueOrDefault(percentOfSegmentsToConsiderPerMove, defaults.getPercentOfSegmentsToConsiderPerMove()),
+          valueOrDefault(useBatchedSegmentSampler, defaults.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/4767)



##########
server/src/test/java/org/apache/druid/server/coordinator/rules/BroadcastDistributionRuleTest.java:
##########
@@ -402,18 +409,40 @@
             largeSegments.get(2),
             largeSegments2.get(0),
             largeSegments2.get(1)
-        ),
-        smallSegment
+        )
     );
 
-    Assert.assertEquals(5L, stats.getGlobalStat(LoadRule.ASSIGNED_COUNT));
-    Assert.assertFalse(stats.hasPerTierStats());
-
+    Assert.assertEquals(5L, stats.getDataSourceStat(Stats.Segments.ASSIGNED_BROADCAST, DS_SMALL));
     Assert.assertTrue(
         druidCluster
             .getAllServers()
             .stream()
             .allMatch(holder -> holder.isLoadingSegment(smallSegment) || holder.isServingSegment(smallSegment))
     );
   }
+
+  private CoordinatorRunStats runRuleAndGetStats(
+      Rule rule,
+      DataSegment segment,
+      DruidCoordinatorRuntimeParams params
+  )
+  {
+    final CoordinatorDynamicConfig dynamicConfig = params.getCoordinatorDynamicConfig();
+    ReplicationThrottler throttler = new ReplicationThrottler(
+        Sets.newHashSet(params.getDruidCluster().getTierNames()),
+        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/4774)



-- 
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: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] kfaraz commented on a diff in pull request #13197: Segment loading: Allow cancellation and prioritization of load queue items

Posted by "kfaraz (via GitHub)" <gi...@apache.org>.
kfaraz commented on code in PR #13197:
URL: https://github.com/apache/druid/pull/13197#discussion_r1231749300


##########
server/src/main/java/org/apache/druid/server/coordinator/SegmentReplicantLookup.java:
##########
@@ -125,49 +117,85 @@ public int getMovingReplicas(SegmentId segmentId, String tier)
    * Number of replicas of the segment which are safely loaded on the given tier
    * and are not being dropped.
    */
-  public int getLoadedReplicas(SegmentId segmentId, String tier)
+  public int getLoadedNotDroppingReplicas(SegmentId segmentId, String tier)
+  {
+    ReplicaCount count = replicaCounts.get(segmentId, tier);
+    return (count == null) ? 0 : count.loadedNotDropping();
+  }

Review Comment:
   Yes, that would be neater, thanks!



-- 
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: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] kfaraz commented on a diff in pull request #13197: Segment loading: Allow cancellation and prioritization of load queue items

Posted by "kfaraz (via GitHub)" <gi...@apache.org>.
kfaraz commented on code in PR #13197:
URL: https://github.com/apache/druid/pull/13197#discussion_r1231754131


##########
server/src/main/java/org/apache/druid/server/coordinator/StrategicSegmentAssigner.java:
##########
@@ -0,0 +1,577 @@
+/*
+ * 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;
+
+import com.google.common.collect.Sets;
+import it.unimi.dsi.fastutil.objects.Object2IntOpenHashMap;
+import org.apache.druid.java.util.emitter.EmittingLogger;
+import org.apache.druid.server.coordinator.balancer.BalancerStrategy;
+import org.apache.druid.server.coordinator.loadqueue.SegmentAction;
+import org.apache.druid.server.coordinator.loadqueue.SegmentLoadQueueManager;
+import org.apache.druid.server.coordinator.rules.SegmentActionHandler;
+import org.apache.druid.server.coordinator.stats.CoordinatorRunStats;
+import org.apache.druid.server.coordinator.stats.CoordinatorStat;
+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 java.util.Comparator;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+
+/**
+ * Used by the coordinator in each run for segment loading, dropping, balancing
+ * and broadcasting.
+ * <p>
+ * An instance of this class is freshly created for each coordinator run.
+ */
+public class StrategicSegmentAssigner implements SegmentActionHandler
+{
+  private static final EmittingLogger log = new EmittingLogger(StrategicSegmentAssigner.class);
+
+  private final SegmentLoadQueueManager loadQueueManager;
+  private final DruidCluster cluster;
+  private final CoordinatorRunStats stats;
+  private final SegmentReplicantLookup replicantLookup;
+  private final ReplicationThrottler replicationThrottler;
+  private final RoundRobinServerSelector serverSelector;
+  private final BalancerStrategy strategy;
+
+  private final boolean useRoundRobinAssignment;
+
+  private final Set<String> tiersWithNoServer = new HashSet<>();
+
+  public StrategicSegmentAssigner(
+      SegmentLoadQueueManager loadQueueManager,
+      DruidCluster cluster,
+      BalancerStrategy strategy,
+      CoordinatorDynamicConfig dynamicConfig,
+      CoordinatorRunStats stats
+  )
+  {
+    this.stats = stats;
+    this.cluster = cluster;
+    this.strategy = strategy;
+    this.loadQueueManager = loadQueueManager;
+    this.replicantLookup = SegmentReplicantLookup.make(cluster);
+    this.replicationThrottler = createReplicationThrottler(dynamicConfig);
+    this.useRoundRobinAssignment = dynamicConfig.isUseRoundRobinSegmentAssignment();
+    this.serverSelector = useRoundRobinAssignment ? new RoundRobinServerSelector(cluster) : null;
+  }
+
+  public CoordinatorRunStats getStats()
+  {
+    return stats;
+  }
+
+  public SegmentReplicantLookup getReplicantLookup()
+  {
+    return replicantLookup;
+  }
+
+  public void makeAlerts()
+  {
+    if (!tiersWithNoServer.isEmpty()) {
+      log.makeAlert("Tiers [%s] have no servers! Check your cluster configuration.", tiersWithNoServer).emit();
+    }
+  }
+
+  /**
+   * Moves the given segment from the source server to an eligible destination
+   * server.
+   * <p>
+   * An eligible destination server must:
+   * <ul>
+   *   <li>be present in the given list of destination servers</li>
+   *   <li>belong to the same tier as the source server</li>
+   *   <li>not already be serving or loading a replica of the segment</li>
+   *   <li>have enough space to load the segment</li>
+   * </ul>
+   * <p>
+   * The segment is not moved if:
+   * <ul>
+   *   <li>there is no eligible destination server, or</li>
+   *   <li>skipIfOptimallyPlaced is true and segment is already optimally placed, or</li>
+   *   <li>some other error occurs</li>
+   * </ul>
+   */
+  public boolean moveSegment(
+      DataSegment segment,
+      ServerHolder sourceServer,
+      List<ServerHolder> destinationServers
+  )
+  {
+    final String tier = sourceServer.getServer().getTier();
+    final List<ServerHolder> eligibleDestinationServers =
+        destinationServers.stream()
+                          .filter(s -> s.getServer().getTier().equals(tier))
+                          .filter(s -> s.canLoadSegment(segment))
+                          .collect(Collectors.toList());
+
+    if (eligibleDestinationServers.isEmpty()) {
+      incrementSkipStat(Stats.Segments.MOVE_SKIPPED, "No eligible server", segment, tier);
+      return false;
+    }
+
+    // If the source server is not decommissioning, move can be skipped if the
+    // segment is already optimally placed
+    if (!sourceServer.isDecommissioning()) {
+      eligibleDestinationServers.add(sourceServer);
+    }
+
+    final ServerHolder destination =
+        strategy.findDestinationServerToMoveSegment(segment, sourceServer, eligibleDestinationServers);
+
+    if (destination == null || destination.getServer().equals(sourceServer.getServer())) {
+      incrementSkipStat(Stats.Segments.MOVE_SKIPPED, "Optimally placed", segment, tier);
+      return false;
+    } else if (moveSegment(segment, sourceServer, destination)) {
+      incrementStat(Stats.Segments.MOVED, segment, tier, 1);
+      return true;
+    } else {
+      incrementSkipStat(Stats.Segments.MOVE_SKIPPED, "Encountered error", segment, tier);
+      return false;
+    }
+  }
+
+  /**
+   * Moves the given segment from serverA to serverB.
+   */
+  private boolean moveSegment(DataSegment segment, ServerHolder serverA, ServerHolder serverB)
+  {
+    final String tier = serverA.getServer().getTier();
+    if (serverA.isLoadingSegment(segment)) {
+      // Cancel the load on serverA and load on serverB instead
+      if (serverA.cancelOperation(SegmentAction.LOAD, segment)) {
+        int loadedCountOnTier = replicantLookup.getLoadedNotDroppingReplicas(segment.getId(), tier);
+        return loadSegment(segment, serverB, loadedCountOnTier >= 1);
+      }
+
+      // Could not cancel load, let the segment load on serverA and count it as unmoved
+      return false;
+    } else if (serverA.isServingSegment(segment)) {
+      return loadQueueManager.moveSegment(segment, serverA, serverB);
+    } else {
+      return false;
+    }
+  }
+
+  @Override
+  public void updateSegmentReplicasInTiers(DataSegment segment, Map<String, Integer> tierToReplicaCount)
+  {
+    // Identify empty tiers and determine total required replicas
+    final AtomicInteger requiredTotalReplicas = new AtomicInteger(0);
+    final Set<String> allTiers = Sets.newHashSet(cluster.getTierNames());
+    tierToReplicaCount.forEach((tier, requiredReplicas) -> {
+      reportTierCapacityStats(segment, requiredReplicas, tier);
+      replicantLookup.setRequiredReplicas(segment.getId(), false, tier, requiredReplicas);
+      if (allTiers.contains(tier)) {
+        requiredTotalReplicas.addAndGet(requiredReplicas);
+      } else {
+        tiersWithNoServer.add(tier);
+      }
+    });
+
+    final int loadedNotDroppingReplicas = replicantLookup.getLoadedReplicas(segment.getId(), false);
+    final int replicaSurplus = loadedNotDroppingReplicas - requiredTotalReplicas.get();
+
+    // Update replicas in every tier
+    int dropsQueued = 0;
+    for (String tier : allTiers) {
+      dropsQueued += updateReplicasInTier(
+          segment,
+          tier,
+          tierToReplicaCount.getOrDefault(tier, 0),
+          replicaSurplus - dropsQueued
+      );
+    }
+  }
+
+  /**
+   * Queues load or drop operations on this tier based on the required
+   * number of replicas and the current state.
+   * <p>
+   * The {@code maxReplicasToDrop} helps to maintain the required level of
+   * replication in the cluster. This ensures that segment read concurrency does
+   * not suffer during a tier shift or load rule change.
+   * <p>
+   * Returns the number of new drop operations queued on this tier.
+   */
+  private int updateReplicasInTier(
+      DataSegment segment,
+      String tier,
+      int requiredReplicas,
+      int maxReplicasToDrop
+  )
+  {
+    final int loadedNotDroppingReplicas =
+        replicantLookup.getLoadedNotDroppingReplicas(segment.getId(), tier);
+    final int loadingReplicas = replicantLookup.getLoadingReplicas(segment.getId(), tier);
+    final int projectedReplicas = loadedNotDroppingReplicas + loadingReplicas;
+
+    final int movingReplicas = replicantLookup.getMovingReplicas(segment.getId(), tier);
+    final boolean shouldCancelMoves = requiredReplicas == 0 && movingReplicas > 0;
+
+    // Check if there is any action required on this tier
+    if (projectedReplicas == requiredReplicas && !shouldCancelMoves) {
+      return 0;
+    }
+
+    final SegmentStatusInTier segmentStatus =
+        new SegmentStatusInTier(segment, cluster.getHistoricalsByTier(tier));
+
+    // Cancel all moves in this tier if it does not need to have replicas
+    if (shouldCancelMoves) {
+      cancelOperations(SegmentAction.MOVE_TO, movingReplicas, segment, segmentStatus);
+      cancelOperations(SegmentAction.MOVE_FROM, movingReplicas, segment, segmentStatus);
+    }
+
+    // Cancel drops and queue loads if the projected count is below the requirement
+    if (projectedReplicas < requiredReplicas) {
+      int replicaDeficit = requiredReplicas - projectedReplicas;
+      int cancelledDrops =
+          cancelOperations(SegmentAction.DROP, replicaDeficit, segment, segmentStatus);
+
+      // Cancelled drops can be counted as loaded replicas, thus reducing deficit
+      int numReplicasToLoad = replicaDeficit - cancelledDrops;
+      if (numReplicasToLoad > 0) {
+        boolean isAlreadyLoadedOnTier = loadedNotDroppingReplicas + cancelledDrops >= 1;
+        int numLoadsQueued = loadReplicas(numReplicasToLoad, segment, tier, segmentStatus, isAlreadyLoadedOnTier);
+        incrementStat(Stats.Segments.ASSIGNED, segment, tier, numLoadsQueued);
+      }
+    }
+
+    // Cancel loads and queue drops if the projected count exceeds the requirement
+    if (projectedReplicas > requiredReplicas) {
+      int replicaSurplus = projectedReplicas - requiredReplicas;
+      int cancelledLoads =
+          cancelOperations(SegmentAction.LOAD, replicaSurplus, segment, segmentStatus);
+
+      int numReplicasToDrop = Math.min(replicaSurplus - cancelledLoads, maxReplicasToDrop);
+      if (numReplicasToDrop > 0) {
+        int dropsQueuedOnTier = dropReplicas(numReplicasToDrop, segment, tier, segmentStatus);
+        incrementStat(Stats.Segments.DROPPED, segment, tier, dropsQueuedOnTier);
+        return dropsQueuedOnTier;
+      }
+    }
+
+    return 0;
+  }
+
+  private void reportTierCapacityStats(DataSegment segment, int requiredReplicas, String tier)
+  {
+    final RowKey rowKey = RowKey.forTier(tier);
+    stats.updateMax(Stats.Tier.REPLICATION_FACTOR, rowKey, requiredReplicas);
+    stats.add(Stats.Tier.REQUIRED_CAPACITY, rowKey, segment.getSize() * requiredReplicas);
+  }
+
+  @Override
+  public void broadcastSegment(DataSegment segment)
+  {
+    final Object2IntOpenHashMap<String> tierToRequiredReplicas = new Object2IntOpenHashMap<>();
+    for (ServerHolder server : cluster.getAllServers()) {
+      // Ignore servers which are not broadcast targets
+      if (!server.getServer().getType().isSegmentBroadcastTarget()) {
+        continue;
+      }
+
+      final String tier = server.getServer().getTier();
+
+      // Drop from decommissioning servers and load on active servers
+      int numDropsQueued = 0;
+      int numLoadsQueued = 0;
+      if (server.isDecommissioning()) {
+        numDropsQueued += dropBroadcastSegment(segment, server) ? 1 : 0;
+      } else {
+        tierToRequiredReplicas.addTo(tier, 1);
+        numLoadsQueued += loadBroadcastSegment(segment, server) ? 1 : 0;
+      }
+
+      if (numLoadsQueued > 0) {
+        incrementStat(Stats.Segments.ASSIGNED, segment, tier, numLoadsQueued);
+      }
+      if (numDropsQueued > 0) {
+        incrementStat(Stats.Segments.DROPPED, segment, tier, numDropsQueued);
+      }
+    }
+
+    // Update required replica counts
+    tierToRequiredReplicas.object2IntEntrySet().fastForEach(
+        entry -> replicantLookup
+            .setRequiredReplicas(segment.getId(), true, entry.getKey(), entry.getIntValue())
+    );
+  }
+
+  @Override
+  public void deleteSegment(DataSegment segment)
+  {
+    loadQueueManager.deleteSegment(segment);
+    stats.addToDatasourceStat(Stats.Segments.DELETED, segment.getDataSource(), 1);
+  }
+
+  /**
+   * Loads the broadcast segment if it is not loaded on the given server.
+   * Returns true only if the segment was successfully queued for load on the server.
+   */
+  private boolean loadBroadcastSegment(DataSegment segment, ServerHolder server)
+  {
+    if (server.isServingSegment(segment) || server.isLoadingSegment(segment)) {
+      return false;
+    } else if (server.isDroppingSegment(segment)) {
+      return server.cancelOperation(SegmentAction.DROP, segment);
+    }
+
+    if (server.canLoadSegment(segment) && loadSegment(segment, server, false)) {
+      return true;
+    } else {
+      log.makeAlert("Could not assign broadcast segment for datasource [%s]", segment.getDataSource())
+         .addData("segmentId", segment.getId())
+         .addData("segmentSize", segment.getSize())
+         .addData("hostName", server.getServer().getHost())
+         .addData("availableSize", server.getAvailableSize())
+         .emit();
+      return false;
+    }
+  }
+
+  /**
+   * Drops the broadcast segment if it is loaded on the given server.
+   * Returns true only if the segment was successfully queued for drop on the server.
+   */
+  private boolean dropBroadcastSegment(DataSegment segment, ServerHolder server)
+  {
+    if (server.isLoadingSegment(segment)) {
+      return server.cancelOperation(SegmentAction.LOAD, segment);
+    } else if (server.isServingSegment(segment)) {
+      return loadQueueManager.dropSegment(segment, server);
+    } else {
+      return false;
+    }
+  }
+
+  /**
+   * Queues drop of {@code numToDrop} replicas of the segment from a tier.
+   * Tries to drop replicas first from decommissioning servers and then from
+   * active servers.
+   * <p>
+   * Returns the number of successfully queued drop operations.
+   */
+  private int dropReplicas(
+      final int numToDrop,
+      DataSegment segment,
+      String tier,
+      SegmentStatusInTier segmentStatus
+  )
+  {
+    if (numToDrop <= 0) {
+      return 0;
+    }
+
+    final List<ServerHolder> eligibleServers = segmentStatus.getServersEligibleToDrop();
+    if (eligibleServers.isEmpty()) {
+      incrementSkipStat(Stats.Segments.DROP_SKIPPED, "No eligible server", segment, tier);
+      return 0;
+    }
+
+    // Keep eligible servers sorted by most full first
+    final TreeSet<ServerHolder> eligibleLiveServers = new TreeSet<>(Comparator.reverseOrder());
+    final TreeSet<ServerHolder> eligibleDyingServers = new TreeSet<>(Comparator.reverseOrder());
+    for (ServerHolder server : eligibleServers) {
+      if (server.isDecommissioning()) {
+        eligibleDyingServers.add(server);
+      } else {
+        eligibleLiveServers.add(server);
+      }
+    }
+
+    // Drop as many replicas as possible from decommissioning servers
+    int remainingNumToDrop = numToDrop;
+    int numDropsQueued =
+        dropReplicasFromServers(remainingNumToDrop, segment, eligibleDyingServers.iterator(), tier);
+
+    // Drop replicas from active servers if required
+    if (numToDrop > numDropsQueued) {
+      remainingNumToDrop = numToDrop - numDropsQueued;
+      Iterator<ServerHolder> serverIterator =
+          (useRoundRobinAssignment || eligibleLiveServers.size() >= remainingNumToDrop)
+          ? eligibleLiveServers.iterator()
+          : strategy.pickServersToDropSegment(segment, eligibleLiveServers);
+      numDropsQueued += dropReplicasFromServers(remainingNumToDrop, segment, serverIterator, tier);
+    }
+
+    return numDropsQueued;
+  }
+
+  /**
+   * Queues drop of {@code numToDrop} replicas of the segment from the servers.
+   * Returns the number of successfully queued drop operations.
+   */
+  private int dropReplicasFromServers(
+      int numToDrop,
+      DataSegment segment,
+      Iterator<ServerHolder> serverIterator,
+      String tier
+  )
+  {
+    int numDropsQueued = 0;
+    while (numToDrop > numDropsQueued && serverIterator.hasNext()) {
+      ServerHolder holder = serverIterator.next();
+      boolean dropped = loadQueueManager.dropSegment(segment, holder);
+
+      if (dropped) {
+        ++numDropsQueued;
+      } else {
+        incrementSkipStat(Stats.Segments.DROP_SKIPPED, "Encountered error", segment, tier);
+      }
+    }
+
+    return numDropsQueued;
+  }
+
+  /**
+   * Queues load of {@code numToLoad} replicas of the segment on a tier.
+   */
+  private int loadReplicas(
+      int numToLoad,
+      DataSegment segment,
+      String tier,
+      SegmentStatusInTier segmentStatus,
+      boolean isAlreadyLoadedOnTier
+  )
+  {
+    // Do not assign replicas if tier is already busy loading some
+    if (isAlreadyLoadedOnTier && replicationThrottler.isTierLoadingReplicas(tier)) {
+      return 0;
+    }
+
+    final List<ServerHolder> eligibleServers = segmentStatus.getServersEligibleToLoad();
+    if (eligibleServers.isEmpty()) {
+      incrementSkipStat(Stats.Segments.ASSIGN_SKIPPED, "No eligible server", segment, tier);
+      return 0;
+    }
+
+    final Iterator<ServerHolder> serverIterator =
+        useRoundRobinAssignment
+        ? serverSelector.getServersInTierToLoadSegment(tier, segment)
+        : strategy.findServersToLoadSegment(segment, eligibleServers);
+    if (!serverIterator.hasNext()) {
+      incrementSkipStat(Stats.Segments.ASSIGN_SKIPPED, "No server chosen by strategy", segment, tier);
+      return 0;
+    }
+
+    // Load the replicas on this tier
+    int numLoadsQueued = 0;
+    while (numLoadsQueued < numToLoad && serverIterator.hasNext()) {
+      numLoadsQueued += loadSegment(segment, serverIterator.next(), isAlreadyLoadedOnTier)
+                        ? 1 : 0;
+    }
+
+    return numLoadsQueued;
+  }
+
+  private boolean loadSegment(DataSegment segment, ServerHolder server, boolean isAlreadyLoadedOnTier)
+  {
+    final String tier = server.getServer().getTier();
+    if (isAlreadyLoadedOnTier && !replicationThrottler.canAssignReplica(tier)) {
+      incrementSkipStat(Stats.Segments.ASSIGN_SKIPPED, "Throttled replication", segment, tier);
+      return false;
+    }
+
+    final SegmentAction action = isAlreadyLoadedOnTier ? SegmentAction.REPLICATE : SegmentAction.LOAD;
+    final boolean assigned = loadQueueManager.loadSegment(segment, server, action);
+
+    if (!assigned) {
+      incrementSkipStat(Stats.Segments.ASSIGN_SKIPPED, "Encountered error", segment, tier);
+    } else if (isAlreadyLoadedOnTier) {
+      replicationThrottler.incrementAssignedReplicas(tier);
+    }
+
+    return assigned;
+  }

Review Comment:
   The boolean here is used to determine if the assignment was successful or not, not distinguish between load and replicate. I can add a comment to that effect. The boolean status is used downstream for other decisions, like counting the number of successful assignments so far.
   
   Please let me know if you think some other approach would be cleaner.



-- 
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: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] github-code-scanning[bot] commented on a diff in pull request #13197: Segment loading: Allow cancellation and prioritization of load queue items

Posted by "github-code-scanning[bot] (via GitHub)" <gi...@apache.org>.
github-code-scanning[bot] commented on code in PR #13197:
URL: https://github.com/apache/druid/pull/13197#discussion_r1232985512


##########
server/src/main/java/org/apache/druid/server/coordinator/loading/SegmentLoadingConfig.java:
##########
@@ -0,0 +1,124 @@
+/*
+ * 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.loading;
+
+import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.server.coordinator.CoordinatorDynamicConfig;
+
+/**
+ * Contains recomputed configs from {@link CoordinatorDynamicConfig} based on
+ * whether {@code smartSegmentLoading} is enabled or not.
+ */
+public class SegmentLoadingConfig
+{
+  private static final Logger log = new Logger(SegmentLoadingConfig.class);
+
+  private final int maxSegmentsInLoadQueue;
+  private final int replicationThrottleLimit;
+  private final int maxReplicaAssignmentsInRun;
+  private final int maxLifetimeInLoadQueue;
+
+  private final int maxSegmentsToMove;
+  private final int percentDecommSegmentsToMove;
+
+  private final boolean useRoundRobinSegmentAssignment;
+  private final boolean emitBalancingStats;
+
+  public SegmentLoadingConfig(CoordinatorDynamicConfig dynamicConfig, int numUsedSegments)
+  {
+    if (dynamicConfig.isSmartSegmentLoading()) {
+      // Compute recommended values
+      this.maxSegmentsInLoadQueue = 0;
+      this.useRoundRobinSegmentAssignment = true;
+      this.emitBalancingStats = false;
+      this.maxLifetimeInLoadQueue = dynamicConfig.getReplicantLifetime();
+      this.maxReplicaAssignmentsInRun = Integer.MAX_VALUE;
+      this.percentDecommSegmentsToMove = 100;
+
+      // Impose a lower bound on both replicationThrottleLimit and maxSegmentsToMove
+      final int throttlePercentage = 2;
+      final int replicationThrottleLimit = Math.max(100, numUsedSegments * throttlePercentage / 100);
+
+      // Impose an upper bound on maxSegmentsToMove to ensure that coordinator
+      // run times are bounded. This limit can be relaxed as performance of
+      // the CostBalancerStrategy.computeCost() is improved.
+      final int maxSegmentsToMove = Math.min(1000, replicationThrottleLimit);
+
+      log.info(
+          "Smart segment loading is enabled. Recomputed replicationThrottleLimit"
+          + " [%,d] (%d%% of used segments [%,d]) and maxSegmentsToMove [%d].",
+          replicationThrottleLimit, throttlePercentage, numUsedSegments, maxSegmentsToMove
+      );
+
+      this.replicationThrottleLimit = replicationThrottleLimit;
+      this.maxSegmentsToMove = maxSegmentsToMove;
+    } else {
+      // Use the configured values
+      this.maxSegmentsInLoadQueue = dynamicConfig.getMaxSegmentsInNodeLoadingQueue();
+      this.replicationThrottleLimit = dynamicConfig.getReplicationThrottleLimit();
+      this.maxLifetimeInLoadQueue = dynamicConfig.getReplicantLifetime();
+      this.maxSegmentsToMove = dynamicConfig.getMaxSegmentsToMove();
+      this.useRoundRobinSegmentAssignment = dynamicConfig.isUseRoundRobinSegmentAssignment();
+      this.emitBalancingStats = dynamicConfig.emitBalancingStats();
+      this.maxReplicaAssignmentsInRun = 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/5111)



##########
server/src/main/java/org/apache/druid/server/coordinator/loading/SegmentLoadingConfig.java:
##########
@@ -0,0 +1,124 @@
+/*
+ * 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.loading;
+
+import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.server.coordinator.CoordinatorDynamicConfig;
+
+/**
+ * Contains recomputed configs from {@link CoordinatorDynamicConfig} based on
+ * whether {@code smartSegmentLoading} is enabled or not.
+ */
+public class SegmentLoadingConfig
+{
+  private static final Logger log = new Logger(SegmentLoadingConfig.class);
+
+  private final int maxSegmentsInLoadQueue;
+  private final int replicationThrottleLimit;
+  private final int maxReplicaAssignmentsInRun;
+  private final int maxLifetimeInLoadQueue;
+
+  private final int maxSegmentsToMove;
+  private final int percentDecommSegmentsToMove;
+
+  private final boolean useRoundRobinSegmentAssignment;
+  private final boolean emitBalancingStats;
+
+  public SegmentLoadingConfig(CoordinatorDynamicConfig dynamicConfig, int numUsedSegments)
+  {
+    if (dynamicConfig.isSmartSegmentLoading()) {
+      // Compute recommended values
+      this.maxSegmentsInLoadQueue = 0;
+      this.useRoundRobinSegmentAssignment = true;
+      this.emitBalancingStats = false;
+      this.maxLifetimeInLoadQueue = dynamicConfig.getReplicantLifetime();
+      this.maxReplicaAssignmentsInRun = Integer.MAX_VALUE;
+      this.percentDecommSegmentsToMove = 100;
+
+      // Impose a lower bound on both replicationThrottleLimit and maxSegmentsToMove
+      final int throttlePercentage = 2;
+      final int replicationThrottleLimit = Math.max(100, numUsedSegments * throttlePercentage / 100);
+
+      // Impose an upper bound on maxSegmentsToMove to ensure that coordinator
+      // run times are bounded. This limit can be relaxed as performance of
+      // the CostBalancerStrategy.computeCost() is improved.
+      final int maxSegmentsToMove = Math.min(1000, replicationThrottleLimit);
+
+      log.info(
+          "Smart segment loading is enabled. Recomputed replicationThrottleLimit"
+          + " [%,d] (%d%% of used segments [%,d]) and maxSegmentsToMove [%d].",
+          replicationThrottleLimit, throttlePercentage, numUsedSegments, maxSegmentsToMove
+      );
+
+      this.replicationThrottleLimit = replicationThrottleLimit;
+      this.maxSegmentsToMove = maxSegmentsToMove;
+    } else {
+      // Use the configured values
+      this.maxSegmentsInLoadQueue = dynamicConfig.getMaxSegmentsInNodeLoadingQueue();
+      this.replicationThrottleLimit = dynamicConfig.getReplicationThrottleLimit();
+      this.maxLifetimeInLoadQueue = dynamicConfig.getReplicantLifetime();
+      this.maxSegmentsToMove = dynamicConfig.getMaxSegmentsToMove();
+      this.useRoundRobinSegmentAssignment = dynamicConfig.isUseRoundRobinSegmentAssignment();
+      this.emitBalancingStats = dynamicConfig.emitBalancingStats();
+      this.maxReplicaAssignmentsInRun = dynamicConfig.getMaxNonPrimaryReplicantsToLoad();
+      this.percentDecommSegmentsToMove = 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/5112)



##########
server/src/test/java/org/apache/druid/server/coordinator/duty/BalanceSegmentsTest.java:
##########
@@ -0,0 +1,491 @@
+/*
+ * 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.util.concurrent.ListeningExecutorService;
+import com.google.common.util.concurrent.MoreExecutors;
+import org.apache.druid.client.DruidServer;
+import org.apache.druid.java.util.common.DateTimes;
+import org.apache.druid.java.util.common.concurrent.Execs;
+import org.apache.druid.server.coordination.ServerType;
+import org.apache.druid.server.coordinator.CoordinatorDynamicConfig;
+import org.apache.druid.server.coordinator.DruidCluster;
+import org.apache.druid.server.coordinator.DruidCoordinatorRuntimeParams;
+import org.apache.druid.server.coordinator.ServerHolder;
+import org.apache.druid.server.coordinator.balancer.BalancerStrategy;
+import org.apache.druid.server.coordinator.balancer.CostBalancerStrategyFactory;
+import org.apache.druid.server.coordinator.loading.LoadQueuePeonTester;
+import org.apache.druid.server.coordinator.loading.SegmentLoadQueueManager;
+import org.apache.druid.server.coordinator.stats.CoordinatorRunStats;
+import org.apache.druid.server.coordinator.stats.Stats;
+import org.apache.druid.timeline.DataSegment;
+import org.apache.druid.timeline.partition.NoneShardSpec;
+import org.easymock.EasyMock;
+import org.joda.time.DateTime;
+import org.joda.time.Interval;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.Collections;
+import java.util.Set;
+
+public class BalanceSegmentsTest
+{
+  private SegmentLoadQueueManager loadQueueManager;
+
+  private DataSegment segment1;
+  private DataSegment segment2;
+  private DataSegment segment3;
+  private DataSegment segment4;
+  private DataSegment segment5;
+
+  private DataSegment[] allSegments;
+
+  private ListeningExecutorService balancerStrategyExecutor;
+  private BalancerStrategy balancerStrategy;
+  private Set<String> broadcastDatasources;
+
+  private DruidServer server1;
+  private DruidServer server2;
+  private DruidServer server3;
+  private DruidServer server4;
+
+  @Before
+  public void setUp()
+  {
+    loadQueueManager = new SegmentLoadQueueManager(null, null, null);
+
+    // Create test segments for multiple datasources
+    final DateTime start1 = DateTimes.of("2012-01-01");
+    final DateTime start2 = DateTimes.of("2012-02-01");
+    final String version = DateTimes.of("2012-03-01").toString();
+
+    segment1 = createHourlySegment("datasource1", start1, version);
+    segment2 = createHourlySegment("datasource1", start2, version);
+    segment3 = createHourlySegment("datasource2", start1, version);
+    segment4 = createHourlySegment("datasource2", start2, version);
+    segment5 = createHourlySegment("datasourceBroadcast", start2, version);
+    allSegments = new DataSegment[]{segment1, segment2, segment3, segment4, segment5};
+
+    server1 = new DruidServer("server1", "server1", null, 100L, ServerType.HISTORICAL, "normal", 0);
+    server2 = new DruidServer("server2", "server2", null, 100L, ServerType.HISTORICAL, "normal", 0);
+    server3 = new DruidServer("server3", "server3", null, 100L, ServerType.HISTORICAL, "normal", 0);
+    server4 = new DruidServer("server4", "server4", null, 100L, ServerType.HISTORICAL, "normal", 0);
+
+    balancerStrategyExecutor = MoreExecutors.listeningDecorator(Execs.multiThreaded(1, "BalanceSegmentsTest-%d"));
+    balancerStrategy = new CostBalancerStrategyFactory().createBalancerStrategy(balancerStrategyExecutor);
+
+    broadcastDatasources = Collections.singleton("datasourceBroadcast");
+  }
+
+  @After
+  public void tearDown()
+  {
+    balancerStrategyExecutor.shutdownNow();
+  }
+
+  @Test
+  public void testMoveToEmptyServerBalancer()
+  {
+    final ServerHolder serverHolder1 = createHolder(server1, segment1, segment2, segment3, segment4);
+    final ServerHolder serverHolder2 = createHolder(server2);
+
+    DruidCoordinatorRuntimeParams params =
+        defaultRuntimeParamsBuilder(serverHolder1, serverHolder2)
+            .withBalancerStrategy(balancerStrategy)
+            .withBroadcastDatasources(broadcastDatasources)
+            .build();
+
+    CoordinatorRunStats stats = runBalancer(params);
+    long totalMoved = stats.getSegmentStat(Stats.Segments.MOVED, "normal", "datasource1")
+                      + stats.getSegmentStat(Stats.Segments.MOVED, "normal", "datasource2");
+    Assert.assertEquals(2L, totalMoved);
+  }
+
+  /**
+   * Server 1 has 2 segments.
+   * Server 2 (decommissioning) has 2 segments.
+   * Server 3 is empty.
+   * Decommissioning percent is 60.
+   * Max segments to move is 3.
+   * 2 (of 2) segments should be moved from Server 2 and 1 (of 2) from Server 1.
+   */
+  @Test
+  public void testMoveDecommissioningMaxPercentOfMaxSegmentsToMove()
+  {
+    final ServerHolder serverHolder1 = createHolder(server1, false, segment1, segment2);
+    final ServerHolder serverHolder2 = createHolder(server2, true, segment3, segment4);
+    final ServerHolder serverHolder3 = createHolder(server3, false);
+
+    BalancerStrategy strategy = EasyMock.createMock(BalancerStrategy.class);
+    expectFindDestinationAndReturn(strategy, serverHolder3);
+    EasyMock.replay(strategy);
+
+    // ceil(3 * 0.6) = 2 segments from decommissioning servers
+    CoordinatorDynamicConfig dynamicConfig =
+        CoordinatorDynamicConfig.builder()
+                                .withSmartSegmentLoading(false)
+                                .withMaxSegmentsToMove(3)
+                                .withDecommissioningMaxPercentOfMaxSegmentsToMove(60)
+                                .build();
+    DruidCoordinatorRuntimeParams params =
+        defaultRuntimeParamsBuilder(serverHolder1, serverHolder2, serverHolder3)
+            .withDynamicConfigs(dynamicConfig)
+            .withBalancerStrategy(strategy)
+            .withBroadcastDatasources(broadcastDatasources)
+            .withSegmentAssignerUsing(loadQueueManager)
+            .build();
+
+    CoordinatorRunStats stats = runBalancer(params);
+
+    EasyMock.verify(strategy);
+
+    // 2 segments are moved from the decommissioning server and 1 from the active server
+    Assert.assertEquals(1L, stats.getSegmentStat(Stats.Segments.MOVED, "normal", "datasource1"));
+    Assert.assertEquals(2L, stats.getSegmentStat(Stats.Segments.MOVED, "normal", "datasource2"));
+    Set<DataSegment> segmentsMoved = serverHolder3.getPeon().getSegmentsToLoad();
+    Assert.assertTrue(segmentsMoved.contains(segment3));
+    Assert.assertTrue(segmentsMoved.contains(segment4));
+    Assert.assertTrue(segmentsMoved.contains(segment1) || segmentsMoved.contains(segment2));
+  }
+
+  @Test
+  public void testZeroDecommissioningMaxPercentOfMaxSegmentsToMove()
+  {
+    final ServerHolder holder1 = createHolder(server1, false, segment1, segment2);
+    final ServerHolder holder2 = createHolder(server2, true, segment3, segment4);
+    final ServerHolder holder3 = createHolder(server3, false);
+
+    CoordinatorDynamicConfig dynamicConfig =
+        CoordinatorDynamicConfig.builder()
+                                .withSmartSegmentLoading(false)
+                                .withDecommissioningMaxPercentOfMaxSegmentsToMove(0)
+                                .withMaxSegmentsToMove(1).build();
+    DruidCoordinatorRuntimeParams params =
+        defaultRuntimeParamsBuilder(holder1, holder2, holder3).withDynamicConfigs(dynamicConfig).build();
+
+    CoordinatorRunStats stats = runBalancer(params);
+
+    // Verify that either segment1 or segment2 is chosen for move
+    Assert.assertEquals(1L, stats.getSegmentStat(Stats.Segments.MOVED, "normal", segment1.getDataSource()));
+    DataSegment movingSegment = holder3.getPeon().getSegmentsToLoad().iterator().next();
+    Assert.assertEquals(segment1.getDataSource(), movingSegment.getDataSource());
+  }
+
+  @Test
+  public void testMaxDecommissioningMaxPercentOfMaxSegmentsToMove()
+  {
+    final ServerHolder holder1 = createHolder(server1, false, segment1, segment2);
+    final ServerHolder holder2 = createHolder(server2, true, segment3, segment4);
+    final ServerHolder holder3 = createHolder(server3, false);
+
+    CoordinatorDynamicConfig dynamicConfig =
+        CoordinatorDynamicConfig.builder()
+                                .withSmartSegmentLoading(false)
+                                .withDecommissioningMaxPercentOfMaxSegmentsToMove(100)
+                                .withMaxSegmentsToMove(1).build();
+    DruidCoordinatorRuntimeParams params =
+        defaultRuntimeParamsBuilder(holder1, holder2, holder3).withDynamicConfigs(dynamicConfig).build();
+
+    CoordinatorRunStats stats = runBalancer(params);
+
+    // Verify that either segment3 or segment4 is chosen for move
+    Assert.assertEquals(1L, stats.getSegmentStat(Stats.Segments.MOVED, "normal", segment3.getDataSource()));
+    DataSegment movingSegment = holder3.getPeon().getSegmentsToLoad().iterator().next();
+    Assert.assertEquals(segment3.getDataSource(), movingSegment.getDataSource());
+  }
+
+  /**
+   * Should balance segments as usual (ignoring percent) with empty decommissioningNodes.
+   */
+  @Test
+  public void testMoveDecommissioningMaxPercentOfMaxSegmentsToMoveWithNoDecommissioning()
+  {
+    final ServerHolder serverHolder1 = createHolder(server1, segment1, segment2);
+    final ServerHolder serverHolder2 = createHolder(server2, segment3, segment4);
+    final ServerHolder serverHolder3 = createHolder(server3);
+
+    BalancerStrategy strategy = EasyMock.createMock(BalancerStrategy.class);
+    expectFindDestinationAndReturn(strategy, serverHolder3);
+    EasyMock.replay(strategy);
+
+    CoordinatorDynamicConfig dynamicConfig =
+        CoordinatorDynamicConfig.builder()
+                                .withSmartSegmentLoading(false)
+                                .withMaxSegmentsToMove(3)
+                                .withDecommissioningMaxPercentOfMaxSegmentsToMove(9)
+                                .build();
+    DruidCoordinatorRuntimeParams params =
+        defaultRuntimeParamsBuilder(serverHolder1, serverHolder2, serverHolder3)
+            .withDynamicConfigs(dynamicConfig)
+            .withBalancerStrategy(strategy)
+            .withSegmentAssignerUsing(loadQueueManager)
+            .build();
+
+    CoordinatorRunStats stats = runBalancer(params);
+    EasyMock.verify(strategy);
+    long totalMoved = stats.getSegmentStat(Stats.Segments.MOVED, "normal", "datasource1")
+                      + stats.getSegmentStat(Stats.Segments.MOVED, "normal", "datasource2");
+    Assert.assertEquals(3L, totalMoved);
+    Assert.assertEquals(3, serverHolder3.getPeon().getSegmentsToLoad().size());
+  }
+
+  /**
+   * Shouldn't move segments to a decommissioning server.
+   */
+  @Test
+  public void testMoveToDecommissioningServer()
+  {
+    final ServerHolder activeServer = createHolder(server1, false, allSegments);
+    final ServerHolder decommissioningServer = createHolder(server2, true);
+
+    BalancerStrategy strategy = EasyMock.createMock(BalancerStrategy.class);
+    expectFindDestinationAndReturn(strategy, decommissioningServer);
+    EasyMock.replay(strategy);
+
+    DruidCoordinatorRuntimeParams params =
+        defaultRuntimeParamsBuilder(activeServer, decommissioningServer)
+            .withBalancerStrategy(strategy)
+            .withBroadcastDatasources(broadcastDatasources)
+            .build();
+
+    CoordinatorRunStats stats = runBalancer(params);
+    EasyMock.verify(strategy);
+    Assert.assertFalse(stats.hasStat(Stats.Segments.MOVED));
+  }
+
+  @Test
+  public void testMoveFromDecommissioningServer()
+  {
+    final ServerHolder decommissioningServer = createHolder(server1, true, allSegments);
+    final ServerHolder activeServer = createHolder(server2);
+
+    BalancerStrategy strategy = EasyMock.createMock(BalancerStrategy.class);
+    expectFindDestinationAndReturn(strategy, activeServer);
+    EasyMock.replay(strategy);
+
+    DruidCoordinatorRuntimeParams params = defaultRuntimeParamsBuilder(decommissioningServer, activeServer)
+        .withDynamicConfigs(
+            CoordinatorDynamicConfig.builder()
+                                    .withSmartSegmentLoading(false)
+                                    .withMaxSegmentsToMove(3).build()
+        )
+        .withBalancerStrategy(strategy)
+        .withBroadcastDatasources(broadcastDatasources)
+        .build();
+
+    runBalancer(params);
+    EasyMock.verify(strategy);
+    Assert.assertEquals(0, decommissioningServer.getPeon().getSegmentsToLoad().size());
+    Assert.assertEquals(3, activeServer.getPeon().getSegmentsToLoad().size());
+  }
+
+  @Test
+  public void testMoveMaxLoadQueueServerBalancer()
+  {
+    final int maxSegmentsInQueue = 1;
+    final ServerHolder holder1 = createHolder(server1, maxSegmentsInQueue, false, allSegments);
+    final ServerHolder holder2 = createHolder(server2, maxSegmentsInQueue, false);
+
+    final CoordinatorDynamicConfig dynamicConfig = CoordinatorDynamicConfig
+        .builder()
+        .withSmartSegmentLoading(false)
+        .withMaxSegmentsInNodeLoadingQueue(maxSegmentsInQueue)
+        .build();
+    DruidCoordinatorRuntimeParams params =
+        defaultRuntimeParamsBuilder(holder1, holder2)
+            .withDynamicConfigs(dynamicConfig)
+            .build();
+
+    CoordinatorRunStats stats = runBalancer(params);
+
+    // max to move is 5, all segments on server 1, but only expect to move 1 to server 2 since max node load queue is 1
+    Assert.assertEquals(maxSegmentsInQueue, stats.getSegmentStat(Stats.Segments.MOVED, "normal", "datasource1"));
+  }
+
+  @Test
+  public void testRun1()
+  {
+    // Mock some servers of different usages
+    DruidCoordinatorRuntimeParams params = defaultRuntimeParamsBuilder(
+        createHolder(server1, allSegments),
+        createHolder(server2)
+    ).build();
+
+    CoordinatorRunStats stats = runBalancer(params);
+    Assert.assertTrue(stats.getSegmentStat(Stats.Segments.MOVED, "normal", "datasource1") > 0);
+  }
+
+  @Test
+  public void testRun2()
+  {
+    // Mock some servers of different usages
+    DruidCoordinatorRuntimeParams params = defaultRuntimeParamsBuilder(
+        createHolder(server1, allSegments),
+        createHolder(server2),
+        createHolder(server3),
+        createHolder(server4)
+    ).build();
+
+    CoordinatorRunStats stats = runBalancer(params);
+    Assert.assertTrue(stats.getSegmentStat(Stats.Segments.MOVED, "normal", "datasource1") > 0);
+  }
+
+  @Test
+  public void testMaxSegmentsToMoveIsHonored()
+  {
+    // Move from non-decomissioning servers
+    final ServerHolder holder1 = createHolder(server1, segment1, segment2);
+    final ServerHolder holder2 = createHolder(server2, segment3, segment4);
+    final ServerHolder holder3 = createHolder(server3);
+
+    BalancerStrategy strategy = EasyMock.createMock(BalancerStrategy.class);
+    expectFindDestinationAndReturn(strategy, holder3);
+    EasyMock.replay(strategy);
+
+    DruidCoordinatorRuntimeParams params =
+        defaultRuntimeParamsBuilder(holder1, holder2, holder3)
+            .withDynamicConfigs(
+                CoordinatorDynamicConfig.builder()
+                                        .withSmartSegmentLoading(false)
+                                        .withMaxSegmentsToMove(1)
+                                        .withUseBatchedSegmentSampler(true)
+                                        .withPercentOfSegmentsToConsiderPerMove(40)
+                                        .build()
+            )
+            .withBalancerStrategy(strategy)
+            .withBroadcastDatasources(broadcastDatasources)
+            .build();
+
+    CoordinatorRunStats stats = runBalancer(params);
+    EasyMock.verify(strategy);
+    long totalMoved = stats.getSegmentStat(Stats.Segments.MOVED, "normal", "datasource1")
+                      + stats.getSegmentStat(Stats.Segments.MOVED, "normal", "datasource2");
+    Assert.assertEquals(1L, totalMoved);
+    Assert.assertEquals(1, holder3.getPeon().getSegmentsToLoad().size());
+  }
+
+  @Test
+  public void testUseBatchedSegmentSampler()
+  {
+    DruidCoordinatorRuntimeParams params = defaultRuntimeParamsBuilder(
+        createHolder(server1, allSegments),
+        createHolder(server2),
+        createHolder(server3),
+        createHolder(server4)
+    )
+        .withDynamicConfigs(
+            CoordinatorDynamicConfig.builder()
+                                    .withSmartSegmentLoading(false)
+                                    .withMaxSegmentsToMove(2)
+                                    .withUseBatchedSegmentSampler(true)

Review Comment:
   ## Deprecated method or constructor invocation
   
   Invoking [Builder.withUseBatchedSegmentSampler](1) should be avoided because it has been deprecated.
   
   [Show more details](https://github.com/apache/druid/security/code-scanning/5115)



##########
server/src/test/java/org/apache/druid/server/coordinator/duty/BalanceSegmentsTest.java:
##########
@@ -0,0 +1,491 @@
+/*
+ * 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.util.concurrent.ListeningExecutorService;
+import com.google.common.util.concurrent.MoreExecutors;
+import org.apache.druid.client.DruidServer;
+import org.apache.druid.java.util.common.DateTimes;
+import org.apache.druid.java.util.common.concurrent.Execs;
+import org.apache.druid.server.coordination.ServerType;
+import org.apache.druid.server.coordinator.CoordinatorDynamicConfig;
+import org.apache.druid.server.coordinator.DruidCluster;
+import org.apache.druid.server.coordinator.DruidCoordinatorRuntimeParams;
+import org.apache.druid.server.coordinator.ServerHolder;
+import org.apache.druid.server.coordinator.balancer.BalancerStrategy;
+import org.apache.druid.server.coordinator.balancer.CostBalancerStrategyFactory;
+import org.apache.druid.server.coordinator.loading.LoadQueuePeonTester;
+import org.apache.druid.server.coordinator.loading.SegmentLoadQueueManager;
+import org.apache.druid.server.coordinator.stats.CoordinatorRunStats;
+import org.apache.druid.server.coordinator.stats.Stats;
+import org.apache.druid.timeline.DataSegment;
+import org.apache.druid.timeline.partition.NoneShardSpec;
+import org.easymock.EasyMock;
+import org.joda.time.DateTime;
+import org.joda.time.Interval;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.Collections;
+import java.util.Set;
+
+public class BalanceSegmentsTest
+{
+  private SegmentLoadQueueManager loadQueueManager;
+
+  private DataSegment segment1;
+  private DataSegment segment2;
+  private DataSegment segment3;
+  private DataSegment segment4;
+  private DataSegment segment5;
+
+  private DataSegment[] allSegments;
+
+  private ListeningExecutorService balancerStrategyExecutor;
+  private BalancerStrategy balancerStrategy;
+  private Set<String> broadcastDatasources;
+
+  private DruidServer server1;
+  private DruidServer server2;
+  private DruidServer server3;
+  private DruidServer server4;
+
+  @Before
+  public void setUp()
+  {
+    loadQueueManager = new SegmentLoadQueueManager(null, null, null);
+
+    // Create test segments for multiple datasources
+    final DateTime start1 = DateTimes.of("2012-01-01");
+    final DateTime start2 = DateTimes.of("2012-02-01");
+    final String version = DateTimes.of("2012-03-01").toString();
+
+    segment1 = createHourlySegment("datasource1", start1, version);
+    segment2 = createHourlySegment("datasource1", start2, version);
+    segment3 = createHourlySegment("datasource2", start1, version);
+    segment4 = createHourlySegment("datasource2", start2, version);
+    segment5 = createHourlySegment("datasourceBroadcast", start2, version);
+    allSegments = new DataSegment[]{segment1, segment2, segment3, segment4, segment5};
+
+    server1 = new DruidServer("server1", "server1", null, 100L, ServerType.HISTORICAL, "normal", 0);
+    server2 = new DruidServer("server2", "server2", null, 100L, ServerType.HISTORICAL, "normal", 0);
+    server3 = new DruidServer("server3", "server3", null, 100L, ServerType.HISTORICAL, "normal", 0);
+    server4 = new DruidServer("server4", "server4", null, 100L, ServerType.HISTORICAL, "normal", 0);
+
+    balancerStrategyExecutor = MoreExecutors.listeningDecorator(Execs.multiThreaded(1, "BalanceSegmentsTest-%d"));
+    balancerStrategy = new CostBalancerStrategyFactory().createBalancerStrategy(balancerStrategyExecutor);
+
+    broadcastDatasources = Collections.singleton("datasourceBroadcast");
+  }
+
+  @After
+  public void tearDown()
+  {
+    balancerStrategyExecutor.shutdownNow();
+  }
+
+  @Test
+  public void testMoveToEmptyServerBalancer()
+  {
+    final ServerHolder serverHolder1 = createHolder(server1, segment1, segment2, segment3, segment4);
+    final ServerHolder serverHolder2 = createHolder(server2);
+
+    DruidCoordinatorRuntimeParams params =
+        defaultRuntimeParamsBuilder(serverHolder1, serverHolder2)
+            .withBalancerStrategy(balancerStrategy)
+            .withBroadcastDatasources(broadcastDatasources)
+            .build();
+
+    CoordinatorRunStats stats = runBalancer(params);
+    long totalMoved = stats.getSegmentStat(Stats.Segments.MOVED, "normal", "datasource1")
+                      + stats.getSegmentStat(Stats.Segments.MOVED, "normal", "datasource2");
+    Assert.assertEquals(2L, totalMoved);
+  }
+
+  /**
+   * Server 1 has 2 segments.
+   * Server 2 (decommissioning) has 2 segments.
+   * Server 3 is empty.
+   * Decommissioning percent is 60.
+   * Max segments to move is 3.
+   * 2 (of 2) segments should be moved from Server 2 and 1 (of 2) from Server 1.
+   */
+  @Test
+  public void testMoveDecommissioningMaxPercentOfMaxSegmentsToMove()
+  {
+    final ServerHolder serverHolder1 = createHolder(server1, false, segment1, segment2);
+    final ServerHolder serverHolder2 = createHolder(server2, true, segment3, segment4);
+    final ServerHolder serverHolder3 = createHolder(server3, false);
+
+    BalancerStrategy strategy = EasyMock.createMock(BalancerStrategy.class);
+    expectFindDestinationAndReturn(strategy, serverHolder3);
+    EasyMock.replay(strategy);
+
+    // ceil(3 * 0.6) = 2 segments from decommissioning servers
+    CoordinatorDynamicConfig dynamicConfig =
+        CoordinatorDynamicConfig.builder()
+                                .withSmartSegmentLoading(false)
+                                .withMaxSegmentsToMove(3)
+                                .withDecommissioningMaxPercentOfMaxSegmentsToMove(60)
+                                .build();
+    DruidCoordinatorRuntimeParams params =
+        defaultRuntimeParamsBuilder(serverHolder1, serverHolder2, serverHolder3)
+            .withDynamicConfigs(dynamicConfig)
+            .withBalancerStrategy(strategy)
+            .withBroadcastDatasources(broadcastDatasources)
+            .withSegmentAssignerUsing(loadQueueManager)
+            .build();
+
+    CoordinatorRunStats stats = runBalancer(params);
+
+    EasyMock.verify(strategy);
+
+    // 2 segments are moved from the decommissioning server and 1 from the active server
+    Assert.assertEquals(1L, stats.getSegmentStat(Stats.Segments.MOVED, "normal", "datasource1"));
+    Assert.assertEquals(2L, stats.getSegmentStat(Stats.Segments.MOVED, "normal", "datasource2"));
+    Set<DataSegment> segmentsMoved = serverHolder3.getPeon().getSegmentsToLoad();
+    Assert.assertTrue(segmentsMoved.contains(segment3));
+    Assert.assertTrue(segmentsMoved.contains(segment4));
+    Assert.assertTrue(segmentsMoved.contains(segment1) || segmentsMoved.contains(segment2));
+  }
+
+  @Test
+  public void testZeroDecommissioningMaxPercentOfMaxSegmentsToMove()
+  {
+    final ServerHolder holder1 = createHolder(server1, false, segment1, segment2);
+    final ServerHolder holder2 = createHolder(server2, true, segment3, segment4);
+    final ServerHolder holder3 = createHolder(server3, false);
+
+    CoordinatorDynamicConfig dynamicConfig =
+        CoordinatorDynamicConfig.builder()
+                                .withSmartSegmentLoading(false)
+                                .withDecommissioningMaxPercentOfMaxSegmentsToMove(0)
+                                .withMaxSegmentsToMove(1).build();
+    DruidCoordinatorRuntimeParams params =
+        defaultRuntimeParamsBuilder(holder1, holder2, holder3).withDynamicConfigs(dynamicConfig).build();
+
+    CoordinatorRunStats stats = runBalancer(params);
+
+    // Verify that either segment1 or segment2 is chosen for move
+    Assert.assertEquals(1L, stats.getSegmentStat(Stats.Segments.MOVED, "normal", segment1.getDataSource()));
+    DataSegment movingSegment = holder3.getPeon().getSegmentsToLoad().iterator().next();
+    Assert.assertEquals(segment1.getDataSource(), movingSegment.getDataSource());
+  }
+
+  @Test
+  public void testMaxDecommissioningMaxPercentOfMaxSegmentsToMove()
+  {
+    final ServerHolder holder1 = createHolder(server1, false, segment1, segment2);
+    final ServerHolder holder2 = createHolder(server2, true, segment3, segment4);
+    final ServerHolder holder3 = createHolder(server3, false);
+
+    CoordinatorDynamicConfig dynamicConfig =
+        CoordinatorDynamicConfig.builder()
+                                .withSmartSegmentLoading(false)
+                                .withDecommissioningMaxPercentOfMaxSegmentsToMove(100)
+                                .withMaxSegmentsToMove(1).build();
+    DruidCoordinatorRuntimeParams params =
+        defaultRuntimeParamsBuilder(holder1, holder2, holder3).withDynamicConfigs(dynamicConfig).build();
+
+    CoordinatorRunStats stats = runBalancer(params);
+
+    // Verify that either segment3 or segment4 is chosen for move
+    Assert.assertEquals(1L, stats.getSegmentStat(Stats.Segments.MOVED, "normal", segment3.getDataSource()));
+    DataSegment movingSegment = holder3.getPeon().getSegmentsToLoad().iterator().next();
+    Assert.assertEquals(segment3.getDataSource(), movingSegment.getDataSource());
+  }
+
+  /**
+   * Should balance segments as usual (ignoring percent) with empty decommissioningNodes.
+   */
+  @Test
+  public void testMoveDecommissioningMaxPercentOfMaxSegmentsToMoveWithNoDecommissioning()
+  {
+    final ServerHolder serverHolder1 = createHolder(server1, segment1, segment2);
+    final ServerHolder serverHolder2 = createHolder(server2, segment3, segment4);
+    final ServerHolder serverHolder3 = createHolder(server3);
+
+    BalancerStrategy strategy = EasyMock.createMock(BalancerStrategy.class);
+    expectFindDestinationAndReturn(strategy, serverHolder3);
+    EasyMock.replay(strategy);
+
+    CoordinatorDynamicConfig dynamicConfig =
+        CoordinatorDynamicConfig.builder()
+                                .withSmartSegmentLoading(false)
+                                .withMaxSegmentsToMove(3)
+                                .withDecommissioningMaxPercentOfMaxSegmentsToMove(9)
+                                .build();
+    DruidCoordinatorRuntimeParams params =
+        defaultRuntimeParamsBuilder(serverHolder1, serverHolder2, serverHolder3)
+            .withDynamicConfigs(dynamicConfig)
+            .withBalancerStrategy(strategy)
+            .withSegmentAssignerUsing(loadQueueManager)
+            .build();
+
+    CoordinatorRunStats stats = runBalancer(params);
+    EasyMock.verify(strategy);
+    long totalMoved = stats.getSegmentStat(Stats.Segments.MOVED, "normal", "datasource1")
+                      + stats.getSegmentStat(Stats.Segments.MOVED, "normal", "datasource2");
+    Assert.assertEquals(3L, totalMoved);
+    Assert.assertEquals(3, serverHolder3.getPeon().getSegmentsToLoad().size());
+  }
+
+  /**
+   * Shouldn't move segments to a decommissioning server.
+   */
+  @Test
+  public void testMoveToDecommissioningServer()
+  {
+    final ServerHolder activeServer = createHolder(server1, false, allSegments);
+    final ServerHolder decommissioningServer = createHolder(server2, true);
+
+    BalancerStrategy strategy = EasyMock.createMock(BalancerStrategy.class);
+    expectFindDestinationAndReturn(strategy, decommissioningServer);
+    EasyMock.replay(strategy);
+
+    DruidCoordinatorRuntimeParams params =
+        defaultRuntimeParamsBuilder(activeServer, decommissioningServer)
+            .withBalancerStrategy(strategy)
+            .withBroadcastDatasources(broadcastDatasources)
+            .build();
+
+    CoordinatorRunStats stats = runBalancer(params);
+    EasyMock.verify(strategy);
+    Assert.assertFalse(stats.hasStat(Stats.Segments.MOVED));
+  }
+
+  @Test
+  public void testMoveFromDecommissioningServer()
+  {
+    final ServerHolder decommissioningServer = createHolder(server1, true, allSegments);
+    final ServerHolder activeServer = createHolder(server2);
+
+    BalancerStrategy strategy = EasyMock.createMock(BalancerStrategy.class);
+    expectFindDestinationAndReturn(strategy, activeServer);
+    EasyMock.replay(strategy);
+
+    DruidCoordinatorRuntimeParams params = defaultRuntimeParamsBuilder(decommissioningServer, activeServer)
+        .withDynamicConfigs(
+            CoordinatorDynamicConfig.builder()
+                                    .withSmartSegmentLoading(false)
+                                    .withMaxSegmentsToMove(3).build()
+        )
+        .withBalancerStrategy(strategy)
+        .withBroadcastDatasources(broadcastDatasources)
+        .build();
+
+    runBalancer(params);
+    EasyMock.verify(strategy);
+    Assert.assertEquals(0, decommissioningServer.getPeon().getSegmentsToLoad().size());
+    Assert.assertEquals(3, activeServer.getPeon().getSegmentsToLoad().size());
+  }
+
+  @Test
+  public void testMoveMaxLoadQueueServerBalancer()
+  {
+    final int maxSegmentsInQueue = 1;
+    final ServerHolder holder1 = createHolder(server1, maxSegmentsInQueue, false, allSegments);
+    final ServerHolder holder2 = createHolder(server2, maxSegmentsInQueue, false);
+
+    final CoordinatorDynamicConfig dynamicConfig = CoordinatorDynamicConfig
+        .builder()
+        .withSmartSegmentLoading(false)
+        .withMaxSegmentsInNodeLoadingQueue(maxSegmentsInQueue)
+        .build();
+    DruidCoordinatorRuntimeParams params =
+        defaultRuntimeParamsBuilder(holder1, holder2)
+            .withDynamicConfigs(dynamicConfig)
+            .build();
+
+    CoordinatorRunStats stats = runBalancer(params);
+
+    // max to move is 5, all segments on server 1, but only expect to move 1 to server 2 since max node load queue is 1
+    Assert.assertEquals(maxSegmentsInQueue, stats.getSegmentStat(Stats.Segments.MOVED, "normal", "datasource1"));
+  }
+
+  @Test
+  public void testRun1()
+  {
+    // Mock some servers of different usages
+    DruidCoordinatorRuntimeParams params = defaultRuntimeParamsBuilder(
+        createHolder(server1, allSegments),
+        createHolder(server2)
+    ).build();
+
+    CoordinatorRunStats stats = runBalancer(params);
+    Assert.assertTrue(stats.getSegmentStat(Stats.Segments.MOVED, "normal", "datasource1") > 0);
+  }
+
+  @Test
+  public void testRun2()
+  {
+    // Mock some servers of different usages
+    DruidCoordinatorRuntimeParams params = defaultRuntimeParamsBuilder(
+        createHolder(server1, allSegments),
+        createHolder(server2),
+        createHolder(server3),
+        createHolder(server4)
+    ).build();
+
+    CoordinatorRunStats stats = runBalancer(params);
+    Assert.assertTrue(stats.getSegmentStat(Stats.Segments.MOVED, "normal", "datasource1") > 0);
+  }
+
+  @Test
+  public void testMaxSegmentsToMoveIsHonored()
+  {
+    // Move from non-decomissioning servers
+    final ServerHolder holder1 = createHolder(server1, segment1, segment2);
+    final ServerHolder holder2 = createHolder(server2, segment3, segment4);
+    final ServerHolder holder3 = createHolder(server3);
+
+    BalancerStrategy strategy = EasyMock.createMock(BalancerStrategy.class);
+    expectFindDestinationAndReturn(strategy, holder3);
+    EasyMock.replay(strategy);
+
+    DruidCoordinatorRuntimeParams params =
+        defaultRuntimeParamsBuilder(holder1, holder2, holder3)
+            .withDynamicConfigs(
+                CoordinatorDynamicConfig.builder()
+                                        .withSmartSegmentLoading(false)
+                                        .withMaxSegmentsToMove(1)
+                                        .withUseBatchedSegmentSampler(true)

Review Comment:
   ## Deprecated method or constructor invocation
   
   Invoking [Builder.withUseBatchedSegmentSampler](1) should be avoided because it has been deprecated.
   
   [Show more details](https://github.com/apache/druid/security/code-scanning/5114)



##########
server/src/test/java/org/apache/druid/server/coordinator/duty/BalanceSegmentsTest.java:
##########
@@ -0,0 +1,491 @@
+/*
+ * 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.util.concurrent.ListeningExecutorService;
+import com.google.common.util.concurrent.MoreExecutors;
+import org.apache.druid.client.DruidServer;
+import org.apache.druid.java.util.common.DateTimes;
+import org.apache.druid.java.util.common.concurrent.Execs;
+import org.apache.druid.server.coordination.ServerType;
+import org.apache.druid.server.coordinator.CoordinatorDynamicConfig;
+import org.apache.druid.server.coordinator.DruidCluster;
+import org.apache.druid.server.coordinator.DruidCoordinatorRuntimeParams;
+import org.apache.druid.server.coordinator.ServerHolder;
+import org.apache.druid.server.coordinator.balancer.BalancerStrategy;
+import org.apache.druid.server.coordinator.balancer.CostBalancerStrategyFactory;
+import org.apache.druid.server.coordinator.loading.LoadQueuePeonTester;
+import org.apache.druid.server.coordinator.loading.SegmentLoadQueueManager;
+import org.apache.druid.server.coordinator.stats.CoordinatorRunStats;
+import org.apache.druid.server.coordinator.stats.Stats;
+import org.apache.druid.timeline.DataSegment;
+import org.apache.druid.timeline.partition.NoneShardSpec;
+import org.easymock.EasyMock;
+import org.joda.time.DateTime;
+import org.joda.time.Interval;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.Collections;
+import java.util.Set;
+
+public class BalanceSegmentsTest
+{
+  private SegmentLoadQueueManager loadQueueManager;
+
+  private DataSegment segment1;
+  private DataSegment segment2;
+  private DataSegment segment3;
+  private DataSegment segment4;
+  private DataSegment segment5;
+
+  private DataSegment[] allSegments;
+
+  private ListeningExecutorService balancerStrategyExecutor;
+  private BalancerStrategy balancerStrategy;
+  private Set<String> broadcastDatasources;
+
+  private DruidServer server1;
+  private DruidServer server2;
+  private DruidServer server3;
+  private DruidServer server4;
+
+  @Before
+  public void setUp()
+  {
+    loadQueueManager = new SegmentLoadQueueManager(null, null, null);
+
+    // Create test segments for multiple datasources
+    final DateTime start1 = DateTimes.of("2012-01-01");
+    final DateTime start2 = DateTimes.of("2012-02-01");
+    final String version = DateTimes.of("2012-03-01").toString();
+
+    segment1 = createHourlySegment("datasource1", start1, version);
+    segment2 = createHourlySegment("datasource1", start2, version);
+    segment3 = createHourlySegment("datasource2", start1, version);
+    segment4 = createHourlySegment("datasource2", start2, version);
+    segment5 = createHourlySegment("datasourceBroadcast", start2, version);
+    allSegments = new DataSegment[]{segment1, segment2, segment3, segment4, segment5};
+
+    server1 = new DruidServer("server1", "server1", null, 100L, ServerType.HISTORICAL, "normal", 0);
+    server2 = new DruidServer("server2", "server2", null, 100L, ServerType.HISTORICAL, "normal", 0);
+    server3 = new DruidServer("server3", "server3", null, 100L, ServerType.HISTORICAL, "normal", 0);
+    server4 = new DruidServer("server4", "server4", null, 100L, ServerType.HISTORICAL, "normal", 0);
+
+    balancerStrategyExecutor = MoreExecutors.listeningDecorator(Execs.multiThreaded(1, "BalanceSegmentsTest-%d"));
+    balancerStrategy = new CostBalancerStrategyFactory().createBalancerStrategy(balancerStrategyExecutor);
+
+    broadcastDatasources = Collections.singleton("datasourceBroadcast");
+  }
+
+  @After
+  public void tearDown()
+  {
+    balancerStrategyExecutor.shutdownNow();
+  }
+
+  @Test
+  public void testMoveToEmptyServerBalancer()
+  {
+    final ServerHolder serverHolder1 = createHolder(server1, segment1, segment2, segment3, segment4);
+    final ServerHolder serverHolder2 = createHolder(server2);
+
+    DruidCoordinatorRuntimeParams params =
+        defaultRuntimeParamsBuilder(serverHolder1, serverHolder2)
+            .withBalancerStrategy(balancerStrategy)
+            .withBroadcastDatasources(broadcastDatasources)
+            .build();
+
+    CoordinatorRunStats stats = runBalancer(params);
+    long totalMoved = stats.getSegmentStat(Stats.Segments.MOVED, "normal", "datasource1")
+                      + stats.getSegmentStat(Stats.Segments.MOVED, "normal", "datasource2");
+    Assert.assertEquals(2L, totalMoved);
+  }
+
+  /**
+   * Server 1 has 2 segments.
+   * Server 2 (decommissioning) has 2 segments.
+   * Server 3 is empty.
+   * Decommissioning percent is 60.
+   * Max segments to move is 3.
+   * 2 (of 2) segments should be moved from Server 2 and 1 (of 2) from Server 1.
+   */
+  @Test
+  public void testMoveDecommissioningMaxPercentOfMaxSegmentsToMove()
+  {
+    final ServerHolder serverHolder1 = createHolder(server1, false, segment1, segment2);
+    final ServerHolder serverHolder2 = createHolder(server2, true, segment3, segment4);
+    final ServerHolder serverHolder3 = createHolder(server3, false);
+
+    BalancerStrategy strategy = EasyMock.createMock(BalancerStrategy.class);
+    expectFindDestinationAndReturn(strategy, serverHolder3);
+    EasyMock.replay(strategy);
+
+    // ceil(3 * 0.6) = 2 segments from decommissioning servers
+    CoordinatorDynamicConfig dynamicConfig =
+        CoordinatorDynamicConfig.builder()
+                                .withSmartSegmentLoading(false)
+                                .withMaxSegmentsToMove(3)
+                                .withDecommissioningMaxPercentOfMaxSegmentsToMove(60)
+                                .build();
+    DruidCoordinatorRuntimeParams params =
+        defaultRuntimeParamsBuilder(serverHolder1, serverHolder2, serverHolder3)
+            .withDynamicConfigs(dynamicConfig)
+            .withBalancerStrategy(strategy)
+            .withBroadcastDatasources(broadcastDatasources)
+            .withSegmentAssignerUsing(loadQueueManager)
+            .build();
+
+    CoordinatorRunStats stats = runBalancer(params);
+
+    EasyMock.verify(strategy);
+
+    // 2 segments are moved from the decommissioning server and 1 from the active server
+    Assert.assertEquals(1L, stats.getSegmentStat(Stats.Segments.MOVED, "normal", "datasource1"));
+    Assert.assertEquals(2L, stats.getSegmentStat(Stats.Segments.MOVED, "normal", "datasource2"));
+    Set<DataSegment> segmentsMoved = serverHolder3.getPeon().getSegmentsToLoad();
+    Assert.assertTrue(segmentsMoved.contains(segment3));
+    Assert.assertTrue(segmentsMoved.contains(segment4));
+    Assert.assertTrue(segmentsMoved.contains(segment1) || segmentsMoved.contains(segment2));
+  }
+
+  @Test
+  public void testZeroDecommissioningMaxPercentOfMaxSegmentsToMove()
+  {
+    final ServerHolder holder1 = createHolder(server1, false, segment1, segment2);
+    final ServerHolder holder2 = createHolder(server2, true, segment3, segment4);
+    final ServerHolder holder3 = createHolder(server3, false);
+
+    CoordinatorDynamicConfig dynamicConfig =
+        CoordinatorDynamicConfig.builder()
+                                .withSmartSegmentLoading(false)
+                                .withDecommissioningMaxPercentOfMaxSegmentsToMove(0)
+                                .withMaxSegmentsToMove(1).build();
+    DruidCoordinatorRuntimeParams params =
+        defaultRuntimeParamsBuilder(holder1, holder2, holder3).withDynamicConfigs(dynamicConfig).build();
+
+    CoordinatorRunStats stats = runBalancer(params);
+
+    // Verify that either segment1 or segment2 is chosen for move
+    Assert.assertEquals(1L, stats.getSegmentStat(Stats.Segments.MOVED, "normal", segment1.getDataSource()));
+    DataSegment movingSegment = holder3.getPeon().getSegmentsToLoad().iterator().next();
+    Assert.assertEquals(segment1.getDataSource(), movingSegment.getDataSource());
+  }
+
+  @Test
+  public void testMaxDecommissioningMaxPercentOfMaxSegmentsToMove()
+  {
+    final ServerHolder holder1 = createHolder(server1, false, segment1, segment2);
+    final ServerHolder holder2 = createHolder(server2, true, segment3, segment4);
+    final ServerHolder holder3 = createHolder(server3, false);
+
+    CoordinatorDynamicConfig dynamicConfig =
+        CoordinatorDynamicConfig.builder()
+                                .withSmartSegmentLoading(false)
+                                .withDecommissioningMaxPercentOfMaxSegmentsToMove(100)
+                                .withMaxSegmentsToMove(1).build();
+    DruidCoordinatorRuntimeParams params =
+        defaultRuntimeParamsBuilder(holder1, holder2, holder3).withDynamicConfigs(dynamicConfig).build();
+
+    CoordinatorRunStats stats = runBalancer(params);
+
+    // Verify that either segment3 or segment4 is chosen for move
+    Assert.assertEquals(1L, stats.getSegmentStat(Stats.Segments.MOVED, "normal", segment3.getDataSource()));
+    DataSegment movingSegment = holder3.getPeon().getSegmentsToLoad().iterator().next();
+    Assert.assertEquals(segment3.getDataSource(), movingSegment.getDataSource());
+  }
+
+  /**
+   * Should balance segments as usual (ignoring percent) with empty decommissioningNodes.
+   */
+  @Test
+  public void testMoveDecommissioningMaxPercentOfMaxSegmentsToMoveWithNoDecommissioning()
+  {
+    final ServerHolder serverHolder1 = createHolder(server1, segment1, segment2);
+    final ServerHolder serverHolder2 = createHolder(server2, segment3, segment4);
+    final ServerHolder serverHolder3 = createHolder(server3);
+
+    BalancerStrategy strategy = EasyMock.createMock(BalancerStrategy.class);
+    expectFindDestinationAndReturn(strategy, serverHolder3);
+    EasyMock.replay(strategy);
+
+    CoordinatorDynamicConfig dynamicConfig =
+        CoordinatorDynamicConfig.builder()
+                                .withSmartSegmentLoading(false)
+                                .withMaxSegmentsToMove(3)
+                                .withDecommissioningMaxPercentOfMaxSegmentsToMove(9)
+                                .build();
+    DruidCoordinatorRuntimeParams params =
+        defaultRuntimeParamsBuilder(serverHolder1, serverHolder2, serverHolder3)
+            .withDynamicConfigs(dynamicConfig)
+            .withBalancerStrategy(strategy)
+            .withSegmentAssignerUsing(loadQueueManager)
+            .build();
+
+    CoordinatorRunStats stats = runBalancer(params);
+    EasyMock.verify(strategy);
+    long totalMoved = stats.getSegmentStat(Stats.Segments.MOVED, "normal", "datasource1")
+                      + stats.getSegmentStat(Stats.Segments.MOVED, "normal", "datasource2");
+    Assert.assertEquals(3L, totalMoved);
+    Assert.assertEquals(3, serverHolder3.getPeon().getSegmentsToLoad().size());
+  }
+
+  /**
+   * Shouldn't move segments to a decommissioning server.
+   */
+  @Test
+  public void testMoveToDecommissioningServer()
+  {
+    final ServerHolder activeServer = createHolder(server1, false, allSegments);
+    final ServerHolder decommissioningServer = createHolder(server2, true);
+
+    BalancerStrategy strategy = EasyMock.createMock(BalancerStrategy.class);
+    expectFindDestinationAndReturn(strategy, decommissioningServer);
+    EasyMock.replay(strategy);
+
+    DruidCoordinatorRuntimeParams params =
+        defaultRuntimeParamsBuilder(activeServer, decommissioningServer)
+            .withBalancerStrategy(strategy)
+            .withBroadcastDatasources(broadcastDatasources)
+            .build();
+
+    CoordinatorRunStats stats = runBalancer(params);
+    EasyMock.verify(strategy);
+    Assert.assertFalse(stats.hasStat(Stats.Segments.MOVED));
+  }
+
+  @Test
+  public void testMoveFromDecommissioningServer()
+  {
+    final ServerHolder decommissioningServer = createHolder(server1, true, allSegments);
+    final ServerHolder activeServer = createHolder(server2);
+
+    BalancerStrategy strategy = EasyMock.createMock(BalancerStrategy.class);
+    expectFindDestinationAndReturn(strategy, activeServer);
+    EasyMock.replay(strategy);
+
+    DruidCoordinatorRuntimeParams params = defaultRuntimeParamsBuilder(decommissioningServer, activeServer)
+        .withDynamicConfigs(
+            CoordinatorDynamicConfig.builder()
+                                    .withSmartSegmentLoading(false)
+                                    .withMaxSegmentsToMove(3).build()
+        )
+        .withBalancerStrategy(strategy)
+        .withBroadcastDatasources(broadcastDatasources)
+        .build();
+
+    runBalancer(params);
+    EasyMock.verify(strategy);
+    Assert.assertEquals(0, decommissioningServer.getPeon().getSegmentsToLoad().size());
+    Assert.assertEquals(3, activeServer.getPeon().getSegmentsToLoad().size());
+  }
+
+  @Test
+  public void testMoveMaxLoadQueueServerBalancer()
+  {
+    final int maxSegmentsInQueue = 1;
+    final ServerHolder holder1 = createHolder(server1, maxSegmentsInQueue, false, allSegments);
+    final ServerHolder holder2 = createHolder(server2, maxSegmentsInQueue, false);
+
+    final CoordinatorDynamicConfig dynamicConfig = CoordinatorDynamicConfig
+        .builder()
+        .withSmartSegmentLoading(false)
+        .withMaxSegmentsInNodeLoadingQueue(maxSegmentsInQueue)
+        .build();
+    DruidCoordinatorRuntimeParams params =
+        defaultRuntimeParamsBuilder(holder1, holder2)
+            .withDynamicConfigs(dynamicConfig)
+            .build();
+
+    CoordinatorRunStats stats = runBalancer(params);
+
+    // max to move is 5, all segments on server 1, but only expect to move 1 to server 2 since max node load queue is 1
+    Assert.assertEquals(maxSegmentsInQueue, stats.getSegmentStat(Stats.Segments.MOVED, "normal", "datasource1"));
+  }
+
+  @Test
+  public void testRun1()
+  {
+    // Mock some servers of different usages
+    DruidCoordinatorRuntimeParams params = defaultRuntimeParamsBuilder(
+        createHolder(server1, allSegments),
+        createHolder(server2)
+    ).build();
+
+    CoordinatorRunStats stats = runBalancer(params);
+    Assert.assertTrue(stats.getSegmentStat(Stats.Segments.MOVED, "normal", "datasource1") > 0);
+  }
+
+  @Test
+  public void testRun2()
+  {
+    // Mock some servers of different usages
+    DruidCoordinatorRuntimeParams params = defaultRuntimeParamsBuilder(
+        createHolder(server1, allSegments),
+        createHolder(server2),
+        createHolder(server3),
+        createHolder(server4)
+    ).build();
+
+    CoordinatorRunStats stats = runBalancer(params);
+    Assert.assertTrue(stats.getSegmentStat(Stats.Segments.MOVED, "normal", "datasource1") > 0);
+  }
+
+  @Test
+  public void testMaxSegmentsToMoveIsHonored()
+  {
+    // Move from non-decomissioning servers
+    final ServerHolder holder1 = createHolder(server1, segment1, segment2);
+    final ServerHolder holder2 = createHolder(server2, segment3, segment4);
+    final ServerHolder holder3 = createHolder(server3);
+
+    BalancerStrategy strategy = EasyMock.createMock(BalancerStrategy.class);
+    expectFindDestinationAndReturn(strategy, holder3);
+    EasyMock.replay(strategy);
+
+    DruidCoordinatorRuntimeParams params =
+        defaultRuntimeParamsBuilder(holder1, holder2, holder3)
+            .withDynamicConfigs(
+                CoordinatorDynamicConfig.builder()
+                                        .withSmartSegmentLoading(false)
+                                        .withMaxSegmentsToMove(1)
+                                        .withUseBatchedSegmentSampler(true)
+                                        .withPercentOfSegmentsToConsiderPerMove(40)

Review Comment:
   ## Deprecated method or constructor invocation
   
   Invoking [Builder.withPercentOfSegmentsToConsiderPerMove](1) should be avoided because it has been deprecated.
   
   [Show more details](https://github.com/apache/druid/security/code-scanning/5113)



##########
server/src/main/java/org/apache/druid/server/coordinator/loading/HttpLoadQueuePeon.java:
##########
@@ -459,145 +500,127 @@
   }
 
   @Override
-  public int getNumberOfSegmentsInQueue()
+  public Set<DataSegment> getSegmentsMarkedToDrop()
   {
-    return segmentsToLoad.size();
+    return Collections.unmodifiableSet(segmentsMarkedToDrop);
   }
 
-  @Override
-  public Set<DataSegment> getSegmentsMarkedToDrop()
+  /**
+   * A request is considered to have timed out if the time elapsed since it was
+   * first sent to the server is greater than the configured load timeout.
+   *
+   * @see DruidCoordinatorConfig#getLoadTimeoutDelay()
+   */
+  private boolean hasRequestTimedOut(SegmentHolder holder, long currentTimeMillis)
   {
-    return Collections.unmodifiableSet(segmentsMarkedToDrop);
+    return holder.isRequestSentToServer()
+           && currentTimeMillis - holder.getFirstRequestMillis()
+              > config.getLoadTimeoutDelay().getMillis();
   }
 
-  private abstract class SegmentHolder
+  private void onRequestFailed(SegmentHolder holder, String failureCause)
   {
-    private final DataSegment segment;
-    private final DataSegmentChangeRequest changeRequest;
-    private final List<LoadPeonCallback> callbacks = new ArrayList<>();
-
-    // Time when this request was sent to target server the first time.
-    private volatile long scheduleTime = -1;
-
-    private SegmentHolder(
-        DataSegment segment,
-        DataSegmentChangeRequest changeRequest,
-        LoadPeonCallback callback
-    )
-    {
-      this.segment = segment;
-      this.changeRequest = changeRequest;
+    log.error(
+        "Server[%s] failed segment[%s] request[%s] with cause [%s].",
+        serverId, holder.getSegment().getId(), holder.getAction(), failureCause
+    );
+    onRequestCompleted(holder, SegmentStatusInQueue.FAILED);
+  }
 
-      if (callback != null) {
-        this.callbacks.add(callback);
-      }
-    }
+  private void onRequestCompleted(SegmentHolder holder, SegmentStatusInQueue status)
+  {
+    final SegmentAction action = holder.getAction();
+    log.trace(
+        "Server[%s] completed request[%s] on segment[%s] with status[%s].",
+        serverId, action, holder.getSegment().getId(), status
+    );
 
-    public void addCallback(LoadPeonCallback newCallback)
-    {
-      synchronized (callbacks) {
-        if (newCallback != null) {
-          callbacks.add(newCallback);
-        }
-      }
+    if (holder.isLoad()) {
+      queuedSize.addAndGet(-holder.getSegment().getSize());
     }
+    incrementStat(holder, status);
+    executeCallbacks(holder, status == SegmentStatusInQueue.SUCCESS);
+  }
 
-    public DataSegment getSegment()
-    {
-      return segment;
+  private void incrementStat(SegmentHolder holder, SegmentStatusInQueue status)
+  {
+    stats.add(status.getStatForAction(holder.getAction()), 1);
+    if (status.datasourceStat != null) {
+      stats.addToDatasourceStat(status.datasourceStat, holder.getSegment().getDataSource(), 1);
     }
+  }
 
-    public DataSegmentChangeRequest getChangeRequest()
-    {
-      return changeRequest;
-    }
+  private void executeCallbacks(SegmentHolder holder, boolean success)
+  {
+    callBackExecutor.execute(() -> {
+      for (LoadPeonCallback callback : holder.getCallbacks()) {
+        callback.execute(success);
+      }
+    });
+  }
 
-    public boolean hasTimedOut()
-    {
-      if (scheduleTime < 0) {
-        scheduleTime = System.currentTimeMillis();
-        return false;
-      } else if (System.currentTimeMillis() - scheduleTime > config.getLoadTimeoutDelay().getMillis()) {
-        return true;
-      } else {
+  /**
+   * Tries to cancel a load/drop operation. An load/drop request can be cancelled
+   * only if it has not already been sent to the corresponding server.
+   */
+  @Override
+  public boolean cancelOperation(DataSegment segment)
+  {
+    synchronized (lock) {
+      if (activeRequestSegments.contains(segment)) {
         return false;
       }
-    }
-
-    public void requestSucceeded()
-    {
-      log.trace(
-          "Server[%s] Successfully processed segment[%s] request[%s].",
-          serverId,
-          segment.getId(),
-          changeRequest.getClass().getSimpleName()
-      );
-
-      callBackExecutor.execute(() -> {
-        for (LoadPeonCallback callback : callbacks) {
-          if (callback != null) {
-            callback.execute(true);
-          }
-        }
-      });
-    }
 
-    public void requestFailed(String failureCause)
-    {
-      log.error(
-          "Server[%s] Failed segment[%s] request[%s] with cause [%s].",
-          serverId,
-          segment.getId(),
-          changeRequest.getClass().getSimpleName(),
-          failureCause
-      );
-
-      failedAssignCount.getAndIncrement();
+      // Find the action on this segment, if any
+      final SegmentHolder holder = segmentsToLoad.containsKey(segment)
+                                   ? segmentsToLoad.remove(segment)
+                                   : segmentsToDrop.remove(segment);
+      if (holder == null) {
+        return false;
+      }
 
-      callBackExecutor.execute(() -> {
-        for (LoadPeonCallback callback : callbacks) {
-          if (callback != null) {
-            callback.execute(false);
-          }
-        }
-      });
-    }
-
-    @Override
-    public String toString()
-    {
-      return changeRequest.toString();
+      queuedSegments.remove(holder);
+      onRequestCompleted(holder, SegmentStatusInQueue.CANCELLED);
+      return true;
     }
   }
 
-  private class LoadSegmentHolder extends SegmentHolder
+  private enum SegmentStatusInQueue
   {
-    public LoadSegmentHolder(DataSegment segment, LoadPeonCallback callback)
-    {
-      super(segment, new SegmentChangeRequestLoad(segment), callback);
-      queuedSize.addAndGet(segment.getSize());
-    }
+    ASSIGNED(Stats.SegmentQueue.ASSIGNED_ACTIONS),
+    SUCCESS(Stats.SegmentQueue.COMPLETED_ACTIONS),
+    FAILED(Stats.SegmentQueue.FAILED_ACTIONS),
+    CANCELLED(Stats.SegmentQueue.CANCELLED_ACTIONS);
 
-    @Override
-    public void requestSucceeded()
-    {
-      queuedSize.addAndGet(-getSegment().getSize());
-      super.requestSucceeded();
-    }
+    final CoordinatorStat loadStat;
+    final CoordinatorStat moveStat;
+    final CoordinatorStat dropStat;
+    final CoordinatorStat datasourceStat;
 
-    @Override
-    public void requestFailed(String failureCause)
+    SegmentStatusInQueue(CoordinatorStat datasourceStat)
     {
-      queuedSize.addAndGet(-getSegment().getSize());
-      super.requestFailed(failureCause);
+      // These stats are not emitted and are tracked for debugging purposes only
+      final String prefix = StringUtils.toLowerCase(name());
+      this.loadStat = new CoordinatorStat(prefix + "Load");
+      this.moveStat = new CoordinatorStat(prefix + "Move");
+      this.dropStat = new CoordinatorStat(prefix + "Drop");
+
+      this.datasourceStat = datasourceStat;
     }
-  }
 
-  private class DropSegmentHolder extends SegmentHolder
-  {
-    public DropSegmentHolder(DataSegment segment, LoadPeonCallback callback)
+    CoordinatorStat getStatForAction(SegmentAction action)
     {
-      super(segment, new SegmentChangeRequestDrop(segment), callback);
+      switch (action) {

Review Comment:
   ## Missing enum case in switch
   
   Switch statement does not have a case for [MOVE_FROM](1).
   
   [Show more details](https://github.com/apache/druid/security/code-scanning/5117)



##########
server/src/test/java/org/apache/druid/server/coordinator/loading/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.loading;
+
+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, load, replicate, 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.getSegmentsToLoad().size());
+
+    boolean cancelled = httpLoadQueuePeon.cancelOperation(segment);
+    Assert.assertTrue(cancelled);
+    Assert.assertEquals(0, httpLoadQueuePeon.getSegmentsToLoad().size());
+
+    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/5116)



-- 
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: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] kfaraz commented on a diff in pull request #13197: Segment loading: Allow cancellation and prioritization of load queue items

Posted by GitBox <gi...@apache.org>.
kfaraz commented on code in PR #13197:
URL: https://github.com/apache/druid/pull/13197#discussion_r1010210402


##########
server/src/main/java/org/apache/druid/server/coordinator/HttpLoadQueuePeon.java:
##########
@@ -380,10 +407,12 @@ public void loadSegment(DataSegment segment, LoadPeonCallback callback)
       }
 
       SegmentHolder holder = segmentsToLoad.get(segment);
-
       if (holder == null) {
         log.trace("Server[%s] to load segment[%s] queued.", serverId, segment.getId());
-        segmentsToLoad.put(segment, new LoadSegmentHolder(segment, callback));
+        queuedSize.addAndGet(segment.getSize());
+        holder = new SegmentHolder(segment, action, callback);
+        segmentsToLoad.put(segment, holder);

Review Comment:
   We might still need the `segmentsToLoad` and `segmentsToDrop` atleast for now because the balancer strategies use these to compute cost. This can be done with `queuedSegments` itself but we might have to filter out the relevant entries on every cost computation. 
   
   I do have a follow up PR which deals with the fixes in the strategy. I will try to clean up this part there.



-- 
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: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] kfaraz commented on a diff in pull request #13197: Segment loading: Allow cancellation and prioritization of load queue items

Posted by GitBox <gi...@apache.org>.
kfaraz commented on code in PR #13197:
URL: https://github.com/apache/druid/pull/13197#discussion_r996554921


##########
server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java:
##########
@@ -255,6 +253,8 @@ public DruidCoordinator(
     this.coordLeaderSelector = coordLeaderSelector;
     this.objectMapper = objectMapper;
     this.compactSegments = initializeCompactSegmentsDuty();
+    this.segmentStateManager =
+        new SegmentStateManager(serverInventoryView, segmentsMetadataManager, taskMaster.isHttpLoading());

Review Comment:
   Added.



-- 
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: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] imply-cheddar commented on a diff in pull request #13197: Segment loading: Allow cancellation and prioritization of load queue items

Posted by GitBox <gi...@apache.org>.
imply-cheddar commented on code in PR #13197:
URL: https://github.com/apache/druid/pull/13197#discussion_r997793124


##########
server/src/main/java/org/apache/druid/server/coordinator/SegmentStateManager.java:
##########
@@ -0,0 +1,261 @@
+/*
+ * 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;
+
+import com.google.inject.Inject;
+import org.apache.druid.client.ServerInventoryView;
+import org.apache.druid.java.util.emitter.EmittingLogger;
+import org.apache.druid.metadata.SegmentsMetadataManager;
+import org.apache.druid.timeline.DataSegment;
+
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * Manages state of segments being loaded.
+ */
+public class SegmentStateManager
+{
+  private static final EmittingLogger log = new EmittingLogger(SegmentStateManager.class);
+
+  private final LoadQueueTaskMaster taskMaster;
+  private final ServerInventoryView serverInventoryView;
+  private final SegmentsMetadataManager segmentsMetadataManager;
+
+  private final ConcurrentHashMap<String, TierLoadingState> currentlyMovingSegments =
+      new ConcurrentHashMap<>();
+  private final ConcurrentHashMap<String, TierLoadingState> currentlyReplicatingSegments
+      = new ConcurrentHashMap<>();
+
+  @Inject
+  public SegmentStateManager(
+      ServerInventoryView serverInventoryView,
+      SegmentsMetadataManager segmentsMetadataManager,
+      LoadQueueTaskMaster taskMaster
+  )
+  {
+    this.serverInventoryView = serverInventoryView;
+    this.segmentsMetadataManager = segmentsMetadataManager;
+    this.taskMaster = taskMaster;
+  }
+
+  /**
+   * Queues load of a replica of the segment on the given server.
+   */
+  public boolean loadSegment(
+      DataSegment segment,
+      ServerHolder server,
+      boolean isPrimary,
+      ReplicationThrottler throttler
+  )
+  {
+    final String tier = server.getServer().getTier();
+    final LoadPeonCallback callback;
+    if (isPrimary) {
+      // Primary replicas are not subject to throttling
+      callback = null;

Review Comment:
   What's the definition of "isPrimary"?  There really isn't a notion of "primary" or "secondary" replication, at least there shouldn't be.  The only semantic difference that should exist is
   
   1) If the data is not available at all inside of the tier -> get the data loaded as fast as humanly possibly
   2) If the data is available somewhere inside of the tier -> throttle the loading so that there is capacity in the cluster for other activities as well as under the assumption that this usually happens due to a failure and the node could just be flapping.



##########
server/src/main/java/org/apache/druid/server/coordinator/SegmentStateManager.java:
##########
@@ -0,0 +1,261 @@
+/*
+ * 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;
+
+import com.google.inject.Inject;
+import org.apache.druid.client.ServerInventoryView;
+import org.apache.druid.java.util.emitter.EmittingLogger;
+import org.apache.druid.metadata.SegmentsMetadataManager;
+import org.apache.druid.timeline.DataSegment;
+
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * Manages state of segments being loaded.
+ */
+public class SegmentStateManager
+{
+  private static final EmittingLogger log = new EmittingLogger(SegmentStateManager.class);
+
+  private final LoadQueueTaskMaster taskMaster;
+  private final ServerInventoryView serverInventoryView;
+  private final SegmentsMetadataManager segmentsMetadataManager;
+
+  private final ConcurrentHashMap<String, TierLoadingState> currentlyMovingSegments =
+      new ConcurrentHashMap<>();
+  private final ConcurrentHashMap<String, TierLoadingState> currentlyReplicatingSegments
+      = new ConcurrentHashMap<>();
+
+  @Inject
+  public SegmentStateManager(
+      ServerInventoryView serverInventoryView,
+      SegmentsMetadataManager segmentsMetadataManager,
+      LoadQueueTaskMaster taskMaster
+  )
+  {
+    this.serverInventoryView = serverInventoryView;
+    this.segmentsMetadataManager = segmentsMetadataManager;
+    this.taskMaster = taskMaster;
+  }
+
+  /**
+   * Queues load of a replica of the segment on the given server.
+   */
+  public boolean loadSegment(
+      DataSegment segment,
+      ServerHolder server,
+      boolean isPrimary,
+      ReplicationThrottler throttler
+  )
+  {
+    final String tier = server.getServer().getTier();
+    final LoadPeonCallback callback;
+    if (isPrimary) {
+      // Primary replicas are not subject to throttling
+      callback = null;
+    } else if (canLoadReplica(tier, throttler)) {
+      throttler.incrementAssignedReplicas(tier);
+
+      final TierLoadingState replicatingInTier = currentlyReplicatingSegments
+          .computeIfAbsent(tier, t -> new TierLoadingState(throttler.getMaxLifetime()));
+      replicatingInTier.addSegment(segment.getId(), server.getServer().getHost());
+      callback = success -> replicatingInTier.removeSegment(segment.getId());
+    } else {
+      throttler.incrementThrottledReplicas(tier);
+      return false;
+    }
+
+    try {
+      if (!server.startOperation(segment, SegmentState.LOADING)) {
+        return false;
+      }
+
+      server.getPeon().loadSegment(
+          segment,
+          isPrimary ? SegmentAction.LOAD_AS_PRIMARY : SegmentAction.LOAD_AS_REPLICA,

Review Comment:
   Let's please kill this false notion of primary/replica and use words that actually align with the semantics that should exist.



##########
server/src/main/java/org/apache/druid/server/coordinator/SegmentStateManager.java:
##########
@@ -0,0 +1,261 @@
+/*
+ * 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;
+
+import com.google.inject.Inject;
+import org.apache.druid.client.ServerInventoryView;
+import org.apache.druid.java.util.emitter.EmittingLogger;
+import org.apache.druid.metadata.SegmentsMetadataManager;
+import org.apache.druid.timeline.DataSegment;
+
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * Manages state of segments being loaded.
+ */
+public class SegmentStateManager
+{
+  private static final EmittingLogger log = new EmittingLogger(SegmentStateManager.class);
+
+  private final LoadQueueTaskMaster taskMaster;
+  private final ServerInventoryView serverInventoryView;
+  private final SegmentsMetadataManager segmentsMetadataManager;
+
+  private final ConcurrentHashMap<String, TierLoadingState> currentlyMovingSegments =
+      new ConcurrentHashMap<>();
+  private final ConcurrentHashMap<String, TierLoadingState> currentlyReplicatingSegments
+      = new ConcurrentHashMap<>();
+
+  @Inject
+  public SegmentStateManager(
+      ServerInventoryView serverInventoryView,
+      SegmentsMetadataManager segmentsMetadataManager,
+      LoadQueueTaskMaster taskMaster
+  )
+  {
+    this.serverInventoryView = serverInventoryView;
+    this.segmentsMetadataManager = segmentsMetadataManager;
+    this.taskMaster = taskMaster;
+  }
+
+  /**
+   * Queues load of a replica of the segment on the given server.
+   */
+  public boolean loadSegment(
+      DataSegment segment,
+      ServerHolder server,
+      boolean isPrimary,
+      ReplicationThrottler throttler
+  )
+  {
+    final String tier = server.getServer().getTier();
+    final LoadPeonCallback callback;
+    if (isPrimary) {
+      // Primary replicas are not subject to throttling
+      callback = null;
+    } else if (canLoadReplica(tier, throttler)) {
+      throttler.incrementAssignedReplicas(tier);
+
+      final TierLoadingState replicatingInTier = currentlyReplicatingSegments
+          .computeIfAbsent(tier, t -> new TierLoadingState(throttler.getMaxLifetime()));
+      replicatingInTier.addSegment(segment.getId(), server.getServer().getHost());
+      callback = success -> replicatingInTier.removeSegment(segment.getId());
+    } else {
+      throttler.incrementThrottledReplicas(tier);
+      return false;
+    }
+
+    try {
+      if (!server.startOperation(segment, SegmentState.LOADING)) {
+        return false;
+      }
+
+      server.getPeon().loadSegment(
+          segment,
+          isPrimary ? SegmentAction.LOAD_AS_PRIMARY : SegmentAction.LOAD_AS_REPLICA,
+          callback
+      );
+      return true;
+    }
+    catch (Exception e) {
+      server.cancelOperation(segment, SegmentState.LOADING);
+      return false;
+    }
+  }
+
+  public boolean dropSegment(DataSegment segment, ServerHolder server)
+  {
+    try {
+      if (!server.startOperation(segment, SegmentState.DROPPING)) {
+        return false;
+      }
+
+      server.getPeon().dropSegment(segment, null);
+      return true;
+    }
+    catch (Exception e) {
+      server.cancelOperation(segment, SegmentState.DROPPING);
+      return false;
+    }
+  }
+
+  public boolean moveSegment(
+      DataSegment segment,
+      ServerHolder fromServer,
+      ServerHolder toServer,
+      int maxLifetimeInBalancingQueue
+  )
+  {
+    final TierLoadingState segmentsMovingInTier = currentlyMovingSegments.computeIfAbsent(
+        toServer.getServer().getTier(),
+        t -> new TierLoadingState(maxLifetimeInBalancingQueue)
+    );
+    final LoadQueuePeon fromServerPeon = fromServer.getPeon();
+    final LoadPeonCallback moveFinishCallback = success -> {
+      fromServerPeon.unmarkSegmentToDrop(segment);
+      segmentsMovingInTier.removeSegment(segment.getId());
+    };
+
+    // mark segment to drop before it is actually loaded on server
+    // to be able to account for this information in BalancerStrategy immediately
+    toServer.startOperation(segment, SegmentState.MOVING_TO);
+    fromServerPeon.markSegmentToDrop(segment);
+    segmentsMovingInTier.addSegment(segment.getId(), fromServer.getServer().getHost());
+
+    final LoadQueuePeon toServerPeon = toServer.getPeon();
+    final String toServerName = toServer.getServer().getName();
+    try {
+      toServerPeon.loadSegment(
+          segment,
+          SegmentAction.MOVE_TO,
+          success -> {
+            // Drop segment only if:
+            // (1) segment load was successful on toServer
+            // AND (2) segment not already queued for drop on fromServer
+            // AND (3a) loading is http-based
+            //     OR (3b) inventory shows segment loaded on toServer
+
+            // Do not check the inventory with http loading as the HTTP
+            // response is enough to determine load success or failure
+            if (success
+                && !fromServerPeon.getSegmentsToDrop().contains(segment)
+                && (taskMaster.isHttpLoading()
+                    || serverInventoryView.isSegmentLoadedByServer(toServerName, segment))) {
+              fromServerPeon.dropSegment(segment, moveFinishCallback);
+            } else {
+              moveFinishCallback.execute(success);
+            }
+          }
+      );
+    }
+    catch (Exception e) {
+      toServer.cancelOperation(segment, SegmentState.MOVING_TO);
+      moveFinishCallback.execute(false);
+      throw new RuntimeException(e);
+    }
+
+    return true;
+  }
+
+  /**
+   * Marks the given segment as unused.
+   */
+  public boolean deleteSegment(DataSegment segment)
+  {
+    return segmentsMetadataManager.markSegmentAsUnused(segment.getId());
+  }
+
+  /**
+   * Cancels the segment operation being performed on a server if the actual
+   * state of the segment on the server matches the given currentState.
+   */
+  public boolean cancelOperation(
+      SegmentState currentState,
+      DataSegment segment,
+      ServerHolder server
+  )
+  {
+    if (!server.cancelOperation(segment, currentState)) {
+      return false;
+    }
+
+    final LoadQueuePeon peon = server.getPeon();
+    switch (currentState) {
+      case DROPPING:
+        return peon.cancelDrop(segment);
+      case MOVING_TO:
+      case LOADING:
+        return peon.cancelLoad(segment);
+      default:
+        return false;
+    }
+  }
+
+  /**
+   * Reduces the lifetimes of the segments currently being moved in all the tiers,
+   * and returns a map from tier names to the corresponding state.
+   */
+  public Map<String, TierLoadingState> reduceLifetimesOfMovingSegments()
+  {
+    return reduceLifetimesAndCreateCopy(currentlyMovingSegments);
+  }
+
+  /**
+   * Reduces the lifetimes of the segments currently being replicated in the tiers,
+   * and returns a map from tier names to the corresponding state.
+   */
+  public Map<String, TierLoadingState> reduceLifetimesOfReplicatingSegments()
+  {
+    return reduceLifetimesAndCreateCopy(currentlyReplicatingSegments);
+  }
+
+  private Map<String, TierLoadingState> reduceLifetimesAndCreateCopy(
+      Map<String, TierLoadingState> inFlightSegments
+  )
+  {
+    final Set<String> inactiveTiers = new HashSet<>();
+    inFlightSegments.forEach((tier, holder) -> {
+      if (holder.getNumProcessingSegments() == 0) {
+        inactiveTiers.add(tier);
+      }
+      holder.reduceLifetime();
+    });
+
+    // Reset state for inactive tiers
+    inactiveTiers.forEach(inFlightSegments::remove);
+
+    return Collections.unmodifiableMap(inFlightSegments);
+  }
+
+  private boolean canLoadReplica(String tier, ReplicationThrottler throttler)
+  {
+    if (!throttler.canAssignReplica(tier)) {
+      return false;
+    }
+
+    TierLoadingState tierState = currentlyReplicatingSegments.get(tier);
+    return tierState == null || tierState.getNumProcessingSegments() < throttler.getReplicationThrottleLimit();

Review Comment:
   The latter part of this really "feels" like it's a throttling decision, but the throttler appears to not be making that decision?  I don't know that I have a specific suggestion, just, it caught my eye as I read the code.



##########
server/src/main/java/org/apache/druid/server/coordinator/SegmentStateManager.java:
##########
@@ -0,0 +1,261 @@
+/*
+ * 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;
+
+import com.google.inject.Inject;
+import org.apache.druid.client.ServerInventoryView;
+import org.apache.druid.java.util.emitter.EmittingLogger;
+import org.apache.druid.metadata.SegmentsMetadataManager;
+import org.apache.druid.timeline.DataSegment;
+
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * Manages state of segments being loaded.
+ */
+public class SegmentStateManager
+{
+  private static final EmittingLogger log = new EmittingLogger(SegmentStateManager.class);
+
+  private final LoadQueueTaskMaster taskMaster;
+  private final ServerInventoryView serverInventoryView;
+  private final SegmentsMetadataManager segmentsMetadataManager;
+
+  private final ConcurrentHashMap<String, TierLoadingState> currentlyMovingSegments =
+      new ConcurrentHashMap<>();
+  private final ConcurrentHashMap<String, TierLoadingState> currentlyReplicatingSegments
+      = new ConcurrentHashMap<>();
+
+  @Inject
+  public SegmentStateManager(
+      ServerInventoryView serverInventoryView,
+      SegmentsMetadataManager segmentsMetadataManager,
+      LoadQueueTaskMaster taskMaster
+  )
+  {
+    this.serverInventoryView = serverInventoryView;
+    this.segmentsMetadataManager = segmentsMetadataManager;
+    this.taskMaster = taskMaster;
+  }
+
+  /**
+   * Queues load of a replica of the segment on the given server.
+   */
+  public boolean loadSegment(
+      DataSegment segment,
+      ServerHolder server,
+      boolean isPrimary,
+      ReplicationThrottler throttler
+  )
+  {
+    final String tier = server.getServer().getTier();
+    final LoadPeonCallback callback;
+    if (isPrimary) {
+      // Primary replicas are not subject to throttling
+      callback = null;
+    } else if (canLoadReplica(tier, throttler)) {
+      throttler.incrementAssignedReplicas(tier);
+
+      final TierLoadingState replicatingInTier = currentlyReplicatingSegments
+          .computeIfAbsent(tier, t -> new TierLoadingState(throttler.getMaxLifetime()));
+      replicatingInTier.addSegment(segment.getId(), server.getServer().getHost());
+      callback = success -> replicatingInTier.removeSegment(segment.getId());

Review Comment:
   Is this just internal bookkeeping?  Or is it actually asking for the segment to be removed?
   
   If it's internal bookkeeping, `.markCompleted()` might be a better name.



##########
server/src/main/java/org/apache/druid/server/coordinator/SegmentStateManager.java:
##########
@@ -0,0 +1,261 @@
+/*
+ * 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;
+
+import com.google.inject.Inject;
+import org.apache.druid.client.ServerInventoryView;
+import org.apache.druid.java.util.emitter.EmittingLogger;
+import org.apache.druid.metadata.SegmentsMetadataManager;
+import org.apache.druid.timeline.DataSegment;
+
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * Manages state of segments being loaded.
+ */
+public class SegmentStateManager
+{
+  private static final EmittingLogger log = new EmittingLogger(SegmentStateManager.class);
+
+  private final LoadQueueTaskMaster taskMaster;
+  private final ServerInventoryView serverInventoryView;
+  private final SegmentsMetadataManager segmentsMetadataManager;
+
+  private final ConcurrentHashMap<String, TierLoadingState> currentlyMovingSegments =
+      new ConcurrentHashMap<>();
+  private final ConcurrentHashMap<String, TierLoadingState> currentlyReplicatingSegments
+      = new ConcurrentHashMap<>();
+
+  @Inject
+  public SegmentStateManager(
+      ServerInventoryView serverInventoryView,
+      SegmentsMetadataManager segmentsMetadataManager,
+      LoadQueueTaskMaster taskMaster
+  )
+  {
+    this.serverInventoryView = serverInventoryView;
+    this.segmentsMetadataManager = segmentsMetadataManager;
+    this.taskMaster = taskMaster;
+  }
+
+  /**
+   * Queues load of a replica of the segment on the given server.
+   */
+  public boolean loadSegment(
+      DataSegment segment,
+      ServerHolder server,
+      boolean isPrimary,
+      ReplicationThrottler throttler
+  )
+  {
+    final String tier = server.getServer().getTier();
+    final LoadPeonCallback callback;
+    if (isPrimary) {
+      // Primary replicas are not subject to throttling
+      callback = null;
+    } else if (canLoadReplica(tier, throttler)) {
+      throttler.incrementAssignedReplicas(tier);
+
+      final TierLoadingState replicatingInTier = currentlyReplicatingSegments
+          .computeIfAbsent(tier, t -> new TierLoadingState(throttler.getMaxLifetime()));
+      replicatingInTier.addSegment(segment.getId(), server.getServer().getHost());
+      callback = success -> replicatingInTier.removeSegment(segment.getId());
+    } else {
+      throttler.incrementThrottledReplicas(tier);
+      return false;
+    }
+
+    try {
+      if (!server.startOperation(segment, SegmentState.LOADING)) {
+        return false;
+      }
+
+      server.getPeon().loadSegment(
+          segment,
+          isPrimary ? SegmentAction.LOAD_AS_PRIMARY : SegmentAction.LOAD_AS_REPLICA,
+          callback
+      );
+      return true;
+    }
+    catch (Exception e) {
+      server.cancelOperation(segment, SegmentState.LOADING);
+      return false;
+    }
+  }
+
+  public boolean dropSegment(DataSegment segment, ServerHolder server)
+  {
+    try {
+      if (!server.startOperation(segment, SegmentState.DROPPING)) {
+        return false;
+      }
+
+      server.getPeon().dropSegment(segment, null);
+      return true;
+    }
+    catch (Exception e) {
+      server.cancelOperation(segment, SegmentState.DROPPING);
+      return false;
+    }
+  }
+
+  public boolean moveSegment(
+      DataSegment segment,
+      ServerHolder fromServer,
+      ServerHolder toServer,
+      int maxLifetimeInBalancingQueue
+  )
+  {
+    final TierLoadingState segmentsMovingInTier = currentlyMovingSegments.computeIfAbsent(
+        toServer.getServer().getTier(),
+        t -> new TierLoadingState(maxLifetimeInBalancingQueue)
+    );
+    final LoadQueuePeon fromServerPeon = fromServer.getPeon();
+    final LoadPeonCallback moveFinishCallback = success -> {
+      fromServerPeon.unmarkSegmentToDrop(segment);
+      segmentsMovingInTier.removeSegment(segment.getId());
+    };
+
+    // mark segment to drop before it is actually loaded on server
+    // to be able to account for this information in BalancerStrategy immediately
+    toServer.startOperation(segment, SegmentState.MOVING_TO);
+    fromServerPeon.markSegmentToDrop(segment);
+    segmentsMovingInTier.addSegment(segment.getId(), fromServer.getServer().getHost());
+
+    final LoadQueuePeon toServerPeon = toServer.getPeon();
+    final String toServerName = toServer.getServer().getName();
+    try {
+      toServerPeon.loadSegment(
+          segment,
+          SegmentAction.MOVE_TO,
+          success -> {
+            // Drop segment only if:
+            // (1) segment load was successful on toServer
+            // AND (2) segment not already queued for drop on fromServer
+            // AND (3a) loading is http-based
+            //     OR (3b) inventory shows segment loaded on toServer
+
+            // Do not check the inventory with http loading as the HTTP
+            // response is enough to determine load success or failure
+            if (success

Review Comment:
   If it wasn't successful, do we need to update the state that was set from the call to `fromServerPeon.markSegmentToDrop`?



-- 
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: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] kfaraz commented on a diff in pull request #13197: Segment loading: Allow cancellation and prioritization of load queue items

Posted by GitBox <gi...@apache.org>.
kfaraz commented on code in PR #13197:
URL: https://github.com/apache/druid/pull/13197#discussion_r997808118


##########
server/src/main/java/org/apache/druid/server/coordinator/SegmentStateManager.java:
##########
@@ -0,0 +1,261 @@
+/*
+ * 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;
+
+import com.google.inject.Inject;
+import org.apache.druid.client.ServerInventoryView;
+import org.apache.druid.java.util.emitter.EmittingLogger;
+import org.apache.druid.metadata.SegmentsMetadataManager;
+import org.apache.druid.timeline.DataSegment;
+
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * Manages state of segments being loaded.
+ */
+public class SegmentStateManager
+{
+  private static final EmittingLogger log = new EmittingLogger(SegmentStateManager.class);
+
+  private final LoadQueueTaskMaster taskMaster;
+  private final ServerInventoryView serverInventoryView;
+  private final SegmentsMetadataManager segmentsMetadataManager;
+
+  private final ConcurrentHashMap<String, TierLoadingState> currentlyMovingSegments =
+      new ConcurrentHashMap<>();
+  private final ConcurrentHashMap<String, TierLoadingState> currentlyReplicatingSegments
+      = new ConcurrentHashMap<>();
+
+  @Inject
+  public SegmentStateManager(
+      ServerInventoryView serverInventoryView,
+      SegmentsMetadataManager segmentsMetadataManager,
+      LoadQueueTaskMaster taskMaster
+  )
+  {
+    this.serverInventoryView = serverInventoryView;
+    this.segmentsMetadataManager = segmentsMetadataManager;
+    this.taskMaster = taskMaster;
+  }
+
+  /**
+   * Queues load of a replica of the segment on the given server.
+   */
+  public boolean loadSegment(
+      DataSegment segment,
+      ServerHolder server,
+      boolean isPrimary,
+      ReplicationThrottler throttler
+  )
+  {
+    final String tier = server.getServer().getTier();
+    final LoadPeonCallback callback;
+    if (isPrimary) {
+      // Primary replicas are not subject to throttling
+      callback = null;
+    } else if (canLoadReplica(tier, throttler)) {
+      throttler.incrementAssignedReplicas(tier);
+
+      final TierLoadingState replicatingInTier = currentlyReplicatingSegments
+          .computeIfAbsent(tier, t -> new TierLoadingState(throttler.getMaxLifetime()));
+      replicatingInTier.addSegment(segment.getId(), server.getServer().getHost());
+      callback = success -> replicatingInTier.removeSegment(segment.getId());
+    } else {
+      throttler.incrementThrottledReplicas(tier);
+      return false;
+    }
+
+    try {
+      if (!server.startOperation(segment, SegmentState.LOADING)) {
+        return false;
+      }
+
+      server.getPeon().loadSegment(
+          segment,
+          isPrimary ? SegmentAction.LOAD_AS_PRIMARY : SegmentAction.LOAD_AS_REPLICA,

Review Comment:
   Okay, I will include those changes in this PR itself. I am going with the term LOAD vs PRIORITY_LOAD.



-- 
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: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] github-code-scanning[bot] commented on a diff in pull request #13197: Segment loading: Allow cancellation and prioritization of load queue items

Posted by "github-code-scanning[bot] (via GitHub)" <gi...@apache.org>.
github-code-scanning[bot] commented on code in PR #13197:
URL: https://github.com/apache/druid/pull/13197#discussion_r1182120694


##########
server/src/test/java/org/apache/druid/server/coordinator/rules/LoadRuleTest.java:
##########
@@ -117,397 +99,276 @@
   @Before
   public void setUp()
   {
-    EmittingLogger.registerEmitter(EMITTER);
-    EMITTER.start();
-    throttler = EasyMock.createMock(ReplicationThrottler.class);
-
     exec = MoreExecutors.listeningDecorator(Execs.multiThreaded(1, "LoadRuleTest-%d"));
     balancerStrategy = new CostBalancerStrategyFactory().createBalancerStrategy(exec);
     cachingCostBalancerStrategy = new CachingCostBalancerStrategy(ClusterCostCache.builder().build(), exec);
 
     mockBalancerStrategy = EasyMock.createMock(BalancerStrategy.class);
+    loadQueueManager = new SegmentLoadQueueManager(null, null, null);
   }
 
   @After
-  public void tearDown() throws Exception
+  public void tearDown()
   {
     exec.shutdown();
-    EMITTER.close();
   }
 
   @Test
   public void testLoad()
   {
-    EasyMock.expect(throttler.canCreateReplicant(EasyMock.anyString())).andReturn(true).anyTimes();
-
     final LoadQueuePeon mockPeon = createEmptyPeon();
-    mockPeon.loadSegment(EasyMock.anyObject(), EasyMock.anyObject());
+    mockPeon.loadSegment(EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyObject());
     EasyMock.expectLastCall().atLeastOnce();
 
-    LoadRule rule = createLoadRule(ImmutableMap.of(
-        "hot", 1,
-        DruidServer.DEFAULT_TIER, 2
-    ));
-
-    final DataSegment segment = createDataSegment("foo");
-
-    throttler.registerReplicantCreation(DruidServer.DEFAULT_TIER, segment.getId(), "hostNorm");
-    EasyMock.expectLastCall().once();
-
     if (!useRoundRobinAssignment) {
-      EasyMock.expect(mockBalancerStrategy.findNewSegmentHomeReplicator(EasyMock.anyObject(), EasyMock.anyObject()))
+      EasyMock.expect(mockBalancerStrategy.findServerToLoadSegment(EasyMock.anyObject(), EasyMock.anyObject()))
               .andDelegateTo(balancerStrategy)
-              .times(3);
+              .times(2);
     }
+    EasyMock.replay(mockPeon, mockBalancerStrategy);
 
-    EasyMock.replay(throttler, mockPeon, mockBalancerStrategy);
-
-    DruidCluster druidCluster = DruidClusterBuilder
-        .newBuilder()
-        .addTier(
-            "hot",
-            new ServerHolder(
-                new DruidServer("serverHot", "hostHot", null, 1000, ServerType.HISTORICAL, "hot", 1)
-                    .toImmutableDruidServer(),
-                mockPeon
-            )
-        )
-        .addTier(
-            DruidServer.DEFAULT_TIER,
-            new ServerHolder(
-                new DruidServer(
-                    "serverNorm",
-                    "hostNorm",
-                    null,
-                    1000,
-                    ServerType.HISTORICAL,
-                    DruidServer.DEFAULT_TIER,
-                    0
-                ).toImmutableDruidServer(),
-                mockPeon
-            )
-        )
+    DruidCluster druidCluster = DruidCluster
+        .builder()
+        .addTier(Tier.T1, createServerHolder(Tier.T1, mockPeon, false))
+        .addTier(Tier.T2, createServerHolder(Tier.T2, mockPeon, false))
         .build();
 
-    CoordinatorStats stats = rule.run(null, makeCoordinatorRuntimeParams(druidCluster, segment), segment);
+    final DataSegment segment = createDataSegment(DS_WIKI);
+    LoadRule rule = loadForever(ImmutableMap.of(Tier.T1, 1, Tier.T2, 2));
+    CoordinatorRunStats stats = runRuleAndGetStats(rule, segment, druidCluster);
+
+    Assert.assertEquals(1L, stats.getSegmentStat(Stats.Segments.ASSIGNED, Tier.T1, DS_WIKI));
+    Assert.assertEquals(1L, stats.getSegmentStat(Stats.Segments.ASSIGNED, Tier.T2, DS_WIKI));
+
+    EasyMock.verify(mockPeon, mockBalancerStrategy);
+  }
 
-    Assert.assertEquals(1L, stats.getTieredStat(LoadRule.ASSIGNED_COUNT, "hot"));
-    Assert.assertEquals(1L, stats.getTieredStat(LoadRule.ASSIGNED_COUNT, DruidServer.DEFAULT_TIER));
+  private CoordinatorRunStats runRuleAndGetStats(
+      LoadRule rule,
+      DataSegment segment,
+      DruidCluster cluster
+  )
+  {
+    return runRuleAndGetStats(rule, segment, makeCoordinatorRuntimeParams(cluster, segment));
+  }
 
-    EasyMock.verify(throttler, mockPeon, mockBalancerStrategy);
+  private CoordinatorRunStats runRuleAndGetStats(
+      LoadRule rule,
+      DataSegment segment,
+      DruidCoordinatorRuntimeParams params
+  )
+  {
+    final StrategicSegmentAssigner segmentAssigner = params.getSegmentAssigner();
+    rule.run(segment, segmentAssigner);
+    return segmentAssigner.getStats();
   }
 
   private DruidCoordinatorRuntimeParams makeCoordinatorRuntimeParams(
       DruidCluster druidCluster,
       DataSegment... usedSegments
   )
   {
-    return CoordinatorRuntimeParamsTestHelpers
-        .newBuilder()
-        .withDruidCluster(druidCluster)
-        .withSegmentReplicantLookup(SegmentReplicantLookup.make(druidCluster, false))
-        .withReplicationManager(throttler)
-        .withDynamicConfigs(
-            CoordinatorDynamicConfig
-                .builder()
-                .withUseRoundRobinSegmentAssignment(useRoundRobinAssignment)
-                .build()
-        )
-        .withRoundRobinServerSelector(useRoundRobinAssignment ? new RoundRobinServerSelector(druidCluster) : null)
-        .withBalancerStrategy(mockBalancerStrategy)
-        .withUsedSegmentsInTest(usedSegments)
-        .build();
+    return makeCoordinatorRuntimeParams(druidCluster, false, usedSegments);
   }
 
-  private DruidCoordinatorRuntimeParams makeCoordinatorRuntimeParamsWithLoadReplicationOnTimeout(
+  private DruidCoordinatorRuntimeParams makeCoordinatorRuntimeParams(
       DruidCluster druidCluster,
+      boolean replicateAfterLoadTimeout,

Review Comment:
   ## Useless parameter
   
   The parameter 'replicateAfterLoadTimeout' is never used.
   
   [Show more details](https://github.com/apache/druid/security/code-scanning/4912)



##########
server/src/main/java/org/apache/druid/server/coordinator/StrategicSegmentAssigner.java:
##########
@@ -0,0 +1,585 @@
+/*
+ * 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;
+
+import com.google.common.collect.Sets;
+import it.unimi.dsi.fastutil.objects.Object2IntOpenHashMap;
+import org.apache.druid.java.util.emitter.EmittingLogger;
+import org.apache.druid.server.coordinator.balancer.BalancerStrategy;
+import org.apache.druid.server.coordinator.loadqueue.SegmentAction;
+import org.apache.druid.server.coordinator.loadqueue.SegmentLoadQueueManager;
+import org.apache.druid.server.coordinator.rules.SegmentActionHandler;
+import org.apache.druid.server.coordinator.stats.CoordinatorRunStats;
+import org.apache.druid.server.coordinator.stats.CoordinatorStat;
+import org.apache.druid.server.coordinator.stats.RowKey;
+import org.apache.druid.server.coordinator.stats.Stats;
+import org.apache.druid.timeline.DataSegment;
+
+import java.util.Comparator;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+
+/**
+ * Used by the coordinator in each run for segment loading, dropping, balancing
+ * and broadcasting.
+ * <p>
+ * An instance of this class is freshly created for each coordinator run.
+ */
+public class StrategicSegmentAssigner implements SegmentActionHandler
+{
+  private static final EmittingLogger log = new EmittingLogger(StrategicSegmentAssigner.class);
+
+  private final SegmentLoadQueueManager loadQueueManager;
+  private final DruidCluster cluster;
+  private final CoordinatorRunStats stats;
+  private final SegmentReplicantLookup replicantLookup;
+  private final ReplicationThrottler replicationThrottler;
+  private final RoundRobinServerSelector serverSelector;
+  private final BalancerStrategy strategy;
+
+  private final boolean useRoundRobinAssignment;
+
+  private final Set<String> tiersWithNoServer = new HashSet<>();
+
+  public StrategicSegmentAssigner(
+      SegmentLoadQueueManager loadQueueManager,
+      DruidCluster cluster,
+      BalancerStrategy strategy,
+      CoordinatorDynamicConfig dynamicConfig,
+      CoordinatorRunStats stats
+  )
+  {
+    this.stats = stats;
+    this.cluster = cluster;
+    this.strategy = strategy;
+    this.loadQueueManager = loadQueueManager;
+    this.replicantLookup = SegmentReplicantLookup.make(cluster);
+    this.replicationThrottler = createReplicationThrottler(dynamicConfig);
+    this.useRoundRobinAssignment = dynamicConfig.isUseRoundRobinSegmentAssignment();
+    this.serverSelector = useRoundRobinAssignment ? new RoundRobinServerSelector(cluster) : null;
+  }
+
+  public CoordinatorRunStats getStats()
+  {
+    return stats;
+  }
+
+  public SegmentReplicantLookup getReplicantLookup()
+  {
+    return replicantLookup;
+  }
+
+  public void makeAlerts()
+  {
+    if (!tiersWithNoServer.isEmpty()) {
+      log.makeAlert("Tiers [%s] have no servers! Check your cluster configuration.", tiersWithNoServer).emit();
+    }
+  }
+
+  /**
+   * Moves the given segment from the source server to an eligible destination
+   * server.
+   * <p>
+   * An eligible destination server must:
+   * <ul>
+   *   <li>be present in the given list of destination servers</li>
+   *   <li>belong to the same tier as the source server</li>
+   *   <li>not already be serving or loading a replica of the segment</li>
+   *   <li>have enough space to load the segment</li>
+   * </ul>
+   * <p>
+   * The segment is not moved if:
+   * <ul>
+   *   <li>there is no eligible destination server, or</li>
+   *   <li>skipIfOptimallyPlaced is true and segment is already optimally placed, or</li>
+   *   <li>some other error occurs</li>
+   * </ul>
+   */
+  public boolean moveSegment(
+      DataSegment segment,
+      ServerHolder sourceServer,
+      List<ServerHolder> destinationServers
+  )
+  {
+    final String tier = sourceServer.getServer().getTier();
+    final List<ServerHolder> eligibleDestinationServers =
+        destinationServers.stream()
+                          .filter(s -> s.getServer().getTier().equals(tier))
+                          .filter(s -> s.canLoadSegment(segment))
+                          .collect(Collectors.toList());
+
+    if (eligibleDestinationServers.isEmpty()) {
+      incrementStat(Error.NO_ELIGIBLE_SERVER_FOR_MOVE, segment, tier);
+      return false;
+    }
+
+    // If the source server is not decommissioning, move can be skipped if the
+    // segment is already optimally placed
+    if (!sourceServer.isDecommissioning()) {
+      eligibleDestinationServers.add(sourceServer);
+    }
+
+    final ServerHolder destination =
+        strategy.findDestinationServerToMoveSegment(segment, sourceServer, eligibleDestinationServers);
+
+    if (destination == null || destination.getServer().equals(sourceServer.getServer())) {
+      incrementStat(Error.MOVE_SKIPPED_OPTIMALLY_PLACED, segment, tier);
+      return false;
+    } else if (moveSegment(segment, sourceServer, destination)) {
+      incrementStat(Stats.Segments.MOVED, segment, tier);
+      return true;
+    } else {
+      incrementStat(Error.MOVE_FAILED, segment, tier);
+      return false;
+    }
+  }
+
+  /**
+   * Moves the given segment from serverA to serverB.
+   */
+  private boolean moveSegment(DataSegment segment, ServerHolder serverA, ServerHolder serverB)
+  {
+    final String tier = serverA.getServer().getTier();
+    if (serverA.isLoadingSegment(segment)) {
+      // Cancel the load on serverA and load on serverB instead
+      if (serverA.cancelOperation(SegmentAction.LOAD, segment)) {
+        int loadedCountOnTier = replicantLookup.getServedReplicas(segment.getId(), tier);
+        return loadSegment(segment, serverB, loadedCountOnTier >= 1);
+      }
+
+      // Could not cancel load, let the segment load on serverA and count it as unmoved
+      return false;
+    } else if (serverA.isServingSegment(segment)) {
+      return loadQueueManager.moveSegment(segment, serverA, serverB);
+    } else {
+      return false;
+    }
+  }
+
+  @Override
+  public void updateSegmentReplicasInTiers(DataSegment segment, Map<String, Integer> tierToReplicaCount)
+  {
+    // Identify empty tiers and determine total required replicas
+    final AtomicInteger requiredTotalReplicas = new AtomicInteger(0);
+    final Set<String> allTiers = Sets.newHashSet(cluster.getTierNames());
+    tierToReplicaCount.forEach((tier, requiredReplicas) -> {
+      reportTierCapacityStats(segment, requiredReplicas, tier);
+      replicantLookup.setRequiredReplicas(segment.getId(), false, tier, requiredReplicas);
+      if (allTiers.contains(tier)) {
+        requiredTotalReplicas.addAndGet(requiredReplicas);
+      } else {
+        tiersWithNoServer.add(tier);
+      }
+    });
+
+    final int totalOverReplication =
+        replicantLookup.getTotalServedReplicas(segment.getId()) - requiredTotalReplicas.get();
+
+    // Update replicas in every tier
+    int totalDropsQueued = 0;
+    for (String tier : allTiers) {
+      totalDropsQueued += updateReplicasInTier(
+          segment,
+          tier,
+          tierToReplicaCount.getOrDefault(tier, 0),
+          totalOverReplication - totalDropsQueued
+      );
+    }
+  }
+
+  /**
+   * Queues load or drop operations on this tier based on the required
+   * number of replicas and the current state.
+   * <p>
+   * The {@code maxReplicasToDrop} helps to maintain the required level of
+   * replication in the cluster. This ensures that segment read concurrency does
+   * not suffer during a tier shift or load rule change.
+   * <p>
+   * Returns the number of new drop operations queued on this tier.
+   */
+  private int updateReplicasInTier(
+      DataSegment segment,
+      String tier,
+      int requiredReplicas,
+      int maxReplicasToDrop
+  )
+  {
+    final int projectedReplicas = replicantLookup.getProjectedReplicas(segment.getId(), tier);
+    final int movingReplicas = replicantLookup.getMovingReplicas(segment.getId(), tier);
+    final boolean shouldCancelMoves = requiredReplicas == 0 && movingReplicas > 0;
+
+    // Check if there is any action required on this tier
+    if (projectedReplicas == requiredReplicas && !shouldCancelMoves) {
+      return 0;
+    }
+
+    final SegmentTierStatus segmentStatus =
+        new SegmentTierStatus(segment, cluster.getHistoricalsByTier(tier));
+
+    // Cancel all moves in this tier if it does not need to have replicas
+    if (shouldCancelMoves) {
+      cancelOperations(SegmentAction.MOVE_TO, movingReplicas, segment, segmentStatus);
+      cancelOperations(SegmentAction.MOVE_FROM, movingReplicas, segment, segmentStatus);
+    }
+
+    // Cancel drops and queue loads if the projected count is below the requirement
+    if (projectedReplicas < requiredReplicas) {
+      int replicaDeficit = requiredReplicas - projectedReplicas;
+      int cancelledDrops =
+          cancelOperations(SegmentAction.DROP, replicaDeficit, segment, segmentStatus);
+
+      // Cancelled drops can be counted as loaded replicas, thus reducing deficit
+      int numReplicasToLoad = replicaDeficit - cancelledDrops;
+      if (numReplicasToLoad > 0) {
+        boolean isAlreadyLoadedOnTier = replicantLookup.getServedReplicas(segment.getId(), tier)
+                                        + cancelledDrops >= 1;
+        int numLoadsQueued = loadReplicas(numReplicasToLoad, segment, tier, segmentStatus, isAlreadyLoadedOnTier);
+        incrementStat(Stats.Segments.ASSIGNED, segment, tier, numLoadsQueued);
+      }
+    }
+
+    // Cancel loads and queue drops if the projected count exceeds the requirement
+    if (projectedReplicas > requiredReplicas) {
+      int replicaSurplus = projectedReplicas - requiredReplicas;
+      int cancelledLoads =
+          cancelOperations(SegmentAction.LOAD, replicaSurplus, segment, segmentStatus);
+
+      int numReplicasToDrop = Math.min(replicaSurplus - cancelledLoads, maxReplicasToDrop);
+      if (numReplicasToDrop > 0) {
+        int dropsQueuedOnTier = dropReplicas(numReplicasToDrop, segment, tier, segmentStatus);
+        incrementStat(Stats.Segments.DROPPED, segment, tier, dropsQueuedOnTier);
+        return dropsQueuedOnTier;
+      }
+    }
+
+    return 0;
+  }
+
+  private void reportTierCapacityStats(DataSegment segment, int requiredReplicas, String tier)
+  {
+    final RowKey rowKey = RowKey.forTier(tier);
+    stats.updateMax(Stats.Tier.REPLICATION_FACTOR, rowKey, requiredReplicas);
+    stats.add(Stats.Tier.REQUIRED_CAPACITY, rowKey, segment.getSize() * requiredReplicas);
+  }
+
+  @Override
+  public void broadcastSegment(DataSegment segment)
+  {
+    final Object2IntOpenHashMap<String> tierToRequiredReplicas = new Object2IntOpenHashMap<>();
+    for (ServerHolder server : cluster.getAllServers()) {
+      // Ignore servers which are not broadcast targets
+      if (!server.getServer().getType().isSegmentBroadcastTarget()) {
+        continue;
+      }
+
+      // Drop from decommissioning servers and load on active servers
+      final String tier = server.getServer().getTier();
+      if (server.isDecommissioning()) {
+        boolean dropQueued = dropBroadcastSegment(segment, server);
+        incrementStat(Stats.Segments.DROPPED_BROADCAST, segment, tier, dropQueued ? 1 : 0);
+      } else if (loadBroadcastSegment(segment, server)) {
+        tierToRequiredReplicas.addTo(tier, 1);
+        incrementStat(Stats.Segments.ASSIGNED_BROADCAST, segment, tier);
+      } else {
+        tierToRequiredReplicas.addTo(tier, 1);
+      }
+    }
+
+    // Update required replica counts
+    tierToRequiredReplicas.object2IntEntrySet().fastForEach(
+        entry -> replicantLookup
+            .setRequiredReplicas(segment.getId(), true, entry.getKey(), entry.getIntValue())
+    );
+  }
+
+  @Override
+  public void deleteSegment(DataSegment segment)
+  {
+    loadQueueManager.deleteSegment(segment);
+    stats.addToDatasourceStat(Stats.Segments.DELETED, segment.getDataSource(), 1);
+  }
+
+  /**
+   * Loads the broadcast segment if it is not loaded on the given server.
+   * Returns true only if the segment was successfully queued for load on the server.
+   */
+  private boolean loadBroadcastSegment(DataSegment segment, ServerHolder server)
+  {
+    if (server.isServingSegment(segment) || server.isLoadingSegment(segment)) {
+      return false;
+    } else if (server.isDroppingSegment(segment)) {
+      return server.cancelOperation(SegmentAction.DROP, segment);
+    }
+
+    if (server.canLoadSegment(segment) && loadSegment(segment, server, false)) {
+      return true;
+    } else {
+      log.makeAlert("Could not assign broadcast segment for datasource [%s]", segment.getDataSource())
+         .addData("segmentId", segment.getId())
+         .addData("segmentSize", segment.getSize())
+         .addData("hostName", server.getServer().getHost())
+         .addData("availableSize", server.getAvailableSize())
+         .emit();
+      return false;
+    }
+  }
+
+  /**
+   * Drops the broadcast segment if it is loaded on the given server.
+   * Returns true only if the segment was successfully queued for drop on the server.
+   */
+  private boolean dropBroadcastSegment(DataSegment segment, ServerHolder server)
+  {
+    if (server.isLoadingSegment(segment)) {
+      return server.cancelOperation(SegmentAction.LOAD, segment);
+    } else if (server.isServingSegment(segment)) {
+      return loadQueueManager.dropSegment(segment, server);
+    } else {
+      return false;
+    }
+  }
+
+  /**
+   * Queues drop of {@code numToDrop} replicas of the segment from a tier.
+   * Tries to drop replicas first from decommissioning servers and then from
+   * active servers.
+   * <p>
+   * Returns the number of successfully queued drop operations.
+   */
+  private int dropReplicas(
+      final int numToDrop,
+      DataSegment segment,
+      String tier,
+      SegmentTierStatus segmentStatus
+  )
+  {
+    if (numToDrop <= 0) {
+      return 0;
+    }
+
+    final List<ServerHolder> eligibleServers = segmentStatus.getServersEligibleToDrop();
+    if (eligibleServers.isEmpty()) {
+      incrementStat(Error.NO_ELIGIBLE_SERVER_FOR_DROP, segment, tier);
+      return 0;
+    }
+
+    // Keep eligible servers sorted by most full first
+    final TreeSet<ServerHolder> eligibleLiveServers = new TreeSet<>(Comparator.reverseOrder());
+    final TreeSet<ServerHolder> eligibleDyingServers = new TreeSet<>(Comparator.reverseOrder());
+    for (ServerHolder server : eligibleServers) {
+      if (server.isDecommissioning()) {
+        eligibleDyingServers.add(server);
+      } else {
+        eligibleLiveServers.add(server);
+      }
+    }
+
+    // Drop as many replicas as possible from decommissioning servers
+    int remainingNumToDrop = numToDrop;
+    int numDropsQueued =
+        dropReplicasFromServers(remainingNumToDrop, segment, eligibleDyingServers.iterator(), tier);
+
+    // Drop replicas from active servers if required
+    if (numToDrop > numDropsQueued) {
+      remainingNumToDrop = numToDrop - numDropsQueued;
+      Iterator<ServerHolder> serverIterator =
+          (useRoundRobinAssignment || eligibleLiveServers.size() >= remainingNumToDrop)
+          ? eligibleLiveServers.iterator()
+          : strategy.pickServersToDrop(segment, eligibleLiveServers);
+      numDropsQueued += dropReplicasFromServers(remainingNumToDrop, segment, serverIterator, tier);
+    }
+
+    return numDropsQueued;
+  }
+
+  /**
+   * Queues drop of {@code numToDrop} replicas of the segment from the servers.
+   * Returns the number of successfully queued drop operations.
+   */
+  private int dropReplicasFromServers(
+      int numToDrop,
+      DataSegment segment,
+      Iterator<ServerHolder> serverIterator,
+      String tier
+  )
+  {
+    int numDropsQueued = 0;
+    while (numToDrop > numDropsQueued && serverIterator.hasNext()) {
+      ServerHolder holder = serverIterator.next();
+      boolean dropped = loadQueueManager.dropSegment(segment, holder);
+
+      if (dropped) {
+        ++numDropsQueued;
+      } else {
+        incrementStat(Error.DROP_FAILED, segment, tier);
+      }
+    }
+
+    return numDropsQueued;
+  }
+
+  /**
+   * Queues load of {@code numToLoad} replicas of the segment on a tier.
+   */
+  private int loadReplicas(
+      int numToLoad,
+      DataSegment segment,
+      String tier,
+      SegmentTierStatus segmentStatus,
+      boolean isAlreadyLoadedOnTier
+  )
+  {
+    // Do not assign replicas if tier is already busy loading some
+    if (isAlreadyLoadedOnTier && replicationThrottler.isTierLoadingReplicas(tier)) {
+      return 0;
+    }
+
+    final List<ServerHolder> eligibleServers = segmentStatus.getServersEligibleToLoad();
+    if (eligibleServers.isEmpty()) {
+      incrementStat(Error.NO_ELIGIBLE_SERVER_FOR_LOAD, segment, tier);
+      return 0;
+    }
+
+    final Iterator<ServerHolder> serverIterator =
+        useRoundRobinAssignment
+        ? serverSelector.getServersInTierToLoadSegment(tier, segment)
+        : strategy.findServerToLoadSegment(segment, eligibleServers);
+    if (!serverIterator.hasNext()) {
+      incrementStat(Error.NO_STRATEGIC_SERVER_FOR_LOAD, segment, tier);
+      return 0;
+    }
+
+    // Load the replicas on this tier
+    int numLoadsQueued = 0;
+    while (numLoadsQueued < numToLoad && serverIterator.hasNext()) {
+      numLoadsQueued += loadSegment(segment, serverIterator.next(), isAlreadyLoadedOnTier)
+                        ? 1 : 0;
+    }
+
+    return numLoadsQueued;
+  }
+
+  private boolean loadSegment(DataSegment segment, ServerHolder server, boolean isAlreadyLoadedOnTier)
+  {
+    final String tier = server.getServer().getTier();
+    if (isAlreadyLoadedOnTier && !replicationThrottler.canAssignReplica(tier)) {
+      incrementStat(Error.REPLICA_THROTTLED, segment, tier);
+      return false;
+    }
+
+    final SegmentAction action = isAlreadyLoadedOnTier ? SegmentAction.REPLICATE : SegmentAction.LOAD;
+    final boolean assigned = loadQueueManager.loadSegment(segment, server, action);
+
+    if (!assigned) {
+      incrementStat(Error.LOAD_FAILED, segment, tier);
+    } else if (isAlreadyLoadedOnTier) {
+      replicationThrottler.incrementAssignedReplicas(tier);
+    }
+
+    return assigned;
+  }
+
+  private ReplicationThrottler createReplicationThrottler(CoordinatorDynamicConfig dynamicConfig)
+  {
+    final Set<String> tiersLoadingReplicas = new HashSet<>();
+
+    cluster.getHistoricals().forEach(
+        (tier, historicals) -> {
+          int numLoadingReplicas = historicals.stream().mapToInt(ServerHolder::getNumLoadingReplicas).sum();
+          if (numLoadingReplicas > 0) {
+            log.info(
+                "Tier [%s] will not be assigned replicas as it is already loading [%d] replicas.",
+                tier, numLoadingReplicas
+            );
+            tiersLoadingReplicas.add(tier);
+          }
+        }
+    );
+    return new ReplicationThrottler(
+        tiersLoadingReplicas,
+        dynamicConfig.getReplicationThrottleLimit(),
+        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/4909)



##########
server/src/main/java/org/apache/druid/server/coordinator/balancer/TierSegmentBalancer.java:
##########
@@ -0,0 +1,252 @@
+/*
+ * 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.balancer;
+
+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.CoordinatorDynamicConfig;
+import org.apache.druid.server.coordinator.DruidCoordinatorRuntimeParams;
+import org.apache.druid.server.coordinator.ServerHolder;
+import org.apache.druid.server.coordinator.StrategicSegmentAssigner;
+import org.apache.druid.server.coordinator.stats.CoordinatorRunStats;
+import org.apache.druid.server.coordinator.stats.CoordinatorStat;
+import org.apache.druid.timeline.DataSegment;
+
+import javax.annotation.Nullable;
+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 StrategicSegmentAssigner segmentAssigner;
+
+  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;
+
+  private final int movingSegmentCount;
+
+  public TierSegmentBalancer(
+      String tier,
+      Set<ServerHolder> servers,
+      DruidCoordinatorRuntimeParams params
+  )
+  {
+    this.tier = tier;
+    this.params = params;
+    this.segmentAssigner = params.getSegmentAssigner();
+
+    this.strategy = params.getBalancerStrategy();
+    this.dynamicConfig = params.getCoordinatorDynamicConfig();
+    this.totalMaxSegmentsToMove = dynamicConfig.getMaxSegmentsToMove();
+    this.runStats = segmentAssigner.getStats();
+
+    Map<Boolean, List<ServerHolder>> partitions =
+        servers.stream().collect(Collectors.partitioningBy(ServerHolder::isDecommissioning));
+    this.decommissioningServers = partitions.get(true);
+    this.activeServers = partitions.get(false);
+    this.allServers = servers;
+
+    this.movingSegmentCount = activeServers.stream().mapToInt(ServerHolder::getNumMovingSegments).sum();
+  }
+
+  public 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(
+        "Moving max [%d] segments in tier [%s] with [%d] active servers and"
+        + " [%d] decommissioning servers. There are [%d] segments already in queue.",
+        totalMaxSegmentsToMove, tier, activeServers.size(), decommissioningServers.size(), movingSegmentCount
+    );
+
+    // Move segments from decommissioning to active servers
+    int movedDecommSegments = 0;
+    if (!decommissioningServers.isEmpty()) {
+      int maxDecommPercentToMove = dynamicConfig.getDecommissioningMaxPercentOfMaxSegmentsToMove();
+      int maxDecommSegmentsToMove = (int) Math.ceil(totalMaxSegmentsToMove * (maxDecommPercentToMove / 100.0));
+      movedDecommSegments +=
+          moveSegmentsFromTo(decommissioningServers, activeServers, maxDecommSegmentsToMove);
+      log.info(
+          "Moved [%d] segments out of max [%d (%d%%)] from decommissioning to active servers in tier [%s].",
+          movedDecommSegments, maxDecommSegmentsToMove, maxDecommPercentToMove, tier
+      );
+    }
+
+    // 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 in tier [%s].",
+        movedGeneralSegments, maxGeneralSegmentsToMove, tier
+    );
+
+    if (dynamicConfig.emitBalancingStats()) {
+      strategy.emitStats(tier, runStats, Lists.newArrayList(allServers));
+    }
+  }
+
+  private int moveSegmentsFromTo(
+      List<ServerHolder> sourceServers,
+      List<ServerHolder> destServers,
+      int maxSegmentsToMove
+  )
+  {
+    if (maxSegmentsToMove <= 0 || sourceServers.isEmpty() || destServers.isEmpty()) {
+      return 0;
+    }
+
+    // Always move loading segments first as it is a cheaper operation
+    Iterator<BalancerSegmentHolder> pickedSegments
+        = pickSegmentsFrom(sourceServers, maxSegmentsToMove, true);
+    int movedCount = moveSegmentsTo(destServers, pickedSegments, maxSegmentsToMove);
+
+    // Move loaded segments only if tier is not already busy moving segments
+    if (movingSegmentCount <= 0) {
+      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) {
+      // non-batched sampler cannot pick loading segments
+      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/4910)



##########
server/src/main/java/org/apache/druid/server/coordinator/balancer/TierSegmentBalancer.java:
##########
@@ -0,0 +1,252 @@
+/*
+ * 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.balancer;
+
+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.CoordinatorDynamicConfig;
+import org.apache.druid.server.coordinator.DruidCoordinatorRuntimeParams;
+import org.apache.druid.server.coordinator.ServerHolder;
+import org.apache.druid.server.coordinator.StrategicSegmentAssigner;
+import org.apache.druid.server.coordinator.stats.CoordinatorRunStats;
+import org.apache.druid.server.coordinator.stats.CoordinatorStat;
+import org.apache.druid.timeline.DataSegment;
+
+import javax.annotation.Nullable;
+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 StrategicSegmentAssigner segmentAssigner;
+
+  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;
+
+  private final int movingSegmentCount;
+
+  public TierSegmentBalancer(
+      String tier,
+      Set<ServerHolder> servers,
+      DruidCoordinatorRuntimeParams params
+  )
+  {
+    this.tier = tier;
+    this.params = params;
+    this.segmentAssigner = params.getSegmentAssigner();
+
+    this.strategy = params.getBalancerStrategy();
+    this.dynamicConfig = params.getCoordinatorDynamicConfig();
+    this.totalMaxSegmentsToMove = dynamicConfig.getMaxSegmentsToMove();
+    this.runStats = segmentAssigner.getStats();
+
+    Map<Boolean, List<ServerHolder>> partitions =
+        servers.stream().collect(Collectors.partitioningBy(ServerHolder::isDecommissioning));
+    this.decommissioningServers = partitions.get(true);
+    this.activeServers = partitions.get(false);
+    this.allServers = servers;
+
+    this.movingSegmentCount = activeServers.stream().mapToInt(ServerHolder::getNumMovingSegments).sum();
+  }
+
+  public 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(
+        "Moving max [%d] segments in tier [%s] with [%d] active servers and"
+        + " [%d] decommissioning servers. There are [%d] segments already in queue.",
+        totalMaxSegmentsToMove, tier, activeServers.size(), decommissioningServers.size(), movingSegmentCount
+    );
+
+    // Move segments from decommissioning to active servers
+    int movedDecommSegments = 0;
+    if (!decommissioningServers.isEmpty()) {
+      int maxDecommPercentToMove = dynamicConfig.getDecommissioningMaxPercentOfMaxSegmentsToMove();
+      int maxDecommSegmentsToMove = (int) Math.ceil(totalMaxSegmentsToMove * (maxDecommPercentToMove / 100.0));
+      movedDecommSegments +=
+          moveSegmentsFromTo(decommissioningServers, activeServers, maxDecommSegmentsToMove);
+      log.info(
+          "Moved [%d] segments out of max [%d (%d%%)] from decommissioning to active servers in tier [%s].",
+          movedDecommSegments, maxDecommSegmentsToMove, maxDecommPercentToMove, tier
+      );
+    }
+
+    // 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 in tier [%s].",
+        movedGeneralSegments, maxGeneralSegmentsToMove, tier
+    );
+
+    if (dynamicConfig.emitBalancingStats()) {
+      strategy.emitStats(tier, runStats, Lists.newArrayList(allServers));
+    }
+  }
+
+  private int moveSegmentsFromTo(
+      List<ServerHolder> sourceServers,
+      List<ServerHolder> destServers,
+      int maxSegmentsToMove
+  )
+  {
+    if (maxSegmentsToMove <= 0 || sourceServers.isEmpty() || destServers.isEmpty()) {
+      return 0;
+    }
+
+    // Always move loading segments first as it is a cheaper operation
+    Iterator<BalancerSegmentHolder> pickedSegments
+        = pickSegmentsFrom(sourceServers, maxSegmentsToMove, true);
+    int movedCount = moveSegmentsTo(destServers, pickedSegments, maxSegmentsToMove);
+
+    // Move loaded segments only if tier is not already busy moving segments
+    if (movingSegmentCount <= 0) {
+      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) {
+      // non-batched sampler cannot pick loading segments
+      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/4911)



-- 
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: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] kfaraz commented on a diff in pull request #13197: Segment loading: Allow cancellation and prioritization of load queue items

Posted by GitBox <gi...@apache.org>.
kfaraz commented on code in PR #13197:
URL: https://github.com/apache/druid/pull/13197#discussion_r993734456


##########
server/src/main/java/org/apache/druid/server/coordinator/TierLoadingState.java:
##########
@@ -0,0 +1,72 @@
+/*
+ * 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;
+
+import org.apache.druid.timeline.SegmentId;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * Contains segments being loaded or replicated in a tier.
+ */
+public class TierLoadingState
+{
+  private int lifetime;
+  private final ConcurrentHashMap<SegmentId, String> processingSegments = new ConcurrentHashMap<>();

Review Comment:
   However, this PR does modify the definition of ~`replicationThrottleLimit` and~ `maxSegmentsInNodeLoadingQueue` slightly (docs to be updated). The limits now apply on number of assignments done during a single coordinator run rather than items in the queue at a given instant.
   This change was done to:
   - make the configs easier to use
   - ensure that the configs actually take effect. In some cases (where loading happens quickly enough), these limits are constantly violated.~
   
   Edit: the only modification done to the behaviour of `replicationThrottleLimit` is that now it doesn't apply to the first replica on any tier.



-- 
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: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] AmatyaAvadhanula commented on a diff in pull request #13197: Segment loading: Allow cancellation and prioritization of load queue items

Posted by GitBox <gi...@apache.org>.
AmatyaAvadhanula commented on code in PR #13197:
URL: https://github.com/apache/druid/pull/13197#discussion_r994866626


##########
server/src/main/java/org/apache/druid/server/coordinator/TierLoadingState.java:
##########
@@ -0,0 +1,72 @@
+/*
+ * 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;
+
+import org.apache.druid.timeline.SegmentId;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * Contains segments being loaded or replicated in a tier.
+ */
+public class TierLoadingState
+{
+  private int lifetime;

Review Comment:
   Apologies, the situation I'm thinking of can occur with the existing code as well.
   I was thinking of timedOut in LoadQueue while making thing comment.



-- 
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: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] github-code-scanning[bot] commented on a diff in pull request #13197: Segment loading: Allow cancellation and prioritization of load queue items

Posted by "github-code-scanning[bot] (via GitHub)" <gi...@apache.org>.
github-code-scanning[bot] commented on code in PR #13197:
URL: https://github.com/apache/druid/pull/13197#discussion_r1227819721


##########
server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java:
##########
@@ -985,26 +799,101 @@
       List<ImmutableDruidServer> currentServers = prepareCurrentServers();
 
       startPeonsForNewServers(currentServers);
+      stopPeonsForDisappearedServers(currentServers);
 
-      cluster = prepareCluster(params, currentServers);
-      segmentReplicantLookup = SegmentReplicantLookup.make(cluster, getDynamicConfigs().getReplicateAfterLoadTimeout());
+      final DruidCluster cluster = prepareCluster(params.getCoordinatorDynamicConfig(), currentServers);
+      cancelLoadsOnDecommissioningServers(cluster);
 
-      stopPeonsForDisappearedServers(currentServers);
+      final CoordinatorDynamicConfig dynamicConfig = params.getCoordinatorDynamicConfig();
 
-      final RoundRobinServerSelector roundRobinServerSelector;
-      if (params.getCoordinatorDynamicConfig().isUseRoundRobinSegmentAssignment()) {
-        roundRobinServerSelector = new RoundRobinServerSelector(cluster);
-        log.info("Using round-robin segment assignment.");
-      } else {
-        roundRobinServerSelector = null;
+      initBalancerExecutor();
+      final BalancerStrategy balancerStrategy = balancerStrategyFactory.createBalancerStrategy(balancerExec);
+      log.info(
+          "Using balancer strategy [%s] with round-robin assignment [%s] and debug dimensions [%s].",
+          balancerStrategy.getClass().getSimpleName(),
+          dynamicConfig.isUseRoundRobinSegmentAssignment(), dynamicConfig.getDebugDimensions()
+      );
+
+      params = params.buildFromExisting()
+                     .withDruidCluster(cluster)
+                     .withDynamicConfigs(recomputeDynamicConfig(params))
+                     .withBalancerStrategy(balancerStrategy)
+                     .withSegmentAssignerUsing(loadQueueManager)
+                     .build();
+
+      segmentReplicantLookup = params.getSegmentReplicantLookup();
+
+      return params;
+    }
+
+    /**
+     * Recomputes dynamic config values if {@code smartLoadQueue} is enabled.
+     */
+    private CoordinatorDynamicConfig recomputeDynamicConfig(DruidCoordinatorRuntimeParams params)
+    {
+      final CoordinatorDynamicConfig dynamicConfig = params.getCoordinatorDynamicConfig();
+      if (!dynamicConfig.isSmartSegmentLoading()) {
+        return dynamicConfig;
       }
 
-      return params.buildFromExisting()
-                   .withDruidCluster(cluster)
-                   .withLoadManagementPeons(loadManagementPeons)
-                   .withSegmentReplicantLookup(segmentReplicantLookup)
-                   .withRoundRobinServerSelector(roundRobinServerSelector)
-                   .build();
+      // Impose a lower bound on both replicationThrottleLimit and maxSegmentsToMove
+      final int throttlePercentage = 2;
+      final int replicationThrottleLimit = Math.max(
+          100,
+          params.getUsedSegments().size() * throttlePercentage / 100
+      );
+
+      // Impose an upper bound on maxSegmentsToMove to ensure that coordinator
+      // run times are bounded. This limit can be relaxed as performance of
+      // the CostBalancerStrategy.computeCost() is improved.
+      final int maxSegmentsToMove = Math.min(1000, replicationThrottleLimit);
+
+      log.info(
+          "Smart segment loading is enabled. Recomputed replicationThrottleLimit"
+          + " [%d] (%d%% of used segments) and maxSegmentsToMove [%d].",
+          replicationThrottleLimit, throttlePercentage, maxSegmentsToMove
+      );
+
+      return CoordinatorDynamicConfig.builder()
+                                     .withMaxSegmentsInNodeLoadingQueue(0)
+                                     .withReplicationThrottleLimit(replicationThrottleLimit)
+                                     .withMaxSegmentsToMove(maxSegmentsToMove)
+                                     .withUseRoundRobinSegmentAssignment(true)
+                                     .withUseBatchedSegmentSampler(true)

Review Comment:
   ## Deprecated method or constructor invocation
   
   Invoking [Builder.withUseBatchedSegmentSampler](1) should be avoided because it has been deprecated.
   
   [Show more details](https://github.com/apache/druid/security/code-scanning/5052)



-- 
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: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] imply-cheddar commented on a diff in pull request #13197: Segment loading: Allow cancellation and prioritization of load queue items

Posted by "imply-cheddar (via GitHub)" <gi...@apache.org>.
imply-cheddar commented on code in PR #13197:
URL: https://github.com/apache/druid/pull/13197#discussion_r1233461731


##########
server/src/main/java/org/apache/druid/server/coordinator/ServerHolder.java:
##########
@@ -337,29 +347,42 @@ private SegmentAction simplify(SegmentAction action)
     return action == SegmentAction.REPLICATE ? SegmentAction.LOAD : action;
   }
 
-  private boolean updateQueuedSegments(DataSegment segment, SegmentAction action, boolean addToQueue)
+  private void addToQueuedSegments(DataSegment segment, SegmentAction action)
   {
-    if (addToQueue) {
-      queuedSegments.put(segment, action);
+    queuedSegments.put(segment, action);
+
+    // Add to projected if load is started, remove from projected if drop has started
+    if (action.isLoad()) {
+      projectedSegments.add(segment);
     } else {
-      queuedSegments.remove(segment);
+      projectedSegments.remove(segment);
     }
 
-    final long sizeDelta = addToQueue ? segment.getSize() : -segment.getSize();
     if (action.isLoad()) {
-      sizeOfLoadingSegments += sizeDelta;
+      sizeOfLoadingSegments += segment.getSize();
     } else if (action == SegmentAction.DROP) {
-      sizeOfDroppingSegments += sizeDelta;
+      sizeOfDroppingSegments += segment.getSize();
+    } else {
+      // MOVE_FROM actions graduate to DROP after the corresponding MOVE_TO has finished
+      // Do not consider size delta until then, otherwise we might over-assign the server
     }
+  }
+
+  private void removeFromQueuedSegments(DataSegment segment, SegmentAction action)
+  {
+    queuedSegments.remove(segment);
 
-    // Remove from projected if load is cancelled or drop is started, add otherwise
-    if (addToQueue ^ action.isLoad()) {
+    if (action.isLoad()) {
       projectedSegments.remove(segment);
     } else {
       projectedSegments.add(segment);
     }
 
-    return true;
+    if (action.isLoad()) {
+      sizeOfLoadingSegments -= segment.getSize();
+    } else if (action == SegmentAction.DROP) {
+      sizeOfDroppingSegments -= segment.getSize();
+    }
   }

Review Comment:
   Same comments here, basically have the same if statement twice.



##########
server/src/main/java/org/apache/druid/server/coordinator/ServerHolder.java:
##########
@@ -337,29 +347,42 @@ private SegmentAction simplify(SegmentAction action)
     return action == SegmentAction.REPLICATE ? SegmentAction.LOAD : action;
   }
 
-  private boolean updateQueuedSegments(DataSegment segment, SegmentAction action, boolean addToQueue)
+  private void addToQueuedSegments(DataSegment segment, SegmentAction action)
   {
-    if (addToQueue) {
-      queuedSegments.put(segment, action);
+    queuedSegments.put(segment, action);
+
+    // Add to projected if load is started, remove from projected if drop has started
+    if (action.isLoad()) {
+      projectedSegments.add(segment);
     } else {
-      queuedSegments.remove(segment);
+      projectedSegments.remove(segment);
     }
 
-    final long sizeDelta = addToQueue ? segment.getSize() : -segment.getSize();
     if (action.isLoad()) {
-      sizeOfLoadingSegments += sizeDelta;
+      sizeOfLoadingSegments += segment.getSize();
     } else if (action == SegmentAction.DROP) {
-      sizeOfDroppingSegments += sizeDelta;
+      sizeOfDroppingSegments += segment.getSize();
+    } else {
+      // MOVE_FROM actions graduate to DROP after the corresponding MOVE_TO has finished
+      // Do not consider size delta until then, otherwise we might over-assign the server
     }
+  }

Review Comment:
   After the change to remove teh `addToQueue` boolean, you now have the exact same if clause at the beginning of 2 subsequence if statements.  You should be able to make it
   
   ```
   if (action.isLoad()) {
     projectedSegments.add(segment);
     sizeOfLoadingSegments += segment.getSize();
   } else {
     projectedSegments.remove(segment);
     // The current else block could be from a DROP or a MOVE_FROM.  The MOVE_FROM will eventually graduate to
     // DROP after ...
     if (action == SegmentAction.DROP) {
       sizeOfDroppingSegments += segment.getSize()
     }
   }
   ```



##########
server/src/main/java/org/apache/druid/server/coordinator/balancer/ReservoirSegmentSampler.java:
##########
@@ -21,28 +21,86 @@
 
 import org.apache.druid.java.util.emitter.EmittingLogger;
 import org.apache.druid.server.coordinator.ServerHolder;
-import org.apache.druid.server.coordinator.loadqueue.SegmentAction;
+import org.apache.druid.server.coordinator.loading.SegmentAction;
 import org.apache.druid.timeline.DataSegment;
 
 import java.util.ArrayList;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.List;
 import java.util.Set;
 import java.util.concurrent.ThreadLocalRandom;
+import java.util.function.Function;
 
-final class ReservoirSegmentSampler
+public final class ReservoirSegmentSampler
 {
 
   private static final EmittingLogger log = new EmittingLogger(ReservoirSegmentSampler.class);
 
-  static List<BalancerSegmentHolder> getRandomBalancerSegmentHolders(
+  /**
+   * Picks segments currently loading on the given set of servers that can be
+   * moved to other servers for balancing.
+   *
+   * @param serverHolders        Set of historicals to consider for picking segments
+   * @param maxSegmentsToPick    Maximum number of segments to pick
+   * @param broadcastDatasources Segments belonging to these datasources will not
+   *                             be picked for balancing, since they should be
+   *                             loaded on all servers anyway.
+   * @return Iterator over {@link BalancerSegmentHolder}s, each of which contains
+   * a segment picked for moving and the server currently loading it.
+   */
+  public static List<BalancerSegmentHolder> pickMovableLoadingSegmentsFrom(

Review Comment:
   nit: the different in name is soooo minute it took me a long time to realize.  One is `ing` and the other is `ed`.
   
   Is there a reason not to expose the 4-argument method as `public` and have the call sites pass in one of the two lambdas?  It looks like there's only a single call-site for each of them except for tests.
   
   That or maybe try to make the names a bit more different from each other.



##########
server/src/main/java/org/apache/druid/server/coordinator/loading/SegmentLoadingConfig.java:
##########
@@ -0,0 +1,124 @@
+/*
+ * 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.loading;
+
+import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.server.coordinator.CoordinatorDynamicConfig;
+
+/**
+ * Contains recomputed configs from {@link CoordinatorDynamicConfig} based on
+ * whether {@code smartSegmentLoading} is enabled or not.
+ */
+public class SegmentLoadingConfig
+{
+  private static final Logger log = new Logger(SegmentLoadingConfig.class);
+
+  private final int maxSegmentsInLoadQueue;
+  private final int replicationThrottleLimit;
+  private final int maxReplicaAssignmentsInRun;
+  private final int maxLifetimeInLoadQueue;
+
+  private final int maxSegmentsToMove;
+  private final int percentDecommSegmentsToMove;
+
+  private final boolean useRoundRobinSegmentAssignment;
+  private final boolean emitBalancingStats;
+
+  public SegmentLoadingConfig(CoordinatorDynamicConfig dynamicConfig, int numUsedSegments)

Review Comment:
   design nit: a constructor should generally define the "dependency relationship" of a class.  I.e. the things passed in on the constructor are the things that the current class is dependent upon.  This constructor is doing a bunch of work, that work is dependent on the `CoordinatorDynamicConfig` object, but `SegmentLoadingConfig` is not dependent on the object.
   
   In this case, it would be preferable for this to be a static method and the constructor just take all of the various values.



-- 
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: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] github-code-scanning[bot] commented on a diff in pull request #13197: Segment loading: Allow cancellation and prioritization of load queue items

Posted by "github-code-scanning[bot] (via GitHub)" <gi...@apache.org>.
github-code-scanning[bot] commented on code in PR #13197:
URL: https://github.com/apache/druid/pull/13197#discussion_r1217664291


##########
server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java:
##########
@@ -345,135 +322,184 @@
         .collect(Collectors.toList());
   }
 
-  private CoordinatorStats doRun(
+  private int getCompactionTaskCapacity(CoordinatorCompactionConfig dynamicConfig)
+  {
+    int totalWorkerCapacity;
+    try {
+      totalWorkerCapacity = dynamicConfig.isUseAutoScaleSlots()
+                            ? indexingServiceClient.getTotalWorkerCapacityWithAutoScale()
+                            : indexingServiceClient.getTotalWorkerCapacity();
+    }
+    catch (Exception e) {
+      LOG.warn("Failed to get total worker capacity with auto scale slots. Falling back to current capacity count");
+      totalWorkerCapacity = indexingServiceClient.getTotalWorkerCapacity();
+    }
+
+    return Math.min(
+        (int) (totalWorkerCapacity * dynamicConfig.getCompactionTaskSlotRatio()),
+        dynamicConfig.getMaxCompactionTaskSlots()
+    );
+  }
+
+  private int getAvailableCompactionTaskSlots(int compactionTaskCapacity, int busyCompactionTaskSlots)
+  {
+    final int availableCompactionTaskSlots;
+    if (busyCompactionTaskSlots > 0) {
+      availableCompactionTaskSlots = Math.max(0, compactionTaskCapacity - busyCompactionTaskSlots);
+    } else {
+      // compactionTaskCapacity might be 0 if totalWorkerCapacity is low.
+      // This guarantees that at least one slot is available if
+      // compaction is enabled and estimatedIncompleteCompactionTasks is 0.
+      availableCompactionTaskSlots = Math.max(1, compactionTaskCapacity);
+    }
+    LOG.info(
+        "Found [%d] available task slots for compaction out of max compaction task capacity [%d]",
+        availableCompactionTaskSlots, compactionTaskCapacity
+    );
+
+    return availableCompactionTaskSlots;
+  }
+
+  /**
+   * Submits compaction tasks to the Overlord. Returns total number of tasks submitted.
+   */
+  private int submitCompactionTasks(
       Map<String, DataSourceCompactionConfig> compactionConfigs,
       Map<String, AutoCompactionSnapshot.Builder> currentRunAutoCompactionSnapshotBuilders,
       int numAvailableCompactionTaskSlots,
       CompactionSegmentIterator iterator
   )
   {
+    if (numAvailableCompactionTaskSlots <= 0) {
+      return 0;
+    }
+
     int numSubmittedTasks = 0;
     int numCompactionTasksAndSubtasks = 0;
 
     while (iterator.hasNext() && numCompactionTasksAndSubtasks < numAvailableCompactionTaskSlots) {
       final List<DataSegment> segmentsToCompact = iterator.next();
+      if (segmentsToCompact.isEmpty()) {
+        throw new ISE("segmentsToCompact is empty?");
+      }
 
-      if (!segmentsToCompact.isEmpty()) {
-        final String dataSourceName = segmentsToCompact.get(0).getDataSource();
-        // As these segments will be compacted, we will aggregates the statistic to the Compacted statistics
+      final String dataSourceName = segmentsToCompact.get(0).getDataSource();
 
-        AutoCompactionSnapshot.Builder snapshotBuilder = currentRunAutoCompactionSnapshotBuilders.computeIfAbsent(
-            dataSourceName,
-            k -> new AutoCompactionSnapshot.Builder(k, AutoCompactionSnapshot.AutoCompactionScheduleStatus.RUNNING)
-        );
-        snapshotBuilder.incrementBytesCompacted(segmentsToCompact.stream().mapToLong(DataSegment::getSize).sum());
-        snapshotBuilder.incrementIntervalCountCompacted(segmentsToCompact.stream().map(DataSegment::getInterval).distinct().count());
-        snapshotBuilder.incrementSegmentCountCompacted(segmentsToCompact.size());
-
-        final DataSourceCompactionConfig config = compactionConfigs.get(dataSourceName);
-
-        // Create granularitySpec to send to compaction task
-        ClientCompactionTaskGranularitySpec granularitySpec;
-        Granularity segmentGranularityToUse = null;
-        if (config.getGranularitySpec() == null || config.getGranularitySpec().getSegmentGranularity() == null) {
-          // Determines segmentGranularity from the segmentsToCompact
-          // Each batch of segmentToCompact from CompactionSegmentIterator will contains the same interval as
-          // segmentGranularity is not set in the compaction config
-          Interval interval = segmentsToCompact.get(0).getInterval();
-          if (segmentsToCompact.stream().allMatch(segment -> interval.overlaps(segment.getInterval()))) {
-            try {
-              segmentGranularityToUse = GranularityType.fromPeriod(interval.toPeriod()).getDefaultGranularity();
-            }
-            catch (IllegalArgumentException iae) {
-              // This case can happen if the existing segment interval result in complicated periods.
-              // Fall back to setting segmentGranularity as null
-              LOG.warn("Cannot determine segmentGranularity from interval [%s]", interval);
-            }
-          } else {
-            LOG.warn("segmentsToCompact does not have the same interval. Fallback to not setting segmentGranularity for auto compaction task");
+      // As these segments will be compacted, we will aggregates the statistic to the Compacted statistics
+      AutoCompactionSnapshot.Builder snapshotBuilder = currentRunAutoCompactionSnapshotBuilders.computeIfAbsent(
+          dataSourceName,
+          k -> new AutoCompactionSnapshot.Builder(k, AutoCompactionSnapshot.AutoCompactionScheduleStatus.RUNNING)
+      );
+      snapshotBuilder.incrementBytesCompacted(
+          segmentsToCompact.stream().mapToLong(DataSegment::getSize).sum()
+      );
+      snapshotBuilder.incrementIntervalCountCompacted(
+          segmentsToCompact.stream()
+                           .map(DataSegment::getInterval)
+                           .distinct().count()
+      );
+      snapshotBuilder.incrementSegmentCountCompacted(segmentsToCompact.size());
+
+      final DataSourceCompactionConfig config = compactionConfigs.get(dataSourceName);
+
+      // Create granularitySpec to send to compaction task
+      ClientCompactionTaskGranularitySpec granularitySpec;
+      Granularity segmentGranularityToUse = null;
+      if (config.getGranularitySpec() == null || config.getGranularitySpec().getSegmentGranularity() == null) {
+        // Determines segmentGranularity from the segmentsToCompact
+        // Each batch of segmentToCompact from CompactionSegmentIterator will contains the same interval as
+        // segmentGranularity is not set in the compaction config
+        Interval interval = segmentsToCompact.get(0).getInterval();
+        if (segmentsToCompact.stream().allMatch(segment -> interval.overlaps(segment.getInterval()))) {
+          try {
+            segmentGranularityToUse = GranularityType.fromPeriod(interval.toPeriod()).getDefaultGranularity();
+          }
+          catch (IllegalArgumentException iae) {
+            // This case can happen if the existing segment interval result in complicated periods.
+            // Fall back to setting segmentGranularity as null
+            LOG.warn("Cannot determine segmentGranularity from interval [%s]", interval);
           }
         } else {
-          segmentGranularityToUse = config.getGranularitySpec().getSegmentGranularity();
+          LOG.warn(
+              "segmentsToCompact does not have the same interval. Fallback to not setting segmentGranularity for auto compaction task");
         }
-        granularitySpec = new ClientCompactionTaskGranularitySpec(
-            segmentGranularityToUse,
-            config.getGranularitySpec() != null ? config.getGranularitySpec().getQueryGranularity() : null,
-            config.getGranularitySpec() != null ? config.getGranularitySpec().isRollup() : null
+      } else {
+        segmentGranularityToUse = config.getGranularitySpec().getSegmentGranularity();
+      }
+      granularitySpec = new ClientCompactionTaskGranularitySpec(
+          segmentGranularityToUse,
+          config.getGranularitySpec() != null ? config.getGranularitySpec().getQueryGranularity() : null,
+          config.getGranularitySpec() != null ? config.getGranularitySpec().isRollup() : null
+      );
 
+      // Create dimensionsSpec to send to compaction task
+      ClientCompactionTaskDimensionsSpec dimensionsSpec;
+      if (config.getDimensionsSpec() != null) {
+        dimensionsSpec = new ClientCompactionTaskDimensionsSpec(
+            config.getDimensionsSpec().getDimensions()
         );
+      } else {
+        dimensionsSpec = null;
+      }
 
-        // Create dimensionsSpec to send to compaction task
-        ClientCompactionTaskDimensionsSpec dimensionsSpec;
-        if (config.getDimensionsSpec() != null) {
-          dimensionsSpec = new ClientCompactionTaskDimensionsSpec(
-              config.getDimensionsSpec().getDimensions()
-          );
-        } else {
-          dimensionsSpec = null;
-        }
-
-        // Create transformSpec to send to compaction task
-        ClientCompactionTaskTransformSpec transformSpec = null;
-        if (config.getTransformSpec() != null) {
-          transformSpec = new ClientCompactionTaskTransformSpec(
-              config.getTransformSpec().getFilter()
-          );
-        }
+      // Create transformSpec to send to compaction task
+      ClientCompactionTaskTransformSpec transformSpec = null;
+      if (config.getTransformSpec() != null) {
+        transformSpec = new ClientCompactionTaskTransformSpec(
+            config.getTransformSpec().getFilter()
+        );
+      }
 
-        Boolean dropExisting = null;
-        if (config.getIoConfig() != null) {
-          dropExisting = config.getIoConfig().isDropExisting();
-        }
+      Boolean dropExisting = null;
+      if (config.getIoConfig() != null) {
+        dropExisting = config.getIoConfig().isDropExisting();
+      }
 
-        // If all the segments found to be compacted are tombstones then dropExisting
-        // needs to be forced to true. This forcing needs to  happen in the case that
-        // the flag is null, or it is false. It is needed when it is null to avoid the
-        // possibility of the code deciding to default it to false later.
-        // Forcing the flag to true will enable the task ingestion code to generate new, compacted, tombstones to
-        // cover the tombstones found to be compacted as well as to mark them
-        // as compacted (update their lastCompactionState). If we don't force the
-        // flag then every time this compact duty runs it will find the same tombstones
-        // in the interval since their lastCompactionState
-        // was not set repeating this over and over and the duty will not make progress; it
-        // will become stuck on this set of tombstones.
-        // This forcing code should be revised
-        // when/if the autocompaction code policy to decide which segments to compact changes
-        if (dropExisting == null || !dropExisting) {
-          if (segmentsToCompact.stream().allMatch(dataSegment -> dataSegment.isTombstone())) {
-            dropExisting = true;
-            LOG.info("Forcing dropExisting to %s since all segments to compact are tombstones", dropExisting);
-          }
+      // If all the segments found to be compacted are tombstones then dropExisting
+      // needs to be forced to true. This forcing needs to  happen in the case that
+      // the flag is null, or it is false. It is needed when it is null to avoid the
+      // possibility of the code deciding to default it to false later.
+      // Forcing the flag to true will enable the task ingestion code to generate new, compacted, tombstones to
+      // cover the tombstones found to be compacted as well as to mark them
+      // as compacted (update their lastCompactionState). If we don't force the
+      // flag then every time this compact duty runs it will find the same tombstones
+      // in the interval since their lastCompactionState
+      // was not set repeating this over and over and the duty will not make progress; it
+      // will become stuck on this set of tombstones.
+      // This forcing code should be revised
+      // when/if the autocompaction code policy to decide which segments to compact changes
+      if (dropExisting == null || !dropExisting) {
+        if (segmentsToCompact.stream().allMatch(DataSegment::isTombstone)) {
+          dropExisting = true;
+          LOG.info("Forcing dropExisting to true since all segments to compact are tombstones.");
         }
+      }
 
-        // make tuningConfig
-        final String taskId = indexingServiceClient.compactSegments(
-            "coordinator-issued",
-            segmentsToCompact,
-            config.getTaskPriority(),
-            ClientCompactionTaskQueryTuningConfig.from(config.getTuningConfig(), config.getMaxRowsPerSegment(), config.getMetricsSpec() != null),
-            granularitySpec,
-            dimensionsSpec,
-            config.getMetricsSpec(),
-            transformSpec,
-            dropExisting,
-            newAutoCompactionContext(config.getTaskContext())
-        );
+      final String taskId = indexingServiceClient.compactSegments(
+          "coordinator-issued",
+          segmentsToCompact,
+          config.getTaskPriority(),
+          ClientCompactionTaskQueryTuningConfig.from(
+              config.getTuningConfig(),
+              config.getMaxRowsPerSegment(),

Review Comment:
   ## Deprecated method or constructor invocation
   
   Invoking [DataSourceCompactionConfig.getMaxRowsPerSegment](1) should be avoided because it has been deprecated.
   
   [Show more details](https://github.com/apache/druid/security/code-scanning/5041)



-- 
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: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] kfaraz commented on a diff in pull request #13197: Segment loading: Allow cancellation and prioritization of load queue items

Posted by GitBox <gi...@apache.org>.
kfaraz commented on code in PR #13197:
URL: https://github.com/apache/druid/pull/13197#discussion_r994052340


##########
server/src/main/java/org/apache/druid/server/coordinator/HttpLoadQueuePeon.java:
##########
@@ -470,134 +486,98 @@ public Set<DataSegment> getSegmentsMarkedToDrop()
     return Collections.unmodifiableSet(segmentsMarkedToDrop);
   }
 
-  private abstract class SegmentHolder
+  /**
+   * A request is considered to have timed out if the time elapsed since it was
+   * first sent to the server is greater than the configured load timeout.
+   *
+   * @see DruidCoordinatorConfig#getLoadTimeoutDelay()
+   */
+  private boolean hasRequestTimedOut(QueuedSegment holder)
   {
-    private final DataSegment segment;
-    private final DataSegmentChangeRequest changeRequest;
-    private final List<LoadPeonCallback> callbacks = new ArrayList<>();
-
-    // Time when this request was sent to target server the first time.
-    private volatile long scheduleTime = -1;
-
-    private SegmentHolder(
-        DataSegment segment,
-        DataSegmentChangeRequest changeRequest,
-        LoadPeonCallback callback
-    )
-    {
-      this.segment = segment;
-      this.changeRequest = changeRequest;
-
-      if (callback != null) {
-        this.callbacks.add(callback);
-      }
-    }
-
-    public void addCallback(LoadPeonCallback newCallback)
-    {
-      synchronized (callbacks) {
-        if (newCallback != null) {
-          callbacks.add(newCallback);
-        }
-      }
-    }
-
-    public DataSegment getSegment()
-    {
-      return segment;
-    }
-
-    public DataSegmentChangeRequest getChangeRequest()
-    {
-      return changeRequest;
-    }
-
-    public boolean hasTimedOut()
-    {
-      if (scheduleTime < 0) {
-        scheduleTime = System.currentTimeMillis();
-        return false;
-      } else if (System.currentTimeMillis() - scheduleTime > config.getLoadTimeoutDelay().getMillis()) {
-        return true;
-      } else {
-        return false;
-      }
-    }
+    return System.currentTimeMillis() - holder.getFirstRequestTimeMillis()

Review Comment:
   Yes, that would be cleaner.



-- 
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: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] AmatyaAvadhanula commented on a diff in pull request #13197: Segment loading: Allow cancellation and prioritization of load queue items

Posted by GitBox <gi...@apache.org>.
AmatyaAvadhanula commented on code in PR #13197:
URL: https://github.com/apache/druid/pull/13197#discussion_r994673055


##########
server/src/main/java/org/apache/druid/server/coordinator/TierLoadingState.java:
##########
@@ -0,0 +1,72 @@
+/*
+ * 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;
+
+import org.apache.druid.timeline.SegmentId;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * Contains segments being loaded or replicated in a tier.

Review Comment:
   This would also mean that the lifetime of primary replicants is not considered, right?



-- 
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: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] AmatyaAvadhanula commented on a diff in pull request #13197: Segment loading: Allow cancellation and prioritization of load queue items

Posted by GitBox <gi...@apache.org>.
AmatyaAvadhanula commented on code in PR #13197:
URL: https://github.com/apache/druid/pull/13197#discussion_r994671386


##########
server/src/main/java/org/apache/druid/server/coordinator/TierLoadingState.java:
##########
@@ -0,0 +1,72 @@
+/*
+ * 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;
+
+import org.apache.druid.timeline.SegmentId;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * Contains segments being loaded or replicated in a tier.

Review Comment:
   Isn't it segments being moved or being replicated, but not as a primary replica? (As replicants are added only by the ReplicationThrottler which doesn't consider primary replicants)



-- 
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: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] kfaraz commented on a diff in pull request #13197: Segment loading: Allow cancellation and prioritization of load queue items

Posted by GitBox <gi...@apache.org>.
kfaraz commented on code in PR #13197:
URL: https://github.com/apache/druid/pull/13197#discussion_r994865469


##########
server/src/main/java/org/apache/druid/server/coordinator/SegmentLoader.java:
##########
@@ -0,0 +1,472 @@
+/*
+ * 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;
+
+import com.google.common.collect.Sets;
+import org.apache.druid.java.util.emitter.EmittingLogger;
+import org.apache.druid.timeline.DataSegment;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.EnumMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.stream.Collectors;
+
+/**
+ * Used by the coordinator in each run for segment loading, dropping, balancing
+ * and broadcasting.
+ * <p>
+ * An instance of this class is freshly created for each coordinator run.
+ */
+public class SegmentLoader
+{
+  private static final EmittingLogger log = new EmittingLogger(SegmentLoader.class);
+
+  private final SegmentStateManager stateManager;
+  private final DruidCluster cluster;
+  private final CoordinatorStats stats = new CoordinatorStats();
+  private final SegmentReplicantLookup replicantLookup;
+  private final BalancerStrategy strategy;
+  private final int maxLoadQueueSize;
+
+  public SegmentLoader(SegmentStateManager stateManager, DruidCoordinatorRuntimeParams runParams)
+  {
+    this.stateManager = stateManager;
+    this.strategy = runParams.getBalancerStrategy();
+    this.cluster = runParams.getDruidCluster();
+    this.replicantLookup = runParams.getSegmentReplicantLookup();
+    this.maxLoadQueueSize = runParams.getCoordinatorDynamicConfig()
+                                     .getMaxSegmentsInNodeLoadingQueue();
+  }
+
+  public CoordinatorStats getStats()
+  {
+    return stats;
+  }
+
+  /**
+   * Moves the given segment between two servers of the same tier.
+   * <p>
+   * See if we can move balancing here.
+   */
+  public boolean moveSegment(DataSegment segment, ServerHolder fromServer, ServerHolder toServer)
+  {
+    if (!fromServer.getServer().getTier().equals(toServer.getServer().getTier())) {
+      return false;
+    }
+
+    // fromServer must be loading or serving the segment
+    // and toServer must be able to load it
+    final SegmentState stateOnSrc = fromServer.getSegmentState(segment);
+    if ((stateOnSrc != SegmentState.LOADING && stateOnSrc != SegmentState.LOADED)
+        || !canLoadSegment(toServer, segment)) {
+      return false;
+    }
+
+    final boolean cancelSuccess = stateOnSrc == SegmentState.LOADING
+                                  && stateManager.cancelOperation(SegmentState.LOADING, segment, fromServer);
+
+    if (cancelSuccess) {
+      int loadedCountOnTier = replicantLookup
+          .getLoadedReplicants(segment.getId(), toServer.getServer().getTier());
+      stateManager.loadSegment(segment, toServer, loadedCountOnTier < 1);
+    } else {
+      return stateManager.moveSegment(segment, fromServer, toServer);
+    }
+
+    return true;
+  }
+
+  /**
+   * Queues load or drop of replicas of the given segment to achieve the
+   * target replication level in all the tiers.
+   */
+  public void updateReplicas(DataSegment segment, Map<String, Integer> tierToReplicaCount)
+  {
+    // Handle every target tier
+    tierToReplicaCount.forEach((tier, numReplicas) -> {
+      updateReplicasOnTier(segment, tier, tierToReplicaCount.get(tier));
+      stats.addToTieredStat(CoordinatorStats.REQUIRED_CAPACITY, tier, segment.getSize() * numReplicas);
+    });
+
+    // Find the minimum number of segments required for fault tolerance
+    final int totalTargetReplicas = tierToReplicaCount.values().stream()
+                                                      .reduce(0, Integer::sum);
+    final int minLoadedSegments = totalTargetReplicas > 1 ? 2 : 1;
+
+    // Drop segment from unneeded tiers if requirement is met across target tiers
+    int loadedTargetReplicas = 0;
+    final Set<String> targetTiers = tierToReplicaCount.keySet();
+    for (String tier : targetTiers) {
+      loadedTargetReplicas += replicantLookup.getLoadedReplicants(segment.getId(), tier);
+    }
+    if (loadedTargetReplicas < minLoadedSegments) {
+      return;
+    }

Review Comment:
   Fixed this to always maintain the currently configured level of replication.



##########
server/src/main/java/org/apache/druid/server/coordinator/SegmentLoader.java:
##########
@@ -0,0 +1,472 @@
+/*
+ * 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;
+
+import com.google.common.collect.Sets;
+import org.apache.druid.java.util.emitter.EmittingLogger;
+import org.apache.druid.timeline.DataSegment;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.EnumMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.stream.Collectors;
+
+/**
+ * Used by the coordinator in each run for segment loading, dropping, balancing
+ * and broadcasting.
+ * <p>
+ * An instance of this class is freshly created for each coordinator run.
+ */
+public class SegmentLoader
+{
+  private static final EmittingLogger log = new EmittingLogger(SegmentLoader.class);
+
+  private final SegmentStateManager stateManager;
+  private final DruidCluster cluster;
+  private final CoordinatorStats stats = new CoordinatorStats();
+  private final SegmentReplicantLookup replicantLookup;
+  private final BalancerStrategy strategy;
+  private final int maxLoadQueueSize;
+
+  public SegmentLoader(SegmentStateManager stateManager, DruidCoordinatorRuntimeParams runParams)
+  {
+    this.stateManager = stateManager;
+    this.strategy = runParams.getBalancerStrategy();
+    this.cluster = runParams.getDruidCluster();
+    this.replicantLookup = runParams.getSegmentReplicantLookup();
+    this.maxLoadQueueSize = runParams.getCoordinatorDynamicConfig()
+                                     .getMaxSegmentsInNodeLoadingQueue();
+  }
+
+  public CoordinatorStats getStats()
+  {
+    return stats;
+  }
+
+  /**
+   * Moves the given segment between two servers of the same tier.
+   * <p>
+   * See if we can move balancing here.
+   */
+  public boolean moveSegment(DataSegment segment, ServerHolder fromServer, ServerHolder toServer)
+  {
+    if (!fromServer.getServer().getTier().equals(toServer.getServer().getTier())) {
+      return false;
+    }
+
+    // fromServer must be loading or serving the segment
+    // and toServer must be able to load it
+    final SegmentState stateOnSrc = fromServer.getSegmentState(segment);
+    if ((stateOnSrc != SegmentState.LOADING && stateOnSrc != SegmentState.LOADED)
+        || !canLoadSegment(toServer, segment)) {
+      return false;
+    }
+
+    final boolean cancelSuccess = stateOnSrc == SegmentState.LOADING
+                                  && stateManager.cancelOperation(SegmentState.LOADING, segment, fromServer);
+
+    if (cancelSuccess) {
+      int loadedCountOnTier = replicantLookup
+          .getLoadedReplicants(segment.getId(), toServer.getServer().getTier());
+      stateManager.loadSegment(segment, toServer, loadedCountOnTier < 1);
+    } else {
+      return stateManager.moveSegment(segment, fromServer, toServer);
+    }
+
+    return true;
+  }
+
+  /**
+   * Queues load or drop of replicas of the given segment to achieve the
+   * target replication level in all the tiers.
+   */
+  public void updateReplicas(DataSegment segment, Map<String, Integer> tierToReplicaCount)
+  {
+    // Handle every target tier
+    tierToReplicaCount.forEach((tier, numReplicas) -> {
+      updateReplicasOnTier(segment, tier, tierToReplicaCount.get(tier));
+      stats.addToTieredStat(CoordinatorStats.REQUIRED_CAPACITY, tier, segment.getSize() * numReplicas);
+    });
+
+    // Find the minimum number of segments required for fault tolerance
+    final int totalTargetReplicas = tierToReplicaCount.values().stream()
+                                                      .reduce(0, Integer::sum);
+    final int minLoadedSegments = totalTargetReplicas > 1 ? 2 : 1;
+
+    // Drop segment from unneeded tiers if requirement is met across target tiers
+    int loadedTargetReplicas = 0;
+    final Set<String> targetTiers = tierToReplicaCount.keySet();
+    for (String tier : targetTiers) {
+      loadedTargetReplicas += replicantLookup.getLoadedReplicants(segment.getId(), tier);
+    }
+    if (loadedTargetReplicas < minLoadedSegments) {
+      return;
+    }
+
+    final Set<String> dropTiers = Sets.newHashSet(cluster.getTierNames());
+    dropTiers.removeAll(targetTiers);
+    for (String dropTier : dropTiers) {
+      updateReplicasOnTier(segment, dropTier, 0);
+    }
+  }
+
+  /**
+   * Broadcasts the given segment to all servers that are broadcast targets and
+   * queues a drop of the segment from decommissioning servers.
+   */
+  public void broadcastSegment(DataSegment segment)
+  {
+    int assignedCount = 0;
+    int droppedCount = 0;
+    for (ServerHolder server : cluster.getAllServers()) {
+      // Ignore servers which are not broadcast targets
+      if (!server.getServer().getType().isSegmentBroadcastTarget()) {
+        continue;
+      }
+
+      if (server.isDecommissioning()) {
+        droppedCount += dropBroadcastSegment(segment, server) ? 1 : 0;
+      } else {
+        assignedCount += loadBroadcastSegment(segment, server) ? 1 : 0;
+      }
+    }
+
+    if (assignedCount > 0) {
+      stats.addToDataSourceStat(CoordinatorStats.BROADCAST_LOADS, segment.getDataSource(), assignedCount);
+    }
+    if (droppedCount > 0) {
+      stats.addToDataSourceStat(CoordinatorStats.BROADCAST_DROPS, segment.getDataSource(), droppedCount);
+    }
+  }
+
+  /**
+   * Marks the given segment as unused.
+   */
+  public void deleteSegment(DataSegment segment)
+  {
+    stateManager.deleteSegment(segment);
+    stats.addToGlobalStat(CoordinatorStats.DELETED_SEGMENTS, 1);
+  }
+
+  /**
+   * Checks if the server can load the given segment.
+   * <p>
+   * A load is possible only if the server meets all of the following criteria:
+   * <ul>
+   *   <li>is not already serving or loading the segment</li>
+   *   <li>is not being decommissioned</li>
+   *   <li>has not already exceeded the load queue limit in this run</li>
+   *   <li>has available disk space</li>
+   * </ul>
+   */
+  public boolean canLoadSegment(ServerHolder server, DataSegment segment)
+  {
+    return server.canLoadSegment(segment)

Review Comment:
   Fixed.



-- 
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: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] kfaraz commented on a diff in pull request #13197: Segment loading: Allow cancellation and prioritization of load queue items

Posted by GitBox <gi...@apache.org>.
kfaraz commented on code in PR #13197:
URL: https://github.com/apache/druid/pull/13197#discussion_r995348391


##########
server/src/main/java/org/apache/druid/server/coordinator/CuratorLoadQueuePeon.java:
##########
@@ -181,10 +183,10 @@ public int getNumberOfSegmentsInQueue()
   }
 
   @Override
-  public void loadSegment(final DataSegment segment, @Nullable final LoadPeonCallback callback)
+  public void loadSegment(final DataSegment segment, SegmentAction action, @Nullable final LoadPeonCallback callback)
   {
-    SegmentHolder segmentHolder = new SegmentHolder(segment, Action.LOAD, Collections.singletonList(callback));
-    final SegmentHolder existingHolder = segmentsToLoad.putIfAbsent(segment, segmentHolder);
+    QueuedSegment segmentHolder = new QueuedSegment(segment, action, callback);

Review Comment:
   > That said, I would've expected the queue itself to return a QueuedSegment rather than something creating one of those before it's actually queued...
   
   Do you mean like a `computeIfAbsent` rather than `putIfAbsent`?



-- 
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: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] kfaraz commented on a diff in pull request #13197: Segment loading: Allow cancellation and prioritization of load queue items

Posted by "kfaraz (via GitHub)" <gi...@apache.org>.
kfaraz commented on code in PR #13197:
URL: https://github.com/apache/druid/pull/13197#discussion_r1231770485


##########
server/src/main/java/org/apache/druid/server/coordinator/SegmentReplicantLookup.java:
##########
@@ -125,49 +117,85 @@ public int getMovingReplicas(SegmentId segmentId, String tier)
    * Number of replicas of the segment which are safely loaded on the given tier
    * and are not being dropped.
    */
-  public int getLoadedReplicas(SegmentId segmentId, String tier)
+  public int getLoadedNotDroppingReplicas(SegmentId segmentId, String tier)
+  {
+    ReplicaCount count = replicaCounts.get(segmentId, tier);
+    return (count == null) ? 0 : count.loadedNotDropping();
+  }
+
+  public int getLoadingReplicas(SegmentId segmentId, String tier)
   {
     ReplicaCount count = replicaCounts.get(segmentId, tier);
-    return (count == null) ? 0 : count.safelyLoaded();
+    return count == null ? 0 : count.loading;
   }
 
   /**
-   * Number of replicas of the segment which are safely loaded on the cluster
-   * and are not being dropped.
+   * Number of replicas of the segment which are loaded on the cluster.
+   *
+   * @param includeDropping Whether segments which are being dropped should be
+   *                        included in the total count.
    */
-  public int getTotalLoadedReplicas(SegmentId segmentId)
+  public int getLoadedReplicas(SegmentId segmentId, boolean includeDropping)
   {
     final Map<String, ReplicaCount> allTiers = replicaCounts.row(segmentId);
     int totalLoaded = 0;
     for (ReplicaCount count : allTiers.values()) {
-      totalLoaded += count.safelyLoaded();
+      totalLoaded += includeDropping ? count.loaded : count.loadedNotDropping();
     }
     return totalLoaded;
   }
 
-  public Object2LongMap<String> getBroadcastUnderReplication(SegmentId segmentId)
+  /**
+   * Sets the number of replicas required for the specified segment in the tier.
+   * In a given coordinator run, this method must be called atleast once for
+   * every segment every tier.
+   */
+  public void setRequiredReplicas(SegmentId segmentId, boolean isBroadcast, String tier, int requiredReplicas)
   {
-    Object2LongOpenHashMap<String> perTier = new Object2LongOpenHashMap<>();
-    for (ServerHolder holder : cluster.getAllServers()) {
-      // Only record tier entry for server that is segment broadcast target
-      if (holder.getServer().getType().isSegmentBroadcastTarget()) {
-        // Every broadcast target server should be serving 1 replica of the segment
-        if (!holder.isServingSegment(segmentId)) {
-          perTier.addTo(holder.getServer().getTier(), 1L);
-        } else {
-          perTier.putIfAbsent(holder.getServer().getTier(), 0);
-        }
+    ReplicaCount counts = computeIfAbsent(replicaCounts, segmentId, tier);
+    counts.required = requiredReplicas;
+    if (isBroadcast) {
+      counts.possible = requiredReplicas;
+    } else {
+      counts.possible = tierToHistoricalCount.getOrDefault(tier, 0);
+    }
+  }
+
+  public Map<String, Object2LongMap<String>> getTierToDatasourceToUnderReplicated(
+      Iterable<DataSegment> usedSegments,
+      boolean ignoreMissingServers
+  )
+  {
+    final Map<String, Object2LongMap<String>> tierToUnderReplicated = new HashMap<>();
+
+    for (DataSegment segment : usedSegments) {
+      final Map<String, ReplicaCount> tierToReplicaCount = replicaCounts.row(segment.getId());
+      if (tierToReplicaCount == null) {
+        continue;
       }
+
+      tierToReplicaCount.forEach((tier, counts) -> {
+        final int underReplicated = counts.underReplicated(ignoreMissingServers);
+        if (underReplicated >= 0) {
+          Object2LongOpenHashMap<String> datasourceToUnderReplicated = (Object2LongOpenHashMap<String>)
+              tierToUnderReplicated.computeIfAbsent(tier, ds -> new Object2LongOpenHashMap<>());
+          datasourceToUnderReplicated.addTo(segment.getDataSource(), underReplicated);
+        }
+      });
     }
-    return perTier;
+
+    return tierToUnderReplicated;
   }

Review Comment:
   Hmm, let me see what can be done here. 
   
   It turned out like this mostly because it is a direct translation of the flag `computeUsingClusterView` passed in from `OverlordResource` to `DruidCoordinator` to `SegmentReplicantLookup`.



-- 
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: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] kfaraz commented on a diff in pull request #13197: Segment loading: Allow cancellation and prioritization of load queue items

Posted by "kfaraz (via GitHub)" <gi...@apache.org>.
kfaraz commented on code in PR #13197:
URL: https://github.com/apache/druid/pull/13197#discussion_r1231754131


##########
server/src/main/java/org/apache/druid/server/coordinator/StrategicSegmentAssigner.java:
##########
@@ -0,0 +1,577 @@
+/*
+ * 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;
+
+import com.google.common.collect.Sets;
+import it.unimi.dsi.fastutil.objects.Object2IntOpenHashMap;
+import org.apache.druid.java.util.emitter.EmittingLogger;
+import org.apache.druid.server.coordinator.balancer.BalancerStrategy;
+import org.apache.druid.server.coordinator.loadqueue.SegmentAction;
+import org.apache.druid.server.coordinator.loadqueue.SegmentLoadQueueManager;
+import org.apache.druid.server.coordinator.rules.SegmentActionHandler;
+import org.apache.druid.server.coordinator.stats.CoordinatorRunStats;
+import org.apache.druid.server.coordinator.stats.CoordinatorStat;
+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 java.util.Comparator;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+
+/**
+ * Used by the coordinator in each run for segment loading, dropping, balancing
+ * and broadcasting.
+ * <p>
+ * An instance of this class is freshly created for each coordinator run.
+ */
+public class StrategicSegmentAssigner implements SegmentActionHandler
+{
+  private static final EmittingLogger log = new EmittingLogger(StrategicSegmentAssigner.class);
+
+  private final SegmentLoadQueueManager loadQueueManager;
+  private final DruidCluster cluster;
+  private final CoordinatorRunStats stats;
+  private final SegmentReplicantLookup replicantLookup;
+  private final ReplicationThrottler replicationThrottler;
+  private final RoundRobinServerSelector serverSelector;
+  private final BalancerStrategy strategy;
+
+  private final boolean useRoundRobinAssignment;
+
+  private final Set<String> tiersWithNoServer = new HashSet<>();
+
+  public StrategicSegmentAssigner(
+      SegmentLoadQueueManager loadQueueManager,
+      DruidCluster cluster,
+      BalancerStrategy strategy,
+      CoordinatorDynamicConfig dynamicConfig,
+      CoordinatorRunStats stats
+  )
+  {
+    this.stats = stats;
+    this.cluster = cluster;
+    this.strategy = strategy;
+    this.loadQueueManager = loadQueueManager;
+    this.replicantLookup = SegmentReplicantLookup.make(cluster);
+    this.replicationThrottler = createReplicationThrottler(dynamicConfig);
+    this.useRoundRobinAssignment = dynamicConfig.isUseRoundRobinSegmentAssignment();
+    this.serverSelector = useRoundRobinAssignment ? new RoundRobinServerSelector(cluster) : null;
+  }
+
+  public CoordinatorRunStats getStats()
+  {
+    return stats;
+  }
+
+  public SegmentReplicantLookup getReplicantLookup()
+  {
+    return replicantLookup;
+  }
+
+  public void makeAlerts()
+  {
+    if (!tiersWithNoServer.isEmpty()) {
+      log.makeAlert("Tiers [%s] have no servers! Check your cluster configuration.", tiersWithNoServer).emit();
+    }
+  }
+
+  /**
+   * Moves the given segment from the source server to an eligible destination
+   * server.
+   * <p>
+   * An eligible destination server must:
+   * <ul>
+   *   <li>be present in the given list of destination servers</li>
+   *   <li>belong to the same tier as the source server</li>
+   *   <li>not already be serving or loading a replica of the segment</li>
+   *   <li>have enough space to load the segment</li>
+   * </ul>
+   * <p>
+   * The segment is not moved if:
+   * <ul>
+   *   <li>there is no eligible destination server, or</li>
+   *   <li>skipIfOptimallyPlaced is true and segment is already optimally placed, or</li>
+   *   <li>some other error occurs</li>
+   * </ul>
+   */
+  public boolean moveSegment(
+      DataSegment segment,
+      ServerHolder sourceServer,
+      List<ServerHolder> destinationServers
+  )
+  {
+    final String tier = sourceServer.getServer().getTier();
+    final List<ServerHolder> eligibleDestinationServers =
+        destinationServers.stream()
+                          .filter(s -> s.getServer().getTier().equals(tier))
+                          .filter(s -> s.canLoadSegment(segment))
+                          .collect(Collectors.toList());
+
+    if (eligibleDestinationServers.isEmpty()) {
+      incrementSkipStat(Stats.Segments.MOVE_SKIPPED, "No eligible server", segment, tier);
+      return false;
+    }
+
+    // If the source server is not decommissioning, move can be skipped if the
+    // segment is already optimally placed
+    if (!sourceServer.isDecommissioning()) {
+      eligibleDestinationServers.add(sourceServer);
+    }
+
+    final ServerHolder destination =
+        strategy.findDestinationServerToMoveSegment(segment, sourceServer, eligibleDestinationServers);
+
+    if (destination == null || destination.getServer().equals(sourceServer.getServer())) {
+      incrementSkipStat(Stats.Segments.MOVE_SKIPPED, "Optimally placed", segment, tier);
+      return false;
+    } else if (moveSegment(segment, sourceServer, destination)) {
+      incrementStat(Stats.Segments.MOVED, segment, tier, 1);
+      return true;
+    } else {
+      incrementSkipStat(Stats.Segments.MOVE_SKIPPED, "Encountered error", segment, tier);
+      return false;
+    }
+  }
+
+  /**
+   * Moves the given segment from serverA to serverB.
+   */
+  private boolean moveSegment(DataSegment segment, ServerHolder serverA, ServerHolder serverB)
+  {
+    final String tier = serverA.getServer().getTier();
+    if (serverA.isLoadingSegment(segment)) {
+      // Cancel the load on serverA and load on serverB instead
+      if (serverA.cancelOperation(SegmentAction.LOAD, segment)) {
+        int loadedCountOnTier = replicantLookup.getLoadedNotDroppingReplicas(segment.getId(), tier);
+        return loadSegment(segment, serverB, loadedCountOnTier >= 1);
+      }
+
+      // Could not cancel load, let the segment load on serverA and count it as unmoved
+      return false;
+    } else if (serverA.isServingSegment(segment)) {
+      return loadQueueManager.moveSegment(segment, serverA, serverB);
+    } else {
+      return false;
+    }
+  }
+
+  @Override
+  public void updateSegmentReplicasInTiers(DataSegment segment, Map<String, Integer> tierToReplicaCount)
+  {
+    // Identify empty tiers and determine total required replicas
+    final AtomicInteger requiredTotalReplicas = new AtomicInteger(0);
+    final Set<String> allTiers = Sets.newHashSet(cluster.getTierNames());
+    tierToReplicaCount.forEach((tier, requiredReplicas) -> {
+      reportTierCapacityStats(segment, requiredReplicas, tier);
+      replicantLookup.setRequiredReplicas(segment.getId(), false, tier, requiredReplicas);
+      if (allTiers.contains(tier)) {
+        requiredTotalReplicas.addAndGet(requiredReplicas);
+      } else {
+        tiersWithNoServer.add(tier);
+      }
+    });
+
+    final int loadedNotDroppingReplicas = replicantLookup.getLoadedReplicas(segment.getId(), false);
+    final int replicaSurplus = loadedNotDroppingReplicas - requiredTotalReplicas.get();
+
+    // Update replicas in every tier
+    int dropsQueued = 0;
+    for (String tier : allTiers) {
+      dropsQueued += updateReplicasInTier(
+          segment,
+          tier,
+          tierToReplicaCount.getOrDefault(tier, 0),
+          replicaSurplus - dropsQueued
+      );
+    }
+  }
+
+  /**
+   * Queues load or drop operations on this tier based on the required
+   * number of replicas and the current state.
+   * <p>
+   * The {@code maxReplicasToDrop} helps to maintain the required level of
+   * replication in the cluster. This ensures that segment read concurrency does
+   * not suffer during a tier shift or load rule change.
+   * <p>
+   * Returns the number of new drop operations queued on this tier.
+   */
+  private int updateReplicasInTier(
+      DataSegment segment,
+      String tier,
+      int requiredReplicas,
+      int maxReplicasToDrop
+  )
+  {
+    final int loadedNotDroppingReplicas =
+        replicantLookup.getLoadedNotDroppingReplicas(segment.getId(), tier);
+    final int loadingReplicas = replicantLookup.getLoadingReplicas(segment.getId(), tier);
+    final int projectedReplicas = loadedNotDroppingReplicas + loadingReplicas;
+
+    final int movingReplicas = replicantLookup.getMovingReplicas(segment.getId(), tier);
+    final boolean shouldCancelMoves = requiredReplicas == 0 && movingReplicas > 0;
+
+    // Check if there is any action required on this tier
+    if (projectedReplicas == requiredReplicas && !shouldCancelMoves) {
+      return 0;
+    }
+
+    final SegmentStatusInTier segmentStatus =
+        new SegmentStatusInTier(segment, cluster.getHistoricalsByTier(tier));
+
+    // Cancel all moves in this tier if it does not need to have replicas
+    if (shouldCancelMoves) {
+      cancelOperations(SegmentAction.MOVE_TO, movingReplicas, segment, segmentStatus);
+      cancelOperations(SegmentAction.MOVE_FROM, movingReplicas, segment, segmentStatus);
+    }
+
+    // Cancel drops and queue loads if the projected count is below the requirement
+    if (projectedReplicas < requiredReplicas) {
+      int replicaDeficit = requiredReplicas - projectedReplicas;
+      int cancelledDrops =
+          cancelOperations(SegmentAction.DROP, replicaDeficit, segment, segmentStatus);
+
+      // Cancelled drops can be counted as loaded replicas, thus reducing deficit
+      int numReplicasToLoad = replicaDeficit - cancelledDrops;
+      if (numReplicasToLoad > 0) {
+        boolean isAlreadyLoadedOnTier = loadedNotDroppingReplicas + cancelledDrops >= 1;
+        int numLoadsQueued = loadReplicas(numReplicasToLoad, segment, tier, segmentStatus, isAlreadyLoadedOnTier);
+        incrementStat(Stats.Segments.ASSIGNED, segment, tier, numLoadsQueued);
+      }
+    }
+
+    // Cancel loads and queue drops if the projected count exceeds the requirement
+    if (projectedReplicas > requiredReplicas) {
+      int replicaSurplus = projectedReplicas - requiredReplicas;
+      int cancelledLoads =
+          cancelOperations(SegmentAction.LOAD, replicaSurplus, segment, segmentStatus);
+
+      int numReplicasToDrop = Math.min(replicaSurplus - cancelledLoads, maxReplicasToDrop);
+      if (numReplicasToDrop > 0) {
+        int dropsQueuedOnTier = dropReplicas(numReplicasToDrop, segment, tier, segmentStatus);
+        incrementStat(Stats.Segments.DROPPED, segment, tier, dropsQueuedOnTier);
+        return dropsQueuedOnTier;
+      }
+    }
+
+    return 0;
+  }
+
+  private void reportTierCapacityStats(DataSegment segment, int requiredReplicas, String tier)
+  {
+    final RowKey rowKey = RowKey.forTier(tier);
+    stats.updateMax(Stats.Tier.REPLICATION_FACTOR, rowKey, requiredReplicas);
+    stats.add(Stats.Tier.REQUIRED_CAPACITY, rowKey, segment.getSize() * requiredReplicas);
+  }
+
+  @Override
+  public void broadcastSegment(DataSegment segment)
+  {
+    final Object2IntOpenHashMap<String> tierToRequiredReplicas = new Object2IntOpenHashMap<>();
+    for (ServerHolder server : cluster.getAllServers()) {
+      // Ignore servers which are not broadcast targets
+      if (!server.getServer().getType().isSegmentBroadcastTarget()) {
+        continue;
+      }
+
+      final String tier = server.getServer().getTier();
+
+      // Drop from decommissioning servers and load on active servers
+      int numDropsQueued = 0;
+      int numLoadsQueued = 0;
+      if (server.isDecommissioning()) {
+        numDropsQueued += dropBroadcastSegment(segment, server) ? 1 : 0;
+      } else {
+        tierToRequiredReplicas.addTo(tier, 1);
+        numLoadsQueued += loadBroadcastSegment(segment, server) ? 1 : 0;
+      }
+
+      if (numLoadsQueued > 0) {
+        incrementStat(Stats.Segments.ASSIGNED, segment, tier, numLoadsQueued);
+      }
+      if (numDropsQueued > 0) {
+        incrementStat(Stats.Segments.DROPPED, segment, tier, numDropsQueued);
+      }
+    }
+
+    // Update required replica counts
+    tierToRequiredReplicas.object2IntEntrySet().fastForEach(
+        entry -> replicantLookup
+            .setRequiredReplicas(segment.getId(), true, entry.getKey(), entry.getIntValue())
+    );
+  }
+
+  @Override
+  public void deleteSegment(DataSegment segment)
+  {
+    loadQueueManager.deleteSegment(segment);
+    stats.addToDatasourceStat(Stats.Segments.DELETED, segment.getDataSource(), 1);
+  }
+
+  /**
+   * Loads the broadcast segment if it is not loaded on the given server.
+   * Returns true only if the segment was successfully queued for load on the server.
+   */
+  private boolean loadBroadcastSegment(DataSegment segment, ServerHolder server)
+  {
+    if (server.isServingSegment(segment) || server.isLoadingSegment(segment)) {
+      return false;
+    } else if (server.isDroppingSegment(segment)) {
+      return server.cancelOperation(SegmentAction.DROP, segment);
+    }
+
+    if (server.canLoadSegment(segment) && loadSegment(segment, server, false)) {
+      return true;
+    } else {
+      log.makeAlert("Could not assign broadcast segment for datasource [%s]", segment.getDataSource())
+         .addData("segmentId", segment.getId())
+         .addData("segmentSize", segment.getSize())
+         .addData("hostName", server.getServer().getHost())
+         .addData("availableSize", server.getAvailableSize())
+         .emit();
+      return false;
+    }
+  }
+
+  /**
+   * Drops the broadcast segment if it is loaded on the given server.
+   * Returns true only if the segment was successfully queued for drop on the server.
+   */
+  private boolean dropBroadcastSegment(DataSegment segment, ServerHolder server)
+  {
+    if (server.isLoadingSegment(segment)) {
+      return server.cancelOperation(SegmentAction.LOAD, segment);
+    } else if (server.isServingSegment(segment)) {
+      return loadQueueManager.dropSegment(segment, server);
+    } else {
+      return false;
+    }
+  }
+
+  /**
+   * Queues drop of {@code numToDrop} replicas of the segment from a tier.
+   * Tries to drop replicas first from decommissioning servers and then from
+   * active servers.
+   * <p>
+   * Returns the number of successfully queued drop operations.
+   */
+  private int dropReplicas(
+      final int numToDrop,
+      DataSegment segment,
+      String tier,
+      SegmentStatusInTier segmentStatus
+  )
+  {
+    if (numToDrop <= 0) {
+      return 0;
+    }
+
+    final List<ServerHolder> eligibleServers = segmentStatus.getServersEligibleToDrop();
+    if (eligibleServers.isEmpty()) {
+      incrementSkipStat(Stats.Segments.DROP_SKIPPED, "No eligible server", segment, tier);
+      return 0;
+    }
+
+    // Keep eligible servers sorted by most full first
+    final TreeSet<ServerHolder> eligibleLiveServers = new TreeSet<>(Comparator.reverseOrder());
+    final TreeSet<ServerHolder> eligibleDyingServers = new TreeSet<>(Comparator.reverseOrder());
+    for (ServerHolder server : eligibleServers) {
+      if (server.isDecommissioning()) {
+        eligibleDyingServers.add(server);
+      } else {
+        eligibleLiveServers.add(server);
+      }
+    }
+
+    // Drop as many replicas as possible from decommissioning servers
+    int remainingNumToDrop = numToDrop;
+    int numDropsQueued =
+        dropReplicasFromServers(remainingNumToDrop, segment, eligibleDyingServers.iterator(), tier);
+
+    // Drop replicas from active servers if required
+    if (numToDrop > numDropsQueued) {
+      remainingNumToDrop = numToDrop - numDropsQueued;
+      Iterator<ServerHolder> serverIterator =
+          (useRoundRobinAssignment || eligibleLiveServers.size() >= remainingNumToDrop)
+          ? eligibleLiveServers.iterator()
+          : strategy.pickServersToDropSegment(segment, eligibleLiveServers);
+      numDropsQueued += dropReplicasFromServers(remainingNumToDrop, segment, serverIterator, tier);
+    }
+
+    return numDropsQueued;
+  }
+
+  /**
+   * Queues drop of {@code numToDrop} replicas of the segment from the servers.
+   * Returns the number of successfully queued drop operations.
+   */
+  private int dropReplicasFromServers(
+      int numToDrop,
+      DataSegment segment,
+      Iterator<ServerHolder> serverIterator,
+      String tier
+  )
+  {
+    int numDropsQueued = 0;
+    while (numToDrop > numDropsQueued && serverIterator.hasNext()) {
+      ServerHolder holder = serverIterator.next();
+      boolean dropped = loadQueueManager.dropSegment(segment, holder);
+
+      if (dropped) {
+        ++numDropsQueued;
+      } else {
+        incrementSkipStat(Stats.Segments.DROP_SKIPPED, "Encountered error", segment, tier);
+      }
+    }
+
+    return numDropsQueued;
+  }
+
+  /**
+   * Queues load of {@code numToLoad} replicas of the segment on a tier.
+   */
+  private int loadReplicas(
+      int numToLoad,
+      DataSegment segment,
+      String tier,
+      SegmentStatusInTier segmentStatus,
+      boolean isAlreadyLoadedOnTier
+  )
+  {
+    // Do not assign replicas if tier is already busy loading some
+    if (isAlreadyLoadedOnTier && replicationThrottler.isTierLoadingReplicas(tier)) {
+      return 0;
+    }
+
+    final List<ServerHolder> eligibleServers = segmentStatus.getServersEligibleToLoad();
+    if (eligibleServers.isEmpty()) {
+      incrementSkipStat(Stats.Segments.ASSIGN_SKIPPED, "No eligible server", segment, tier);
+      return 0;
+    }
+
+    final Iterator<ServerHolder> serverIterator =
+        useRoundRobinAssignment
+        ? serverSelector.getServersInTierToLoadSegment(tier, segment)
+        : strategy.findServersToLoadSegment(segment, eligibleServers);
+    if (!serverIterator.hasNext()) {
+      incrementSkipStat(Stats.Segments.ASSIGN_SKIPPED, "No server chosen by strategy", segment, tier);
+      return 0;
+    }
+
+    // Load the replicas on this tier
+    int numLoadsQueued = 0;
+    while (numLoadsQueued < numToLoad && serverIterator.hasNext()) {
+      numLoadsQueued += loadSegment(segment, serverIterator.next(), isAlreadyLoadedOnTier)
+                        ? 1 : 0;
+    }
+
+    return numLoadsQueued;
+  }
+
+  private boolean loadSegment(DataSegment segment, ServerHolder server, boolean isAlreadyLoadedOnTier)
+  {
+    final String tier = server.getServer().getTier();
+    if (isAlreadyLoadedOnTier && !replicationThrottler.canAssignReplica(tier)) {
+      incrementSkipStat(Stats.Segments.ASSIGN_SKIPPED, "Throttled replication", segment, tier);
+      return false;
+    }
+
+    final SegmentAction action = isAlreadyLoadedOnTier ? SegmentAction.REPLICATE : SegmentAction.LOAD;
+    final boolean assigned = loadQueueManager.loadSegment(segment, server, action);
+
+    if (!assigned) {
+      incrementSkipStat(Stats.Segments.ASSIGN_SKIPPED, "Encountered error", segment, tier);
+    } else if (isAlreadyLoadedOnTier) {
+      replicationThrottler.incrementAssignedReplicas(tier);
+    }
+
+    return assigned;
+  }

Review Comment:
   The boolean here is used to determine if the assignment was successful or not, not distinguish between load and replicate. I can add a comment to that effect. The boolean status is used downstream for other decisions, like counting the number of successful assignments so far.
   
   Please let me know if you think some other approach would be cleaner.



-- 
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: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] AmatyaAvadhanula commented on a diff in pull request #13197: Segment loading: Allow cancellation and prioritization of load queue items

Posted by GitBox <gi...@apache.org>.
AmatyaAvadhanula commented on code in PR #13197:
URL: https://github.com/apache/druid/pull/13197#discussion_r993678277


##########
server/src/main/java/org/apache/druid/server/coordinator/TierLoadingState.java:
##########
@@ -0,0 +1,72 @@
+/*
+ * 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;
+
+import org.apache.druid.timeline.SegmentId;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * Contains segments being loaded or replicated in a tier.
+ */
+public class TierLoadingState
+{
+  private int lifetime;

Review Comment:
   Please consider the usage in ReplicationThrottler where a TierLoadingState is added to `currentlyReplicating`.
   A tier may be busy for a period greater than the lifetime without the load queue being stuck. In such a case the lifetime would still be decreased and may become < 0.



-- 
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: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] kfaraz commented on a diff in pull request #13197: Segment loading: Allow cancellation and prioritization of load queue items

Posted by GitBox <gi...@apache.org>.
kfaraz commented on code in PR #13197:
URL: https://github.com/apache/druid/pull/13197#discussion_r1000146676


##########
server/src/main/java/org/apache/druid/server/coordinator/SegmentStateManager.java:
##########
@@ -0,0 +1,261 @@
+/*
+ * 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;
+
+import com.google.inject.Inject;
+import org.apache.druid.client.ServerInventoryView;
+import org.apache.druid.java.util.emitter.EmittingLogger;
+import org.apache.druid.metadata.SegmentsMetadataManager;
+import org.apache.druid.timeline.DataSegment;
+
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * Manages state of segments being loaded.
+ */
+public class SegmentStateManager
+{
+  private static final EmittingLogger log = new EmittingLogger(SegmentStateManager.class);
+
+  private final LoadQueueTaskMaster taskMaster;
+  private final ServerInventoryView serverInventoryView;
+  private final SegmentsMetadataManager segmentsMetadataManager;
+
+  private final ConcurrentHashMap<String, TierLoadingState> currentlyMovingSegments =
+      new ConcurrentHashMap<>();
+  private final ConcurrentHashMap<String, TierLoadingState> currentlyReplicatingSegments
+      = new ConcurrentHashMap<>();
+
+  @Inject
+  public SegmentStateManager(
+      ServerInventoryView serverInventoryView,
+      SegmentsMetadataManager segmentsMetadataManager,
+      LoadQueueTaskMaster taskMaster
+  )
+  {
+    this.serverInventoryView = serverInventoryView;
+    this.segmentsMetadataManager = segmentsMetadataManager;
+    this.taskMaster = taskMaster;
+  }
+
+  /**
+   * Queues load of a replica of the segment on the given server.
+   */
+  public boolean loadSegment(
+      DataSegment segment,
+      ServerHolder server,
+      boolean isPrimary,
+      ReplicationThrottler throttler
+  )
+  {
+    final String tier = server.getServer().getTier();
+    final LoadPeonCallback callback;
+    if (isPrimary) {
+      // Primary replicas are not subject to throttling
+      callback = null;
+    } else if (canLoadReplica(tier, throttler)) {
+      throttler.incrementAssignedReplicas(tier);
+
+      final TierLoadingState replicatingInTier = currentlyReplicatingSegments
+          .computeIfAbsent(tier, t -> new TierLoadingState(throttler.getMaxLifetime()));
+      replicatingInTier.addSegment(segment.getId(), server.getServer().getHost());
+      callback = success -> replicatingInTier.removeSegment(segment.getId());
+    } else {
+      throttler.incrementThrottledReplicas(tier);
+      return false;
+    }
+
+    try {
+      if (!server.startOperation(segment, SegmentState.LOADING)) {
+        return false;
+      }
+
+      server.getPeon().loadSegment(
+          segment,
+          isPrimary ? SegmentAction.LOAD_AS_PRIMARY : SegmentAction.LOAD_AS_REPLICA,

Review Comment:
   Fixed.



-- 
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: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] AmatyaAvadhanula commented on a diff in pull request #13197: Segment loading: Allow cancellation and prioritization of load queue items

Posted by GitBox <gi...@apache.org>.
AmatyaAvadhanula commented on code in PR #13197:
URL: https://github.com/apache/druid/pull/13197#discussion_r993483143


##########
server/src/main/java/org/apache/druid/server/coordinator/TierLoadingState.java:
##########
@@ -0,0 +1,72 @@
+/*
+ * 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;
+
+import org.apache.druid.timeline.SegmentId;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * Contains segments being loaded or replicated in a tier.
+ */
+public class TierLoadingState
+{
+  private int lifetime;
+  private final ConcurrentHashMap<SegmentId, String> processingSegments = new ConcurrentHashMap<>();

Review Comment:
   Can't a tier load / balance the same segment on multiple servers simultaneously with these changes?



-- 
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: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] kfaraz commented on a diff in pull request #13197: Segment loading: Allow cancellation and prioritization of load queue items

Posted by GitBox <gi...@apache.org>.
kfaraz commented on code in PR #13197:
URL: https://github.com/apache/druid/pull/13197#discussion_r994050684


##########
server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java:
##########
@@ -255,6 +253,8 @@ public DruidCoordinator(
     this.coordLeaderSelector = coordLeaderSelector;
     this.objectMapper = objectMapper;
     this.compactSegments = initializeCompactSegmentsDuty();
+    this.segmentStateManager =
+        new SegmentStateManager(serverInventoryView, segmentsMetadataManager, taskMaster.isHttpLoading());

Review Comment:
   Agreed, I do have that in a separate patch. Will include it here.



-- 
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: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] kfaraz commented on a diff in pull request #13197: Segment loading: Allow cancellation and prioritization of load queue items

Posted by GitBox <gi...@apache.org>.
kfaraz commented on code in PR #13197:
URL: https://github.com/apache/druid/pull/13197#discussion_r994048811


##########
server/src/main/java/org/apache/druid/server/coordinator/SegmentLoader.java:
##########
@@ -0,0 +1,472 @@
+/*
+ * 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;
+
+import com.google.common.collect.Sets;
+import org.apache.druid.java.util.emitter.EmittingLogger;
+import org.apache.druid.timeline.DataSegment;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.EnumMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.stream.Collectors;
+
+/**
+ * Used by the coordinator in each run for segment loading, dropping, balancing
+ * and broadcasting.
+ * <p>
+ * An instance of this class is freshly created for each coordinator run.
+ */
+public class SegmentLoader
+{
+  private static final EmittingLogger log = new EmittingLogger(SegmentLoader.class);
+
+  private final SegmentStateManager stateManager;
+  private final DruidCluster cluster;
+  private final CoordinatorStats stats = new CoordinatorStats();
+  private final SegmentReplicantLookup replicantLookup;
+  private final BalancerStrategy strategy;
+  private final int maxLoadQueueSize;
+
+  public SegmentLoader(SegmentStateManager stateManager, DruidCoordinatorRuntimeParams runParams)
+  {
+    this.stateManager = stateManager;
+    this.strategy = runParams.getBalancerStrategy();
+    this.cluster = runParams.getDruidCluster();
+    this.replicantLookup = runParams.getSegmentReplicantLookup();
+    this.maxLoadQueueSize = runParams.getCoordinatorDynamicConfig()
+                                     .getMaxSegmentsInNodeLoadingQueue();
+  }
+
+  public CoordinatorStats getStats()
+  {
+    return stats;
+  }
+
+  /**
+   * Moves the given segment between two servers of the same tier.
+   * <p>
+   * See if we can move balancing here.
+   */
+  public boolean moveSegment(DataSegment segment, ServerHolder fromServer, ServerHolder toServer)
+  {
+    if (!fromServer.getServer().getTier().equals(toServer.getServer().getTier())) {
+      return false;
+    }
+
+    // fromServer must be loading or serving the segment
+    // and toServer must be able to load it
+    final SegmentState stateOnSrc = fromServer.getSegmentState(segment);
+    if ((stateOnSrc != SegmentState.LOADING && stateOnSrc != SegmentState.LOADED)
+        || !canLoadSegment(toServer, segment)) {
+      return false;
+    }
+
+    final boolean cancelSuccess = stateOnSrc == SegmentState.LOADING
+                                  && stateManager.cancelOperation(SegmentState.LOADING, segment, fromServer);
+
+    if (cancelSuccess) {
+      int loadedCountOnTier = replicantLookup
+          .getLoadedReplicants(segment.getId(), toServer.getServer().getTier());
+      stateManager.loadSegment(segment, toServer, loadedCountOnTier < 1);
+    } else {
+      return stateManager.moveSegment(segment, fromServer, toServer);
+    }
+
+    return true;
+  }
+
+  /**
+   * Queues load or drop of replicas of the given segment to achieve the
+   * target replication level in all the tiers.
+   */
+  public void updateReplicas(DataSegment segment, Map<String, Integer> tierToReplicaCount)
+  {
+    // Handle every target tier
+    tierToReplicaCount.forEach((tier, numReplicas) -> {
+      updateReplicasOnTier(segment, tier, tierToReplicaCount.get(tier));
+      stats.addToTieredStat(CoordinatorStats.REQUIRED_CAPACITY, tier, segment.getSize() * numReplicas);
+    });
+
+    // Find the minimum number of segments required for fault tolerance
+    final int totalTargetReplicas = tierToReplicaCount.values().stream()
+                                                      .reduce(0, Integer::sum);
+    final int minLoadedSegments = totalTargetReplicas > 1 ? 2 : 1;
+
+    // Drop segment from unneeded tiers if requirement is met across target tiers
+    int loadedTargetReplicas = 0;
+    final Set<String> targetTiers = tierToReplicaCount.keySet();
+    for (String tier : targetTiers) {
+      loadedTargetReplicas += replicantLookup.getLoadedReplicants(segment.getId(), tier);
+    }
+    if (loadedTargetReplicas < minLoadedSegments) {
+      return;
+    }
+
+    final Set<String> dropTiers = Sets.newHashSet(cluster.getTierNames());
+    dropTiers.removeAll(targetTiers);
+    for (String dropTier : dropTiers) {
+      updateReplicasOnTier(segment, dropTier, 0);
+    }
+  }
+
+  /**
+   * Broadcasts the given segment to all servers that are broadcast targets and
+   * queues a drop of the segment from decommissioning servers.
+   */
+  public void broadcastSegment(DataSegment segment)
+  {
+    int assignedCount = 0;
+    int droppedCount = 0;
+    for (ServerHolder server : cluster.getAllServers()) {
+      // Ignore servers which are not broadcast targets
+      if (!server.getServer().getType().isSegmentBroadcastTarget()) {
+        continue;
+      }
+
+      if (server.isDecommissioning()) {
+        droppedCount += dropBroadcastSegment(segment, server) ? 1 : 0;
+      } else {
+        assignedCount += loadBroadcastSegment(segment, server) ? 1 : 0;
+      }
+    }
+
+    if (assignedCount > 0) {
+      stats.addToDataSourceStat(CoordinatorStats.BROADCAST_LOADS, segment.getDataSource(), assignedCount);
+    }
+    if (droppedCount > 0) {
+      stats.addToDataSourceStat(CoordinatorStats.BROADCAST_DROPS, segment.getDataSource(), droppedCount);
+    }
+  }
+
+  /**
+   * Marks the given segment as unused.
+   */
+  public void deleteSegment(DataSegment segment)
+  {
+    stateManager.deleteSegment(segment);
+    stats.addToGlobalStat(CoordinatorStats.DELETED_SEGMENTS, 1);
+  }
+
+  /**
+   * Checks if the server can load the given segment.
+   * <p>
+   * A load is possible only if the server meets all of the following criteria:
+   * <ul>
+   *   <li>is not already serving or loading the segment</li>
+   *   <li>is not being decommissioned</li>
+   *   <li>has not already exceeded the load queue limit in this run</li>
+   *   <li>has available disk space</li>
+   * </ul>
+   */
+  public boolean canLoadSegment(ServerHolder server, DataSegment segment)
+  {
+    return server.canLoadSegment(segment)

Review Comment:
   It does and I did start with that, in fact. But then did this because of laziness in fixing the tests to pass the config `maxSegmentsInNodeLoadingQueue` to every `ServerHolder`. 😅
   
   I will fix this up.



-- 
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: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] AmatyaAvadhanula commented on a diff in pull request #13197: Segment loading: Allow cancellation and prioritization of load queue items

Posted by GitBox <gi...@apache.org>.
AmatyaAvadhanula commented on code in PR #13197:
URL: https://github.com/apache/druid/pull/13197#discussion_r993502484


##########
server/src/main/java/org/apache/druid/server/coordinator/TierLoadingState.java:
##########
@@ -0,0 +1,72 @@
+/*
+ * 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;
+
+import org.apache.druid.timeline.SegmentId;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * Contains segments being loaded or replicated in a tier.
+ */
+public class TierLoadingState
+{
+  private int lifetime;

Review Comment:
   I think that a lifetime is associated with a single segment action currently and adding this in QueuedSegment would be the analogous change.
   Could you please explain how a lifetime at the Tier level would help?



-- 
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: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] imply-cheddar commented on a diff in pull request #13197: Segment loading: Allow cancellation and prioritization of load queue items

Posted by GitBox <gi...@apache.org>.
imply-cheddar commented on code in PR #13197:
URL: https://github.com/apache/druid/pull/13197#discussion_r997785885


##########
server/src/main/java/org/apache/druid/server/coordinator/RandomBalancerStrategy.java:
##########
@@ -26,24 +26,20 @@
 import java.util.Iterator;
 import java.util.List;
 import java.util.NavigableSet;
-import java.util.concurrent.ThreadLocalRandom;
 import java.util.stream.Collectors;
 
 public class RandomBalancerStrategy implements BalancerStrategy
 {
   @Override
-  public ServerHolder findNewSegmentHomeReplicator(DataSegment proposalSegment, List<ServerHolder> serverHolders)
+  public Iterator<ServerHolder> findNewSegmentHomeReplicator(DataSegment proposalSegment, List<ServerHolder> serverHolders)

Review Comment:
   I don't feel overly strong about it.  Was more of a stream-of-conciousness/feeling comment.  So now that you've thought about it directly and like Iterator better, let's keep that.



-- 
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: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] kfaraz commented on a diff in pull request #13197: Segment loading: Allow cancellation and prioritization of load queue items

Posted by GitBox <gi...@apache.org>.
kfaraz commented on code in PR #13197:
URL: https://github.com/apache/druid/pull/13197#discussion_r997837747


##########
server/src/main/java/org/apache/druid/server/coordinator/SegmentStateManager.java:
##########
@@ -0,0 +1,261 @@
+/*
+ * 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;
+
+import com.google.inject.Inject;
+import org.apache.druid.client.ServerInventoryView;
+import org.apache.druid.java.util.emitter.EmittingLogger;
+import org.apache.druid.metadata.SegmentsMetadataManager;
+import org.apache.druid.timeline.DataSegment;
+
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * Manages state of segments being loaded.
+ */
+public class SegmentStateManager
+{
+  private static final EmittingLogger log = new EmittingLogger(SegmentStateManager.class);
+
+  private final LoadQueueTaskMaster taskMaster;
+  private final ServerInventoryView serverInventoryView;
+  private final SegmentsMetadataManager segmentsMetadataManager;
+
+  private final ConcurrentHashMap<String, TierLoadingState> currentlyMovingSegments =
+      new ConcurrentHashMap<>();
+  private final ConcurrentHashMap<String, TierLoadingState> currentlyReplicatingSegments
+      = new ConcurrentHashMap<>();
+
+  @Inject
+  public SegmentStateManager(
+      ServerInventoryView serverInventoryView,
+      SegmentsMetadataManager segmentsMetadataManager,
+      LoadQueueTaskMaster taskMaster
+  )
+  {
+    this.serverInventoryView = serverInventoryView;
+    this.segmentsMetadataManager = segmentsMetadataManager;
+    this.taskMaster = taskMaster;
+  }
+
+  /**
+   * Queues load of a replica of the segment on the given server.
+   */
+  public boolean loadSegment(
+      DataSegment segment,
+      ServerHolder server,
+      boolean isPrimary,
+      ReplicationThrottler throttler
+  )
+  {
+    final String tier = server.getServer().getTier();
+    final LoadPeonCallback callback;
+    if (isPrimary) {
+      // Primary replicas are not subject to throttling
+      callback = null;
+    } else if (canLoadReplica(tier, throttler)) {
+      throttler.incrementAssignedReplicas(tier);
+
+      final TierLoadingState replicatingInTier = currentlyReplicatingSegments
+          .computeIfAbsent(tier, t -> new TierLoadingState(throttler.getMaxLifetime()));
+      replicatingInTier.addSegment(segment.getId(), server.getServer().getHost());
+      callback = success -> replicatingInTier.removeSegment(segment.getId());
+    } else {
+      throttler.incrementThrottledReplicas(tier);
+      return false;
+    }
+
+    try {
+      if (!server.startOperation(segment, SegmentState.LOADING)) {
+        return false;
+      }
+
+      server.getPeon().loadSegment(
+          segment,
+          isPrimary ? SegmentAction.LOAD_AS_PRIMARY : SegmentAction.LOAD_AS_REPLICA,
+          callback
+      );
+      return true;
+    }
+    catch (Exception e) {
+      server.cancelOperation(segment, SegmentState.LOADING);
+      return false;
+    }
+  }
+
+  public boolean dropSegment(DataSegment segment, ServerHolder server)
+  {
+    try {
+      if (!server.startOperation(segment, SegmentState.DROPPING)) {
+        return false;
+      }
+
+      server.getPeon().dropSegment(segment, null);
+      return true;
+    }
+    catch (Exception e) {
+      server.cancelOperation(segment, SegmentState.DROPPING);
+      return false;
+    }
+  }
+
+  public boolean moveSegment(
+      DataSegment segment,
+      ServerHolder fromServer,
+      ServerHolder toServer,
+      int maxLifetimeInBalancingQueue
+  )
+  {
+    final TierLoadingState segmentsMovingInTier = currentlyMovingSegments.computeIfAbsent(
+        toServer.getServer().getTier(),
+        t -> new TierLoadingState(maxLifetimeInBalancingQueue)
+    );
+    final LoadQueuePeon fromServerPeon = fromServer.getPeon();
+    final LoadPeonCallback moveFinishCallback = success -> {
+      fromServerPeon.unmarkSegmentToDrop(segment);
+      segmentsMovingInTier.removeSegment(segment.getId());
+    };
+
+    // mark segment to drop before it is actually loaded on server
+    // to be able to account for this information in BalancerStrategy immediately
+    toServer.startOperation(segment, SegmentState.MOVING_TO);
+    fromServerPeon.markSegmentToDrop(segment);
+    segmentsMovingInTier.addSegment(segment.getId(), fromServer.getServer().getHost());
+
+    final LoadQueuePeon toServerPeon = toServer.getPeon();
+    final String toServerName = toServer.getServer().getName();
+    try {
+      toServerPeon.loadSegment(
+          segment,
+          SegmentAction.MOVE_TO,
+          success -> {
+            // Drop segment only if:
+            // (1) segment load was successful on toServer
+            // AND (2) segment not already queued for drop on fromServer
+            // AND (3a) loading is http-based
+            //     OR (3b) inventory shows segment loaded on toServer
+
+            // Do not check the inventory with http loading as the HTTP
+            // response is enough to determine load success or failure
+            if (success

Review Comment:
   Yes, it happens through the `moveFinishCallback.execute()` in the `else` block.



-- 
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: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] kfaraz commented on a diff in pull request #13197: Segment loading: Allow cancellation and prioritization of load queue items

Posted by GitBox <gi...@apache.org>.
kfaraz commented on code in PR #13197:
URL: https://github.com/apache/druid/pull/13197#discussion_r1010208794


##########
server/src/main/java/org/apache/druid/server/coordinator/SegmentLoader.java:
##########
@@ -0,0 +1,460 @@
+/*
+ * 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;
+
+import com.google.common.collect.Sets;
+import org.apache.druid.java.util.emitter.EmittingLogger;
+import org.apache.druid.timeline.DataSegment;
+
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+
+/**
+ * Used by the coordinator in each run for segment loading, dropping, balancing
+ * and broadcasting.
+ * <p>
+ * An instance of this class is freshly created for each coordinator run.
+ */
+public class SegmentLoader
+{
+  private static final EmittingLogger log = new EmittingLogger(SegmentLoader.class);
+
+  private final SegmentStateManager stateManager;
+  private final DruidCluster cluster;
+  private final CoordinatorStats stats = new CoordinatorStats();
+  private final SegmentReplicantLookup replicantLookup;
+  private final ReplicationThrottler replicationThrottler;
+  private final BalancerStrategy strategy;
+
+  private final Set<String> emptyTiers = new HashSet<>();
+
+  public SegmentLoader(
+      SegmentStateManager stateManager,
+      DruidCluster cluster,
+      SegmentReplicantLookup replicantLookup,
+      ReplicationThrottler replicationThrottler,
+      BalancerStrategy strategy
+  )
+  {
+    this.cluster = cluster;
+    this.strategy = strategy;
+    this.stateManager = stateManager;
+    this.replicantLookup = replicantLookup;
+    this.replicationThrottler = replicationThrottler;
+  }
+
+  public CoordinatorStats getStats()
+  {
+    return stats;
+  }
+
+  public void makeAlerts()
+  {
+    if (!emptyTiers.isEmpty()) {
+      log.makeAlert("Tiers %s have no servers! Check your cluster configuration.", emptyTiers).emit();
+    }
+  }
+
+  /**
+   * Moves the given segment between two servers of the same tier.
+   * <p>
+   * See if we can move balancing here.
+   */
+  public boolean moveSegment(DataSegment segment, ServerHolder fromServer, ServerHolder toServer)
+  {
+    final String tier = toServer.getServer().getTier();
+    if (!fromServer.getServer().getTier().equals(tier)) {
+      return false;
+    }
+
+    if (fromServer.isServingSegment(segment)) {
+      // Segment is loaded on fromServer, move it to toServer
+      return stateManager.moveSegment(segment, fromServer, toServer, replicationThrottler.getMaxLifetime());
+    } else if (!fromServer.isLoadingSegment(segment)) {

Review Comment:
   Yeah, it does make reading a little difficult. Fixed this here and a couple other places where I had used them.



##########
server/src/main/java/org/apache/druid/server/coordinator/SegmentHolder.java:
##########
@@ -0,0 +1,172 @@
+/*
+ * 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;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Ordering;
+import org.apache.druid.server.coordination.DataSegmentChangeRequest;
+import org.apache.druid.server.coordination.SegmentChangeRequestDrop;
+import org.apache.druid.server.coordination.SegmentChangeRequestLoad;
+import org.apache.druid.timeline.DataSegment;
+
+import javax.annotation.Nullable;
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Objects;
+import java.util.concurrent.atomic.AtomicLong;
+
+/**
+ * Represents a segment queued for a load or drop operation in a LoadQueuePeon.
+ * <p>
+ * Requests are naturally ordered using the {@link #COMPARE_ACTION_THEN_INTERVAL}.
+ */
+public class SegmentHolder implements Comparable<SegmentHolder>
+{
+  /**
+   * Orders segment requests:
+   * <ul>
+   *   <li>first by action: all drops, then all loads, then all moves</li>
+   *   <li>then by interval: newest segments first</li>
+   * </ul>
+   */
+  public static final Comparator<SegmentHolder> COMPARE_ACTION_THEN_INTERVAL =
+      Ordering.explicit(SegmentAction.DROP, SegmentAction.LOAD, SegmentAction.REPLICATE, SegmentAction.MOVE_TO)
+              .onResultOf(SegmentHolder::getAction)
+              .compound(DruidCoordinator.SEGMENT_COMPARATOR_RECENT_FIRST.onResultOf(SegmentHolder::getSegment));
+
+  private final DataSegment segment;
+  private final DataSegmentChangeRequest changeRequest;
+  private final SegmentAction action;
+
+  // Guaranteed to store only non-null elements
+  private final List<LoadPeonCallback> callbacks = new ArrayList<>();
+  private final AtomicLong firstRequestMillis = new AtomicLong(0);
+
+  SegmentHolder(
+      DataSegment segment,
+      SegmentAction action,
+      @Nullable LoadPeonCallback callback
+  )
+  {
+    this.segment = segment;
+    this.action = action;
+    this.changeRequest = (action == SegmentAction.DROP)
+                         ? new SegmentChangeRequestDrop(segment)
+                         : new SegmentChangeRequestLoad(segment);
+    if (callback != null) {
+      callbacks.add(callback);
+    }
+  }
+
+  public DataSegment getSegment()
+  {
+    return segment;
+  }
+
+  public SegmentAction getAction()
+  {
+    return action;
+  }
+
+  public boolean isLoad()
+  {
+    return action != SegmentAction.DROP;
+  }
+
+  public DataSegmentChangeRequest getChangeRequest()
+  {
+    return changeRequest;
+  }
+
+  public String getSegmentIdentifier()
+  {
+    return segment.getId().toString();
+  }
+
+  public void addCallback(@Nullable LoadPeonCallback callback)
+  {
+    if (callback != null) {
+      synchronized (callbacks) {
+        callbacks.add(callback);
+      }
+    }
+  }
+
+  /**
+   * Returns an immutable copy of all non-null callbacks for this queued segment.
+   */
+  public List<LoadPeonCallback> getCallbacks()
+  {
+    synchronized (callbacks) {
+      return ImmutableList.copyOf(callbacks);
+    }
+  }
+
+  public void markRequestSentToServer()
+  {
+    firstRequestMillis.compareAndSet(0L, System.currentTimeMillis());
+  }
+
+  public boolean isRequestSentToServer()
+  {
+    return firstRequestMillis.get() > 0;
+  }
+
+  public long getMillisSinceFirstRequestToServer()

Review Comment:
   Thanks for the suggestion!



##########
server/src/main/java/org/apache/druid/server/coordinator/SegmentAction.java:
##########
@@ -0,0 +1,33 @@
+/*
+ * 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;
+
+/**
+ * Represents actions that can be performed on a server for a single segment.
+ * <p>
+ * The different action types can be used to prioritize items in a LoadQueuePeon.
+ */
+public enum SegmentAction
+{
+  DROP,
+  LOAD,
+  REPLICATE,
+  MOVE_TO,

Review Comment:
   Added.



-- 
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: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] kfaraz commented on a diff in pull request #13197: Segment loading: Allow cancellation and prioritization of load queue items

Posted by "kfaraz (via GitHub)" <gi...@apache.org>.
kfaraz commented on code in PR #13197:
URL: https://github.com/apache/druid/pull/13197#discussion_r991362374


##########
server/src/main/java/org/apache/druid/server/coordinator/duty/EmitClusterStatsAndMetrics.java:
##########
@@ -322,18 +310,14 @@ private void emitStatsForHistoricalManagementDuties(DruidCluster cluster, Coordi
           );
         });
 
-    coordinator.computeNumsUnavailableUsedSegmentsPerDataSource().object2IntEntrySet().forEach(
-        (final Object2IntMap.Entry<String> entry) -> {
-          final String dataSource = entry.getKey();
-          final int numUnavailableUsedSegmentsInDataSource = entry.getIntValue();
-          emitter.emit(
-              new ServiceMetricEvent.Builder()
-                  .setDimension(DruidMetrics.DUTY_GROUP, groupName)
-                  .setDimension(DruidMetrics.DATASOURCE, dataSource).build(
-                  "segment/unavailable/count", numUnavailableUsedSegmentsInDataSource
-              )
-          );
-        }
+    coordinator.computeNumsUnavailableUsedSegmentsPerDataSource().forEach(

Review Comment:
   Style change: Use simpler iterator.



##########
server/src/main/java/org/apache/druid/server/coordinator/duty/RunRules.java:
##########
@@ -46,38 +45,16 @@ public class RunRules implements CoordinatorDuty
   private static final EmittingLogger log = new EmittingLogger(RunRules.class);
   private static final int MAX_MISSING_RULES = 10;
 
-  private final ReplicationThrottler replicatorThrottler;
+  private final SegmentStateManager stateManager;
 
-  private final DruidCoordinator coordinator;
-
-  public RunRules(DruidCoordinator coordinator)

Review Comment:
   Change summary:
   Move replication throttling logic to `SegmentStateManager`.
   Move check on `maxNonPrimaryReplicantsToLoad` to `ReplicationThrottler` (requires revisit).



-- 
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: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] github-code-scanning[bot] commented on a diff in pull request #13197: Segment loading: Allow cancellation and prioritization of load queue items

Posted by "github-code-scanning[bot] (via GitHub)" <gi...@apache.org>.
github-code-scanning[bot] commented on code in PR #13197:
URL: https://github.com/apache/druid/pull/13197#discussion_r1182266785


##########
server/src/test/java/org/apache/druid/server/coordinator/rules/LoadRuleTest.java:
##########
@@ -117,397 +98,276 @@
   @Before
   public void setUp()
   {
-    EmittingLogger.registerEmitter(EMITTER);
-    EMITTER.start();
-    throttler = EasyMock.createMock(ReplicationThrottler.class);
-
     exec = MoreExecutors.listeningDecorator(Execs.multiThreaded(1, "LoadRuleTest-%d"));
     balancerStrategy = new CostBalancerStrategyFactory().createBalancerStrategy(exec);
     cachingCostBalancerStrategy = new CachingCostBalancerStrategy(ClusterCostCache.builder().build(), exec);
 
     mockBalancerStrategy = EasyMock.createMock(BalancerStrategy.class);
+    loadQueueManager = new SegmentLoadQueueManager(null, null, null);
   }
 
   @After
-  public void tearDown() throws Exception
+  public void tearDown()
   {
     exec.shutdown();
-    EMITTER.close();
   }
 
   @Test
   public void testLoad()
   {
-    EasyMock.expect(throttler.canCreateReplicant(EasyMock.anyString())).andReturn(true).anyTimes();
-
     final LoadQueuePeon mockPeon = createEmptyPeon();
-    mockPeon.loadSegment(EasyMock.anyObject(), EasyMock.anyObject());
+    mockPeon.loadSegment(EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyObject());
     EasyMock.expectLastCall().atLeastOnce();
 
-    LoadRule rule = createLoadRule(ImmutableMap.of(
-        "hot", 1,
-        DruidServer.DEFAULT_TIER, 2
-    ));
-
-    final DataSegment segment = createDataSegment("foo");
-
-    throttler.registerReplicantCreation(DruidServer.DEFAULT_TIER, segment.getId(), "hostNorm");
-    EasyMock.expectLastCall().once();
-
     if (!useRoundRobinAssignment) {
-      EasyMock.expect(mockBalancerStrategy.findNewSegmentHomeReplicator(EasyMock.anyObject(), EasyMock.anyObject()))
+      EasyMock.expect(mockBalancerStrategy.findServerToLoadSegment(EasyMock.anyObject(), EasyMock.anyObject()))
               .andDelegateTo(balancerStrategy)
-              .times(3);
+              .times(2);
     }
+    EasyMock.replay(mockPeon, mockBalancerStrategy);
 
-    EasyMock.replay(throttler, mockPeon, mockBalancerStrategy);
-
-    DruidCluster druidCluster = DruidClusterBuilder
-        .newBuilder()
-        .addTier(
-            "hot",
-            new ServerHolder(
-                new DruidServer("serverHot", "hostHot", null, 1000, ServerType.HISTORICAL, "hot", 1)
-                    .toImmutableDruidServer(),
-                mockPeon
-            )
-        )
-        .addTier(
-            DruidServer.DEFAULT_TIER,
-            new ServerHolder(
-                new DruidServer(
-                    "serverNorm",
-                    "hostNorm",
-                    null,
-                    1000,
-                    ServerType.HISTORICAL,
-                    DruidServer.DEFAULT_TIER,
-                    0
-                ).toImmutableDruidServer(),
-                mockPeon
-            )
-        )
+    DruidCluster druidCluster = DruidCluster
+        .builder()
+        .addTier(Tier.T1, createServerHolder(Tier.T1, mockPeon, false))
+        .addTier(Tier.T2, createServerHolder(Tier.T2, mockPeon, false))
         .build();
 
-    CoordinatorStats stats = rule.run(null, makeCoordinatorRuntimeParams(druidCluster, segment), segment);
+    final DataSegment segment = createDataSegment(DS_WIKI);
+    LoadRule rule = loadForever(ImmutableMap.of(Tier.T1, 1, Tier.T2, 2));
+    CoordinatorRunStats stats = runRuleAndGetStats(rule, segment, druidCluster);
+
+    Assert.assertEquals(1L, stats.getSegmentStat(Stats.Segments.ASSIGNED, Tier.T1, DS_WIKI));
+    Assert.assertEquals(1L, stats.getSegmentStat(Stats.Segments.ASSIGNED, Tier.T2, DS_WIKI));
 
-    Assert.assertEquals(1L, stats.getTieredStat(LoadRule.ASSIGNED_COUNT, "hot"));
-    Assert.assertEquals(1L, stats.getTieredStat(LoadRule.ASSIGNED_COUNT, DruidServer.DEFAULT_TIER));
+    EasyMock.verify(mockPeon, mockBalancerStrategy);
+  }
+
+  private CoordinatorRunStats runRuleAndGetStats(
+      LoadRule rule,
+      DataSegment segment,
+      DruidCluster cluster
+  )
+  {
+    return runRuleAndGetStats(rule, segment, makeCoordinatorRuntimeParams(cluster, segment));
+  }
 
-    EasyMock.verify(throttler, mockPeon, mockBalancerStrategy);
+  private CoordinatorRunStats runRuleAndGetStats(
+      LoadRule rule,
+      DataSegment segment,
+      DruidCoordinatorRuntimeParams params
+  )
+  {
+    final StrategicSegmentAssigner segmentAssigner = params.getSegmentAssigner();
+    rule.run(segment, segmentAssigner);
+    return segmentAssigner.getStats();
   }
 
   private DruidCoordinatorRuntimeParams makeCoordinatorRuntimeParams(
       DruidCluster druidCluster,
       DataSegment... usedSegments
   )
   {
-    return CoordinatorRuntimeParamsTestHelpers
-        .newBuilder()
-        .withDruidCluster(druidCluster)
-        .withSegmentReplicantLookup(SegmentReplicantLookup.make(druidCluster, false))
-        .withReplicationManager(throttler)
-        .withDynamicConfigs(
-            CoordinatorDynamicConfig
-                .builder()
-                .withUseRoundRobinSegmentAssignment(useRoundRobinAssignment)
-                .build()
-        )
-        .withRoundRobinServerSelector(useRoundRobinAssignment ? new RoundRobinServerSelector(druidCluster) : null)
-        .withBalancerStrategy(mockBalancerStrategy)
-        .withUsedSegmentsInTest(usedSegments)
-        .build();
+    return makeCoordinatorRuntimeParams(druidCluster, false, usedSegments);
   }
 
-  private DruidCoordinatorRuntimeParams makeCoordinatorRuntimeParamsWithLoadReplicationOnTimeout(
+  private DruidCoordinatorRuntimeParams makeCoordinatorRuntimeParams(
       DruidCluster druidCluster,
+      boolean replicateAfterLoadTimeout,

Review Comment:
   ## Useless parameter
   
   The parameter 'replicateAfterLoadTimeout' is never used.
   
   [Show more details](https://github.com/apache/druid/security/code-scanning/4913)



-- 
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: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] kfaraz commented on a diff in pull request #13197: Segment loading: Allow cancellation and prioritization of load queue items

Posted by "kfaraz (via GitHub)" <gi...@apache.org>.
kfaraz commented on code in PR #13197:
URL: https://github.com/apache/druid/pull/13197#discussion_r1231758154


##########
server/src/main/java/org/apache/druid/server/coordinator/CoordinatorDynamicConfig.java:
##########
@@ -194,21 +211,46 @@ public CoordinatorDynamicConfig(
           + "reflect this configuration being added to Druid in a recent release. Druid is defaulting the value "
           + "to the Druid default of %d. It is recommended that you re-submit your dynamic config with your "
           + "desired value for maxNonPrimaryReplicantsToLoad",
-          Builder.DEFAULT_MAX_NON_PRIMARY_REPLICANTS_TO_LOAD
+          Defaults.MAX_NON_PRIMARY_REPLICANTS_TO_LOAD
       );
-      maxNonPrimaryReplicantsToLoad = Builder.DEFAULT_MAX_NON_PRIMARY_REPLICANTS_TO_LOAD;
+      maxNonPrimaryReplicantsToLoad = Defaults.MAX_NON_PRIMARY_REPLICANTS_TO_LOAD;
     }
     Preconditions.checkArgument(
         maxNonPrimaryReplicantsToLoad >= 0,
         "maxNonPrimaryReplicantsToLoad must be greater than or equal to 0."
     );
     this.maxNonPrimaryReplicantsToLoad = maxNonPrimaryReplicantsToLoad;
 
-    if (useRoundRobinSegmentAssignment == null) {
-      this.useRoundRobinSegmentAssignment = Builder.DEFAULT_USE_ROUND_ROBIN_ASSIGNMENT;
-    } else {
-      this.useRoundRobinSegmentAssignment = useRoundRobinSegmentAssignment;
+    this.useRoundRobinSegmentAssignment = Builder.valueOrDefault(
+        useRoundRobinSegmentAssignment,
+        Defaults.USE_ROUND_ROBIN_ASSIGNMENT
+    );
+    this.debugDimensions = debugDimensions;
+    this.validDebugDimensions = validateDebugDimensions(debugDimensions);
+  }
+
+  private Map<Dimension, String> validateDebugDimensions(Map<String, String> debugDimensions)
+  {
+    final Map<Dimension, String> validDebugDimensions = new EnumMap<>(Dimension.class);
+    if (debugDimensions == null || debugDimensions.isEmpty()) {
+      return validDebugDimensions;
+    }
+
+    final Map<String, Dimension> nameToDimension = new HashMap<>();
+    for (Dimension dimension : Dimension.values()) {
+      nameToDimension.put(dimension.reportedName(), dimension);
     }
+
+    debugDimensions.forEach(
+        (dimensionName, value) -> {
+          Dimension dimension = nameToDimension.get(dimensionName);
+          if (dimension != null && value != null) {
+            validDebugDimensions.put(dimension, value);
+          }
+        }
+    );

Review Comment:
   Don't know why I had done it in such a round about way in the first place 😅 , thanks for pointing this out!



-- 
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: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] kfaraz commented on a diff in pull request #13197: Segment loading: Allow cancellation and prioritization of load queue items

Posted by "kfaraz (via GitHub)" <gi...@apache.org>.
kfaraz commented on code in PR #13197:
URL: https://github.com/apache/druid/pull/13197#discussion_r1232279827


##########
server/src/main/java/org/apache/druid/server/coordinator/CoordinatorDynamicConfig.java:
##########
@@ -428,66 +489,36 @@ public boolean equals(Object o)
 
     CoordinatorDynamicConfig that = (CoordinatorDynamicConfig) o;
 
-    if (leadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments !=
-        that.leadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments) {
-      return false;
-    }
-    if (mergeBytesLimit != that.mergeBytesLimit) {
-      return false;
-    }
-    if (mergeSegmentsLimit != that.mergeSegmentsLimit) {
-      return false;
-    }
-    if (maxSegmentsToMove != that.maxSegmentsToMove) {
-      return false;
-    }
-    if (percentOfSegmentsToConsiderPerMove != that.percentOfSegmentsToConsiderPerMove) {
-      return false;
-    }
-    if (useBatchedSegmentSampler != that.useBatchedSegmentSampler) {
-      return false;
-    }
-    if (replicantLifetime != that.replicantLifetime) {
-      return false;
-    }
-    if (replicationThrottleLimit != that.replicationThrottleLimit) {
-      return false;
-    }
-    if (balancerComputeThreads != that.balancerComputeThreads) {
-      return false;
-    }
-    if (emitBalancingStats != that.emitBalancingStats) {
-      return false;
-    }
-    if (maxSegmentsInNodeLoadingQueue != that.maxSegmentsInNodeLoadingQueue) {
-      return false;
-    }
-    if (!Objects.equals(specificDataSourcesToKillUnusedSegmentsIn, that.specificDataSourcesToKillUnusedSegmentsIn)) {
-      return false;
-    }
-    if (!Objects.equals(dataSourcesToNotKillStalePendingSegmentsIn, that.dataSourcesToNotKillStalePendingSegmentsIn)) {
-      return false;
-    }
-    if (!Objects.equals(decommissioningNodes, that.decommissioningNodes)) {
-      return false;
-    }
-    if (pauseCoordination != that.pauseCoordination) {
-      return false;
-    }
-    if (replicateAfterLoadTimeout != that.replicateAfterLoadTimeout) {
-      return false;
-    }
-    if (maxNonPrimaryReplicantsToLoad != that.maxNonPrimaryReplicantsToLoad) {
-      return false;
-    }
-    return decommissioningMaxPercentOfMaxSegmentsToMove == that.decommissioningMaxPercentOfMaxSegmentsToMove;
+    return markSegmentAsUnusedDelayMillis == that.markSegmentAsUnusedDelayMillis
+           && mergeBytesLimit == that.mergeBytesLimit
+           && mergeSegmentsLimit == that.mergeSegmentsLimit
+           && maxSegmentsToMove == that.maxSegmentsToMove
+           && percentOfSegmentsToConsiderPerMove == that.percentOfSegmentsToConsiderPerMove
+           && decommissioningMaxPercentOfMaxSegmentsToMove == that.decommissioningMaxPercentOfMaxSegmentsToMove
+           && useBatchedSegmentSampler == that.useBatchedSegmentSampler
+           && balancerComputeThreads == that.balancerComputeThreads
+           && emitBalancingStats == that.emitBalancingStats
+           && replicantLifetime == that.replicantLifetime
+           && replicationThrottleLimit == that.replicationThrottleLimit
+           && replicateAfterLoadTimeout == that.replicateAfterLoadTimeout
+           && maxSegmentsInNodeLoadingQueue == that.maxSegmentsInNodeLoadingQueue
+           && maxNonPrimaryReplicantsToLoad == that.maxNonPrimaryReplicantsToLoad
+           && useRoundRobinSegmentAssignment == that.useRoundRobinSegmentAssignment
+           && pauseCoordination == that.pauseCoordination
+           && Objects.equals(
+               specificDataSourcesToKillUnusedSegmentsIn,
+               that.specificDataSourcesToKillUnusedSegmentsIn)
+           && Objects.equals(
+               dataSourcesToNotKillStalePendingSegmentsIn,
+               that.dataSourcesToNotKillStalePendingSegmentsIn)
+           && Objects.equals(decommissioningNodes, that.decommissioningNodes);

Review Comment:
   I was even thinking of getting rid of the `equals` and `hashCode` in this class. Don't see them serving any purpose. Even the tests do an item-by-item comparison.



-- 
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: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] github-code-scanning[bot] commented on a diff in pull request #13197: Segment loading: Allow cancellation and prioritization of load queue items

Posted by "github-code-scanning[bot] (via GitHub)" <gi...@apache.org>.
github-code-scanning[bot] commented on code in PR #13197:
URL: https://github.com/apache/druid/pull/13197#discussion_r1233538197


##########
server/src/main/java/org/apache/druid/server/coordinator/loading/SegmentReplicationStatus.java:
##########
@@ -0,0 +1,84 @@
+/*
+ * 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.loading;
+
+import com.google.common.collect.ImmutableMap;
+import it.unimi.dsi.fastutil.objects.Object2LongMap;
+import it.unimi.dsi.fastutil.objects.Object2LongOpenHashMap;
+import org.apache.druid.timeline.DataSegment;
+import org.apache.druid.timeline.SegmentId;
+
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * An immutable object that contains information about the under-replicated
+ * or unavailable status of all used segments. This state is recomputed by
+ * the {@link StrategicSegmentAssigner} in every run.
+ */
+public class SegmentReplicationStatus
+{
+  private final Map<SegmentId, SegmentReplicaCount> totalReplicaCounts;
+  private final Map<SegmentId, Map<String, SegmentReplicaCount>> replicaCountsInTier;
+
+  public SegmentReplicationStatus(Map<SegmentId, Map<String, SegmentReplicaCount>> replicaCountsInTier)
+  {
+    this.replicaCountsInTier = ImmutableMap.copyOf(replicaCountsInTier);
+
+    final Map<SegmentId, SegmentReplicaCount> totalReplicaCounts = new HashMap<>();

Review Comment:
   ## Possible confusion of local and field
   
   Potentially confusing name: [SegmentReplicationStatus](1) also refers to field [totalReplicaCounts](2) (as this.totalReplicaCounts).
   
   [Show more details](https://github.com/apache/druid/security/code-scanning/5119)



##########
server/src/main/java/org/apache/druid/server/coordinator/loading/SegmentLoadingConfig.java:
##########
@@ -0,0 +1,153 @@
+/*
+ * 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.loading;
+
+import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.server.coordinator.CoordinatorDynamicConfig;
+
+/**
+ * Contains recomputed configs from {@link CoordinatorDynamicConfig} based on
+ * whether {@link CoordinatorDynamicConfig#isSmartSegmentLoading} is enabled or not.
+ */
+public class SegmentLoadingConfig
+{
+  private static final Logger log = new Logger(SegmentLoadingConfig.class);
+
+  private final int maxSegmentsInLoadQueue;
+  private final int replicationThrottleLimit;
+  private final int maxReplicaAssignmentsInRun;
+  private final int maxLifetimeInLoadQueue;
+
+  private final int maxSegmentsToMove;
+  private final int percentDecommSegmentsToMove;
+
+  private final boolean useRoundRobinSegmentAssignment;
+  private final boolean emitBalancingStats;
+
+  /**
+   * Creates a new SegmentLoadingConfig with recomputed coordinator config values from
+   * based on whether {@link CoordinatorDynamicConfig#isSmartSegmentLoading()}
+   * is enabled or not.
+   */
+  public static SegmentLoadingConfig create(CoordinatorDynamicConfig dynamicConfig, int numUsedSegments)
+  {
+    if (dynamicConfig.isSmartSegmentLoading()) {
+      // Compute recommended values
+      // Impose a lower bound on both replicationThrottleLimit and maxSegmentsToMove
+      final int throttlePercentage = 2;
+      final int replicationThrottleLimit = Math.max(100, numUsedSegments * throttlePercentage / 100);
+
+      // Impose an upper bound on maxSegmentsToMove to ensure that coordinator
+      // run times are bounded. This limit can be relaxed as performance of
+      // the CostBalancerStrategy.computeCost() is improved.
+      final int maxSegmentsToMove = Math.min(1000, replicationThrottleLimit);
+
+      log.info(
+          "Smart segment loading is enabled. Recomputed replicationThrottleLimit"
+          + " [%,d] (%d%% of used segments [%,d]) and maxSegmentsToMove [%d].",
+          replicationThrottleLimit, throttlePercentage, numUsedSegments, maxSegmentsToMove
+      );
+
+      return new SegmentLoadingConfig(
+          0,
+          replicationThrottleLimit,
+          Integer.MAX_VALUE,
+          dynamicConfig.getReplicantLifetime(),
+          maxSegmentsToMove,
+          100,
+          true,
+          false
+      );
+    } else {
+      // Use the configured values
+      return new SegmentLoadingConfig(
+          dynamicConfig.getMaxSegmentsInNodeLoadingQueue(),
+          dynamicConfig.getReplicationThrottleLimit(),
+          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/5120)



##########
server/src/main/java/org/apache/druid/server/coordinator/loading/SegmentLoadingConfig.java:
##########
@@ -0,0 +1,153 @@
+/*
+ * 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.loading;
+
+import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.server.coordinator.CoordinatorDynamicConfig;
+
+/**
+ * Contains recomputed configs from {@link CoordinatorDynamicConfig} based on
+ * whether {@link CoordinatorDynamicConfig#isSmartSegmentLoading} is enabled or not.
+ */
+public class SegmentLoadingConfig
+{
+  private static final Logger log = new Logger(SegmentLoadingConfig.class);
+
+  private final int maxSegmentsInLoadQueue;
+  private final int replicationThrottleLimit;
+  private final int maxReplicaAssignmentsInRun;
+  private final int maxLifetimeInLoadQueue;
+
+  private final int maxSegmentsToMove;
+  private final int percentDecommSegmentsToMove;
+
+  private final boolean useRoundRobinSegmentAssignment;
+  private final boolean emitBalancingStats;
+
+  /**
+   * Creates a new SegmentLoadingConfig with recomputed coordinator config values from
+   * based on whether {@link CoordinatorDynamicConfig#isSmartSegmentLoading()}
+   * is enabled or not.
+   */
+  public static SegmentLoadingConfig create(CoordinatorDynamicConfig dynamicConfig, int numUsedSegments)
+  {
+    if (dynamicConfig.isSmartSegmentLoading()) {
+      // Compute recommended values
+      // Impose a lower bound on both replicationThrottleLimit and maxSegmentsToMove
+      final int throttlePercentage = 2;
+      final int replicationThrottleLimit = Math.max(100, numUsedSegments * throttlePercentage / 100);
+
+      // Impose an upper bound on maxSegmentsToMove to ensure that coordinator
+      // run times are bounded. This limit can be relaxed as performance of
+      // the CostBalancerStrategy.computeCost() is improved.
+      final int maxSegmentsToMove = Math.min(1000, replicationThrottleLimit);
+
+      log.info(
+          "Smart segment loading is enabled. Recomputed replicationThrottleLimit"
+          + " [%,d] (%d%% of used segments [%,d]) and maxSegmentsToMove [%d].",
+          replicationThrottleLimit, throttlePercentage, numUsedSegments, maxSegmentsToMove
+      );
+
+      return new SegmentLoadingConfig(
+          0,
+          replicationThrottleLimit,
+          Integer.MAX_VALUE,
+          dynamicConfig.getReplicantLifetime(),
+          maxSegmentsToMove,
+          100,
+          true,
+          false
+      );
+    } else {
+      // Use the configured values
+      return new SegmentLoadingConfig(
+          dynamicConfig.getMaxSegmentsInNodeLoadingQueue(),
+          dynamicConfig.getReplicationThrottleLimit(),
+          dynamicConfig.getMaxNonPrimaryReplicantsToLoad(),
+          dynamicConfig.getReplicantLifetime(),
+          dynamicConfig.getMaxSegmentsToMove(),
+          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/5121)



-- 
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: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] kfaraz commented on pull request #13197: Segment loading: Allow cancellation and prioritization of load queue items

Posted by GitBox <gi...@apache.org>.
kfaraz commented on PR #13197:
URL: https://github.com/apache/druid/pull/13197#issuecomment-1280169633

   @imply-cheddar , thanks a lot for your review! I have addressed your comments.
   More improvements to logging, metrics and docs will be included in a quick follow-up PR.


-- 
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: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] kfaraz commented on a diff in pull request #13197: Segment loading: Allow cancellation and prioritization of load queue items

Posted by GitBox <gi...@apache.org>.
kfaraz commented on code in PR #13197:
URL: https://github.com/apache/druid/pull/13197#discussion_r994062008


##########
server/src/main/java/org/apache/druid/server/coordinator/RandomBalancerStrategy.java:
##########
@@ -26,24 +26,20 @@
 import java.util.Iterator;
 import java.util.List;
 import java.util.NavigableSet;
-import java.util.concurrent.ThreadLocalRandom;
 import java.util.stream.Collectors;
 
 public class RandomBalancerStrategy implements BalancerStrategy
 {
   @Override
-  public ServerHolder findNewSegmentHomeReplicator(DataSegment proposalSegment, List<ServerHolder> serverHolders)
+  public Iterator<ServerHolder> findNewSegmentHomeReplicator(DataSegment proposalSegment, List<ServerHolder> serverHolders)

Review Comment:
   Sounds good.



-- 
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: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] kfaraz commented on a diff in pull request #13197: Segment loading: Allow cancellation and prioritization of load queue items

Posted by GitBox <gi...@apache.org>.
kfaraz commented on code in PR #13197:
URL: https://github.com/apache/druid/pull/13197#discussion_r994052485


##########
server/src/main/java/org/apache/druid/server/coordinator/SegmentLoader.java:
##########
@@ -0,0 +1,472 @@
+/*
+ * 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;
+
+import com.google.common.collect.Sets;
+import org.apache.druid.java.util.emitter.EmittingLogger;
+import org.apache.druid.timeline.DataSegment;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.EnumMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.stream.Collectors;
+
+/**
+ * Used by the coordinator in each run for segment loading, dropping, balancing
+ * and broadcasting.
+ * <p>
+ * An instance of this class is freshly created for each coordinator run.
+ */
+public class SegmentLoader
+{
+  private static final EmittingLogger log = new EmittingLogger(SegmentLoader.class);
+
+  private final SegmentStateManager stateManager;
+  private final DruidCluster cluster;
+  private final CoordinatorStats stats = new CoordinatorStats();
+  private final SegmentReplicantLookup replicantLookup;
+  private final BalancerStrategy strategy;
+  private final int maxLoadQueueSize;
+
+  public SegmentLoader(SegmentStateManager stateManager, DruidCoordinatorRuntimeParams runParams)
+  {
+    this.stateManager = stateManager;
+    this.strategy = runParams.getBalancerStrategy();
+    this.cluster = runParams.getDruidCluster();
+    this.replicantLookup = runParams.getSegmentReplicantLookup();
+    this.maxLoadQueueSize = runParams.getCoordinatorDynamicConfig()
+                                     .getMaxSegmentsInNodeLoadingQueue();
+  }
+
+  public CoordinatorStats getStats()
+  {
+    return stats;
+  }
+
+  /**
+   * Moves the given segment between two servers of the same tier.
+   * <p>
+   * See if we can move balancing here.
+   */
+  public boolean moveSegment(DataSegment segment, ServerHolder fromServer, ServerHolder toServer)
+  {
+    if (!fromServer.getServer().getTier().equals(toServer.getServer().getTier())) {
+      return false;
+    }
+
+    // fromServer must be loading or serving the segment
+    // and toServer must be able to load it
+    final SegmentState stateOnSrc = fromServer.getSegmentState(segment);
+    if ((stateOnSrc != SegmentState.LOADING && stateOnSrc != SegmentState.LOADED)
+        || !canLoadSegment(toServer, segment)) {
+      return false;
+    }
+
+    final boolean cancelSuccess = stateOnSrc == SegmentState.LOADING
+                                  && stateManager.cancelOperation(SegmentState.LOADING, segment, fromServer);
+
+    if (cancelSuccess) {
+      int loadedCountOnTier = replicantLookup
+          .getLoadedReplicants(segment.getId(), toServer.getServer().getTier());
+      stateManager.loadSegment(segment, toServer, loadedCountOnTier < 1);
+    } else {
+      return stateManager.moveSegment(segment, fromServer, toServer);
+    }
+
+    return true;
+  }
+
+  /**
+   * Queues load or drop of replicas of the given segment to achieve the
+   * target replication level in all the tiers.
+   */
+  public void updateReplicas(DataSegment segment, Map<String, Integer> tierToReplicaCount)
+  {
+    // Handle every target tier
+    tierToReplicaCount.forEach((tier, numReplicas) -> {
+      updateReplicasOnTier(segment, tier, tierToReplicaCount.get(tier));
+      stats.addToTieredStat(CoordinatorStats.REQUIRED_CAPACITY, tier, segment.getSize() * numReplicas);
+    });
+
+    // Find the minimum number of segments required for fault tolerance
+    final int totalTargetReplicas = tierToReplicaCount.values().stream()
+                                                      .reduce(0, Integer::sum);
+    final int minLoadedSegments = totalTargetReplicas > 1 ? 2 : 1;
+
+    // Drop segment from unneeded tiers if requirement is met across target tiers
+    int loadedTargetReplicas = 0;
+    final Set<String> targetTiers = tierToReplicaCount.keySet();
+    for (String tier : targetTiers) {
+      loadedTargetReplicas += replicantLookup.getLoadedReplicants(segment.getId(), tier);
+    }
+    if (loadedTargetReplicas < minLoadedSegments) {
+      return;
+    }

Review Comment:
   This is the check for queueing up drops in unwanted tiers. If there is even the minimum level of fault tolerance (2 replicas) in the wanted tiers, we can proceed with the drops on unwanted tiers while full replication continues on the wanted tiers.
   
   The assignment to new target tiers is already done by this point.



-- 
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: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] kfaraz commented on a diff in pull request #13197: Segment loading: Allow cancellation and prioritization of load queue items

Posted by GitBox <gi...@apache.org>.
kfaraz commented on code in PR #13197:
URL: https://github.com/apache/druid/pull/13197#discussion_r994864548


##########
server/src/main/java/org/apache/druid/server/coordinator/CuratorLoadQueuePeon.java:
##########
@@ -181,10 +183,10 @@ public int getNumberOfSegmentsInQueue()
   }
 
   @Override
-  public void loadSegment(final DataSegment segment, @Nullable final LoadPeonCallback callback)
+  public void loadSegment(final DataSegment segment, SegmentAction action, @Nullable final LoadPeonCallback callback)
   {
-    SegmentHolder segmentHolder = new SegmentHolder(segment, Action.LOAD, Collections.singletonList(callback));
-    final SegmentHolder existingHolder = segmentsToLoad.putIfAbsent(segment, segmentHolder);
+    QueuedSegment segmentHolder = new QueuedSegment(segment, action, callback);

Review Comment:
   Reverted to use `SegmentHolder`.



-- 
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: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] kfaraz commented on a diff in pull request #13197: Segment loading: Allow cancellation and prioritization of load queue items

Posted by GitBox <gi...@apache.org>.
kfaraz commented on code in PR #13197:
URL: https://github.com/apache/druid/pull/13197#discussion_r994863657


##########
server/src/main/java/org/apache/druid/server/coordinator/HttpLoadQueuePeon.java:
##########
@@ -65,7 +66,7 @@
 
 /**
  */
-public class HttpLoadQueuePeon extends LoadQueuePeon
+public class HttpLoadQueuePeon implements LoadQueuePeon

Review Comment:
   Reverted.



-- 
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: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] kfaraz commented on a diff in pull request #13197: Segment loading: Allow cancellation and prioritization of load queue items

Posted by GitBox <gi...@apache.org>.
kfaraz commented on code in PR #13197:
URL: https://github.com/apache/druid/pull/13197#discussion_r1010208298


##########
server/src/main/java/org/apache/druid/server/coordinator/SegmentLoader.java:
##########
@@ -0,0 +1,460 @@
+/*
+ * 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;
+
+import com.google.common.collect.Sets;
+import org.apache.druid.java.util.emitter.EmittingLogger;
+import org.apache.druid.timeline.DataSegment;
+
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+
+/**
+ * Used by the coordinator in each run for segment loading, dropping, balancing
+ * and broadcasting.
+ * <p>
+ * An instance of this class is freshly created for each coordinator run.
+ */
+public class SegmentLoader
+{
+  private static final EmittingLogger log = new EmittingLogger(SegmentLoader.class);
+
+  private final SegmentStateManager stateManager;
+  private final DruidCluster cluster;
+  private final CoordinatorStats stats = new CoordinatorStats();
+  private final SegmentReplicantLookup replicantLookup;
+  private final ReplicationThrottler replicationThrottler;
+  private final BalancerStrategy strategy;
+
+  private final Set<String> emptyTiers = new HashSet<>();
+
+  public SegmentLoader(
+      SegmentStateManager stateManager,
+      DruidCluster cluster,
+      SegmentReplicantLookup replicantLookup,
+      ReplicationThrottler replicationThrottler,
+      BalancerStrategy strategy
+  )
+  {
+    this.cluster = cluster;
+    this.strategy = strategy;
+    this.stateManager = stateManager;
+    this.replicantLookup = replicantLookup;
+    this.replicationThrottler = replicationThrottler;
+  }
+
+  public CoordinatorStats getStats()
+  {
+    return stats;
+  }
+
+  public void makeAlerts()
+  {
+    if (!emptyTiers.isEmpty()) {
+      log.makeAlert("Tiers %s have no servers! Check your cluster configuration.", emptyTiers).emit();
+    }
+  }
+
+  /**
+   * Moves the given segment between two servers of the same tier.
+   * <p>
+   * See if we can move balancing here.
+   */
+  public boolean moveSegment(DataSegment segment, ServerHolder fromServer, ServerHolder toServer)
+  {
+    final String tier = toServer.getServer().getTier();
+    if (!fromServer.getServer().getTier().equals(tier)) {
+      return false;
+    }
+
+    if (fromServer.isServingSegment(segment)) {
+      // Segment is loaded on fromServer, move it to toServer
+      return stateManager.moveSegment(segment, fromServer, toServer, replicationThrottler.getMaxLifetime());
+    } else if (!fromServer.isLoadingSegment(segment)) {
+      // Cannot move if fromServer is neither loading nor serving the segment
+      return false;
+    }
+
+    // Cancel the load on fromServer and load on toServer instead
+    final boolean loadCancelledOnFromServer =
+        stateManager.cancelOperation(SegmentState.LOADING, segment, fromServer);
+    if (loadCancelledOnFromServer) {
+      stats.addToTieredStat(CoordinatorStats.CANCELLED_LOADS, tier, 1);
+      int loadedCountOnTier = replicantLookup.getLoadedReplicas(segment.getId(), tier);
+      return stateManager.loadSegment(segment, toServer, loadedCountOnTier < 1, replicationThrottler);
+    }
+
+    return false;
+  }
+
+  /**
+   * Queues load or drop of replicas of the given segment to achieve the
+   * target replication level on all the tiers.
+   */
+  public void updateReplicas(DataSegment segment, Map<String, Integer> tierToReplicaCount)
+  {
+    // Identify empty tiers and determine total required replicas
+    final AtomicInteger requiredTotalReplicas = new AtomicInteger(0);
+    final Set<String> allTiers = Sets.newHashSet(cluster.getTierNames());
+    tierToReplicaCount.forEach((tier, requiredReplicas) -> {
+      reportTierCapacityStats(segment, requiredReplicas, tier);
+      if (!allTiers.contains(tier)) {
+        emptyTiers.add(tier);
+      } else {
+        requiredTotalReplicas.addAndGet(requiredReplicas);
+      }
+    });
+
+    final int totalOverReplication =
+        replicantLookup.getTotalLoadedReplicas(segment.getId()) - requiredTotalReplicas.get();
+
+    // Update replicas in every tier
+    int totalDropsQueued = 0;
+    for (String tier : allTiers) {
+      totalDropsQueued += updateReplicasInTier(
+          segment,
+          tier,
+          tierToReplicaCount.getOrDefault(tier, 0),
+          totalOverReplication - totalDropsQueued
+      );
+    }
+  }
+
+  /**
+   * Queues load or drop operations on this tier based on the required
+   * number of replicas and the current state.
+   * <p>
+   * The {@code maxReplicasToDrop} helps to maintain the required level of
+   * replication in the cluster. This ensures that segment read concurrency does
+   * not suffer during a tier shift or load rule change.
+   * <p>
+   * Returns the number of new drop operations queued on this tier.
+   */
+  private int updateReplicasInTier(
+      DataSegment segment,
+      String tier,
+      int requiredReplicas,
+      int maxReplicasToDrop
+  )
+  {
+    final int projectedReplicas = replicantLookup.getProjectedReplicas(segment.getId(), tier);
+    final int movingReplicas = replicantLookup.getMovingReplicas(segment.getId(), tier);
+    final boolean shouldCancelMoves = requiredReplicas == 0 && movingReplicas > 0;
+
+    // Check if there is any action required on this tier
+    if (projectedReplicas == requiredReplicas && !shouldCancelMoves) {
+      return 0;
+    }
+
+    SegmentTierStatus segmentStatus = new SegmentTierStatus(segment, cluster.getHistoricalsByTier(tier));
+
+    // Cancel all moves in this tier if it does not need to have replicas
+    if (shouldCancelMoves) {
+      int cancelledMoves =
+          cancelOperations(movingReplicas, SegmentState.MOVING_TO, segment, segmentStatus);
+      stats.addToTieredStat(CoordinatorStats.CANCELLED_MOVES, tier, cancelledMoves);
+    }
+
+    // Cancel drops and queue loads if the projected count is below the requirement
+    if (projectedReplicas < requiredReplicas) {
+      int replicaDeficit = requiredReplicas - projectedReplicas;
+      int cancelledDrops =
+          cancelOperations(replicaDeficit, SegmentState.DROPPING, segment, segmentStatus);
+
+      // Cancelled drops can be counted as loaded replicas, thus reducing deficit
+      int numReplicasToLoad = replicaDeficit - cancelledDrops;
+      if (numReplicasToLoad > 0) {
+        boolean isFirstLoadOnTier = replicantLookup.getLoadedReplicas(segment.getId(), tier)
+                                    + cancelledDrops < 1;
+        int numLoadsQueued = loadReplicas(numReplicasToLoad, segment, tier, segmentStatus, isFirstLoadOnTier);
+        stats.addToTieredStat(CoordinatorStats.ASSIGNED_COUNT, tier, numLoadsQueued);
+      }
+    }
+
+    // Cancel loads and queue drops if the projected count exceeds the requirement
+    if (projectedReplicas > requiredReplicas) {
+      int replicaSurplus = projectedReplicas - requiredReplicas;
+      int cancelledLoads =
+          cancelOperations(replicaSurplus, SegmentState.LOADING, segment, segmentStatus);
+      stats.addToTieredStat(CoordinatorStats.CANCELLED_LOADS, tier, cancelledLoads);
+
+      int numReplicasToDrop = Math.min(replicaSurplus - cancelledLoads, maxReplicasToDrop);
+      if (numReplicasToDrop > 0) {
+        int dropsQueuedOnTier = dropReplicas(numReplicasToDrop, segment, tier, segmentStatus);
+        stats.addToTieredStat(CoordinatorStats.DROPPED_COUNT, tier, dropsQueuedOnTier);
+        return dropsQueuedOnTier;
+      }
+    }
+
+    return 0;
+  }
+
+  private void reportTierCapacityStats(DataSegment segment, int requiredReplicas, String tier)
+  {
+    stats.accumulateMaxTieredStat(
+        CoordinatorStats.MAX_REPLICATION_FACTOR,
+        tier,
+        requiredReplicas
+    );
+    stats.addToTieredStat(
+        CoordinatorStats.REQUIRED_CAPACITY,
+        tier,
+        segment.getSize() * requiredReplicas
+    );
+  }
+
+  /**
+   * Broadcasts the given segment to all servers that are broadcast targets and
+   * queues a drop of the segment from decommissioning servers.
+   */
+  public void broadcastSegment(DataSegment segment)
+  {
+    int assignedCount = 0;
+    int droppedCount = 0;
+    for (ServerHolder server : cluster.getAllServers()) {
+      // Ignore servers which are not broadcast targets
+      if (!server.getServer().getType().isSegmentBroadcastTarget()) {
+        continue;
+      }
+
+      if (server.isDecommissioning()) {
+        droppedCount += dropBroadcastSegment(segment, server) ? 1 : 0;
+      } else {
+        assignedCount += loadBroadcastSegment(segment, server) ? 1 : 0;
+      }
+    }
+
+    if (assignedCount > 0) {
+      stats.addToDataSourceStat(CoordinatorStats.BROADCAST_LOADS, segment.getDataSource(), assignedCount);
+    }
+    if (droppedCount > 0) {
+      stats.addToDataSourceStat(CoordinatorStats.BROADCAST_DROPS, segment.getDataSource(), droppedCount);
+    }
+  }
+
+  /**
+   * Marks the given segment as unused.
+   */
+  public void deleteSegment(DataSegment segment)
+  {
+    stateManager.deleteSegment(segment);
+    stats.addToGlobalStat(CoordinatorStats.DELETED_COUNT, 1);
+  }
+
+  /**
+   * Loads the broadcast segment if it is not loaded on the given server.
+   * Returns true only if the segment was successfully queued for load on the server.
+   */
+  private boolean loadBroadcastSegment(DataSegment segment, ServerHolder server)
+  {
+    final SegmentState state = server.getSegmentState(segment);
+    if (state == SegmentState.LOADED || state == SegmentState.LOADING) {
+      return false;
+    }
+
+    // Cancel drop if it is in progress
+    boolean dropCancelled = stateManager.cancelOperation(SegmentState.DROPPING, segment, server);
+    if (dropCancelled) {
+      return false;
+    }
+
+    if (server.canLoadSegment(segment)
+        && stateManager.loadSegment(segment, server, true, replicationThrottler)) {
+      return true;
+    } else {
+      log.makeAlert("Failed to assign broadcast segment for datasource [%s]", segment.getDataSource())
+         .addData("segmentId", segment.getId())
+         .addData("segmentSize", segment.getSize())
+         .addData("hostName", server.getServer().getHost())
+         .addData("availableSize", server.getAvailableSize())
+         .emit();
+      return false;
+    }
+  }
+
+  /**
+   * Drops the broadcast segment if it is loaded on the given server.
+   * Returns true only if the segment was successfully queued for drop on the server.
+   */
+  private boolean dropBroadcastSegment(DataSegment segment, ServerHolder server)
+  {
+    final SegmentState state = server.getSegmentState(segment);
+    if (state == SegmentState.NONE || state == SegmentState.DROPPING) {
+      return false;
+    }
+
+    // Cancel load if it is in progress
+    boolean loadCancelled = stateManager.cancelOperation(SegmentState.LOADING, segment, server);
+    if (loadCancelled) {
+      return false;
+    }
+
+    return stateManager.dropSegment(segment, server);
+  }
+
+  /**
+   * Queues drop of {@code numToDrop} replicas of the segment from a tier.
+   * Tries to drop replicas first from decommissioning servers and then from
+   * active servers.
+   * <p>
+   * Returns the number of successfully queued drop operations.
+   */
+  private int dropReplicas(
+      int numToDrop,
+      DataSegment segment,
+      String tier,
+      SegmentTierStatus segmentStatus
+  )
+  {
+    final List<ServerHolder> eligibleServers = segmentStatus.getServers(SegmentState.LOADED);
+    if (eligibleServers.isEmpty() || numToDrop <= 0) {
+      return 0;
+    }
+
+    final TreeSet<ServerHolder> eligibleLiveServers = new TreeSet<>();
+    final TreeSet<ServerHolder> eligibleDyingServers = new TreeSet<>();
+    for (ServerHolder server : eligibleServers) {
+      if (!server.isServingSegment(segment)) {
+        // ignore this server
+      } else if (server.isDecommissioning()) {
+        eligibleDyingServers.add(server);
+      } else {
+        eligibleLiveServers.add(server);
+      }
+    }
+
+    // Drop as many replicas as possible from decommissioning servers

Review Comment:
   Thanks for the suggestion. Will include this changes along with the changes for full node replication in a follow up PR.



-- 
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: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] kfaraz commented on a diff in pull request #13197: Segment loading: Allow cancellation and prioritization of load queue items

Posted by "kfaraz (via GitHub)" <gi...@apache.org>.
kfaraz commented on code in PR #13197:
URL: https://github.com/apache/druid/pull/13197#discussion_r1232279827


##########
server/src/main/java/org/apache/druid/server/coordinator/CoordinatorDynamicConfig.java:
##########
@@ -428,66 +489,36 @@ public boolean equals(Object o)
 
     CoordinatorDynamicConfig that = (CoordinatorDynamicConfig) o;
 
-    if (leadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments !=
-        that.leadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments) {
-      return false;
-    }
-    if (mergeBytesLimit != that.mergeBytesLimit) {
-      return false;
-    }
-    if (mergeSegmentsLimit != that.mergeSegmentsLimit) {
-      return false;
-    }
-    if (maxSegmentsToMove != that.maxSegmentsToMove) {
-      return false;
-    }
-    if (percentOfSegmentsToConsiderPerMove != that.percentOfSegmentsToConsiderPerMove) {
-      return false;
-    }
-    if (useBatchedSegmentSampler != that.useBatchedSegmentSampler) {
-      return false;
-    }
-    if (replicantLifetime != that.replicantLifetime) {
-      return false;
-    }
-    if (replicationThrottleLimit != that.replicationThrottleLimit) {
-      return false;
-    }
-    if (balancerComputeThreads != that.balancerComputeThreads) {
-      return false;
-    }
-    if (emitBalancingStats != that.emitBalancingStats) {
-      return false;
-    }
-    if (maxSegmentsInNodeLoadingQueue != that.maxSegmentsInNodeLoadingQueue) {
-      return false;
-    }
-    if (!Objects.equals(specificDataSourcesToKillUnusedSegmentsIn, that.specificDataSourcesToKillUnusedSegmentsIn)) {
-      return false;
-    }
-    if (!Objects.equals(dataSourcesToNotKillStalePendingSegmentsIn, that.dataSourcesToNotKillStalePendingSegmentsIn)) {
-      return false;
-    }
-    if (!Objects.equals(decommissioningNodes, that.decommissioningNodes)) {
-      return false;
-    }
-    if (pauseCoordination != that.pauseCoordination) {
-      return false;
-    }
-    if (replicateAfterLoadTimeout != that.replicateAfterLoadTimeout) {
-      return false;
-    }
-    if (maxNonPrimaryReplicantsToLoad != that.maxNonPrimaryReplicantsToLoad) {
-      return false;
-    }
-    return decommissioningMaxPercentOfMaxSegmentsToMove == that.decommissioningMaxPercentOfMaxSegmentsToMove;
+    return markSegmentAsUnusedDelayMillis == that.markSegmentAsUnusedDelayMillis
+           && mergeBytesLimit == that.mergeBytesLimit
+           && mergeSegmentsLimit == that.mergeSegmentsLimit
+           && maxSegmentsToMove == that.maxSegmentsToMove
+           && percentOfSegmentsToConsiderPerMove == that.percentOfSegmentsToConsiderPerMove
+           && decommissioningMaxPercentOfMaxSegmentsToMove == that.decommissioningMaxPercentOfMaxSegmentsToMove
+           && useBatchedSegmentSampler == that.useBatchedSegmentSampler
+           && balancerComputeThreads == that.balancerComputeThreads
+           && emitBalancingStats == that.emitBalancingStats
+           && replicantLifetime == that.replicantLifetime
+           && replicationThrottleLimit == that.replicationThrottleLimit
+           && replicateAfterLoadTimeout == that.replicateAfterLoadTimeout
+           && maxSegmentsInNodeLoadingQueue == that.maxSegmentsInNodeLoadingQueue
+           && maxNonPrimaryReplicantsToLoad == that.maxNonPrimaryReplicantsToLoad
+           && useRoundRobinSegmentAssignment == that.useRoundRobinSegmentAssignment
+           && pauseCoordination == that.pauseCoordination
+           && Objects.equals(
+               specificDataSourcesToKillUnusedSegmentsIn,
+               that.specificDataSourcesToKillUnusedSegmentsIn)
+           && Objects.equals(
+               dataSourcesToNotKillStalePendingSegmentsIn,
+               that.dataSourcesToNotKillStalePendingSegmentsIn)
+           && Objects.equals(decommissioningNodes, that.decommissioningNodes);

Review Comment:
   I was even thinking of getting rid of the `equals` and `hashCode` in this class. Don't see them serving any purpose (unless the `JsonConfigProvider` does something with it, need to double check). Even the tests do an item-by-item comparison. But will do it later.



-- 
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: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] kfaraz commented on a diff in pull request #13197: Segment loading: Allow cancellation and prioritization of load queue items

Posted by "kfaraz (via GitHub)" <gi...@apache.org>.
kfaraz commented on code in PR #13197:
URL: https://github.com/apache/druid/pull/13197#discussion_r991339473


##########
server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java:
##########
@@ -405,111 +405,6 @@ public String getCurrentLeader()
     return coordLeaderSelector.getCurrentLeader();
   }
 
-  public void moveSegment(

Review Comment:
   Change summary:
   Moved this method to `SegmentLoader`/`SegmentStateManager`.



-- 
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: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] kfaraz closed pull request #13197: Segment loading: Allow cancellation and prioritization of load queue items

Posted by "kfaraz (via GitHub)" <gi...@apache.org>.
kfaraz closed pull request #13197: Segment loading: Allow cancellation and prioritization of load queue items
URL: https://github.com/apache/druid/pull/13197


-- 
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: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] kfaraz commented on a diff in pull request #13197: Segment loading: Allow cancellation and prioritization of load queue items

Posted by "kfaraz (via GitHub)" <gi...@apache.org>.
kfaraz commented on code in PR #13197:
URL: https://github.com/apache/druid/pull/13197#discussion_r1116501774


##########
server/src/main/java/org/apache/druid/server/coordinator/BalancerStrategy.java:
##########
@@ -46,13 +46,18 @@
   ServerHolder findNewSegmentHomeBalancer(DataSegment proposalSegment, List<ServerHolder> serverHolders);
 
   /**
-   * Find the best server on which to place a {@link DataSegment} replica according to the balancing strategy
+   * Finds the best servers on which to place a replica of the {@code proposalSegment}
+   * according to the balancing strategy.

Review Comment:
   Updated.



-- 
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: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] kfaraz commented on a diff in pull request #13197: Segment loading: Allow cancellation and prioritization of load queue items

Posted by GitBox <gi...@apache.org>.
kfaraz commented on code in PR #13197:
URL: https://github.com/apache/druid/pull/13197#discussion_r997808987


##########
server/src/main/java/org/apache/druid/server/coordinator/SegmentStateManager.java:
##########
@@ -0,0 +1,261 @@
+/*
+ * 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;
+
+import com.google.inject.Inject;
+import org.apache.druid.client.ServerInventoryView;
+import org.apache.druid.java.util.emitter.EmittingLogger;
+import org.apache.druid.metadata.SegmentsMetadataManager;
+import org.apache.druid.timeline.DataSegment;
+
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * Manages state of segments being loaded.
+ */
+public class SegmentStateManager
+{
+  private static final EmittingLogger log = new EmittingLogger(SegmentStateManager.class);
+
+  private final LoadQueueTaskMaster taskMaster;
+  private final ServerInventoryView serverInventoryView;
+  private final SegmentsMetadataManager segmentsMetadataManager;
+
+  private final ConcurrentHashMap<String, TierLoadingState> currentlyMovingSegments =
+      new ConcurrentHashMap<>();
+  private final ConcurrentHashMap<String, TierLoadingState> currentlyReplicatingSegments
+      = new ConcurrentHashMap<>();
+
+  @Inject
+  public SegmentStateManager(
+      ServerInventoryView serverInventoryView,
+      SegmentsMetadataManager segmentsMetadataManager,
+      LoadQueueTaskMaster taskMaster
+  )
+  {
+    this.serverInventoryView = serverInventoryView;
+    this.segmentsMetadataManager = segmentsMetadataManager;
+    this.taskMaster = taskMaster;
+  }
+
+  /**
+   * Queues load of a replica of the segment on the given server.
+   */
+  public boolean loadSegment(
+      DataSegment segment,
+      ServerHolder server,
+      boolean isPrimary,
+      ReplicationThrottler throttler
+  )
+  {
+    final String tier = server.getServer().getTier();
+    final LoadPeonCallback callback;
+    if (isPrimary) {
+      // Primary replicas are not subject to throttling
+      callback = null;
+    } else if (canLoadReplica(tier, throttler)) {
+      throttler.incrementAssignedReplicas(tier);
+
+      final TierLoadingState replicatingInTier = currentlyReplicatingSegments
+          .computeIfAbsent(tier, t -> new TierLoadingState(throttler.getMaxLifetime()));
+      replicatingInTier.addSegment(segment.getId(), server.getServer().getHost());
+      callback = success -> replicatingInTier.removeSegment(segment.getId());
+    } else {
+      throttler.incrementThrottledReplicas(tier);
+      return false;
+    }
+
+    try {
+      if (!server.startOperation(segment, SegmentState.LOADING)) {
+        return false;
+      }
+
+      server.getPeon().loadSegment(
+          segment,
+          isPrimary ? SegmentAction.LOAD_AS_PRIMARY : SegmentAction.LOAD_AS_REPLICA,
+          callback
+      );
+      return true;
+    }
+    catch (Exception e) {
+      server.cancelOperation(segment, SegmentState.LOADING);
+      return false;
+    }
+  }
+
+  public boolean dropSegment(DataSegment segment, ServerHolder server)
+  {
+    try {
+      if (!server.startOperation(segment, SegmentState.DROPPING)) {
+        return false;
+      }
+
+      server.getPeon().dropSegment(segment, null);
+      return true;
+    }
+    catch (Exception e) {
+      server.cancelOperation(segment, SegmentState.DROPPING);
+      return false;
+    }
+  }
+
+  public boolean moveSegment(
+      DataSegment segment,
+      ServerHolder fromServer,
+      ServerHolder toServer,
+      int maxLifetimeInBalancingQueue
+  )
+  {
+    final TierLoadingState segmentsMovingInTier = currentlyMovingSegments.computeIfAbsent(
+        toServer.getServer().getTier(),
+        t -> new TierLoadingState(maxLifetimeInBalancingQueue)
+    );
+    final LoadQueuePeon fromServerPeon = fromServer.getPeon();
+    final LoadPeonCallback moveFinishCallback = success -> {
+      fromServerPeon.unmarkSegmentToDrop(segment);
+      segmentsMovingInTier.removeSegment(segment.getId());
+    };
+
+    // mark segment to drop before it is actually loaded on server
+    // to be able to account for this information in BalancerStrategy immediately
+    toServer.startOperation(segment, SegmentState.MOVING_TO);
+    fromServerPeon.markSegmentToDrop(segment);
+    segmentsMovingInTier.addSegment(segment.getId(), fromServer.getServer().getHost());
+
+    final LoadQueuePeon toServerPeon = toServer.getPeon();
+    final String toServerName = toServer.getServer().getName();
+    try {
+      toServerPeon.loadSegment(
+          segment,
+          SegmentAction.MOVE_TO,
+          success -> {
+            // Drop segment only if:
+            // (1) segment load was successful on toServer
+            // AND (2) segment not already queued for drop on fromServer
+            // AND (3a) loading is http-based
+            //     OR (3b) inventory shows segment loaded on toServer
+
+            // Do not check the inventory with http loading as the HTTP
+            // response is enough to determine load success or failure
+            if (success
+                && !fromServerPeon.getSegmentsToDrop().contains(segment)
+                && (taskMaster.isHttpLoading()
+                    || serverInventoryView.isSegmentLoadedByServer(toServerName, segment))) {
+              fromServerPeon.dropSegment(segment, moveFinishCallback);
+            } else {
+              moveFinishCallback.execute(success);
+            }
+          }
+      );
+    }
+    catch (Exception e) {
+      toServer.cancelOperation(segment, SegmentState.MOVING_TO);
+      moveFinishCallback.execute(false);
+      throw new RuntimeException(e);
+    }
+
+    return true;
+  }
+
+  /**
+   * Marks the given segment as unused.
+   */
+  public boolean deleteSegment(DataSegment segment)
+  {
+    return segmentsMetadataManager.markSegmentAsUnused(segment.getId());
+  }
+
+  /**
+   * Cancels the segment operation being performed on a server if the actual
+   * state of the segment on the server matches the given currentState.
+   */
+  public boolean cancelOperation(
+      SegmentState currentState,
+      DataSegment segment,
+      ServerHolder server
+  )
+  {
+    if (!server.cancelOperation(segment, currentState)) {
+      return false;
+    }
+
+    final LoadQueuePeon peon = server.getPeon();
+    switch (currentState) {
+      case DROPPING:
+        return peon.cancelDrop(segment);
+      case MOVING_TO:
+      case LOADING:
+        return peon.cancelLoad(segment);
+      default:
+        return false;
+    }
+  }
+
+  /**
+   * Reduces the lifetimes of the segments currently being moved in all the tiers,
+   * and returns a map from tier names to the corresponding state.
+   */
+  public Map<String, TierLoadingState> reduceLifetimesOfMovingSegments()
+  {
+    return reduceLifetimesAndCreateCopy(currentlyMovingSegments);
+  }
+
+  /**
+   * Reduces the lifetimes of the segments currently being replicated in the tiers,
+   * and returns a map from tier names to the corresponding state.
+   */
+  public Map<String, TierLoadingState> reduceLifetimesOfReplicatingSegments()
+  {
+    return reduceLifetimesAndCreateCopy(currentlyReplicatingSegments);
+  }
+
+  private Map<String, TierLoadingState> reduceLifetimesAndCreateCopy(
+      Map<String, TierLoadingState> inFlightSegments
+  )
+  {
+    final Set<String> inactiveTiers = new HashSet<>();
+    inFlightSegments.forEach((tier, holder) -> {
+      if (holder.getNumProcessingSegments() == 0) {
+        inactiveTiers.add(tier);
+      }
+      holder.reduceLifetime();
+    });
+
+    // Reset state for inactive tiers
+    inactiveTiers.forEach(inFlightSegments::remove);
+
+    return Collections.unmodifiableMap(inFlightSegments);
+  }
+
+  private boolean canLoadReplica(String tier, ReplicationThrottler throttler)
+  {
+    if (!throttler.canAssignReplica(tier)) {
+      return false;
+    }
+
+    TierLoadingState tierState = currentlyReplicatingSegments.get(tier);
+    return tierState == null || tierState.getNumProcessingSegments() < throttler.getReplicationThrottleLimit();

Review Comment:
   Fair point. I will clean it up.



-- 
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: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] kfaraz commented on a diff in pull request #13197: Segment loading: Allow cancellation and prioritization of load queue items

Posted by GitBox <gi...@apache.org>.
kfaraz commented on code in PR #13197:
URL: https://github.com/apache/druid/pull/13197#discussion_r997806161


##########
server/src/main/java/org/apache/druid/server/coordinator/SegmentStateManager.java:
##########
@@ -0,0 +1,261 @@
+/*
+ * 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;
+
+import com.google.inject.Inject;
+import org.apache.druid.client.ServerInventoryView;
+import org.apache.druid.java.util.emitter.EmittingLogger;
+import org.apache.druid.metadata.SegmentsMetadataManager;
+import org.apache.druid.timeline.DataSegment;
+
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * Manages state of segments being loaded.
+ */
+public class SegmentStateManager
+{
+  private static final EmittingLogger log = new EmittingLogger(SegmentStateManager.class);
+
+  private final LoadQueueTaskMaster taskMaster;
+  private final ServerInventoryView serverInventoryView;
+  private final SegmentsMetadataManager segmentsMetadataManager;
+
+  private final ConcurrentHashMap<String, TierLoadingState> currentlyMovingSegments =
+      new ConcurrentHashMap<>();
+  private final ConcurrentHashMap<String, TierLoadingState> currentlyReplicatingSegments
+      = new ConcurrentHashMap<>();
+
+  @Inject
+  public SegmentStateManager(
+      ServerInventoryView serverInventoryView,
+      SegmentsMetadataManager segmentsMetadataManager,
+      LoadQueueTaskMaster taskMaster
+  )
+  {
+    this.serverInventoryView = serverInventoryView;
+    this.segmentsMetadataManager = segmentsMetadataManager;
+    this.taskMaster = taskMaster;
+  }
+
+  /**
+   * Queues load of a replica of the segment on the given server.
+   */
+  public boolean loadSegment(
+      DataSegment segment,
+      ServerHolder server,
+      boolean isPrimary,
+      ReplicationThrottler throttler
+  )
+  {
+    final String tier = server.getServer().getTier();
+    final LoadPeonCallback callback;
+    if (isPrimary) {
+      // Primary replicas are not subject to throttling
+      callback = null;

Review Comment:
   Yes, I have made this change in a follow up PR which fully enables prioritisation and does away with the notion of primary and secondary. Should I include that here itself?



-- 
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: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] kfaraz commented on a diff in pull request #13197: Segment loading: Allow cancellation and prioritization of load queue items

Posted by "kfaraz (via GitHub)" <gi...@apache.org>.
kfaraz commented on code in PR #13197:
URL: https://github.com/apache/druid/pull/13197#discussion_r1232981597


##########
server/src/main/java/org/apache/druid/server/coordinator/SegmentReplicantLookup.java:
##########
@@ -125,49 +117,85 @@ public int getMovingReplicas(SegmentId segmentId, String tier)
    * Number of replicas of the segment which are safely loaded on the given tier
    * and are not being dropped.
    */
-  public int getLoadedReplicas(SegmentId segmentId, String tier)
+  public int getLoadedNotDroppingReplicas(SegmentId segmentId, String tier)
+  {
+    ReplicaCount count = replicaCounts.get(segmentId, tier);
+    return (count == null) ? 0 : count.loadedNotDropping();
+  }
+
+  public int getLoadingReplicas(SegmentId segmentId, String tier)
   {
     ReplicaCount count = replicaCounts.get(segmentId, tier);
-    return (count == null) ? 0 : count.safelyLoaded();
+    return count == null ? 0 : count.loading;
   }
 
   /**
-   * Number of replicas of the segment which are safely loaded on the cluster
-   * and are not being dropped.
+   * Number of replicas of the segment which are loaded on the cluster.
+   *
+   * @param includeDropping Whether segments which are being dropped should be
+   *                        included in the total count.
    */
-  public int getTotalLoadedReplicas(SegmentId segmentId)
+  public int getLoadedReplicas(SegmentId segmentId, boolean includeDropping)
   {
     final Map<String, ReplicaCount> allTiers = replicaCounts.row(segmentId);
     int totalLoaded = 0;
     for (ReplicaCount count : allTiers.values()) {
-      totalLoaded += count.safelyLoaded();
+      totalLoaded += includeDropping ? count.loaded : count.loadedNotDropping();
     }
     return totalLoaded;
   }

Review Comment:
   Updated.



-- 
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: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] kfaraz commented on a diff in pull request #13197: Segment loading: Allow cancellation and prioritization of load queue items

Posted by GitBox <gi...@apache.org>.
kfaraz commented on code in PR #13197:
URL: https://github.com/apache/druid/pull/13197#discussion_r994060186


##########
server/src/main/java/org/apache/druid/server/coordinator/BalancerStrategy.java:
##########
@@ -46,13 +46,18 @@
   ServerHolder findNewSegmentHomeBalancer(DataSegment proposalSegment, List<ServerHolder> serverHolders);
 
   /**
-   * Find the best server on which to place a {@link DataSegment} replica according to the balancing strategy
+   * Finds the best servers on which to place a replica of the {@code proposalSegment}
+   * according to the balancing strategy.

Review Comment:
   Okay, I will try to update the javadoc to reflect this distinction. But, as it happens, the same method is used for both "new/unavailable" segments as well "replicating/available" segments. (this would go away if we do a round-robin assignment when the segment is unavailable)
   
   So I guess the method name should just be `findSegmentHomes`?



-- 
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: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] kfaraz commented on a diff in pull request #13197: Segment loading: Allow cancellation and prioritization of load queue items

Posted by GitBox <gi...@apache.org>.
kfaraz commented on code in PR #13197:
URL: https://github.com/apache/druid/pull/13197#discussion_r994743260


##########
server/src/main/java/org/apache/druid/server/coordinator/TierLoadingState.java:
##########
@@ -0,0 +1,72 @@
+/*
+ * 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;
+
+import org.apache.druid.timeline.SegmentId;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * Contains segments being loaded or replicated in a tier.
+ */
+public class TierLoadingState
+{
+  private int lifetime;

Review Comment:
   This is the current code:
   https://github.com/apache/druid/blob/45dfd679e92e668172c470c37615e7447b601af1/server/src/main/java/org/apache/druid/server/coordinator/ReplicationThrottler.java#L73-L98
   
   Here too a tier may be busy for a period greater than the lifetime, the lifetime would keep decreasing and we would keep alerting for this tier.
   
   Saying that the load queue is stuck is probably not the correct message here and is causing the confusion. But AFAICT, the behaviour is unchanged.
   
   Or are you talking about a `TierLoadingState` that doesn't have any segments inside and is empty? That has been handled like this:
   ```
   // Reset state for inactive tiers
   inactiveTiers.forEach(currentlyReplicating::remove);
   ```
   The next time when a replica comes for this tier, a new entry would be created with a fresh lifetime value. The same handling has been done for moving segments as well.



-- 
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: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] kfaraz commented on a diff in pull request #13197: Segment loading: Allow cancellation and prioritization of load queue items

Posted by GitBox <gi...@apache.org>.
kfaraz commented on code in PR #13197:
URL: https://github.com/apache/druid/pull/13197#discussion_r993730815


##########
server/src/main/java/org/apache/druid/server/coordinator/TierLoadingState.java:
##########
@@ -0,0 +1,72 @@
+/*
+ * 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;
+
+import org.apache.druid.timeline.SegmentId;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * Contains segments being loaded or replicated in a tier.
+ */
+public class TierLoadingState
+{
+  private int lifetime;
+  private final ConcurrentHashMap<SegmentId, String> processingSegments = new ConcurrentHashMap<>();

Review Comment:
   Yes, as you mention, that is the current implementation and we have retained it here.
   We are hoping to relax replication throttle limit further anyway. So changing this behaviour is not really required.
   
   Incidentally, it is also the desirable implementation, being simpler to understand and reason about than the alternative. From the docs, definition of `replicationThrottleLimit`:
   `The maximum number of segments that can be replicated at one time.`
   Alternative would be: 
   `The number of replica segments (i.e. non primary) that can be in the queue at any time`.
   
   We could try an alternative implementation but it would have these issues:
   - Counting each replica separately would hit the throttle limit sooner (i.e. with fewer _unique_ segments), thus increasing the time to full replication. We could always increase the throttle limit but we would have to keep increasing it whenever we decided to increase the replication level on _any_ datasource.
   - Assigning 1st replica of each segment, then 2nd replica of each segment is not an option either because we must make all decisions for a given segment in a single invocation of the `LoadRule.run()`.
   
   Hope that clarifies the design choice 🙂



-- 
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: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] kfaraz commented on a diff in pull request #13197: Segment loading: Allow cancellation and prioritization of load queue items

Posted by GitBox <gi...@apache.org>.
kfaraz commented on code in PR #13197:
URL: https://github.com/apache/druid/pull/13197#discussion_r994054356


##########
server/src/main/java/org/apache/druid/server/coordinator/CuratorLoadQueuePeon.java:
##########
@@ -181,10 +183,10 @@ public int getNumberOfSegmentsInQueue()
   }
 
   @Override
-  public void loadSegment(final DataSegment segment, @Nullable final LoadPeonCallback callback)
+  public void loadSegment(final DataSegment segment, SegmentAction action, @Nullable final LoadPeonCallback callback)
   {
-    SegmentHolder segmentHolder = new SegmentHolder(segment, Action.LOAD, Collections.singletonList(callback));
-    final SegmentHolder existingHolder = segmentsToLoad.putIfAbsent(segment, segmentHolder);
+    QueuedSegment segmentHolder = new QueuedSegment(segment, action, callback);

Review Comment:
   Yes, I will rename `QueuedSegment` back to `SegmentHolder` as it is adding unnecessarily to the diff clutter.



-- 
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: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] kfaraz commented on a diff in pull request #13197: Segment loading: Allow cancellation and prioritization of load queue items

Posted by GitBox <gi...@apache.org>.
kfaraz commented on code in PR #13197:
URL: https://github.com/apache/druid/pull/13197#discussion_r994054777


##########
server/src/main/java/org/apache/druid/server/coordinator/DruidCluster.java:
##########
@@ -138,7 +138,7 @@ public Set<ServerHolder> getBrokers()
     return brokers;
   }
 
-  public Iterable<String> getTierNames()
+  public Set<String> getTierNames()

Review Comment:
   `DruidCluster` is immutable, and I ended up using this method just once in the new code. So I will just stick to using the original `Iterable<String>`.



-- 
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: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] kfaraz commented on a diff in pull request #13197: Segment loading: Allow cancellation and prioritization of load queue items

Posted by GitBox <gi...@apache.org>.
kfaraz commented on code in PR #13197:
URL: https://github.com/apache/druid/pull/13197#discussion_r993734456


##########
server/src/main/java/org/apache/druid/server/coordinator/TierLoadingState.java:
##########
@@ -0,0 +1,72 @@
+/*
+ * 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;
+
+import org.apache.druid.timeline.SegmentId;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * Contains segments being loaded or replicated in a tier.
+ */
+public class TierLoadingState
+{
+  private int lifetime;
+  private final ConcurrentHashMap<SegmentId, String> processingSegments = new ConcurrentHashMap<>();

Review Comment:
   However, this PR does modify the definition of `replicationThrottleLimit` and `maxSegmentsInNodeLoadingQueue` slightly (docs to be updated). The limits now apply on number of assignments done during a single coordinator run rather than items in the queue at a given instant.
   This change was done to:
   - make the configs easier to use
   - ensure that the configs actually take effect. In some cases (where loading happens quickly enough), these limits are constantly violated.



-- 
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: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] imply-cheddar commented on a diff in pull request #13197: Segment loading: Allow cancellation and prioritization of load queue items

Posted by GitBox <gi...@apache.org>.
imply-cheddar commented on code in PR #13197:
URL: https://github.com/apache/druid/pull/13197#discussion_r1009974733


##########
server/src/main/java/org/apache/druid/server/coordinator/SegmentHolder.java:
##########
@@ -0,0 +1,172 @@
+/*
+ * 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;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Ordering;
+import org.apache.druid.server.coordination.DataSegmentChangeRequest;
+import org.apache.druid.server.coordination.SegmentChangeRequestDrop;
+import org.apache.druid.server.coordination.SegmentChangeRequestLoad;
+import org.apache.druid.timeline.DataSegment;
+
+import javax.annotation.Nullable;
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Objects;
+import java.util.concurrent.atomic.AtomicLong;
+
+/**
+ * Represents a segment queued for a load or drop operation in a LoadQueuePeon.
+ * <p>
+ * Requests are naturally ordered using the {@link #COMPARE_ACTION_THEN_INTERVAL}.
+ */
+public class SegmentHolder implements Comparable<SegmentHolder>
+{
+  /**
+   * Orders segment requests:
+   * <ul>
+   *   <li>first by action: all drops, then all loads, then all moves</li>
+   *   <li>then by interval: newest segments first</li>
+   * </ul>
+   */
+  public static final Comparator<SegmentHolder> COMPARE_ACTION_THEN_INTERVAL =
+      Ordering.explicit(SegmentAction.DROP, SegmentAction.LOAD, SegmentAction.REPLICATE, SegmentAction.MOVE_TO)
+              .onResultOf(SegmentHolder::getAction)
+              .compound(DruidCoordinator.SEGMENT_COMPARATOR_RECENT_FIRST.onResultOf(SegmentHolder::getSegment));
+
+  private final DataSegment segment;
+  private final DataSegmentChangeRequest changeRequest;
+  private final SegmentAction action;
+
+  // Guaranteed to store only non-null elements
+  private final List<LoadPeonCallback> callbacks = new ArrayList<>();
+  private final AtomicLong firstRequestMillis = new AtomicLong(0);
+
+  SegmentHolder(
+      DataSegment segment,
+      SegmentAction action,
+      @Nullable LoadPeonCallback callback
+  )
+  {
+    this.segment = segment;
+    this.action = action;
+    this.changeRequest = (action == SegmentAction.DROP)
+                         ? new SegmentChangeRequestDrop(segment)
+                         : new SegmentChangeRequestLoad(segment);
+    if (callback != null) {
+      callbacks.add(callback);
+    }
+  }
+
+  public DataSegment getSegment()
+  {
+    return segment;
+  }
+
+  public SegmentAction getAction()
+  {
+    return action;
+  }
+
+  public boolean isLoad()
+  {
+    return action != SegmentAction.DROP;
+  }
+
+  public DataSegmentChangeRequest getChangeRequest()
+  {
+    return changeRequest;
+  }
+
+  public String getSegmentIdentifier()
+  {
+    return segment.getId().toString();
+  }
+
+  public void addCallback(@Nullable LoadPeonCallback callback)
+  {
+    if (callback != null) {
+      synchronized (callbacks) {
+        callbacks.add(callback);
+      }
+    }
+  }
+
+  /**
+   * Returns an immutable copy of all non-null callbacks for this queued segment.
+   */
+  public List<LoadPeonCallback> getCallbacks()
+  {
+    synchronized (callbacks) {
+      return ImmutableList.copyOf(callbacks);
+    }
+  }
+
+  public void markRequestSentToServer()
+  {
+    firstRequestMillis.compareAndSet(0L, System.currentTimeMillis());
+  }
+
+  public boolean isRequestSentToServer()
+  {
+    return firstRequestMillis.get() > 0;
+  }
+
+  public long getMillisSinceFirstRequestToServer()

Review Comment:
   stream of conciousness: should we really have this doing the subtraction (and taking the current time millis with each call)?  Or should we have this be able to return the firstRequestMillis and do the `.currentTimeMillis()` externally once?



##########
server/src/main/java/org/apache/druid/server/coordinator/SegmentAction.java:
##########
@@ -0,0 +1,33 @@
+/*
+ * 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;
+
+/**
+ * Represents actions that can be performed on a server for a single segment.
+ * <p>
+ * The different action types can be used to prioritize items in a LoadQueuePeon.
+ */
+public enum SegmentAction
+{
+  DROP,
+  LOAD,
+  REPLICATE,
+  MOVE_TO,

Review Comment:
   Please add Javadoc on each of these to explain what they mean.  "REPLICATE" and "LOAD" are especially important given that it is not unbelievable that some people will come and think that some usage of LOAD should be REPLICATE, when we are very explicitly defining them to mean throttled versus unthrottled.



##########
server/src/main/java/org/apache/druid/server/coordinator/HttpLoadQueuePeon.java:
##########
@@ -380,10 +407,12 @@ public void loadSegment(DataSegment segment, LoadPeonCallback callback)
       }
 
       SegmentHolder holder = segmentsToLoad.get(segment);
-
       if (holder == null) {
         log.trace("Server[%s] to load segment[%s] queued.", serverId, segment.getId());
-        segmentsToLoad.put(segment, new LoadSegmentHolder(segment, callback));
+        queuedSize.addAndGet(segment.getSize());
+        holder = new SegmentHolder(segment, action, callback);
+        segmentsToLoad.put(segment, holder);

Review Comment:
   The old code separated `segmentsToLoad` and `segmentsToDrop` so that it could prioritize drops over loads.  If I'm udnerstanding correctly, we are doing that prioriziation through the `queuedSegments` prioritization now, which makes me wonder if we need to keep the old `segmentsToLoad` and `segmentsToDrop` around anymore?  Are those data structures still used for some meaningful purpose?



##########
server/src/main/java/org/apache/druid/server/coordinator/SegmentLoader.java:
##########
@@ -0,0 +1,460 @@
+/*
+ * 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;
+
+import com.google.common.collect.Sets;
+import org.apache.druid.java.util.emitter.EmittingLogger;
+import org.apache.druid.timeline.DataSegment;
+
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+
+/**
+ * Used by the coordinator in each run for segment loading, dropping, balancing
+ * and broadcasting.
+ * <p>
+ * An instance of this class is freshly created for each coordinator run.
+ */
+public class SegmentLoader
+{
+  private static final EmittingLogger log = new EmittingLogger(SegmentLoader.class);
+
+  private final SegmentStateManager stateManager;
+  private final DruidCluster cluster;
+  private final CoordinatorStats stats = new CoordinatorStats();
+  private final SegmentReplicantLookup replicantLookup;
+  private final ReplicationThrottler replicationThrottler;
+  private final BalancerStrategy strategy;
+
+  private final Set<String> emptyTiers = new HashSet<>();
+
+  public SegmentLoader(
+      SegmentStateManager stateManager,
+      DruidCluster cluster,
+      SegmentReplicantLookup replicantLookup,
+      ReplicationThrottler replicationThrottler,
+      BalancerStrategy strategy
+  )
+  {
+    this.cluster = cluster;
+    this.strategy = strategy;
+    this.stateManager = stateManager;
+    this.replicantLookup = replicantLookup;
+    this.replicationThrottler = replicationThrottler;
+  }
+
+  public CoordinatorStats getStats()
+  {
+    return stats;
+  }
+
+  public void makeAlerts()
+  {
+    if (!emptyTiers.isEmpty()) {
+      log.makeAlert("Tiers %s have no servers! Check your cluster configuration.", emptyTiers).emit();
+    }
+  }
+
+  /**
+   * Moves the given segment between two servers of the same tier.
+   * <p>
+   * See if we can move balancing here.
+   */
+  public boolean moveSegment(DataSegment segment, ServerHolder fromServer, ServerHolder toServer)
+  {
+    final String tier = toServer.getServer().getTier();
+    if (!fromServer.getServer().getTier().equals(tier)) {
+      return false;
+    }
+
+    if (fromServer.isServingSegment(segment)) {
+      // Segment is loaded on fromServer, move it to toServer
+      return stateManager.moveSegment(segment, fromServer, toServer, replicationThrottler.getMaxLifetime());
+    } else if (!fromServer.isLoadingSegment(segment)) {
+      // Cannot move if fromServer is neither loading nor serving the segment
+      return false;
+    }
+
+    // Cancel the load on fromServer and load on toServer instead
+    final boolean loadCancelledOnFromServer =
+        stateManager.cancelOperation(SegmentState.LOADING, segment, fromServer);
+    if (loadCancelledOnFromServer) {
+      stats.addToTieredStat(CoordinatorStats.CANCELLED_LOADS, tier, 1);
+      int loadedCountOnTier = replicantLookup.getLoadedReplicas(segment.getId(), tier);
+      return stateManager.loadSegment(segment, toServer, loadedCountOnTier < 1, replicationThrottler);
+    }
+
+    return false;
+  }
+
+  /**
+   * Queues load or drop of replicas of the given segment to achieve the
+   * target replication level on all the tiers.
+   */
+  public void updateReplicas(DataSegment segment, Map<String, Integer> tierToReplicaCount)
+  {
+    // Identify empty tiers and determine total required replicas
+    final AtomicInteger requiredTotalReplicas = new AtomicInteger(0);
+    final Set<String> allTiers = Sets.newHashSet(cluster.getTierNames());
+    tierToReplicaCount.forEach((tier, requiredReplicas) -> {
+      reportTierCapacityStats(segment, requiredReplicas, tier);
+      if (!allTiers.contains(tier)) {

Review Comment:
   Once again a nit, but you are doing an `if/else` and have a negative clause here.  You can just as easily have the positive clause and use the else statement for the `!` case.



##########
server/src/main/java/org/apache/druid/server/coordinator/SegmentReplicantLookup.java:
##########
@@ -156,4 +161,45 @@ public Object2LongMap<String> getBroadcastUnderReplication(SegmentId segmentId)
     }
     return perTier;
   }
+
+  /**
+   * Counts of replicas of a segment in different states.
+   */
+  private static class ReplicaCount
+  {
+    int loaded;
+    int loading;
+    int dropping;
+    int moving;
+
+    void addLoaded()
+    {
+      ++loaded;
+    }
+
+    void addQueued(SegmentState state)
+    {
+      switch (state) {

Review Comment:
   Why no increment for `REPLICANT`?



##########
server/src/main/java/org/apache/druid/server/coordinator/SegmentLoader.java:
##########
@@ -0,0 +1,460 @@
+/*
+ * 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;
+
+import com.google.common.collect.Sets;
+import org.apache.druid.java.util.emitter.EmittingLogger;
+import org.apache.druid.timeline.DataSegment;
+
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+
+/**
+ * Used by the coordinator in each run for segment loading, dropping, balancing
+ * and broadcasting.
+ * <p>
+ * An instance of this class is freshly created for each coordinator run.
+ */
+public class SegmentLoader
+{
+  private static final EmittingLogger log = new EmittingLogger(SegmentLoader.class);
+
+  private final SegmentStateManager stateManager;
+  private final DruidCluster cluster;
+  private final CoordinatorStats stats = new CoordinatorStats();
+  private final SegmentReplicantLookup replicantLookup;
+  private final ReplicationThrottler replicationThrottler;
+  private final BalancerStrategy strategy;
+
+  private final Set<String> emptyTiers = new HashSet<>();
+
+  public SegmentLoader(
+      SegmentStateManager stateManager,
+      DruidCluster cluster,
+      SegmentReplicantLookup replicantLookup,
+      ReplicationThrottler replicationThrottler,
+      BalancerStrategy strategy
+  )
+  {
+    this.cluster = cluster;
+    this.strategy = strategy;
+    this.stateManager = stateManager;
+    this.replicantLookup = replicantLookup;
+    this.replicationThrottler = replicationThrottler;
+  }
+
+  public CoordinatorStats getStats()
+  {
+    return stats;
+  }
+
+  public void makeAlerts()
+  {
+    if (!emptyTiers.isEmpty()) {
+      log.makeAlert("Tiers %s have no servers! Check your cluster configuration.", emptyTiers).emit();
+    }
+  }
+
+  /**
+   * Moves the given segment between two servers of the same tier.
+   * <p>
+   * See if we can move balancing here.
+   */
+  public boolean moveSegment(DataSegment segment, ServerHolder fromServer, ServerHolder toServer)
+  {
+    final String tier = toServer.getServer().getTier();
+    if (!fromServer.getServer().getTier().equals(tier)) {
+      return false;
+    }
+
+    if (fromServer.isServingSegment(segment)) {
+      // Segment is loaded on fromServer, move it to toServer
+      return stateManager.moveSegment(segment, fromServer, toServer, replicationThrottler.getMaxLifetime());
+    } else if (!fromServer.isLoadingSegment(segment)) {
+      // Cannot move if fromServer is neither loading nor serving the segment
+      return false;
+    }
+
+    // Cancel the load on fromServer and load on toServer instead
+    final boolean loadCancelledOnFromServer =
+        stateManager.cancelOperation(SegmentState.LOADING, segment, fromServer);
+    if (loadCancelledOnFromServer) {
+      stats.addToTieredStat(CoordinatorStats.CANCELLED_LOADS, tier, 1);
+      int loadedCountOnTier = replicantLookup.getLoadedReplicas(segment.getId(), tier);
+      return stateManager.loadSegment(segment, toServer, loadedCountOnTier < 1, replicationThrottler);
+    }
+
+    return false;
+  }
+
+  /**
+   * Queues load or drop of replicas of the given segment to achieve the
+   * target replication level on all the tiers.
+   */
+  public void updateReplicas(DataSegment segment, Map<String, Integer> tierToReplicaCount)
+  {
+    // Identify empty tiers and determine total required replicas
+    final AtomicInteger requiredTotalReplicas = new AtomicInteger(0);
+    final Set<String> allTiers = Sets.newHashSet(cluster.getTierNames());
+    tierToReplicaCount.forEach((tier, requiredReplicas) -> {
+      reportTierCapacityStats(segment, requiredReplicas, tier);
+      if (!allTiers.contains(tier)) {
+        emptyTiers.add(tier);
+      } else {
+        requiredTotalReplicas.addAndGet(requiredReplicas);
+      }
+    });
+
+    final int totalOverReplication =
+        replicantLookup.getTotalLoadedReplicas(segment.getId()) - requiredTotalReplicas.get();
+
+    // Update replicas in every tier
+    int totalDropsQueued = 0;
+    for (String tier : allTiers) {
+      totalDropsQueued += updateReplicasInTier(
+          segment,
+          tier,
+          tierToReplicaCount.getOrDefault(tier, 0),
+          totalOverReplication - totalDropsQueued
+      );
+    }
+  }
+
+  /**
+   * Queues load or drop operations on this tier based on the required
+   * number of replicas and the current state.
+   * <p>
+   * The {@code maxReplicasToDrop} helps to maintain the required level of
+   * replication in the cluster. This ensures that segment read concurrency does
+   * not suffer during a tier shift or load rule change.
+   * <p>
+   * Returns the number of new drop operations queued on this tier.
+   */
+  private int updateReplicasInTier(
+      DataSegment segment,
+      String tier,
+      int requiredReplicas,
+      int maxReplicasToDrop
+  )
+  {
+    final int projectedReplicas = replicantLookup.getProjectedReplicas(segment.getId(), tier);
+    final int movingReplicas = replicantLookup.getMovingReplicas(segment.getId(), tier);
+    final boolean shouldCancelMoves = requiredReplicas == 0 && movingReplicas > 0;
+
+    // Check if there is any action required on this tier
+    if (projectedReplicas == requiredReplicas && !shouldCancelMoves) {
+      return 0;
+    }
+
+    SegmentTierStatus segmentStatus = new SegmentTierStatus(segment, cluster.getHistoricalsByTier(tier));
+
+    // Cancel all moves in this tier if it does not need to have replicas
+    if (shouldCancelMoves) {
+      int cancelledMoves =
+          cancelOperations(movingReplicas, SegmentState.MOVING_TO, segment, segmentStatus);
+      stats.addToTieredStat(CoordinatorStats.CANCELLED_MOVES, tier, cancelledMoves);
+    }
+
+    // Cancel drops and queue loads if the projected count is below the requirement
+    if (projectedReplicas < requiredReplicas) {
+      int replicaDeficit = requiredReplicas - projectedReplicas;
+      int cancelledDrops =
+          cancelOperations(replicaDeficit, SegmentState.DROPPING, segment, segmentStatus);
+
+      // Cancelled drops can be counted as loaded replicas, thus reducing deficit
+      int numReplicasToLoad = replicaDeficit - cancelledDrops;
+      if (numReplicasToLoad > 0) {
+        boolean isFirstLoadOnTier = replicantLookup.getLoadedReplicas(segment.getId(), tier)
+                                    + cancelledDrops < 1;
+        int numLoadsQueued = loadReplicas(numReplicasToLoad, segment, tier, segmentStatus, isFirstLoadOnTier);
+        stats.addToTieredStat(CoordinatorStats.ASSIGNED_COUNT, tier, numLoadsQueued);
+      }
+    }
+
+    // Cancel loads and queue drops if the projected count exceeds the requirement
+    if (projectedReplicas > requiredReplicas) {
+      int replicaSurplus = projectedReplicas - requiredReplicas;
+      int cancelledLoads =
+          cancelOperations(replicaSurplus, SegmentState.LOADING, segment, segmentStatus);
+      stats.addToTieredStat(CoordinatorStats.CANCELLED_LOADS, tier, cancelledLoads);
+
+      int numReplicasToDrop = Math.min(replicaSurplus - cancelledLoads, maxReplicasToDrop);
+      if (numReplicasToDrop > 0) {
+        int dropsQueuedOnTier = dropReplicas(numReplicasToDrop, segment, tier, segmentStatus);
+        stats.addToTieredStat(CoordinatorStats.DROPPED_COUNT, tier, dropsQueuedOnTier);
+        return dropsQueuedOnTier;
+      }
+    }
+
+    return 0;
+  }
+
+  private void reportTierCapacityStats(DataSegment segment, int requiredReplicas, String tier)
+  {
+    stats.accumulateMaxTieredStat(
+        CoordinatorStats.MAX_REPLICATION_FACTOR,
+        tier,
+        requiredReplicas
+    );
+    stats.addToTieredStat(
+        CoordinatorStats.REQUIRED_CAPACITY,
+        tier,
+        segment.getSize() * requiredReplicas
+    );
+  }
+
+  /**
+   * Broadcasts the given segment to all servers that are broadcast targets and
+   * queues a drop of the segment from decommissioning servers.
+   */
+  public void broadcastSegment(DataSegment segment)
+  {
+    int assignedCount = 0;
+    int droppedCount = 0;
+    for (ServerHolder server : cluster.getAllServers()) {
+      // Ignore servers which are not broadcast targets
+      if (!server.getServer().getType().isSegmentBroadcastTarget()) {
+        continue;
+      }
+
+      if (server.isDecommissioning()) {
+        droppedCount += dropBroadcastSegment(segment, server) ? 1 : 0;
+      } else {
+        assignedCount += loadBroadcastSegment(segment, server) ? 1 : 0;
+      }
+    }
+
+    if (assignedCount > 0) {
+      stats.addToDataSourceStat(CoordinatorStats.BROADCAST_LOADS, segment.getDataSource(), assignedCount);
+    }
+    if (droppedCount > 0) {
+      stats.addToDataSourceStat(CoordinatorStats.BROADCAST_DROPS, segment.getDataSource(), droppedCount);
+    }
+  }
+
+  /**
+   * Marks the given segment as unused.
+   */
+  public void deleteSegment(DataSegment segment)
+  {
+    stateManager.deleteSegment(segment);
+    stats.addToGlobalStat(CoordinatorStats.DELETED_COUNT, 1);
+  }
+
+  /**
+   * Loads the broadcast segment if it is not loaded on the given server.
+   * Returns true only if the segment was successfully queued for load on the server.
+   */
+  private boolean loadBroadcastSegment(DataSegment segment, ServerHolder server)
+  {
+    final SegmentState state = server.getSegmentState(segment);
+    if (state == SegmentState.LOADED || state == SegmentState.LOADING) {
+      return false;
+    }
+
+    // Cancel drop if it is in progress
+    boolean dropCancelled = stateManager.cancelOperation(SegmentState.DROPPING, segment, server);
+    if (dropCancelled) {
+      return false;
+    }
+
+    if (server.canLoadSegment(segment)
+        && stateManager.loadSegment(segment, server, true, replicationThrottler)) {
+      return true;
+    } else {
+      log.makeAlert("Failed to assign broadcast segment for datasource [%s]", segment.getDataSource())
+         .addData("segmentId", segment.getId())
+         .addData("segmentSize", segment.getSize())
+         .addData("hostName", server.getServer().getHost())
+         .addData("availableSize", server.getAvailableSize())
+         .emit();
+      return false;
+    }
+  }
+
+  /**
+   * Drops the broadcast segment if it is loaded on the given server.
+   * Returns true only if the segment was successfully queued for drop on the server.
+   */
+  private boolean dropBroadcastSegment(DataSegment segment, ServerHolder server)
+  {
+    final SegmentState state = server.getSegmentState(segment);
+    if (state == SegmentState.NONE || state == SegmentState.DROPPING) {
+      return false;
+    }
+
+    // Cancel load if it is in progress
+    boolean loadCancelled = stateManager.cancelOperation(SegmentState.LOADING, segment, server);
+    if (loadCancelled) {
+      return false;
+    }
+
+    return stateManager.dropSegment(segment, server);
+  }
+
+  /**
+   * Queues drop of {@code numToDrop} replicas of the segment from a tier.
+   * Tries to drop replicas first from decommissioning servers and then from
+   * active servers.
+   * <p>
+   * Returns the number of successfully queued drop operations.
+   */
+  private int dropReplicas(
+      int numToDrop,
+      DataSegment segment,
+      String tier,
+      SegmentTierStatus segmentStatus
+  )
+  {
+    final List<ServerHolder> eligibleServers = segmentStatus.getServers(SegmentState.LOADED);
+    if (eligibleServers.isEmpty() || numToDrop <= 0) {
+      return 0;
+    }
+
+    final TreeSet<ServerHolder> eligibleLiveServers = new TreeSet<>();
+    final TreeSet<ServerHolder> eligibleDyingServers = new TreeSet<>();
+    for (ServerHolder server : eligibleServers) {
+      if (!server.isServingSegment(segment)) {
+        // ignore this server
+      } else if (server.isDecommissioning()) {
+        eligibleDyingServers.add(server);
+      } else {
+        eligibleLiveServers.add(server);
+      }
+    }
+
+    // Drop as many replicas as possible from decommissioning servers

Review Comment:
   Why drop things from a decommissioning server?  As long as the server is up, the dta is available it can be used.  If you don't want the server to be used for anything, just `kill -9` the process.  If it's up and working, keep using it until it's `kill -9`d.  If we are going to support decommissioning, it shouldn't be a "I need to remove things from this server" but rather "I'm going to pretend as if that server doesn't exist anymore".
   
   That said, decommissioning for historicals is not a really good model.  Instead, we need the ability to start up as a replica.



##########
server/src/test/java/org/apache/druid/server/coordinator/BalancerStrategyTest.java:
##########
@@ -84,29 +84,32 @@ public void findNewSegmentHomeReplicatorNotEnoughSpace()
         new LoadQueuePeonTester());
     serverHolders = new ArrayList<>();
     serverHolders.add(serverHolder);
-    final ServerHolder foundServerHolder = balancerStrategy.findNewSegmentHomeReplicator(proposedDataSegment, serverHolders);
     // since there is not enough space on server having available size 10L to host a segment of size 11L, it should be null
-    Assert.assertNull(foundServerHolder);
+    Assert.assertFalse(
+        balancerStrategy.findNewSegmentHomeReplicator(proposedDataSegment, serverHolders)
+                        .hasNext()
+    );

Review Comment:
   This change is doing what I always tell people happens to comments: the code changed and left the comment stale.  Please fix the comment along with the code (or maybe we don't need the comment anymore?



##########
server/src/test/java/org/apache/druid/server/coordinator/RunRulesTest.java:
##########
@@ -119,35 +118,39 @@ public void testOneTierTwoReplicantsWithStrictReplicantLimit()
         )).atLeastOnce();
     EasyMock.replay(databaseRuleManager);
 
-    DruidCluster druidCluster = DruidClusterBuilder
+    // server1 has all the segments already loaded
+    final DruidServer server1 =
+        new DruidServer("server1", "hostNorm", null, 1000, ServerType.HISTORICAL, "normal", 0);
+    usedSegments.forEach(server1::addDataSegment);
+
+    final DruidServer server2 =
+        new DruidServer("server2", "hostNorm", null, 1000, ServerType.HISTORICAL, "normal", 0);
+
+    final DruidCluster druidCluster = DruidClusterBuilder
         .newBuilder()
         .addTier(
             "normal",
-            new ServerHolder(
-                new DruidServer("serverNorm", "hostNorm", null, 1000, ServerType.HISTORICAL, "normal", 0)
-                    .toImmutableDruidServer(),
-                mockPeon
-            ),
-            new ServerHolder(
-                new DruidServer("serverNorm2", "hostNorm2", null, 1000, ServerType.HISTORICAL, "normal", 0)
-                    .toImmutableDruidServer(),
-                mockPeon
-            )
-        )
-        .build();
+            new ServerHolder(server1.toImmutableDruidServer(), mockPeon),
+            new ServerHolder(server2.toImmutableDruidServer(), mockPeon)
+        ).build();
 
     ListeningExecutorService exec = MoreExecutors.listeningDecorator(Executors.newFixedThreadPool(1));
     BalancerStrategy balancerStrategy = new CostBalancerStrategyFactory().createBalancerStrategy(exec);
 
     DruidCoordinatorRuntimeParams params = makeCoordinatorRuntimeParams(druidCluster, balancerStrategy)
-        .withDynamicConfigs(CoordinatorDynamicConfig.builder().withMaxSegmentsToMove(5).withMaxNonPrimaryReplicantsToLoad(10).build())
+        .withDynamicConfigs(
+            CoordinatorDynamicConfig
+                .builder()
+                .withMaxNonPrimaryReplicantsToLoad(10)
+                .build()
+        )
         .build();
 
     DruidCoordinatorRuntimeParams afterParams = ruleRunner.run(params);
     CoordinatorStats stats = afterParams.getCoordinatorStats();
 
-    Assert.assertEquals(34L, stats.getTieredStat("assignedCount", "normal"));
-    Assert.assertEquals(10L, stats.getGlobalStat("totalNonPrimaryReplicantsLoaded"));
+    // There are 24 under-replicated segments, but only 10 replicas are assigned

Review Comment:
   Is there a stat that can be looked at and asserted on the 24 number?



##########
server/src/main/java/org/apache/druid/server/coordinator/SegmentLoader.java:
##########
@@ -0,0 +1,460 @@
+/*
+ * 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;
+
+import com.google.common.collect.Sets;
+import org.apache.druid.java.util.emitter.EmittingLogger;
+import org.apache.druid.timeline.DataSegment;
+
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+
+/**
+ * Used by the coordinator in each run for segment loading, dropping, balancing
+ * and broadcasting.
+ * <p>
+ * An instance of this class is freshly created for each coordinator run.
+ */
+public class SegmentLoader
+{
+  private static final EmittingLogger log = new EmittingLogger(SegmentLoader.class);
+
+  private final SegmentStateManager stateManager;
+  private final DruidCluster cluster;
+  private final CoordinatorStats stats = new CoordinatorStats();
+  private final SegmentReplicantLookup replicantLookup;
+  private final ReplicationThrottler replicationThrottler;
+  private final BalancerStrategy strategy;
+
+  private final Set<String> emptyTiers = new HashSet<>();
+
+  public SegmentLoader(
+      SegmentStateManager stateManager,
+      DruidCluster cluster,
+      SegmentReplicantLookup replicantLookup,
+      ReplicationThrottler replicationThrottler,
+      BalancerStrategy strategy
+  )
+  {
+    this.cluster = cluster;
+    this.strategy = strategy;
+    this.stateManager = stateManager;
+    this.replicantLookup = replicantLookup;
+    this.replicationThrottler = replicationThrottler;
+  }
+
+  public CoordinatorStats getStats()
+  {
+    return stats;
+  }
+
+  public void makeAlerts()
+  {
+    if (!emptyTiers.isEmpty()) {
+      log.makeAlert("Tiers %s have no servers! Check your cluster configuration.", emptyTiers).emit();
+    }
+  }
+
+  /**
+   * Moves the given segment between two servers of the same tier.
+   * <p>
+   * See if we can move balancing here.
+   */
+  public boolean moveSegment(DataSegment segment, ServerHolder fromServer, ServerHolder toServer)
+  {
+    final String tier = toServer.getServer().getTier();
+    if (!fromServer.getServer().getTier().equals(tier)) {
+      return false;
+    }
+
+    if (fromServer.isServingSegment(segment)) {
+      // Segment is loaded on fromServer, move it to toServer
+      return stateManager.moveSegment(segment, fromServer, toServer, replicationThrottler.getMaxLifetime());
+    } else if (!fromServer.isLoadingSegment(segment)) {

Review Comment:
   This is a nit, but perhaps remove the negation here and swap the bodies of the clauses?  Every `!` in a boolean clause just invites confusion about what the clause is trying to say.



-- 
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: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] kfaraz commented on a diff in pull request #13197: Segment loading: Allow cancellation and prioritization of load queue items

Posted by "kfaraz (via GitHub)" <gi...@apache.org>.
kfaraz commented on code in PR #13197:
URL: https://github.com/apache/druid/pull/13197#discussion_r1231768921


##########
server/src/main/java/org/apache/druid/server/coordinator/loadqueue/HttpLoadQueuePeon.java:
##########
@@ -470,134 +517,127 @@
     return Collections.unmodifiableSet(segmentsMarkedToDrop);
   }
 
-  private abstract class SegmentHolder
+  /**
+   * A request is considered to have timed out if the time elapsed since it was
+   * first sent to the server is greater than the configured load timeout.
+   *
+   * @see DruidCoordinatorConfig#getLoadTimeoutDelay()
+   */
+  private boolean hasRequestTimedOut(SegmentHolder holder, long currentTimeMillis)
   {
-    private final DataSegment segment;
-    private final DataSegmentChangeRequest changeRequest;
-    private final List<LoadPeonCallback> callbacks = new ArrayList<>();
-
-    // Time when this request was sent to target server the first time.
-    private volatile long scheduleTime = -1;
-
-    private SegmentHolder(
-        DataSegment segment,
-        DataSegmentChangeRequest changeRequest,
-        LoadPeonCallback callback
-    )
-    {
-      this.segment = segment;
-      this.changeRequest = changeRequest;
+    return holder.isRequestSentToServer()
+           && currentTimeMillis - holder.getFirstRequestMillis()
+              > config.getLoadTimeoutDelay().getMillis();
+  }
 
-      if (callback != null) {
-        this.callbacks.add(callback);
-      }
-    }
+  private void onRequestFailed(SegmentHolder holder, String failureCause)
+  {
+    log.error(
+        "Server[%s] failed segment[%s] request[%s] with cause [%s].",
+        serverId, holder.getSegment().getId(), holder.getAction(), failureCause
+    );
+    onRequestCompleted(holder, QueueStatus.FAILED);
+  }
 
-    public void addCallback(LoadPeonCallback newCallback)
-    {
-      synchronized (callbacks) {
-        if (newCallback != null) {
-          callbacks.add(newCallback);
-        }
-      }
-    }
+  private void onRequestCompleted(SegmentHolder holder, QueueStatus status)
+  {
+    final SegmentAction action = holder.getAction();
+    log.trace(
+        "Server[%s] completed request[%s] on segment[%s] with status[%s].",
+        serverId, action, holder.getSegment().getId(), status
+    );
 
-    public DataSegment getSegment()
-    {
-      return segment;
+    if (holder.isLoad()) {
+      queuedSize.addAndGet(-holder.getSegment().getSize());
     }
+    incrementStat(holder, status);
+    executeCallbacks(holder, status == QueueStatus.SUCCESS);
+  }
 
-    public DataSegmentChangeRequest getChangeRequest()
-    {
-      return changeRequest;
+  private void incrementStat(SegmentHolder holder, QueueStatus status)
+  {
+    stats.add(status.getStatForAction(holder.getAction()), 1);
+    if (status.datasourceStat != null) {
+      stats.addToDatasourceStat(status.datasourceStat, holder.getSegment().getDataSource(), 1);
     }
+  }
 
-    public boolean hasTimedOut()
-    {
-      if (scheduleTime < 0) {
-        scheduleTime = System.currentTimeMillis();
-        return false;
-      } else if (System.currentTimeMillis() - scheduleTime > config.getLoadTimeoutDelay().getMillis()) {
-        return true;
-      } else {
+  private void executeCallbacks(SegmentHolder holder, boolean success)
+  {
+    callBackExecutor.execute(() -> {
+      for (LoadPeonCallback callback : holder.getCallbacks()) {
+        callback.execute(success);
+      }
+    });
+  }
+
+  /**
+   * Tries to cancel a load/drop operation. An load/drop request can be cancelled
+   * only if it has not already been sent to the corresponding server.
+   */
+  @Override
+  public boolean cancelOperation(DataSegment segment)
+  {
+    synchronized (lock) {
+      if (activeRequestSegments.contains(segment)) {
         return false;
       }
-    }
 
-    public void requestSucceeded()
-    {
-      log.trace(
-          "Server[%s] Successfully processed segment[%s] request[%s].",
-          serverId,
-          segment.getId(),
-          changeRequest.getClass().getSimpleName()
-      );
+      // Find the action on this segment, if any
+      final SegmentHolder holder = segmentsToLoad.containsKey(segment)
+                                   ? segmentsToLoad.remove(segment)
+                                   : segmentsToDrop.remove(segment);
+      if (holder == null) {
+        return false;
+      }
 
-      callBackExecutor.execute(() -> {
-        for (LoadPeonCallback callback : callbacks) {
-          if (callback != null) {
-            callback.execute(true);
-          }
-        }
-      });
+      queuedSegments.remove(holder);
+      onRequestCompleted(holder, QueueStatus.CANCELLED);
+      return true;
     }
+  }
 
-    public void requestFailed(String failureCause)
-    {
-      log.error(
-          "Server[%s] Failed segment[%s] request[%s] with cause [%s].",
-          serverId,
-          segment.getId(),
-          changeRequest.getClass().getSimpleName(),
-          failureCause
-      );
-
-      failedAssignCount.getAndIncrement();
+  private enum QueueStatus
+  {
+    ASSIGNED(Stats.SegmentQueue.ASSIGNED_ACTIONS),
+    SUCCESS(Stats.SegmentQueue.COMPLETED_ACTIONS),
+    FAILED(Stats.SegmentQueue.FAILED_ACTIONS),
+    CANCELLED(Stats.SegmentQueue.CANCELLED_ACTIONS);
 
-      callBackExecutor.execute(() -> {
-        for (LoadPeonCallback callback : callbacks) {
-          if (callback != null) {
-            callback.execute(false);
-          }
-        }
-      });
-    }
+    final CoordinatorStat loadStat;
+    final CoordinatorStat moveStat;
+    final CoordinatorStat dropStat;
+    final CoordinatorStat datasourceStat;
 
-    @Override
-    public String toString()
+    QueueStatus()
     {
-      return changeRequest.toString();
+      this(null);
     }
-  }
 
-  private class LoadSegmentHolder extends SegmentHolder
-  {
-    public LoadSegmentHolder(DataSegment segment, LoadPeonCallback callback)
+    QueueStatus(CoordinatorStat datasourceStat)
     {
-      super(segment, new SegmentChangeRequestLoad(segment), callback);
-      queuedSize.addAndGet(segment.getSize());
-    }
+      // These stats are not emitted and are tracked for debugging purposes only
+      final String prefix = StringUtils.toLowerCase(name());
+      this.loadStat = new CoordinatorStat(prefix + "Load");
+      this.moveStat = new CoordinatorStat(prefix + "Move");
+      this.dropStat = new CoordinatorStat(prefix + "Drop");
 
-    @Override
-    public void requestSucceeded()
-    {
-      queuedSize.addAndGet(-getSegment().getSize());
-      super.requestSucceeded();
+      this.datasourceStat = datasourceStat;
     }
 
-    @Override
-    public void requestFailed(String failureCause)
+    CoordinatorStat getStatForAction(SegmentAction action)
     {
-      queuedSize.addAndGet(-getSegment().getSize());
-      super.requestFailed(failureCause);
-    }
-  }
+      switch (action) {

Review Comment:
   No, it is okay, `MOVE_FROM` operations aren't supposed to come to the peon. So the next line will throw an IAE. I can just have the `default` case throw the exception too so that `CodeQL` does not complain.



-- 
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: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] kfaraz commented on a diff in pull request #13197: Segment loading: Allow cancellation and prioritization of load queue items

Posted by "kfaraz (via GitHub)" <gi...@apache.org>.
kfaraz commented on code in PR #13197:
URL: https://github.com/apache/druid/pull/13197#discussion_r1232279827


##########
server/src/main/java/org/apache/druid/server/coordinator/CoordinatorDynamicConfig.java:
##########
@@ -428,66 +489,36 @@ public boolean equals(Object o)
 
     CoordinatorDynamicConfig that = (CoordinatorDynamicConfig) o;
 
-    if (leadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments !=
-        that.leadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments) {
-      return false;
-    }
-    if (mergeBytesLimit != that.mergeBytesLimit) {
-      return false;
-    }
-    if (mergeSegmentsLimit != that.mergeSegmentsLimit) {
-      return false;
-    }
-    if (maxSegmentsToMove != that.maxSegmentsToMove) {
-      return false;
-    }
-    if (percentOfSegmentsToConsiderPerMove != that.percentOfSegmentsToConsiderPerMove) {
-      return false;
-    }
-    if (useBatchedSegmentSampler != that.useBatchedSegmentSampler) {
-      return false;
-    }
-    if (replicantLifetime != that.replicantLifetime) {
-      return false;
-    }
-    if (replicationThrottleLimit != that.replicationThrottleLimit) {
-      return false;
-    }
-    if (balancerComputeThreads != that.balancerComputeThreads) {
-      return false;
-    }
-    if (emitBalancingStats != that.emitBalancingStats) {
-      return false;
-    }
-    if (maxSegmentsInNodeLoadingQueue != that.maxSegmentsInNodeLoadingQueue) {
-      return false;
-    }
-    if (!Objects.equals(specificDataSourcesToKillUnusedSegmentsIn, that.specificDataSourcesToKillUnusedSegmentsIn)) {
-      return false;
-    }
-    if (!Objects.equals(dataSourcesToNotKillStalePendingSegmentsIn, that.dataSourcesToNotKillStalePendingSegmentsIn)) {
-      return false;
-    }
-    if (!Objects.equals(decommissioningNodes, that.decommissioningNodes)) {
-      return false;
-    }
-    if (pauseCoordination != that.pauseCoordination) {
-      return false;
-    }
-    if (replicateAfterLoadTimeout != that.replicateAfterLoadTimeout) {
-      return false;
-    }
-    if (maxNonPrimaryReplicantsToLoad != that.maxNonPrimaryReplicantsToLoad) {
-      return false;
-    }
-    return decommissioningMaxPercentOfMaxSegmentsToMove == that.decommissioningMaxPercentOfMaxSegmentsToMove;
+    return markSegmentAsUnusedDelayMillis == that.markSegmentAsUnusedDelayMillis
+           && mergeBytesLimit == that.mergeBytesLimit
+           && mergeSegmentsLimit == that.mergeSegmentsLimit
+           && maxSegmentsToMove == that.maxSegmentsToMove
+           && percentOfSegmentsToConsiderPerMove == that.percentOfSegmentsToConsiderPerMove
+           && decommissioningMaxPercentOfMaxSegmentsToMove == that.decommissioningMaxPercentOfMaxSegmentsToMove
+           && useBatchedSegmentSampler == that.useBatchedSegmentSampler
+           && balancerComputeThreads == that.balancerComputeThreads
+           && emitBalancingStats == that.emitBalancingStats
+           && replicantLifetime == that.replicantLifetime
+           && replicationThrottleLimit == that.replicationThrottleLimit
+           && replicateAfterLoadTimeout == that.replicateAfterLoadTimeout
+           && maxSegmentsInNodeLoadingQueue == that.maxSegmentsInNodeLoadingQueue
+           && maxNonPrimaryReplicantsToLoad == that.maxNonPrimaryReplicantsToLoad
+           && useRoundRobinSegmentAssignment == that.useRoundRobinSegmentAssignment
+           && pauseCoordination == that.pauseCoordination
+           && Objects.equals(
+               specificDataSourcesToKillUnusedSegmentsIn,
+               that.specificDataSourcesToKillUnusedSegmentsIn)
+           && Objects.equals(
+               dataSourcesToNotKillStalePendingSegmentsIn,
+               that.dataSourcesToNotKillStalePendingSegmentsIn)
+           && Objects.equals(decommissioningNodes, that.decommissioningNodes);

Review Comment:
   I was even thinking of getting rid of the `equals` and `hashCode` in this class. Don't see them serving any purpose. Even the tests do an item-by-item comparison. But will do it later.



-- 
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: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] kfaraz commented on a diff in pull request #13197: Segment loading: Allow cancellation and prioritization of load queue items

Posted by "kfaraz (via GitHub)" <gi...@apache.org>.
kfaraz commented on code in PR #13197:
URL: https://github.com/apache/druid/pull/13197#discussion_r1231756343


##########
server/src/main/java/org/apache/druid/server/coordinator/CoordinatorDynamicConfig.java:
##########
@@ -428,66 +489,36 @@ public boolean equals(Object o)
 
     CoordinatorDynamicConfig that = (CoordinatorDynamicConfig) o;
 
-    if (leadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments !=
-        that.leadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments) {
-      return false;
-    }
-    if (mergeBytesLimit != that.mergeBytesLimit) {
-      return false;
-    }
-    if (mergeSegmentsLimit != that.mergeSegmentsLimit) {
-      return false;
-    }
-    if (maxSegmentsToMove != that.maxSegmentsToMove) {
-      return false;
-    }
-    if (percentOfSegmentsToConsiderPerMove != that.percentOfSegmentsToConsiderPerMove) {
-      return false;
-    }
-    if (useBatchedSegmentSampler != that.useBatchedSegmentSampler) {
-      return false;
-    }
-    if (replicantLifetime != that.replicantLifetime) {
-      return false;
-    }
-    if (replicationThrottleLimit != that.replicationThrottleLimit) {
-      return false;
-    }
-    if (balancerComputeThreads != that.balancerComputeThreads) {
-      return false;
-    }
-    if (emitBalancingStats != that.emitBalancingStats) {
-      return false;
-    }
-    if (maxSegmentsInNodeLoadingQueue != that.maxSegmentsInNodeLoadingQueue) {
-      return false;
-    }
-    if (!Objects.equals(specificDataSourcesToKillUnusedSegmentsIn, that.specificDataSourcesToKillUnusedSegmentsIn)) {
-      return false;
-    }
-    if (!Objects.equals(dataSourcesToNotKillStalePendingSegmentsIn, that.dataSourcesToNotKillStalePendingSegmentsIn)) {
-      return false;
-    }
-    if (!Objects.equals(decommissioningNodes, that.decommissioningNodes)) {
-      return false;
-    }
-    if (pauseCoordination != that.pauseCoordination) {
-      return false;
-    }
-    if (replicateAfterLoadTimeout != that.replicateAfterLoadTimeout) {
-      return false;
-    }
-    if (maxNonPrimaryReplicantsToLoad != that.maxNonPrimaryReplicantsToLoad) {
-      return false;
-    }
-    return decommissioningMaxPercentOfMaxSegmentsToMove == that.decommissioningMaxPercentOfMaxSegmentsToMove;
+    return markSegmentAsUnusedDelayMillis == that.markSegmentAsUnusedDelayMillis
+           && mergeBytesLimit == that.mergeBytesLimit
+           && mergeSegmentsLimit == that.mergeSegmentsLimit
+           && maxSegmentsToMove == that.maxSegmentsToMove
+           && percentOfSegmentsToConsiderPerMove == that.percentOfSegmentsToConsiderPerMove
+           && decommissioningMaxPercentOfMaxSegmentsToMove == that.decommissioningMaxPercentOfMaxSegmentsToMove
+           && useBatchedSegmentSampler == that.useBatchedSegmentSampler
+           && balancerComputeThreads == that.balancerComputeThreads
+           && emitBalancingStats == that.emitBalancingStats
+           && replicantLifetime == that.replicantLifetime
+           && replicationThrottleLimit == that.replicationThrottleLimit
+           && replicateAfterLoadTimeout == that.replicateAfterLoadTimeout
+           && maxSegmentsInNodeLoadingQueue == that.maxSegmentsInNodeLoadingQueue
+           && maxNonPrimaryReplicantsToLoad == that.maxNonPrimaryReplicantsToLoad
+           && useRoundRobinSegmentAssignment == that.useRoundRobinSegmentAssignment
+           && pauseCoordination == that.pauseCoordination
+           && Objects.equals(
+               specificDataSourcesToKillUnusedSegmentsIn,
+               that.specificDataSourcesToKillUnusedSegmentsIn)
+           && Objects.equals(
+               dataSourcesToNotKillStalePendingSegmentsIn,
+               that.dataSourcesToNotKillStalePendingSegmentsIn)
+           && Objects.equals(decommissioningNodes, that.decommissioningNodes);

Review Comment:
   Thanks for catching this, must have missed adding it.



-- 
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: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] AmatyaAvadhanula commented on a diff in pull request #13197: Segment loading: Allow cancellation and prioritization of load queue items

Posted by GitBox <gi...@apache.org>.
AmatyaAvadhanula commented on code in PR #13197:
URL: https://github.com/apache/druid/pull/13197#discussion_r993455840


##########
server/src/main/java/org/apache/druid/server/coordinator/CostBalancerStrategy.java:
##########
@@ -49,6 +50,14 @@ public class CostBalancerStrategy implements BalancerStrategy
   private static final double MILLIS_IN_HOUR = 3_600_000.0;
   private static final double MILLIS_FACTOR = MILLIS_IN_HOUR / LAMBDA;
 
+  /**
+   * Comparator that prioritizes servers by cost. Cheaper servers come before
+   * costlier servers. Servers with the same cost may appear in a random order.
+   */
+  private static final Comparator<Pair<Double, ServerHolder>> CHEAPEST_SERVERS_FIRST
+      = Comparator.<Pair<Double, ServerHolder>, Double>comparing(pair -> pair.lhs)
+      .thenComparing(pair -> ThreadLocalRandom.current().nextInt());

Review Comment:
   Would it help to use ServerHolder.getSizeUsed instead of a random integer for the second comparison?



-- 
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: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] kfaraz commented on a diff in pull request #13197: Segment loading: Allow cancellation and prioritization of load queue items

Posted by GitBox <gi...@apache.org>.
kfaraz commented on code in PR #13197:
URL: https://github.com/apache/druid/pull/13197#discussion_r993657773


##########
server/src/main/java/org/apache/druid/server/coordinator/TierLoadingState.java:
##########
@@ -0,0 +1,72 @@
+/*
+ * 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;
+
+import org.apache.druid.timeline.SegmentId;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * Contains segments being loaded or replicated in a tier.
+ */
+public class TierLoadingState
+{
+  private int lifetime;

Review Comment:
   That would be the ideal behaviour but as it turns out, this is the existing implementation. I have tried to keep it unchanged for now.
   
   Let's say, in a coordinator run, we queue some segments to move within a tier. All the segments queued would start with the same lifecycle. Only when all these queued segments have been processed will we consider this tier for moving again. So, at it turns out, in `BalanceSegments.currentlyMovingSegments` in the existing code, all segment holders for a given tier will always have the same lifecycle.



-- 
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: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] AmatyaAvadhanula commented on a diff in pull request #13197: Segment loading: Allow cancellation and prioritization of load queue items

Posted by GitBox <gi...@apache.org>.
AmatyaAvadhanula commented on code in PR #13197:
URL: https://github.com/apache/druid/pull/13197#discussion_r993665756


##########
server/src/main/java/org/apache/druid/server/coordinator/TierLoadingState.java:
##########
@@ -0,0 +1,72 @@
+/*
+ * 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;
+
+import org.apache.druid.timeline.SegmentId;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * Contains segments being loaded or replicated in a tier.
+ */
+public class TierLoadingState
+{
+  private int lifetime;
+  private final ConcurrentHashMap<SegmentId, String> processingSegments = new ConcurrentHashMap<>();

Review Comment:
   The code doesn't enforce it, and it is possible for the same segment to be replicated / moved on different servers within the same tier.
   This datastructure would lead to a loss of that information  (which also happens to be the case with the current implementation of ReplicationThrottler)



-- 
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: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] kfaraz commented on a diff in pull request #13197: Segment loading: Allow cancellation and prioritization of load queue items

Posted by GitBox <gi...@apache.org>.
kfaraz commented on code in PR #13197:
URL: https://github.com/apache/druid/pull/13197#discussion_r1010207839


##########
server/src/test/java/org/apache/druid/server/coordinator/BalancerStrategyTest.java:
##########
@@ -84,29 +84,32 @@ public void findNewSegmentHomeReplicatorNotEnoughSpace()
         new LoadQueuePeonTester());
     serverHolders = new ArrayList<>();
     serverHolders.add(serverHolder);
-    final ServerHolder foundServerHolder = balancerStrategy.findNewSegmentHomeReplicator(proposedDataSegment, serverHolders);
     // since there is not enough space on server having available size 10L to host a segment of size 11L, it should be null
-    Assert.assertNull(foundServerHolder);
+    Assert.assertFalse(
+        balancerStrategy.findNewSegmentHomeReplicator(proposedDataSegment, serverHolders)
+                        .hasNext()
+    );

Review Comment:
   Removed as we don't need the comment anymore.



##########
server/src/main/java/org/apache/druid/server/coordinator/SegmentReplicantLookup.java:
##########
@@ -156,4 +161,45 @@ public Object2LongMap<String> getBroadcastUnderReplication(SegmentId segmentId)
     }
     return perTier;
   }
+
+  /**
+   * Counts of replicas of a segment in different states.
+   */
+  private static class ReplicaCount
+  {
+    int loaded;
+    int loading;
+    int dropping;
+    int moving;
+
+    void addLoaded()
+    {
+      ++loaded;
+    }
+
+    void addQueued(SegmentState state)
+    {
+      switch (state) {

Review Comment:
   Fixed.



-- 
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: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] github-code-scanning[bot] commented on a diff in pull request #13197: Segment loading: Allow cancellation and prioritization of load queue items

Posted by "github-code-scanning[bot] (via GitHub)" <gi...@apache.org>.
github-code-scanning[bot] commented on code in PR #13197:
URL: https://github.com/apache/druid/pull/13197#discussion_r1170939508


##########
server/src/main/java/org/apache/druid/server/coordinator/loadqueue/HttpLoadQueuePeon.java:
##########
@@ -470,134 +517,127 @@
     return Collections.unmodifiableSet(segmentsMarkedToDrop);
   }
 
-  private abstract class SegmentHolder
+  /**
+   * A request is considered to have timed out if the time elapsed since it was
+   * first sent to the server is greater than the configured load timeout.
+   *
+   * @see DruidCoordinatorConfig#getLoadTimeoutDelay()
+   */
+  private boolean hasRequestTimedOut(SegmentHolder holder, long currentTimeMillis)
   {
-    private final DataSegment segment;
-    private final DataSegmentChangeRequest changeRequest;
-    private final List<LoadPeonCallback> callbacks = new ArrayList<>();
-
-    // Time when this request was sent to target server the first time.
-    private volatile long scheduleTime = -1;
-
-    private SegmentHolder(
-        DataSegment segment,
-        DataSegmentChangeRequest changeRequest,
-        LoadPeonCallback callback
-    )
-    {
-      this.segment = segment;
-      this.changeRequest = changeRequest;
+    return holder.isRequestSentToServer()
+           && currentTimeMillis - holder.getFirstRequestMillis()
+              > config.getLoadTimeoutDelay().getMillis();
+  }
 
-      if (callback != null) {
-        this.callbacks.add(callback);
-      }
-    }
+  private void onRequestFailed(SegmentHolder holder, String failureCause)
+  {
+    log.error(
+        "Server[%s] failed segment[%s] request[%s] with cause [%s].",
+        serverId, holder.getSegment().getId(), holder.getAction(), failureCause
+    );
+    onRequestCompleted(holder, QueueStatus.FAILED);
+  }
 
-    public void addCallback(LoadPeonCallback newCallback)
-    {
-      synchronized (callbacks) {
-        if (newCallback != null) {
-          callbacks.add(newCallback);
-        }
-      }
-    }
+  private void onRequestCompleted(SegmentHolder holder, QueueStatus status)
+  {
+    final SegmentAction action = holder.getAction();
+    log.trace(
+        "Server[%s] completed request[%s] on segment[%s] with status[%s].",
+        serverId, action, holder.getSegment().getId(), status
+    );
 
-    public DataSegment getSegment()
-    {
-      return segment;
+    if (holder.isLoad()) {
+      queuedSize.addAndGet(-holder.getSegment().getSize());
     }
+    incrementStat(holder, status);
+    executeCallbacks(holder, status == QueueStatus.SUCCESS);
+  }
 
-    public DataSegmentChangeRequest getChangeRequest()
-    {
-      return changeRequest;
+  private void incrementStat(SegmentHolder holder, QueueStatus status)
+  {
+    stats.add(status.getStatForAction(holder.getAction()), 1);
+    if (status.datasourceStat != null) {
+      stats.addToDatasourceStat(status.datasourceStat, holder.getSegment().getDataSource(), 1);
     }
+  }
 
-    public boolean hasTimedOut()
-    {
-      if (scheduleTime < 0) {
-        scheduleTime = System.currentTimeMillis();
-        return false;
-      } else if (System.currentTimeMillis() - scheduleTime > config.getLoadTimeoutDelay().getMillis()) {
-        return true;
-      } else {
+  private void executeCallbacks(SegmentHolder holder, boolean success)
+  {
+    callBackExecutor.execute(() -> {
+      for (LoadPeonCallback callback : holder.getCallbacks()) {
+        callback.execute(success);
+      }
+    });
+  }
+
+  /**
+   * Tries to cancel a load/drop operation. An load/drop request can be cancelled
+   * only if it has not already been sent to the corresponding server.
+   */
+  @Override
+  public boolean cancelOperation(DataSegment segment)
+  {
+    synchronized (lock) {
+      if (activeRequestSegments.contains(segment)) {
         return false;
       }
-    }
 
-    public void requestSucceeded()
-    {
-      log.trace(
-          "Server[%s] Successfully processed segment[%s] request[%s].",
-          serverId,
-          segment.getId(),
-          changeRequest.getClass().getSimpleName()
-      );
+      // Find the action on this segment, if any
+      final SegmentHolder holder = segmentsToLoad.containsKey(segment)
+                                   ? segmentsToLoad.remove(segment)
+                                   : segmentsToDrop.remove(segment);
+      if (holder == null) {
+        return false;
+      }
 
-      callBackExecutor.execute(() -> {
-        for (LoadPeonCallback callback : callbacks) {
-          if (callback != null) {
-            callback.execute(true);
-          }
-        }
-      });
+      queuedSegments.remove(holder);
+      onRequestCompleted(holder, QueueStatus.CANCELLED);
+      return true;
     }
+  }
 
-    public void requestFailed(String failureCause)
-    {
-      log.error(
-          "Server[%s] Failed segment[%s] request[%s] with cause [%s].",
-          serverId,
-          segment.getId(),
-          changeRequest.getClass().getSimpleName(),
-          failureCause
-      );
-
-      failedAssignCount.getAndIncrement();
+  private enum QueueStatus
+  {
+    ASSIGNED(Stats.SegmentQueue.ASSIGNED_ACTIONS),
+    SUCCESS(Stats.SegmentQueue.COMPLETED_ACTIONS),
+    FAILED(Stats.SegmentQueue.FAILED_ACTIONS),
+    CANCELLED(Stats.SegmentQueue.CANCELLED_ACTIONS);
 
-      callBackExecutor.execute(() -> {
-        for (LoadPeonCallback callback : callbacks) {
-          if (callback != null) {
-            callback.execute(false);
-          }
-        }
-      });
-    }
+    final CoordinatorStat loadStat;
+    final CoordinatorStat moveStat;
+    final CoordinatorStat dropStat;
+    final CoordinatorStat datasourceStat;
 
-    @Override
-    public String toString()
+    QueueStatus()
     {
-      return changeRequest.toString();
+      this(null);
     }
-  }
 
-  private class LoadSegmentHolder extends SegmentHolder
-  {
-    public LoadSegmentHolder(DataSegment segment, LoadPeonCallback callback)
+    QueueStatus(CoordinatorStat datasourceStat)
     {
-      super(segment, new SegmentChangeRequestLoad(segment), callback);
-      queuedSize.addAndGet(segment.getSize());
-    }
+      // These stats are not emitted and are tracked for debugging purposes only
+      final String prefix = StringUtils.toLowerCase(name());
+      this.loadStat = new CoordinatorStat(prefix + "Load");
+      this.moveStat = new CoordinatorStat(prefix + "Move");
+      this.dropStat = new CoordinatorStat(prefix + "Drop");
 
-    @Override
-    public void requestSucceeded()
-    {
-      queuedSize.addAndGet(-getSegment().getSize());
-      super.requestSucceeded();
+      this.datasourceStat = datasourceStat;
     }
 
-    @Override
-    public void requestFailed(String failureCause)
+    CoordinatorStat getStatForAction(SegmentAction action)
     {
-      queuedSize.addAndGet(-getSegment().getSize());
-      super.requestFailed(failureCause);
-    }
-  }
+      switch (action) {

Review Comment:
   ## Missing enum case in switch
   
   Switch statement does not have a case for [MOVE_FROM](1).
   
   [Show more details](https://github.com/apache/druid/security/code-scanning/4841)



##########
server/src/test/java/org/apache/druid/server/coordinator/duty/BalanceSegmentsTest.java:
##########
@@ -0,0 +1,554 @@
+/*
+ * 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.ImmutableSet;
+import com.google.common.util.concurrent.ListeningExecutorService;
+import com.google.common.util.concurrent.MoreExecutors;
+import org.apache.druid.client.DruidServer;
+import org.apache.druid.java.util.common.DateTimes;
+import org.apache.druid.java.util.common.concurrent.Execs;
+import org.apache.druid.server.coordination.ServerType;
+import org.apache.druid.server.coordinator.CoordinatorDynamicConfig;
+import org.apache.druid.server.coordinator.CoordinatorRuntimeParamsTestHelpers;
+import org.apache.druid.server.coordinator.DruidCluster;
+import org.apache.druid.server.coordinator.DruidCoordinatorRuntimeParams;
+import org.apache.druid.server.coordinator.ReplicationThrottler;
+import org.apache.druid.server.coordinator.ServerHolder;
+import org.apache.druid.server.coordinator.balancer.BalancerSegmentHolder;
+import org.apache.druid.server.coordinator.balancer.BalancerStrategy;
+import org.apache.druid.server.coordinator.balancer.CostBalancerStrategyFactory;
+import org.apache.druid.server.coordinator.loadqueue.LoadQueuePeonTester;
+import org.apache.druid.server.coordinator.loadqueue.SegmentLoadQueueManager;
+import org.apache.druid.server.coordinator.stats.CoordinatorRunStats;
+import org.apache.druid.server.coordinator.stats.Stats;
+import org.apache.druid.timeline.DataSegment;
+import org.apache.druid.timeline.partition.NoneShardSpec;
+import org.easymock.EasyMock;
+import org.easymock.IExpectationSetters;
+import org.joda.time.DateTime;
+import org.joda.time.Interval;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+
+public class BalanceSegmentsTest
+{
+  private SegmentLoadQueueManager loadQueueManager;
+
+  private DataSegment segment1;
+  private DataSegment segment2;
+  private DataSegment segment3;
+  private DataSegment segment4;
+  private DataSegment segment5;
+
+  private DataSegment[] allSegments;
+
+  private ListeningExecutorService balancerStrategyExecutor;
+  private BalancerStrategy balancerStrategy;
+  private Set<String> broadcastDatasources;
+
+  private DruidServer server1;
+  private DruidServer server2;
+  private DruidServer server3;
+  private DruidServer server4;
+
+  @Before
+  public void setUp()
+  {
+    loadQueueManager = new SegmentLoadQueueManager(null, null, null);
+
+    // Create test segments for multiple datasources
+    final DateTime start1 = DateTimes.of("2012-01-01");
+    final DateTime start2 = DateTimes.of("2012-02-01");
+    final String version = DateTimes.of("2012-03-01").toString();
+
+    segment1 = createHourlySegment("datasource1", start1, version);
+    segment2 = createHourlySegment("datasource1", start2, version);
+    segment3 = createHourlySegment("datasource2", start1, version);
+    segment4 = createHourlySegment("datasource2", start2, version);
+    segment5 = createHourlySegment("datasourceBroadcast", start2, version);
+    allSegments = new DataSegment[]{segment1, segment2, segment3, segment4, segment5};
+
+    server1 = new DruidServer("server1", "server1", null, 100L, ServerType.HISTORICAL, "normal", 0);
+    server2 = new DruidServer("server2", "server2", null, 100L, ServerType.HISTORICAL, "normal", 0);
+    server3 = new DruidServer("server3", "server3", null, 100L, ServerType.HISTORICAL, "normal", 0);
+    server4 = new DruidServer("server4", "server4", null, 100L, ServerType.HISTORICAL, "normal", 0);
+
+    balancerStrategyExecutor = MoreExecutors.listeningDecorator(Execs.multiThreaded(1, "BalanceSegmentsTest-%d"));
+    balancerStrategy = new CostBalancerStrategyFactory().createBalancerStrategy(balancerStrategyExecutor);
+
+    broadcastDatasources = Collections.singleton("datasourceBroadcast");
+  }
+
+  @After
+  public void tearDown()
+  {
+    balancerStrategyExecutor.shutdownNow();
+  }
+
+  @Test
+  public void testMoveToEmptyServerBalancer()
+  {
+    final ServerHolder serverHolder1 = createHolder(server1, allSegments);
+    final ServerHolder serverHolder2 = createHolder(server2);
+
+    DruidCoordinatorRuntimeParams params =
+        defaultRuntimeParamsBuilder(serverHolder1, serverHolder2)
+            .withBalancerStrategy(balancerStrategy)
+            .withBroadcastDatasources(broadcastDatasources)
+            .build();
+
+    CoordinatorRunStats stats = runBalancer(params);
+    Assert.assertEquals(2, stats.getSegmentStat(Stats.Segments.MOVED, "normal", "datasource1"));
+    Assert.assertEquals(1, stats.getSegmentStat(Stats.Segments.MOVED, "normal", "datasource2"));
+  }
+
+  /**
+   * Server 1 has 2 segments.
+   * Server 2 (decommissioning) has 2 segments.
+   * Server 3 is empty.
+   * Decommissioning percent is 60.
+   * Max segments to move is 3.
+   * 2 (of 2) segments should be moved from Server 2 and 1 (of 2) from Server 1.
+   */
+  @Test
+  public void testMoveDecommissioningMaxPercentOfMaxSegmentsToMove()
+  {
+    final ServerHolder serverHolder1 = createHolder(server1, false, segment1, segment2);
+    final ServerHolder serverHolder2 = createHolder(server2, true, segment3, segment4);
+    final ServerHolder serverHolder3 = createHolder(server3, false);
+
+    BalancerStrategy strategy = EasyMock.createMock(BalancerStrategy.class);
+    expectPickLoadingSegmentsAndReturnEmpty(strategy).times(2);
+    expectPickLoadedSegmentsAndReturn(
+        strategy,
+        new BalancerSegmentHolder(serverHolder2, segment3),
+        new BalancerSegmentHolder(serverHolder2, segment4)
+    ).once();
+    expectPickLoadedSegmentsAndReturn(
+        strategy,
+        new BalancerSegmentHolder(serverHolder1, segment1),
+        new BalancerSegmentHolder(serverHolder1, segment2)
+    ).once();
+    expectFindDestinationAndReturn(strategy, serverHolder3);
+    EasyMock.replay(strategy);
+
+    // ceil(3 * 0.6) = 2 segments from decommissioning servers
+    CoordinatorDynamicConfig dynamicConfig =
+        CoordinatorDynamicConfig.builder()
+                                .withMaxSegmentsToMove(3)
+                                .withDecommissioningMaxPercentOfMaxSegmentsToMove(60)
+                                .build();
+    DruidCoordinatorRuntimeParams params =
+        defaultRuntimeParamsBuilder(serverHolder1, serverHolder2, serverHolder3)
+            .withDynamicConfigs(dynamicConfig)
+            .withBalancerStrategy(strategy)
+            .withBroadcastDatasources(broadcastDatasources)
+            .build();
+
+    CoordinatorRunStats stats = runBalancer(params);
+
+    EasyMock.verify(strategy);
+    Assert.assertEquals(1L, stats.getSegmentStat(Stats.Segments.MOVED, "normal", "datasource1"));
+    Assert.assertEquals(2L, stats.getSegmentStat(Stats.Segments.MOVED, "normal", "datasource2"));
+    Assert.assertEquals(
+        ImmutableSet.of(segment1, segment3, segment4),
+        serverHolder3.getPeon().getSegmentsToLoad()
+    );
+  }
+
+  @Test
+  public void testZeroDecommissioningMaxPercentOfMaxSegmentsToMove()
+  {
+    final ServerHolder holder1 = createHolder(server1, false, segment1, segment2);
+    final ServerHolder holder2 = createHolder(server2, true, segment3, segment4);
+    final ServerHolder holder3 = createHolder(server3, false);
+
+    CoordinatorDynamicConfig dynamicConfig =
+        CoordinatorDynamicConfig.builder().withDecommissioningMaxPercentOfMaxSegmentsToMove(0)
+                                .withMaxSegmentsToMove(1).build();
+    DruidCoordinatorRuntimeParams params =
+        defaultRuntimeParamsBuilder(holder1, holder2, holder3).withDynamicConfigs(dynamicConfig).build();
+
+    CoordinatorRunStats stats = runBalancer(params);
+
+    // Verify that either segment1 or segment2 is chosen for move
+    Assert.assertEquals(1L, stats.getSegmentStat(Stats.Segments.MOVED, "normal", segment1.getDataSource()));
+    DataSegment movingSegment = holder3.getPeon().getSegmentsToLoad().iterator().next();
+    Assert.assertEquals(segment1.getDataSource(), movingSegment.getDataSource());
+  }
+
+  @Test
+  public void testMaxDecommissioningMaxPercentOfMaxSegmentsToMove()
+  {
+    final ServerHolder holder1 = createHolder(server1, false, segment1, segment2);
+    final ServerHolder holder2 = createHolder(server2, true, segment3, segment4);
+    final ServerHolder holder3 = createHolder(server3, false);
+
+    CoordinatorDynamicConfig dynamicConfig =
+        CoordinatorDynamicConfig.builder()
+                                .withDecommissioningMaxPercentOfMaxSegmentsToMove(100)
+                                .withMaxSegmentsToMove(1).build();
+    DruidCoordinatorRuntimeParams params =
+        defaultRuntimeParamsBuilder(holder1, holder2, holder3).withDynamicConfigs(dynamicConfig).build();
+
+    CoordinatorRunStats stats = runBalancer(params);
+
+    // Verify that either segment3 or segment4 is chosen for move
+    Assert.assertEquals(1L, stats.getSegmentStat(Stats.Segments.MOVED, "normal", segment3.getDataSource()));
+    DataSegment movingSegment = holder3.getPeon().getSegmentsToLoad().iterator().next();
+    Assert.assertEquals(segment3.getDataSource(), movingSegment.getDataSource());
+  }
+
+  /**
+   * Should balance segments as usual (ignoring percent) with empty decommissioningNodes.
+   */
+  @Test
+  public void testMoveDecommissioningMaxPercentOfMaxSegmentsToMoveWithNoDecommissioning()
+  {
+    final ServerHolder serverHolder1 = createHolder(server1, segment1, segment2);
+    final ServerHolder serverHolder2 = createHolder(server2, segment3, segment4);
+    final ServerHolder serverHolder3 = createHolder(server3);
+    BalancerStrategy strategy = EasyMock.createMock(BalancerStrategy.class);
+    expectPickLoadingSegmentsAndReturnEmpty(strategy).once();
+    expectPickLoadedSegmentsAndReturn(
+        strategy,
+        new BalancerSegmentHolder(serverHolder1, segment2),
+        new BalancerSegmentHolder(serverHolder2, segment3),
+        new BalancerSegmentHolder(serverHolder2, segment4)
+    ).once();
+    expectFindDestinationAndReturn(strategy, serverHolder3);
+    EasyMock.replay(strategy);
+
+    CoordinatorDynamicConfig dynamicConfig =
+        CoordinatorDynamicConfig.builder()
+                                .withMaxSegmentsToMove(3)
+                                .withDecommissioningMaxPercentOfMaxSegmentsToMove(9)
+                                .build();
+    DruidCoordinatorRuntimeParams params =
+        defaultRuntimeParamsBuilder(serverHolder1, serverHolder2, serverHolder3)
+            .withDynamicConfigs(dynamicConfig)
+            .withBalancerStrategy(strategy)
+            .build();
+
+    CoordinatorRunStats stats = runBalancer(params);
+    EasyMock.verify(strategy);
+    Assert.assertEquals(1L, stats.getSegmentStat(Stats.Segments.MOVED, "normal", "datasource1"));
+    Assert.assertEquals(2L, stats.getSegmentStat(Stats.Segments.MOVED, "normal", "datasource2"));
+    Assert.assertEquals(
+        ImmutableSet.of(segment2, segment3, segment4),
+        serverHolder3.getPeon().getSegmentsToLoad()
+    );
+  }
+
+  /**
+   * Shouldn't move segments to a decommissioning server.
+   */
+  @Test
+  public void testMoveToDecommissioningServer()
+  {
+    final ServerHolder serverHolder1 = createHolder(server1, false, allSegments);
+    final ServerHolder serverHolder2 = createHolder(server2, true);
+
+    BalancerStrategy strategy = EasyMock.createMock(BalancerStrategy.class);
+    expectPickLoadingSegmentsAndReturnEmpty(strategy).times(2);
+    expectPickLoadedSegmentsAndReturn(
+        strategy,
+        new BalancerSegmentHolder(serverHolder1, segment1)
+    ).times(2);
+    EasyMock.expect(
+        strategy.findDestinationServerToMoveSegment(
+            EasyMock.anyObject(),
+            EasyMock.anyObject(),
+            EasyMock.anyObject()
+        )
+    ).andAnswer(() -> ((List<ServerHolder>) EasyMock.getCurrentArguments()[2]).get(0)).anyTimes();
+    EasyMock.replay(strategy);
+
+    DruidCoordinatorRuntimeParams params =
+        defaultRuntimeParamsBuilder(serverHolder1, serverHolder2)
+            .withBalancerStrategy(strategy)
+            .withBroadcastDatasources(broadcastDatasources)
+            .build();
+
+    CoordinatorRunStats stats = runBalancer(params);
+    EasyMock.verify(strategy);
+    Assert.assertEquals(0, stats.getTieredStat(Stats.Segments.MOVED, "normal"));
+  }
+
+  @Test
+  public void testMoveFromDecommissioningServer()
+  {
+    final ServerHolder holder1 = createHolder(server1, allSegments);
+    final ServerHolder holder2 = createHolder(server2);
+
+    BalancerStrategy strategy = EasyMock.createMock(BalancerStrategy.class);
+    expectPickLoadingSegmentsAndReturnEmpty(strategy).once();
+    expectPickLoadedSegmentsAndReturn(strategy, new BalancerSegmentHolder(holder1, segment1)).once();
+    expectFindDestinationAndReturn(strategy, holder2);
+    EasyMock.replay(strategy);
+
+    DruidCoordinatorRuntimeParams params = defaultRuntimeParamsBuilder(holder1, holder2)
+        .withDynamicConfigs(CoordinatorDynamicConfig.builder().withMaxSegmentsToMove(1).build())
+        .withBalancerStrategy(strategy)
+        .withBroadcastDatasources(broadcastDatasources)
+        .build();
+
+    CoordinatorRunStats stats = runBalancer(params);
+    EasyMock.verify(strategy);
+    Assert.assertEquals(1, stats.getSegmentStat(Stats.Segments.MOVED, "normal", segment1.getDataSource()));
+    Assert.assertEquals(0, holder1.getPeon().getNumberOfSegmentsToLoad());
+    Assert.assertEquals(1, holder2.getPeon().getNumberOfSegmentsToLoad());
+  }
+
+  @Test
+  public void testMoveMaxLoadQueueServerBalancer()
+  {
+    final int maxSegmentsInQueue = 1;
+    final ServerHolder holder1 = createHolder(server1, maxSegmentsInQueue, false, allSegments);
+    final ServerHolder holder2 = createHolder(server2, maxSegmentsInQueue, false);
+
+    final CoordinatorDynamicConfig dynamicConfig = CoordinatorDynamicConfig
+        .builder()
+        .withMaxSegmentsInNodeLoadingQueue(maxSegmentsInQueue)
+        .build();
+    DruidCoordinatorRuntimeParams params =
+        defaultRuntimeParamsBuilder(holder1, holder2)
+            .withDynamicConfigs(dynamicConfig)
+            .build();
+
+    CoordinatorRunStats stats = runBalancer(params);
+
+    // max to move is 5, all segments on server 1, but only expect to move 1 to server 2 since max node load queue is 1
+    Assert.assertEquals(maxSegmentsInQueue, stats.getSegmentStat(Stats.Segments.MOVED, "normal", "datasource1"));
+  }
+
+  @Test
+  public void testRun1()
+  {
+    // Mock some servers of different usages
+    DruidCoordinatorRuntimeParams params = defaultRuntimeParamsBuilder(
+        createHolder(server1, allSegments),
+        createHolder(server2)
+    ).build();
+
+    CoordinatorRunStats stats = runBalancer(params);
+    Assert.assertTrue(stats.getSegmentStat(Stats.Segments.MOVED, "normal", "datasource1") > 0);
+  }
+
+  @Test
+  public void testRun2()
+  {
+    // Mock some servers of different usages
+    DruidCoordinatorRuntimeParams params = defaultRuntimeParamsBuilder(
+        createHolder(server1, allSegments),
+        createHolder(server2),
+        createHolder(server3),
+        createHolder(server4)
+    ).build();
+
+    CoordinatorRunStats stats = runBalancer(params);
+    Assert.assertTrue(stats.getSegmentStat(Stats.Segments.MOVED, "normal", "datasource1") > 0);
+  }
+
+  @Test
+  public void testThatMaxSegmentsToMoveIsHonored()
+  {
+    // Move from non-decomissioning servers
+    final ServerHolder holder1 = createHolder(server1, segment1, segment2);
+    final ServerHolder holder2 = createHolder(server2, segment3, segment4);
+    final ServerHolder holder3 = createHolder(server3);
+
+    BalancerStrategy strategy = EasyMock.createMock(BalancerStrategy.class);
+    expectPickLoadingSegmentsAndReturnEmpty(strategy).once();
+    expectPickLoadedSegmentsAndReturn(strategy, new BalancerSegmentHolder(holder2, segment3)).once();
+    expectFindDestinationAndReturn(strategy, holder3);
+    EasyMock.replay(strategy);
+
+    DruidCoordinatorRuntimeParams params =
+        defaultRuntimeParamsBuilder(holder1, holder2, holder3)
+            .withDynamicConfigs(
+                CoordinatorDynamicConfig.builder()
+                                        .withMaxSegmentsToMove(1)
+                                        .withUseBatchedSegmentSampler(true)
+                                        .withPercentOfSegmentsToConsiderPerMove(40)

Review Comment:
   ## Deprecated method or constructor invocation
   
   Invoking [Builder.withPercentOfSegmentsToConsiderPerMove](1) should be avoided because it has been deprecated.
   
   [Show more details](https://github.com/apache/druid/security/code-scanning/4836)



##########
server/src/test/java/org/apache/druid/server/coordinator/duty/BalanceSegmentsTest.java:
##########
@@ -0,0 +1,554 @@
+/*
+ * 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.ImmutableSet;
+import com.google.common.util.concurrent.ListeningExecutorService;
+import com.google.common.util.concurrent.MoreExecutors;
+import org.apache.druid.client.DruidServer;
+import org.apache.druid.java.util.common.DateTimes;
+import org.apache.druid.java.util.common.concurrent.Execs;
+import org.apache.druid.server.coordination.ServerType;
+import org.apache.druid.server.coordinator.CoordinatorDynamicConfig;
+import org.apache.druid.server.coordinator.CoordinatorRuntimeParamsTestHelpers;
+import org.apache.druid.server.coordinator.DruidCluster;
+import org.apache.druid.server.coordinator.DruidCoordinatorRuntimeParams;
+import org.apache.druid.server.coordinator.ReplicationThrottler;
+import org.apache.druid.server.coordinator.ServerHolder;
+import org.apache.druid.server.coordinator.balancer.BalancerSegmentHolder;
+import org.apache.druid.server.coordinator.balancer.BalancerStrategy;
+import org.apache.druid.server.coordinator.balancer.CostBalancerStrategyFactory;
+import org.apache.druid.server.coordinator.loadqueue.LoadQueuePeonTester;
+import org.apache.druid.server.coordinator.loadqueue.SegmentLoadQueueManager;
+import org.apache.druid.server.coordinator.stats.CoordinatorRunStats;
+import org.apache.druid.server.coordinator.stats.Stats;
+import org.apache.druid.timeline.DataSegment;
+import org.apache.druid.timeline.partition.NoneShardSpec;
+import org.easymock.EasyMock;
+import org.easymock.IExpectationSetters;
+import org.joda.time.DateTime;
+import org.joda.time.Interval;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+
+public class BalanceSegmentsTest
+{
+  private SegmentLoadQueueManager loadQueueManager;
+
+  private DataSegment segment1;
+  private DataSegment segment2;
+  private DataSegment segment3;
+  private DataSegment segment4;
+  private DataSegment segment5;
+
+  private DataSegment[] allSegments;
+
+  private ListeningExecutorService balancerStrategyExecutor;
+  private BalancerStrategy balancerStrategy;
+  private Set<String> broadcastDatasources;
+
+  private DruidServer server1;
+  private DruidServer server2;
+  private DruidServer server3;
+  private DruidServer server4;
+
+  @Before
+  public void setUp()
+  {
+    loadQueueManager = new SegmentLoadQueueManager(null, null, null);
+
+    // Create test segments for multiple datasources
+    final DateTime start1 = DateTimes.of("2012-01-01");
+    final DateTime start2 = DateTimes.of("2012-02-01");
+    final String version = DateTimes.of("2012-03-01").toString();
+
+    segment1 = createHourlySegment("datasource1", start1, version);
+    segment2 = createHourlySegment("datasource1", start2, version);
+    segment3 = createHourlySegment("datasource2", start1, version);
+    segment4 = createHourlySegment("datasource2", start2, version);
+    segment5 = createHourlySegment("datasourceBroadcast", start2, version);
+    allSegments = new DataSegment[]{segment1, segment2, segment3, segment4, segment5};
+
+    server1 = new DruidServer("server1", "server1", null, 100L, ServerType.HISTORICAL, "normal", 0);
+    server2 = new DruidServer("server2", "server2", null, 100L, ServerType.HISTORICAL, "normal", 0);
+    server3 = new DruidServer("server3", "server3", null, 100L, ServerType.HISTORICAL, "normal", 0);
+    server4 = new DruidServer("server4", "server4", null, 100L, ServerType.HISTORICAL, "normal", 0);
+
+    balancerStrategyExecutor = MoreExecutors.listeningDecorator(Execs.multiThreaded(1, "BalanceSegmentsTest-%d"));
+    balancerStrategy = new CostBalancerStrategyFactory().createBalancerStrategy(balancerStrategyExecutor);
+
+    broadcastDatasources = Collections.singleton("datasourceBroadcast");
+  }
+
+  @After
+  public void tearDown()
+  {
+    balancerStrategyExecutor.shutdownNow();
+  }
+
+  @Test
+  public void testMoveToEmptyServerBalancer()
+  {
+    final ServerHolder serverHolder1 = createHolder(server1, allSegments);
+    final ServerHolder serverHolder2 = createHolder(server2);
+
+    DruidCoordinatorRuntimeParams params =
+        defaultRuntimeParamsBuilder(serverHolder1, serverHolder2)
+            .withBalancerStrategy(balancerStrategy)
+            .withBroadcastDatasources(broadcastDatasources)
+            .build();
+
+    CoordinatorRunStats stats = runBalancer(params);
+    Assert.assertEquals(2, stats.getSegmentStat(Stats.Segments.MOVED, "normal", "datasource1"));
+    Assert.assertEquals(1, stats.getSegmentStat(Stats.Segments.MOVED, "normal", "datasource2"));
+  }
+
+  /**
+   * Server 1 has 2 segments.
+   * Server 2 (decommissioning) has 2 segments.
+   * Server 3 is empty.
+   * Decommissioning percent is 60.
+   * Max segments to move is 3.
+   * 2 (of 2) segments should be moved from Server 2 and 1 (of 2) from Server 1.
+   */
+  @Test
+  public void testMoveDecommissioningMaxPercentOfMaxSegmentsToMove()
+  {
+    final ServerHolder serverHolder1 = createHolder(server1, false, segment1, segment2);
+    final ServerHolder serverHolder2 = createHolder(server2, true, segment3, segment4);
+    final ServerHolder serverHolder3 = createHolder(server3, false);
+
+    BalancerStrategy strategy = EasyMock.createMock(BalancerStrategy.class);
+    expectPickLoadingSegmentsAndReturnEmpty(strategy).times(2);
+    expectPickLoadedSegmentsAndReturn(
+        strategy,
+        new BalancerSegmentHolder(serverHolder2, segment3),
+        new BalancerSegmentHolder(serverHolder2, segment4)
+    ).once();
+    expectPickLoadedSegmentsAndReturn(
+        strategy,
+        new BalancerSegmentHolder(serverHolder1, segment1),
+        new BalancerSegmentHolder(serverHolder1, segment2)
+    ).once();
+    expectFindDestinationAndReturn(strategy, serverHolder3);
+    EasyMock.replay(strategy);
+
+    // ceil(3 * 0.6) = 2 segments from decommissioning servers
+    CoordinatorDynamicConfig dynamicConfig =
+        CoordinatorDynamicConfig.builder()
+                                .withMaxSegmentsToMove(3)
+                                .withDecommissioningMaxPercentOfMaxSegmentsToMove(60)
+                                .build();
+    DruidCoordinatorRuntimeParams params =
+        defaultRuntimeParamsBuilder(serverHolder1, serverHolder2, serverHolder3)
+            .withDynamicConfigs(dynamicConfig)
+            .withBalancerStrategy(strategy)
+            .withBroadcastDatasources(broadcastDatasources)
+            .build();
+
+    CoordinatorRunStats stats = runBalancer(params);
+
+    EasyMock.verify(strategy);
+    Assert.assertEquals(1L, stats.getSegmentStat(Stats.Segments.MOVED, "normal", "datasource1"));
+    Assert.assertEquals(2L, stats.getSegmentStat(Stats.Segments.MOVED, "normal", "datasource2"));
+    Assert.assertEquals(
+        ImmutableSet.of(segment1, segment3, segment4),
+        serverHolder3.getPeon().getSegmentsToLoad()
+    );
+  }
+
+  @Test
+  public void testZeroDecommissioningMaxPercentOfMaxSegmentsToMove()
+  {
+    final ServerHolder holder1 = createHolder(server1, false, segment1, segment2);
+    final ServerHolder holder2 = createHolder(server2, true, segment3, segment4);
+    final ServerHolder holder3 = createHolder(server3, false);
+
+    CoordinatorDynamicConfig dynamicConfig =
+        CoordinatorDynamicConfig.builder().withDecommissioningMaxPercentOfMaxSegmentsToMove(0)
+                                .withMaxSegmentsToMove(1).build();
+    DruidCoordinatorRuntimeParams params =
+        defaultRuntimeParamsBuilder(holder1, holder2, holder3).withDynamicConfigs(dynamicConfig).build();
+
+    CoordinatorRunStats stats = runBalancer(params);
+
+    // Verify that either segment1 or segment2 is chosen for move
+    Assert.assertEquals(1L, stats.getSegmentStat(Stats.Segments.MOVED, "normal", segment1.getDataSource()));
+    DataSegment movingSegment = holder3.getPeon().getSegmentsToLoad().iterator().next();
+    Assert.assertEquals(segment1.getDataSource(), movingSegment.getDataSource());
+  }
+
+  @Test
+  public void testMaxDecommissioningMaxPercentOfMaxSegmentsToMove()
+  {
+    final ServerHolder holder1 = createHolder(server1, false, segment1, segment2);
+    final ServerHolder holder2 = createHolder(server2, true, segment3, segment4);
+    final ServerHolder holder3 = createHolder(server3, false);
+
+    CoordinatorDynamicConfig dynamicConfig =
+        CoordinatorDynamicConfig.builder()
+                                .withDecommissioningMaxPercentOfMaxSegmentsToMove(100)
+                                .withMaxSegmentsToMove(1).build();
+    DruidCoordinatorRuntimeParams params =
+        defaultRuntimeParamsBuilder(holder1, holder2, holder3).withDynamicConfigs(dynamicConfig).build();
+
+    CoordinatorRunStats stats = runBalancer(params);
+
+    // Verify that either segment3 or segment4 is chosen for move
+    Assert.assertEquals(1L, stats.getSegmentStat(Stats.Segments.MOVED, "normal", segment3.getDataSource()));
+    DataSegment movingSegment = holder3.getPeon().getSegmentsToLoad().iterator().next();
+    Assert.assertEquals(segment3.getDataSource(), movingSegment.getDataSource());
+  }
+
+  /**
+   * Should balance segments as usual (ignoring percent) with empty decommissioningNodes.
+   */
+  @Test
+  public void testMoveDecommissioningMaxPercentOfMaxSegmentsToMoveWithNoDecommissioning()
+  {
+    final ServerHolder serverHolder1 = createHolder(server1, segment1, segment2);
+    final ServerHolder serverHolder2 = createHolder(server2, segment3, segment4);
+    final ServerHolder serverHolder3 = createHolder(server3);
+    BalancerStrategy strategy = EasyMock.createMock(BalancerStrategy.class);
+    expectPickLoadingSegmentsAndReturnEmpty(strategy).once();
+    expectPickLoadedSegmentsAndReturn(
+        strategy,
+        new BalancerSegmentHolder(serverHolder1, segment2),
+        new BalancerSegmentHolder(serverHolder2, segment3),
+        new BalancerSegmentHolder(serverHolder2, segment4)
+    ).once();
+    expectFindDestinationAndReturn(strategy, serverHolder3);
+    EasyMock.replay(strategy);
+
+    CoordinatorDynamicConfig dynamicConfig =
+        CoordinatorDynamicConfig.builder()
+                                .withMaxSegmentsToMove(3)
+                                .withDecommissioningMaxPercentOfMaxSegmentsToMove(9)
+                                .build();
+    DruidCoordinatorRuntimeParams params =
+        defaultRuntimeParamsBuilder(serverHolder1, serverHolder2, serverHolder3)
+            .withDynamicConfigs(dynamicConfig)
+            .withBalancerStrategy(strategy)
+            .build();
+
+    CoordinatorRunStats stats = runBalancer(params);
+    EasyMock.verify(strategy);
+    Assert.assertEquals(1L, stats.getSegmentStat(Stats.Segments.MOVED, "normal", "datasource1"));
+    Assert.assertEquals(2L, stats.getSegmentStat(Stats.Segments.MOVED, "normal", "datasource2"));
+    Assert.assertEquals(
+        ImmutableSet.of(segment2, segment3, segment4),
+        serverHolder3.getPeon().getSegmentsToLoad()
+    );
+  }
+
+  /**
+   * Shouldn't move segments to a decommissioning server.
+   */
+  @Test
+  public void testMoveToDecommissioningServer()
+  {
+    final ServerHolder serverHolder1 = createHolder(server1, false, allSegments);
+    final ServerHolder serverHolder2 = createHolder(server2, true);
+
+    BalancerStrategy strategy = EasyMock.createMock(BalancerStrategy.class);
+    expectPickLoadingSegmentsAndReturnEmpty(strategy).times(2);
+    expectPickLoadedSegmentsAndReturn(
+        strategy,
+        new BalancerSegmentHolder(serverHolder1, segment1)
+    ).times(2);
+    EasyMock.expect(
+        strategy.findDestinationServerToMoveSegment(
+            EasyMock.anyObject(),
+            EasyMock.anyObject(),
+            EasyMock.anyObject()
+        )
+    ).andAnswer(() -> ((List<ServerHolder>) EasyMock.getCurrentArguments()[2]).get(0)).anyTimes();
+    EasyMock.replay(strategy);
+
+    DruidCoordinatorRuntimeParams params =
+        defaultRuntimeParamsBuilder(serverHolder1, serverHolder2)
+            .withBalancerStrategy(strategy)
+            .withBroadcastDatasources(broadcastDatasources)
+            .build();
+
+    CoordinatorRunStats stats = runBalancer(params);
+    EasyMock.verify(strategy);
+    Assert.assertEquals(0, stats.getTieredStat(Stats.Segments.MOVED, "normal"));
+  }
+
+  @Test
+  public void testMoveFromDecommissioningServer()
+  {
+    final ServerHolder holder1 = createHolder(server1, allSegments);
+    final ServerHolder holder2 = createHolder(server2);
+
+    BalancerStrategy strategy = EasyMock.createMock(BalancerStrategy.class);
+    expectPickLoadingSegmentsAndReturnEmpty(strategy).once();
+    expectPickLoadedSegmentsAndReturn(strategy, new BalancerSegmentHolder(holder1, segment1)).once();
+    expectFindDestinationAndReturn(strategy, holder2);
+    EasyMock.replay(strategy);
+
+    DruidCoordinatorRuntimeParams params = defaultRuntimeParamsBuilder(holder1, holder2)
+        .withDynamicConfigs(CoordinatorDynamicConfig.builder().withMaxSegmentsToMove(1).build())
+        .withBalancerStrategy(strategy)
+        .withBroadcastDatasources(broadcastDatasources)
+        .build();
+
+    CoordinatorRunStats stats = runBalancer(params);
+    EasyMock.verify(strategy);
+    Assert.assertEquals(1, stats.getSegmentStat(Stats.Segments.MOVED, "normal", segment1.getDataSource()));
+    Assert.assertEquals(0, holder1.getPeon().getNumberOfSegmentsToLoad());
+    Assert.assertEquals(1, holder2.getPeon().getNumberOfSegmentsToLoad());
+  }
+
+  @Test
+  public void testMoveMaxLoadQueueServerBalancer()
+  {
+    final int maxSegmentsInQueue = 1;
+    final ServerHolder holder1 = createHolder(server1, maxSegmentsInQueue, false, allSegments);
+    final ServerHolder holder2 = createHolder(server2, maxSegmentsInQueue, false);
+
+    final CoordinatorDynamicConfig dynamicConfig = CoordinatorDynamicConfig
+        .builder()
+        .withMaxSegmentsInNodeLoadingQueue(maxSegmentsInQueue)
+        .build();
+    DruidCoordinatorRuntimeParams params =
+        defaultRuntimeParamsBuilder(holder1, holder2)
+            .withDynamicConfigs(dynamicConfig)
+            .build();
+
+    CoordinatorRunStats stats = runBalancer(params);
+
+    // max to move is 5, all segments on server 1, but only expect to move 1 to server 2 since max node load queue is 1
+    Assert.assertEquals(maxSegmentsInQueue, stats.getSegmentStat(Stats.Segments.MOVED, "normal", "datasource1"));
+  }
+
+  @Test
+  public void testRun1()
+  {
+    // Mock some servers of different usages
+    DruidCoordinatorRuntimeParams params = defaultRuntimeParamsBuilder(
+        createHolder(server1, allSegments),
+        createHolder(server2)
+    ).build();
+
+    CoordinatorRunStats stats = runBalancer(params);
+    Assert.assertTrue(stats.getSegmentStat(Stats.Segments.MOVED, "normal", "datasource1") > 0);
+  }
+
+  @Test
+  public void testRun2()
+  {
+    // Mock some servers of different usages
+    DruidCoordinatorRuntimeParams params = defaultRuntimeParamsBuilder(
+        createHolder(server1, allSegments),
+        createHolder(server2),
+        createHolder(server3),
+        createHolder(server4)
+    ).build();
+
+    CoordinatorRunStats stats = runBalancer(params);
+    Assert.assertTrue(stats.getSegmentStat(Stats.Segments.MOVED, "normal", "datasource1") > 0);
+  }
+
+  @Test
+  public void testThatMaxSegmentsToMoveIsHonored()
+  {
+    // Move from non-decomissioning servers
+    final ServerHolder holder1 = createHolder(server1, segment1, segment2);
+    final ServerHolder holder2 = createHolder(server2, segment3, segment4);
+    final ServerHolder holder3 = createHolder(server3);
+
+    BalancerStrategy strategy = EasyMock.createMock(BalancerStrategy.class);
+    expectPickLoadingSegmentsAndReturnEmpty(strategy).once();
+    expectPickLoadedSegmentsAndReturn(strategy, new BalancerSegmentHolder(holder2, segment3)).once();
+    expectFindDestinationAndReturn(strategy, holder3);
+    EasyMock.replay(strategy);
+
+    DruidCoordinatorRuntimeParams params =
+        defaultRuntimeParamsBuilder(holder1, holder2, holder3)
+            .withDynamicConfigs(
+                CoordinatorDynamicConfig.builder()
+                                        .withMaxSegmentsToMove(1)
+                                        .withUseBatchedSegmentSampler(true)
+                                        .withPercentOfSegmentsToConsiderPerMove(40)
+                                        .build()
+            )
+            .withBalancerStrategy(strategy)
+            .withBroadcastDatasources(broadcastDatasources)
+            .build();
+
+    CoordinatorRunStats stats = runBalancer(params);
+    EasyMock.verify(strategy);
+    Assert.assertEquals(1L, stats.getSegmentStat(Stats.Segments.MOVED, "normal", "datasource2"));
+    Assert.assertEquals(ImmutableSet.of(segment3), holder3.getPeon().getSegmentsToLoad());
+  }
+
+  @Test
+  public void testUseBatchedSegmentSampler()
+  {
+    DruidCoordinatorRuntimeParams params = defaultRuntimeParamsBuilder(
+        createHolder(server1, allSegments),
+        createHolder(server2),
+        createHolder(server3),
+        createHolder(server4)
+    )
+        .withDynamicConfigs(
+            CoordinatorDynamicConfig.builder()
+                                    .withMaxSegmentsToMove(2)
+                                    .withUseBatchedSegmentSampler(true)
+                                    .build()
+        )
+        .withBroadcastDatasources(broadcastDatasources)
+        .build();
+
+    CoordinatorRunStats stats = runBalancer(params);
+    long totalMoved = stats.getSegmentStat(Stats.Segments.MOVED, "normal", "datasource1")
+                      + stats.getSegmentStat(Stats.Segments.MOVED, "normal", "datasource2");
+    Assert.assertEquals(2L, totalMoved);
+  }
+
+  private CoordinatorRunStats runBalancer(DruidCoordinatorRuntimeParams params)
+  {
+    params = new BalanceSegments(loadQueueManager).run(params);
+    if (params == null) {
+      Assert.fail("BalanceSegments duty returned null params");
+      return new CoordinatorRunStats();
+    } else {
+      return params.getCoordinatorStats();
+    }
+  }
+
+  private DruidCoordinatorRuntimeParams.Builder defaultRuntimeParamsBuilder(
+      ServerHolder... servers
+  )
+  {
+    return CoordinatorRuntimeParamsTestHelpers
+        .newBuilder()
+        .withDruidCluster(DruidCluster.builder().addTier("normal", servers).build())
+        .withUsedSegmentsInTest(allSegments)
+        .withBroadcastDatasources(broadcastDatasources)
+        .withBalancerStrategy(balancerStrategy)
+        .withReplicationManager(createReplicationThrottler());
+  }
+
+  private ServerHolder createHolder(DruidServer server, DataSegment... loadedSegments)
+  {
+    return createHolder(server, false, loadedSegments);
+  }
+
+  private ServerHolder createHolder(DruidServer server, boolean isDecommissioning, DataSegment... loadedSegments)
+  {
+    return createHolder(server, 0, isDecommissioning, loadedSegments);
+  }
+
+  private ServerHolder createHolder(
+      DruidServer server,
+      int maxSegmentsInLoadQueue,
+      boolean isDecommissioning,
+      DataSegment... loadedSegments
+  )
+  {
+    for (DataSegment segment : loadedSegments) {
+      server.addDataSegment(segment);
+    }
+
+    return new ServerHolder(
+        server.toImmutableDruidServer(),
+        new LoadQueuePeonTester(),
+        isDecommissioning,
+        maxSegmentsInLoadQueue
+    );
+  }
+
+  private IExpectationSetters<Iterator<BalancerSegmentHolder>> expectPickLoadingSegmentsAndReturnEmpty(
+      BalancerStrategy strategy
+  )
+  {
+    return EasyMock.expect(
+        strategy.pickSegmentsToMove(
+            EasyMock.anyObject(),
+            EasyMock.anyObject(),
+            EasyMock.anyInt(),
+            EasyMock.eq(true)
+        )
+    ).andReturn(Collections.emptyIterator());
+  }
+
+  private IExpectationSetters<Iterator<BalancerSegmentHolder>> expectPickLoadedSegmentsAndReturn(
+      BalancerStrategy strategy,
+      BalancerSegmentHolder... pickedLoadedSegments
+  )
+  {
+    return EasyMock.expect(
+        strategy.pickSegmentsToMove(
+            EasyMock.anyObject(),
+            EasyMock.anyObject(),
+            EasyMock.anyInt(),
+            EasyMock.eq(false)
+        )
+    ).andReturn(Arrays.asList(pickedLoadedSegments).iterator());
+  }
+
+  private void expectFindDestinationAndReturn(BalancerStrategy strategy, ServerHolder chosenServer)
+  {
+    EasyMock.expect(
+        strategy.findDestinationServerToMoveSegment(
+            EasyMock.anyObject(),
+            EasyMock.anyObject(),
+            EasyMock.anyObject()
+        )
+    ).andReturn(chosenServer).anyTimes();
+  }
+
+  private ReplicationThrottler createReplicationThrottler()
+  {
+    CoordinatorDynamicConfig dynamicConfig = CoordinatorDynamicConfig.builder().build();
+    return new ReplicationThrottler(
+        Collections.singleton("normal"),
+        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/4839)



##########
server/src/main/java/org/apache/druid/server/coordinator/balancer/TierSegmentBalancer.java:
##########
@@ -0,0 +1,245 @@
+/*
+ * 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.balancer;
+
+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.CoordinatorDynamicConfig;
+import org.apache.druid.server.coordinator.DruidCoordinatorRuntimeParams;
+import org.apache.druid.server.coordinator.ServerHolder;
+import org.apache.druid.server.coordinator.StrategicSegmentAssigner;
+import org.apache.druid.server.coordinator.stats.CoordinatorRunStats;
+import org.apache.druid.server.coordinator.stats.CoordinatorStat;
+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 StrategicSegmentAssigner segmentAssigner;
+
+  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,
+      StrategicSegmentAssigner segmentAssigner,
+      DruidCoordinatorRuntimeParams params
+  )
+  {
+    this.tier = tier;
+    this.params = params;
+    this.segmentAssigner = segmentAssigner;
+
+    this.strategy = params.getBalancerStrategy();
+    this.dynamicConfig = params.getCoordinatorDynamicConfig();
+    this.totalMaxSegmentsToMove = dynamicConfig.getMaxSegmentsToMove();
+    this.runStats = segmentAssigner.getStats();
+
+    Map<Boolean, List<ServerHolder>> partitions =
+        servers.stream().collect(Collectors.partitioningBy(ServerHolder::isDecommissioning));
+    decommissioningServers = partitions.get(true);
+    activeServers = partitions.get(false);
+    this.allServers = servers;
+  }
+
+  public 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 movedDecommSegments = 0;
+    if (!decommissioningServers.isEmpty()) {
+      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/4835)



##########
server/src/test/java/org/apache/druid/server/coordinator/duty/BalanceSegmentsTest.java:
##########
@@ -0,0 +1,554 @@
+/*
+ * 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.ImmutableSet;
+import com.google.common.util.concurrent.ListeningExecutorService;
+import com.google.common.util.concurrent.MoreExecutors;
+import org.apache.druid.client.DruidServer;
+import org.apache.druid.java.util.common.DateTimes;
+import org.apache.druid.java.util.common.concurrent.Execs;
+import org.apache.druid.server.coordination.ServerType;
+import org.apache.druid.server.coordinator.CoordinatorDynamicConfig;
+import org.apache.druid.server.coordinator.CoordinatorRuntimeParamsTestHelpers;
+import org.apache.druid.server.coordinator.DruidCluster;
+import org.apache.druid.server.coordinator.DruidCoordinatorRuntimeParams;
+import org.apache.druid.server.coordinator.ReplicationThrottler;
+import org.apache.druid.server.coordinator.ServerHolder;
+import org.apache.druid.server.coordinator.balancer.BalancerSegmentHolder;
+import org.apache.druid.server.coordinator.balancer.BalancerStrategy;
+import org.apache.druid.server.coordinator.balancer.CostBalancerStrategyFactory;
+import org.apache.druid.server.coordinator.loadqueue.LoadQueuePeonTester;
+import org.apache.druid.server.coordinator.loadqueue.SegmentLoadQueueManager;
+import org.apache.druid.server.coordinator.stats.CoordinatorRunStats;
+import org.apache.druid.server.coordinator.stats.Stats;
+import org.apache.druid.timeline.DataSegment;
+import org.apache.druid.timeline.partition.NoneShardSpec;
+import org.easymock.EasyMock;
+import org.easymock.IExpectationSetters;
+import org.joda.time.DateTime;
+import org.joda.time.Interval;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+
+public class BalanceSegmentsTest
+{
+  private SegmentLoadQueueManager loadQueueManager;
+
+  private DataSegment segment1;
+  private DataSegment segment2;
+  private DataSegment segment3;
+  private DataSegment segment4;
+  private DataSegment segment5;
+
+  private DataSegment[] allSegments;
+
+  private ListeningExecutorService balancerStrategyExecutor;
+  private BalancerStrategy balancerStrategy;
+  private Set<String> broadcastDatasources;
+
+  private DruidServer server1;
+  private DruidServer server2;
+  private DruidServer server3;
+  private DruidServer server4;
+
+  @Before
+  public void setUp()
+  {
+    loadQueueManager = new SegmentLoadQueueManager(null, null, null);
+
+    // Create test segments for multiple datasources
+    final DateTime start1 = DateTimes.of("2012-01-01");
+    final DateTime start2 = DateTimes.of("2012-02-01");
+    final String version = DateTimes.of("2012-03-01").toString();
+
+    segment1 = createHourlySegment("datasource1", start1, version);
+    segment2 = createHourlySegment("datasource1", start2, version);
+    segment3 = createHourlySegment("datasource2", start1, version);
+    segment4 = createHourlySegment("datasource2", start2, version);
+    segment5 = createHourlySegment("datasourceBroadcast", start2, version);
+    allSegments = new DataSegment[]{segment1, segment2, segment3, segment4, segment5};
+
+    server1 = new DruidServer("server1", "server1", null, 100L, ServerType.HISTORICAL, "normal", 0);
+    server2 = new DruidServer("server2", "server2", null, 100L, ServerType.HISTORICAL, "normal", 0);
+    server3 = new DruidServer("server3", "server3", null, 100L, ServerType.HISTORICAL, "normal", 0);
+    server4 = new DruidServer("server4", "server4", null, 100L, ServerType.HISTORICAL, "normal", 0);
+
+    balancerStrategyExecutor = MoreExecutors.listeningDecorator(Execs.multiThreaded(1, "BalanceSegmentsTest-%d"));
+    balancerStrategy = new CostBalancerStrategyFactory().createBalancerStrategy(balancerStrategyExecutor);
+
+    broadcastDatasources = Collections.singleton("datasourceBroadcast");
+  }
+
+  @After
+  public void tearDown()
+  {
+    balancerStrategyExecutor.shutdownNow();
+  }
+
+  @Test
+  public void testMoveToEmptyServerBalancer()
+  {
+    final ServerHolder serverHolder1 = createHolder(server1, allSegments);
+    final ServerHolder serverHolder2 = createHolder(server2);
+
+    DruidCoordinatorRuntimeParams params =
+        defaultRuntimeParamsBuilder(serverHolder1, serverHolder2)
+            .withBalancerStrategy(balancerStrategy)
+            .withBroadcastDatasources(broadcastDatasources)
+            .build();
+
+    CoordinatorRunStats stats = runBalancer(params);
+    Assert.assertEquals(2, stats.getSegmentStat(Stats.Segments.MOVED, "normal", "datasource1"));
+    Assert.assertEquals(1, stats.getSegmentStat(Stats.Segments.MOVED, "normal", "datasource2"));
+  }
+
+  /**
+   * Server 1 has 2 segments.
+   * Server 2 (decommissioning) has 2 segments.
+   * Server 3 is empty.
+   * Decommissioning percent is 60.
+   * Max segments to move is 3.
+   * 2 (of 2) segments should be moved from Server 2 and 1 (of 2) from Server 1.
+   */
+  @Test
+  public void testMoveDecommissioningMaxPercentOfMaxSegmentsToMove()
+  {
+    final ServerHolder serverHolder1 = createHolder(server1, false, segment1, segment2);
+    final ServerHolder serverHolder2 = createHolder(server2, true, segment3, segment4);
+    final ServerHolder serverHolder3 = createHolder(server3, false);
+
+    BalancerStrategy strategy = EasyMock.createMock(BalancerStrategy.class);
+    expectPickLoadingSegmentsAndReturnEmpty(strategy).times(2);
+    expectPickLoadedSegmentsAndReturn(
+        strategy,
+        new BalancerSegmentHolder(serverHolder2, segment3),
+        new BalancerSegmentHolder(serverHolder2, segment4)
+    ).once();
+    expectPickLoadedSegmentsAndReturn(
+        strategy,
+        new BalancerSegmentHolder(serverHolder1, segment1),
+        new BalancerSegmentHolder(serverHolder1, segment2)
+    ).once();
+    expectFindDestinationAndReturn(strategy, serverHolder3);
+    EasyMock.replay(strategy);
+
+    // ceil(3 * 0.6) = 2 segments from decommissioning servers
+    CoordinatorDynamicConfig dynamicConfig =
+        CoordinatorDynamicConfig.builder()
+                                .withMaxSegmentsToMove(3)
+                                .withDecommissioningMaxPercentOfMaxSegmentsToMove(60)
+                                .build();
+    DruidCoordinatorRuntimeParams params =
+        defaultRuntimeParamsBuilder(serverHolder1, serverHolder2, serverHolder3)
+            .withDynamicConfigs(dynamicConfig)
+            .withBalancerStrategy(strategy)
+            .withBroadcastDatasources(broadcastDatasources)
+            .build();
+
+    CoordinatorRunStats stats = runBalancer(params);
+
+    EasyMock.verify(strategy);
+    Assert.assertEquals(1L, stats.getSegmentStat(Stats.Segments.MOVED, "normal", "datasource1"));
+    Assert.assertEquals(2L, stats.getSegmentStat(Stats.Segments.MOVED, "normal", "datasource2"));
+    Assert.assertEquals(
+        ImmutableSet.of(segment1, segment3, segment4),
+        serverHolder3.getPeon().getSegmentsToLoad()
+    );
+  }
+
+  @Test
+  public void testZeroDecommissioningMaxPercentOfMaxSegmentsToMove()
+  {
+    final ServerHolder holder1 = createHolder(server1, false, segment1, segment2);
+    final ServerHolder holder2 = createHolder(server2, true, segment3, segment4);
+    final ServerHolder holder3 = createHolder(server3, false);
+
+    CoordinatorDynamicConfig dynamicConfig =
+        CoordinatorDynamicConfig.builder().withDecommissioningMaxPercentOfMaxSegmentsToMove(0)
+                                .withMaxSegmentsToMove(1).build();
+    DruidCoordinatorRuntimeParams params =
+        defaultRuntimeParamsBuilder(holder1, holder2, holder3).withDynamicConfigs(dynamicConfig).build();
+
+    CoordinatorRunStats stats = runBalancer(params);
+
+    // Verify that either segment1 or segment2 is chosen for move
+    Assert.assertEquals(1L, stats.getSegmentStat(Stats.Segments.MOVED, "normal", segment1.getDataSource()));
+    DataSegment movingSegment = holder3.getPeon().getSegmentsToLoad().iterator().next();
+    Assert.assertEquals(segment1.getDataSource(), movingSegment.getDataSource());
+  }
+
+  @Test
+  public void testMaxDecommissioningMaxPercentOfMaxSegmentsToMove()
+  {
+    final ServerHolder holder1 = createHolder(server1, false, segment1, segment2);
+    final ServerHolder holder2 = createHolder(server2, true, segment3, segment4);
+    final ServerHolder holder3 = createHolder(server3, false);
+
+    CoordinatorDynamicConfig dynamicConfig =
+        CoordinatorDynamicConfig.builder()
+                                .withDecommissioningMaxPercentOfMaxSegmentsToMove(100)
+                                .withMaxSegmentsToMove(1).build();
+    DruidCoordinatorRuntimeParams params =
+        defaultRuntimeParamsBuilder(holder1, holder2, holder3).withDynamicConfigs(dynamicConfig).build();
+
+    CoordinatorRunStats stats = runBalancer(params);
+
+    // Verify that either segment3 or segment4 is chosen for move
+    Assert.assertEquals(1L, stats.getSegmentStat(Stats.Segments.MOVED, "normal", segment3.getDataSource()));
+    DataSegment movingSegment = holder3.getPeon().getSegmentsToLoad().iterator().next();
+    Assert.assertEquals(segment3.getDataSource(), movingSegment.getDataSource());
+  }
+
+  /**
+   * Should balance segments as usual (ignoring percent) with empty decommissioningNodes.
+   */
+  @Test
+  public void testMoveDecommissioningMaxPercentOfMaxSegmentsToMoveWithNoDecommissioning()
+  {
+    final ServerHolder serverHolder1 = createHolder(server1, segment1, segment2);
+    final ServerHolder serverHolder2 = createHolder(server2, segment3, segment4);
+    final ServerHolder serverHolder3 = createHolder(server3);
+    BalancerStrategy strategy = EasyMock.createMock(BalancerStrategy.class);
+    expectPickLoadingSegmentsAndReturnEmpty(strategy).once();
+    expectPickLoadedSegmentsAndReturn(
+        strategy,
+        new BalancerSegmentHolder(serverHolder1, segment2),
+        new BalancerSegmentHolder(serverHolder2, segment3),
+        new BalancerSegmentHolder(serverHolder2, segment4)
+    ).once();
+    expectFindDestinationAndReturn(strategy, serverHolder3);
+    EasyMock.replay(strategy);
+
+    CoordinatorDynamicConfig dynamicConfig =
+        CoordinatorDynamicConfig.builder()
+                                .withMaxSegmentsToMove(3)
+                                .withDecommissioningMaxPercentOfMaxSegmentsToMove(9)
+                                .build();
+    DruidCoordinatorRuntimeParams params =
+        defaultRuntimeParamsBuilder(serverHolder1, serverHolder2, serverHolder3)
+            .withDynamicConfigs(dynamicConfig)
+            .withBalancerStrategy(strategy)
+            .build();
+
+    CoordinatorRunStats stats = runBalancer(params);
+    EasyMock.verify(strategy);
+    Assert.assertEquals(1L, stats.getSegmentStat(Stats.Segments.MOVED, "normal", "datasource1"));
+    Assert.assertEquals(2L, stats.getSegmentStat(Stats.Segments.MOVED, "normal", "datasource2"));
+    Assert.assertEquals(
+        ImmutableSet.of(segment2, segment3, segment4),
+        serverHolder3.getPeon().getSegmentsToLoad()
+    );
+  }
+
+  /**
+   * Shouldn't move segments to a decommissioning server.
+   */
+  @Test
+  public void testMoveToDecommissioningServer()
+  {
+    final ServerHolder serverHolder1 = createHolder(server1, false, allSegments);
+    final ServerHolder serverHolder2 = createHolder(server2, true);
+
+    BalancerStrategy strategy = EasyMock.createMock(BalancerStrategy.class);
+    expectPickLoadingSegmentsAndReturnEmpty(strategy).times(2);
+    expectPickLoadedSegmentsAndReturn(
+        strategy,
+        new BalancerSegmentHolder(serverHolder1, segment1)
+    ).times(2);
+    EasyMock.expect(
+        strategy.findDestinationServerToMoveSegment(
+            EasyMock.anyObject(),
+            EasyMock.anyObject(),
+            EasyMock.anyObject()
+        )
+    ).andAnswer(() -> ((List<ServerHolder>) EasyMock.getCurrentArguments()[2]).get(0)).anyTimes();
+    EasyMock.replay(strategy);
+
+    DruidCoordinatorRuntimeParams params =
+        defaultRuntimeParamsBuilder(serverHolder1, serverHolder2)
+            .withBalancerStrategy(strategy)
+            .withBroadcastDatasources(broadcastDatasources)
+            .build();
+
+    CoordinatorRunStats stats = runBalancer(params);
+    EasyMock.verify(strategy);
+    Assert.assertEquals(0, stats.getTieredStat(Stats.Segments.MOVED, "normal"));
+  }
+
+  @Test
+  public void testMoveFromDecommissioningServer()
+  {
+    final ServerHolder holder1 = createHolder(server1, allSegments);
+    final ServerHolder holder2 = createHolder(server2);
+
+    BalancerStrategy strategy = EasyMock.createMock(BalancerStrategy.class);
+    expectPickLoadingSegmentsAndReturnEmpty(strategy).once();
+    expectPickLoadedSegmentsAndReturn(strategy, new BalancerSegmentHolder(holder1, segment1)).once();
+    expectFindDestinationAndReturn(strategy, holder2);
+    EasyMock.replay(strategy);
+
+    DruidCoordinatorRuntimeParams params = defaultRuntimeParamsBuilder(holder1, holder2)
+        .withDynamicConfigs(CoordinatorDynamicConfig.builder().withMaxSegmentsToMove(1).build())
+        .withBalancerStrategy(strategy)
+        .withBroadcastDatasources(broadcastDatasources)
+        .build();
+
+    CoordinatorRunStats stats = runBalancer(params);
+    EasyMock.verify(strategy);
+    Assert.assertEquals(1, stats.getSegmentStat(Stats.Segments.MOVED, "normal", segment1.getDataSource()));
+    Assert.assertEquals(0, holder1.getPeon().getNumberOfSegmentsToLoad());
+    Assert.assertEquals(1, holder2.getPeon().getNumberOfSegmentsToLoad());
+  }
+
+  @Test
+  public void testMoveMaxLoadQueueServerBalancer()
+  {
+    final int maxSegmentsInQueue = 1;
+    final ServerHolder holder1 = createHolder(server1, maxSegmentsInQueue, false, allSegments);
+    final ServerHolder holder2 = createHolder(server2, maxSegmentsInQueue, false);
+
+    final CoordinatorDynamicConfig dynamicConfig = CoordinatorDynamicConfig
+        .builder()
+        .withMaxSegmentsInNodeLoadingQueue(maxSegmentsInQueue)
+        .build();
+    DruidCoordinatorRuntimeParams params =
+        defaultRuntimeParamsBuilder(holder1, holder2)
+            .withDynamicConfigs(dynamicConfig)
+            .build();
+
+    CoordinatorRunStats stats = runBalancer(params);
+
+    // max to move is 5, all segments on server 1, but only expect to move 1 to server 2 since max node load queue is 1
+    Assert.assertEquals(maxSegmentsInQueue, stats.getSegmentStat(Stats.Segments.MOVED, "normal", "datasource1"));
+  }
+
+  @Test
+  public void testRun1()
+  {
+    // Mock some servers of different usages
+    DruidCoordinatorRuntimeParams params = defaultRuntimeParamsBuilder(
+        createHolder(server1, allSegments),
+        createHolder(server2)
+    ).build();
+
+    CoordinatorRunStats stats = runBalancer(params);
+    Assert.assertTrue(stats.getSegmentStat(Stats.Segments.MOVED, "normal", "datasource1") > 0);
+  }
+
+  @Test
+  public void testRun2()
+  {
+    // Mock some servers of different usages
+    DruidCoordinatorRuntimeParams params = defaultRuntimeParamsBuilder(
+        createHolder(server1, allSegments),
+        createHolder(server2),
+        createHolder(server3),
+        createHolder(server4)
+    ).build();
+
+    CoordinatorRunStats stats = runBalancer(params);
+    Assert.assertTrue(stats.getSegmentStat(Stats.Segments.MOVED, "normal", "datasource1") > 0);
+  }
+
+  @Test
+  public void testThatMaxSegmentsToMoveIsHonored()
+  {
+    // Move from non-decomissioning servers
+    final ServerHolder holder1 = createHolder(server1, segment1, segment2);
+    final ServerHolder holder2 = createHolder(server2, segment3, segment4);
+    final ServerHolder holder3 = createHolder(server3);
+
+    BalancerStrategy strategy = EasyMock.createMock(BalancerStrategy.class);
+    expectPickLoadingSegmentsAndReturnEmpty(strategy).once();
+    expectPickLoadedSegmentsAndReturn(strategy, new BalancerSegmentHolder(holder2, segment3)).once();
+    expectFindDestinationAndReturn(strategy, holder3);
+    EasyMock.replay(strategy);
+
+    DruidCoordinatorRuntimeParams params =
+        defaultRuntimeParamsBuilder(holder1, holder2, holder3)
+            .withDynamicConfigs(
+                CoordinatorDynamicConfig.builder()
+                                        .withMaxSegmentsToMove(1)
+                                        .withUseBatchedSegmentSampler(true)
+                                        .withPercentOfSegmentsToConsiderPerMove(40)
+                                        .build()
+            )
+            .withBalancerStrategy(strategy)
+            .withBroadcastDatasources(broadcastDatasources)
+            .build();
+
+    CoordinatorRunStats stats = runBalancer(params);
+    EasyMock.verify(strategy);
+    Assert.assertEquals(1L, stats.getSegmentStat(Stats.Segments.MOVED, "normal", "datasource2"));
+    Assert.assertEquals(ImmutableSet.of(segment3), holder3.getPeon().getSegmentsToLoad());
+  }
+
+  @Test
+  public void testUseBatchedSegmentSampler()
+  {
+    DruidCoordinatorRuntimeParams params = defaultRuntimeParamsBuilder(
+        createHolder(server1, allSegments),
+        createHolder(server2),
+        createHolder(server3),
+        createHolder(server4)
+    )
+        .withDynamicConfigs(
+            CoordinatorDynamicConfig.builder()
+                                    .withMaxSegmentsToMove(2)
+                                    .withUseBatchedSegmentSampler(true)

Review Comment:
   ## Deprecated method or constructor invocation
   
   Invoking [Builder.withUseBatchedSegmentSampler](1) should be avoided because it has been deprecated.
   
   [Show more details](https://github.com/apache/druid/security/code-scanning/4838)



##########
server/src/test/java/org/apache/druid/server/coordinator/duty/BalanceSegmentsTest.java:
##########
@@ -0,0 +1,554 @@
+/*
+ * 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.ImmutableSet;
+import com.google.common.util.concurrent.ListeningExecutorService;
+import com.google.common.util.concurrent.MoreExecutors;
+import org.apache.druid.client.DruidServer;
+import org.apache.druid.java.util.common.DateTimes;
+import org.apache.druid.java.util.common.concurrent.Execs;
+import org.apache.druid.server.coordination.ServerType;
+import org.apache.druid.server.coordinator.CoordinatorDynamicConfig;
+import org.apache.druid.server.coordinator.CoordinatorRuntimeParamsTestHelpers;
+import org.apache.druid.server.coordinator.DruidCluster;
+import org.apache.druid.server.coordinator.DruidCoordinatorRuntimeParams;
+import org.apache.druid.server.coordinator.ReplicationThrottler;
+import org.apache.druid.server.coordinator.ServerHolder;
+import org.apache.druid.server.coordinator.balancer.BalancerSegmentHolder;
+import org.apache.druid.server.coordinator.balancer.BalancerStrategy;
+import org.apache.druid.server.coordinator.balancer.CostBalancerStrategyFactory;
+import org.apache.druid.server.coordinator.loadqueue.LoadQueuePeonTester;
+import org.apache.druid.server.coordinator.loadqueue.SegmentLoadQueueManager;
+import org.apache.druid.server.coordinator.stats.CoordinatorRunStats;
+import org.apache.druid.server.coordinator.stats.Stats;
+import org.apache.druid.timeline.DataSegment;
+import org.apache.druid.timeline.partition.NoneShardSpec;
+import org.easymock.EasyMock;
+import org.easymock.IExpectationSetters;
+import org.joda.time.DateTime;
+import org.joda.time.Interval;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+
+public class BalanceSegmentsTest
+{
+  private SegmentLoadQueueManager loadQueueManager;
+
+  private DataSegment segment1;
+  private DataSegment segment2;
+  private DataSegment segment3;
+  private DataSegment segment4;
+  private DataSegment segment5;
+
+  private DataSegment[] allSegments;
+
+  private ListeningExecutorService balancerStrategyExecutor;
+  private BalancerStrategy balancerStrategy;
+  private Set<String> broadcastDatasources;
+
+  private DruidServer server1;
+  private DruidServer server2;
+  private DruidServer server3;
+  private DruidServer server4;
+
+  @Before
+  public void setUp()
+  {
+    loadQueueManager = new SegmentLoadQueueManager(null, null, null);
+
+    // Create test segments for multiple datasources
+    final DateTime start1 = DateTimes.of("2012-01-01");
+    final DateTime start2 = DateTimes.of("2012-02-01");
+    final String version = DateTimes.of("2012-03-01").toString();
+
+    segment1 = createHourlySegment("datasource1", start1, version);
+    segment2 = createHourlySegment("datasource1", start2, version);
+    segment3 = createHourlySegment("datasource2", start1, version);
+    segment4 = createHourlySegment("datasource2", start2, version);
+    segment5 = createHourlySegment("datasourceBroadcast", start2, version);
+    allSegments = new DataSegment[]{segment1, segment2, segment3, segment4, segment5};
+
+    server1 = new DruidServer("server1", "server1", null, 100L, ServerType.HISTORICAL, "normal", 0);
+    server2 = new DruidServer("server2", "server2", null, 100L, ServerType.HISTORICAL, "normal", 0);
+    server3 = new DruidServer("server3", "server3", null, 100L, ServerType.HISTORICAL, "normal", 0);
+    server4 = new DruidServer("server4", "server4", null, 100L, ServerType.HISTORICAL, "normal", 0);
+
+    balancerStrategyExecutor = MoreExecutors.listeningDecorator(Execs.multiThreaded(1, "BalanceSegmentsTest-%d"));
+    balancerStrategy = new CostBalancerStrategyFactory().createBalancerStrategy(balancerStrategyExecutor);
+
+    broadcastDatasources = Collections.singleton("datasourceBroadcast");
+  }
+
+  @After
+  public void tearDown()
+  {
+    balancerStrategyExecutor.shutdownNow();
+  }
+
+  @Test
+  public void testMoveToEmptyServerBalancer()
+  {
+    final ServerHolder serverHolder1 = createHolder(server1, allSegments);
+    final ServerHolder serverHolder2 = createHolder(server2);
+
+    DruidCoordinatorRuntimeParams params =
+        defaultRuntimeParamsBuilder(serverHolder1, serverHolder2)
+            .withBalancerStrategy(balancerStrategy)
+            .withBroadcastDatasources(broadcastDatasources)
+            .build();
+
+    CoordinatorRunStats stats = runBalancer(params);
+    Assert.assertEquals(2, stats.getSegmentStat(Stats.Segments.MOVED, "normal", "datasource1"));
+    Assert.assertEquals(1, stats.getSegmentStat(Stats.Segments.MOVED, "normal", "datasource2"));
+  }
+
+  /**
+   * Server 1 has 2 segments.
+   * Server 2 (decommissioning) has 2 segments.
+   * Server 3 is empty.
+   * Decommissioning percent is 60.
+   * Max segments to move is 3.
+   * 2 (of 2) segments should be moved from Server 2 and 1 (of 2) from Server 1.
+   */
+  @Test
+  public void testMoveDecommissioningMaxPercentOfMaxSegmentsToMove()
+  {
+    final ServerHolder serverHolder1 = createHolder(server1, false, segment1, segment2);
+    final ServerHolder serverHolder2 = createHolder(server2, true, segment3, segment4);
+    final ServerHolder serverHolder3 = createHolder(server3, false);
+
+    BalancerStrategy strategy = EasyMock.createMock(BalancerStrategy.class);
+    expectPickLoadingSegmentsAndReturnEmpty(strategy).times(2);
+    expectPickLoadedSegmentsAndReturn(
+        strategy,
+        new BalancerSegmentHolder(serverHolder2, segment3),
+        new BalancerSegmentHolder(serverHolder2, segment4)
+    ).once();
+    expectPickLoadedSegmentsAndReturn(
+        strategy,
+        new BalancerSegmentHolder(serverHolder1, segment1),
+        new BalancerSegmentHolder(serverHolder1, segment2)
+    ).once();
+    expectFindDestinationAndReturn(strategy, serverHolder3);
+    EasyMock.replay(strategy);
+
+    // ceil(3 * 0.6) = 2 segments from decommissioning servers
+    CoordinatorDynamicConfig dynamicConfig =
+        CoordinatorDynamicConfig.builder()
+                                .withMaxSegmentsToMove(3)
+                                .withDecommissioningMaxPercentOfMaxSegmentsToMove(60)
+                                .build();
+    DruidCoordinatorRuntimeParams params =
+        defaultRuntimeParamsBuilder(serverHolder1, serverHolder2, serverHolder3)
+            .withDynamicConfigs(dynamicConfig)
+            .withBalancerStrategy(strategy)
+            .withBroadcastDatasources(broadcastDatasources)
+            .build();
+
+    CoordinatorRunStats stats = runBalancer(params);
+
+    EasyMock.verify(strategy);
+    Assert.assertEquals(1L, stats.getSegmentStat(Stats.Segments.MOVED, "normal", "datasource1"));
+    Assert.assertEquals(2L, stats.getSegmentStat(Stats.Segments.MOVED, "normal", "datasource2"));
+    Assert.assertEquals(
+        ImmutableSet.of(segment1, segment3, segment4),
+        serverHolder3.getPeon().getSegmentsToLoad()
+    );
+  }
+
+  @Test
+  public void testZeroDecommissioningMaxPercentOfMaxSegmentsToMove()
+  {
+    final ServerHolder holder1 = createHolder(server1, false, segment1, segment2);
+    final ServerHolder holder2 = createHolder(server2, true, segment3, segment4);
+    final ServerHolder holder3 = createHolder(server3, false);
+
+    CoordinatorDynamicConfig dynamicConfig =
+        CoordinatorDynamicConfig.builder().withDecommissioningMaxPercentOfMaxSegmentsToMove(0)
+                                .withMaxSegmentsToMove(1).build();
+    DruidCoordinatorRuntimeParams params =
+        defaultRuntimeParamsBuilder(holder1, holder2, holder3).withDynamicConfigs(dynamicConfig).build();
+
+    CoordinatorRunStats stats = runBalancer(params);
+
+    // Verify that either segment1 or segment2 is chosen for move
+    Assert.assertEquals(1L, stats.getSegmentStat(Stats.Segments.MOVED, "normal", segment1.getDataSource()));
+    DataSegment movingSegment = holder3.getPeon().getSegmentsToLoad().iterator().next();
+    Assert.assertEquals(segment1.getDataSource(), movingSegment.getDataSource());
+  }
+
+  @Test
+  public void testMaxDecommissioningMaxPercentOfMaxSegmentsToMove()
+  {
+    final ServerHolder holder1 = createHolder(server1, false, segment1, segment2);
+    final ServerHolder holder2 = createHolder(server2, true, segment3, segment4);
+    final ServerHolder holder3 = createHolder(server3, false);
+
+    CoordinatorDynamicConfig dynamicConfig =
+        CoordinatorDynamicConfig.builder()
+                                .withDecommissioningMaxPercentOfMaxSegmentsToMove(100)
+                                .withMaxSegmentsToMove(1).build();
+    DruidCoordinatorRuntimeParams params =
+        defaultRuntimeParamsBuilder(holder1, holder2, holder3).withDynamicConfigs(dynamicConfig).build();
+
+    CoordinatorRunStats stats = runBalancer(params);
+
+    // Verify that either segment3 or segment4 is chosen for move
+    Assert.assertEquals(1L, stats.getSegmentStat(Stats.Segments.MOVED, "normal", segment3.getDataSource()));
+    DataSegment movingSegment = holder3.getPeon().getSegmentsToLoad().iterator().next();
+    Assert.assertEquals(segment3.getDataSource(), movingSegment.getDataSource());
+  }
+
+  /**
+   * Should balance segments as usual (ignoring percent) with empty decommissioningNodes.
+   */
+  @Test
+  public void testMoveDecommissioningMaxPercentOfMaxSegmentsToMoveWithNoDecommissioning()
+  {
+    final ServerHolder serverHolder1 = createHolder(server1, segment1, segment2);
+    final ServerHolder serverHolder2 = createHolder(server2, segment3, segment4);
+    final ServerHolder serverHolder3 = createHolder(server3);
+    BalancerStrategy strategy = EasyMock.createMock(BalancerStrategy.class);
+    expectPickLoadingSegmentsAndReturnEmpty(strategy).once();
+    expectPickLoadedSegmentsAndReturn(
+        strategy,
+        new BalancerSegmentHolder(serverHolder1, segment2),
+        new BalancerSegmentHolder(serverHolder2, segment3),
+        new BalancerSegmentHolder(serverHolder2, segment4)
+    ).once();
+    expectFindDestinationAndReturn(strategy, serverHolder3);
+    EasyMock.replay(strategy);
+
+    CoordinatorDynamicConfig dynamicConfig =
+        CoordinatorDynamicConfig.builder()
+                                .withMaxSegmentsToMove(3)
+                                .withDecommissioningMaxPercentOfMaxSegmentsToMove(9)
+                                .build();
+    DruidCoordinatorRuntimeParams params =
+        defaultRuntimeParamsBuilder(serverHolder1, serverHolder2, serverHolder3)
+            .withDynamicConfigs(dynamicConfig)
+            .withBalancerStrategy(strategy)
+            .build();
+
+    CoordinatorRunStats stats = runBalancer(params);
+    EasyMock.verify(strategy);
+    Assert.assertEquals(1L, stats.getSegmentStat(Stats.Segments.MOVED, "normal", "datasource1"));
+    Assert.assertEquals(2L, stats.getSegmentStat(Stats.Segments.MOVED, "normal", "datasource2"));
+    Assert.assertEquals(
+        ImmutableSet.of(segment2, segment3, segment4),
+        serverHolder3.getPeon().getSegmentsToLoad()
+    );
+  }
+
+  /**
+   * Shouldn't move segments to a decommissioning server.
+   */
+  @Test
+  public void testMoveToDecommissioningServer()
+  {
+    final ServerHolder serverHolder1 = createHolder(server1, false, allSegments);
+    final ServerHolder serverHolder2 = createHolder(server2, true);
+
+    BalancerStrategy strategy = EasyMock.createMock(BalancerStrategy.class);
+    expectPickLoadingSegmentsAndReturnEmpty(strategy).times(2);
+    expectPickLoadedSegmentsAndReturn(
+        strategy,
+        new BalancerSegmentHolder(serverHolder1, segment1)
+    ).times(2);
+    EasyMock.expect(
+        strategy.findDestinationServerToMoveSegment(
+            EasyMock.anyObject(),
+            EasyMock.anyObject(),
+            EasyMock.anyObject()
+        )
+    ).andAnswer(() -> ((List<ServerHolder>) EasyMock.getCurrentArguments()[2]).get(0)).anyTimes();
+    EasyMock.replay(strategy);
+
+    DruidCoordinatorRuntimeParams params =
+        defaultRuntimeParamsBuilder(serverHolder1, serverHolder2)
+            .withBalancerStrategy(strategy)
+            .withBroadcastDatasources(broadcastDatasources)
+            .build();
+
+    CoordinatorRunStats stats = runBalancer(params);
+    EasyMock.verify(strategy);
+    Assert.assertEquals(0, stats.getTieredStat(Stats.Segments.MOVED, "normal"));
+  }
+
+  @Test
+  public void testMoveFromDecommissioningServer()
+  {
+    final ServerHolder holder1 = createHolder(server1, allSegments);
+    final ServerHolder holder2 = createHolder(server2);
+
+    BalancerStrategy strategy = EasyMock.createMock(BalancerStrategy.class);
+    expectPickLoadingSegmentsAndReturnEmpty(strategy).once();
+    expectPickLoadedSegmentsAndReturn(strategy, new BalancerSegmentHolder(holder1, segment1)).once();
+    expectFindDestinationAndReturn(strategy, holder2);
+    EasyMock.replay(strategy);
+
+    DruidCoordinatorRuntimeParams params = defaultRuntimeParamsBuilder(holder1, holder2)
+        .withDynamicConfigs(CoordinatorDynamicConfig.builder().withMaxSegmentsToMove(1).build())
+        .withBalancerStrategy(strategy)
+        .withBroadcastDatasources(broadcastDatasources)
+        .build();
+
+    CoordinatorRunStats stats = runBalancer(params);
+    EasyMock.verify(strategy);
+    Assert.assertEquals(1, stats.getSegmentStat(Stats.Segments.MOVED, "normal", segment1.getDataSource()));
+    Assert.assertEquals(0, holder1.getPeon().getNumberOfSegmentsToLoad());
+    Assert.assertEquals(1, holder2.getPeon().getNumberOfSegmentsToLoad());
+  }
+
+  @Test
+  public void testMoveMaxLoadQueueServerBalancer()
+  {
+    final int maxSegmentsInQueue = 1;
+    final ServerHolder holder1 = createHolder(server1, maxSegmentsInQueue, false, allSegments);
+    final ServerHolder holder2 = createHolder(server2, maxSegmentsInQueue, false);
+
+    final CoordinatorDynamicConfig dynamicConfig = CoordinatorDynamicConfig
+        .builder()
+        .withMaxSegmentsInNodeLoadingQueue(maxSegmentsInQueue)
+        .build();
+    DruidCoordinatorRuntimeParams params =
+        defaultRuntimeParamsBuilder(holder1, holder2)
+            .withDynamicConfigs(dynamicConfig)
+            .build();
+
+    CoordinatorRunStats stats = runBalancer(params);
+
+    // max to move is 5, all segments on server 1, but only expect to move 1 to server 2 since max node load queue is 1
+    Assert.assertEquals(maxSegmentsInQueue, stats.getSegmentStat(Stats.Segments.MOVED, "normal", "datasource1"));
+  }
+
+  @Test
+  public void testRun1()
+  {
+    // Mock some servers of different usages
+    DruidCoordinatorRuntimeParams params = defaultRuntimeParamsBuilder(
+        createHolder(server1, allSegments),
+        createHolder(server2)
+    ).build();
+
+    CoordinatorRunStats stats = runBalancer(params);
+    Assert.assertTrue(stats.getSegmentStat(Stats.Segments.MOVED, "normal", "datasource1") > 0);
+  }
+
+  @Test
+  public void testRun2()
+  {
+    // Mock some servers of different usages
+    DruidCoordinatorRuntimeParams params = defaultRuntimeParamsBuilder(
+        createHolder(server1, allSegments),
+        createHolder(server2),
+        createHolder(server3),
+        createHolder(server4)
+    ).build();
+
+    CoordinatorRunStats stats = runBalancer(params);
+    Assert.assertTrue(stats.getSegmentStat(Stats.Segments.MOVED, "normal", "datasource1") > 0);
+  }
+
+  @Test
+  public void testThatMaxSegmentsToMoveIsHonored()
+  {
+    // Move from non-decomissioning servers
+    final ServerHolder holder1 = createHolder(server1, segment1, segment2);
+    final ServerHolder holder2 = createHolder(server2, segment3, segment4);
+    final ServerHolder holder3 = createHolder(server3);
+
+    BalancerStrategy strategy = EasyMock.createMock(BalancerStrategy.class);
+    expectPickLoadingSegmentsAndReturnEmpty(strategy).once();
+    expectPickLoadedSegmentsAndReturn(strategy, new BalancerSegmentHolder(holder2, segment3)).once();
+    expectFindDestinationAndReturn(strategy, holder3);
+    EasyMock.replay(strategy);
+
+    DruidCoordinatorRuntimeParams params =
+        defaultRuntimeParamsBuilder(holder1, holder2, holder3)
+            .withDynamicConfigs(
+                CoordinatorDynamicConfig.builder()
+                                        .withMaxSegmentsToMove(1)
+                                        .withUseBatchedSegmentSampler(true)

Review Comment:
   ## Deprecated method or constructor invocation
   
   Invoking [Builder.withUseBatchedSegmentSampler](1) should be avoided because it has been deprecated.
   
   [Show more details](https://github.com/apache/druid/security/code-scanning/4837)



##########
server/src/main/java/org/apache/druid/server/coordinator/balancer/BalancerStrategy.java:
##########
@@ -37,41 +40,55 @@
 public interface BalancerStrategy
 {
   /**
-   * Find the best server to move a {@link DataSegment} to according the balancing strategy.
-   * @param proposalSegment segment to move
-   * @param serverHolders servers to consider as move destinations
+   * Finds the best server to move a segment to according to the balancing strategy.
+   *
+   * @param proposalSegment    segment to move
+   * @param sourceServer       Server the segment is currently placed on.
+   * @param destinationServers servers to consider as move destinations
    * @return The server to move to, or null if no move should be made or no server is suitable
    */
   @Nullable
-  ServerHolder findNewSegmentHomeBalancer(DataSegment proposalSegment, List<ServerHolder> serverHolders);
+  ServerHolder findDestinationServerToMoveSegment(
+      DataSegment proposalSegment,
+      ServerHolder sourceServer,

Review Comment:
   ## Useless parameter
   
   The parameter 'sourceServer' is never used.
   
   [Show more details](https://github.com/apache/druid/security/code-scanning/4840)



-- 
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: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] kfaraz commented on a diff in pull request #13197: Segment loading: Allow cancellation and prioritization of load queue items

Posted by "kfaraz (via GitHub)" <gi...@apache.org>.
kfaraz commented on code in PR #13197:
URL: https://github.com/apache/druid/pull/13197#discussion_r991356721


##########
server/src/main/java/org/apache/druid/server/coordinator/duty/BalanceSegments.java:
##########
@@ -26,85 +26,65 @@
 import org.apache.druid.server.coordinator.BalancerSegmentHolder;
 import org.apache.druid.server.coordinator.BalancerStrategy;
 import org.apache.druid.server.coordinator.CoordinatorStats;
-import org.apache.druid.server.coordinator.DruidCoordinator;
 import org.apache.druid.server.coordinator.DruidCoordinatorRuntimeParams;
-import org.apache.druid.server.coordinator.LoadPeonCallback;
-import org.apache.druid.server.coordinator.LoadQueuePeon;
+import org.apache.druid.server.coordinator.SegmentLoader;
+import org.apache.druid.server.coordinator.SegmentStateManager;
 import org.apache.druid.server.coordinator.ServerHolder;
 import org.apache.druid.timeline.DataSegment;
-import org.apache.druid.timeline.SegmentId;
 
-import java.util.HashMap;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.NavigableSet;
 import java.util.SortedSet;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
 import java.util.stream.Collectors;
 
 /**
  */
 public class BalanceSegments implements CoordinatorDuty

Review Comment:
   Change summary:
   Move state of `currentlyMovingSegments` to `SegmentStateManager`.



-- 
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: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] github-code-scanning[bot] commented on a diff in pull request #13197: Segment loading: Allow cancellation and prioritization of load queue items

Posted by "github-code-scanning[bot] (via GitHub)" <gi...@apache.org>.
github-code-scanning[bot] commented on code in PR #13197:
URL: https://github.com/apache/druid/pull/13197#discussion_r1162306189


##########
server/src/main/java/org/apache/druid/server/coordinator/duty/BalanceSegments.java:
##########
@@ -20,292 +20,365 @@
 package org.apache.druid.server.coordinator.duty;
 
 import com.google.common.collect.Lists;
-import org.apache.druid.client.ImmutableDruidServer;
-import org.apache.druid.java.util.common.Pair;
+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.CoordinatorStats;
-import org.apache.druid.server.coordinator.DruidCoordinator;
 import org.apache.druid.server.coordinator.DruidCoordinatorRuntimeParams;
-import org.apache.druid.server.coordinator.LoadPeonCallback;
-import org.apache.druid.server.coordinator.LoadQueuePeon;
+import org.apache.druid.server.coordinator.SegmentLoader;
+import org.apache.druid.server.coordinator.SegmentStateManager;
 import org.apache.druid.server.coordinator.ServerHolder;
+import org.apache.druid.server.coordinator.stats.CoordinatorRunStats;
+import org.apache.druid.server.coordinator.stats.Stats;
 import org.apache.druid.timeline.DataSegment;
 import org.apache.druid.timeline.SegmentId;
 
-import java.util.HashMap;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
-import java.util.NavigableSet;
+import java.util.Set;
 import java.util.SortedSet;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
 import java.util.stream.Collectors;
 
 /**
+ *
  */
 public class BalanceSegments implements CoordinatorDuty
 {
   protected static final EmittingLogger log = new EmittingLogger(BalanceSegments.class);
+  private final SegmentStateManager stateManager;
 
-  protected final DruidCoordinator coordinator;
-
-  protected final Map<String, ConcurrentHashMap<SegmentId, BalancerSegmentHolder>> currentlyMovingSegments =
-      new HashMap<>();
-
-  public BalanceSegments(DruidCoordinator coordinator)
+  public BalanceSegments(SegmentStateManager stateManager)
   {
-    this.coordinator = coordinator;
+    this.stateManager = stateManager;
   }
 
-  protected void reduceLifetimes(String tier)
+  /**
+   * Reduces the lifetimes of segments currently being moved in all the tiers.
+   * Raises alerts for segments stuck in queue
+   * Returns the set of tiers that are currently moving some segments and won't be
+   * eligible for assigning more balancing moves in this run.
+   */
+  private void reduceLifetimesAndAlert(int maxLifetime)

Review Comment:
   ## Useless parameter
   
   The parameter 'maxLifetime' is never used.
   
   [Show more details](https://github.com/apache/druid/security/code-scanning/4746)



##########
server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java:
##########
@@ -439,18 +426,21 @@
         // This forcing code should be revised
         // when/if the autocompaction code policy to decide which segments to compact changes
         if (dropExisting == null || !dropExisting) {
-          if (segmentsToCompact.stream().allMatch(dataSegment -> dataSegment.isTombstone())) {
+          if (segmentsToCompact.stream().allMatch(DataSegment::isTombstone)) {
             dropExisting = true;
-            LOG.info("Forcing dropExisting to %s since all segments to compact are tombstones", dropExisting);
+            LOG.info("Forcing dropExisting to true since all segments to compact are tombstones.");
           }
         }
 
-        // make tuningConfig
         final String taskId = indexingServiceClient.compactSegments(
             "coordinator-issued",
             segmentsToCompact,
             config.getTaskPriority(),
-            ClientCompactionTaskQueryTuningConfig.from(config.getTuningConfig(), config.getMaxRowsPerSegment(), config.getMetricsSpec() != null),
+            ClientCompactionTaskQueryTuningConfig.from(
+                config.getTuningConfig(),
+                config.getMaxRowsPerSegment(),

Review Comment:
   ## Deprecated method or constructor invocation
   
   Invoking [DataSourceCompactionConfig.getMaxRowsPerSegment](1) should be avoided because it has been deprecated.
   
   [Show more details](https://github.com/apache/druid/security/code-scanning/4366)



##########
server/src/test/java/org/apache/druid/server/coordinator/BalanceSegmentsTest.java:
##########
@@ -358,265 +257,172 @@
   @Test
   public void testMoveToDecommissioningServer()
   {
-    mockDruidServer(druidServer1, "1", "normal", 30L, 100L, segments);
-    mockDruidServer(druidServer2, "2", "normal", 0L, 100L, Collections.emptyList());
-
-    EasyMock.replay(druidServer3);
-    EasyMock.replay(druidServer4);
-
-    mockCoordinator(coordinator);
+    final ServerHolder serverHolder1 = createHolder(server1, false, allSegments);
+    final ServerHolder serverHolder2 = createHolder(server2, true);
 
     BalancerStrategy strategy = EasyMock.createMock(BalancerStrategy.class);
-    EasyMock.expect(strategy.pickSegmentsToMove(EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyInt()))
-            .andReturn(ImmutableList.of(new BalancerSegmentHolder(druidServer1, segment1)).iterator())
-            .anyTimes();
+    expectPickLoadingSegmentsAndReturnEmpty(strategy, 2);
+    expectPickLoadedSegmentsAndReturn(
+        strategy, 2,
+        new BalancerSegmentHolder(serverHolder1, segment1)
+    );
     EasyMock.expect(strategy.findNewSegmentHomeBalancer(EasyMock.anyObject(), EasyMock.anyObject())).andAnswer(() -> {
       List<ServerHolder> holders = (List<ServerHolder>) EasyMock.getCurrentArguments()[1];
       return holders.get(0);
     }).anyTimes();
     EasyMock.replay(strategy);
 
-    DruidCoordinatorRuntimeParams params = defaultRuntimeParamsBuilder(
-        ImmutableList.of(druidServer1, druidServer2),
-        ImmutableList.of(peon1, peon2),
-        ImmutableList.of(false, true)
-    )
-        .withBalancerStrategy(strategy)
-        .withBroadcastDatasources(broadcastDatasources)
-        .build();
+    DruidCoordinatorRuntimeParams params =
+        defaultRuntimeParamsBuilder(serverHolder1, serverHolder2)
+            .withBalancerStrategy(strategy)
+            .withBroadcastDatasources(broadcastDatasources)
+            .build();
 
-    params = new BalanceSegmentsTester(coordinator).run(params);
+    params = new BalanceSegments(stateManager).run(params);
     EasyMock.verify(strategy);
-    Assert.assertEquals(0, params.getCoordinatorStats().getTieredStat("movedCount", "normal"));
+    Assert.assertEquals(0, params.getCoordinatorStats().getTieredStat(Stats.Segments.MOVED, "normal"));
   }
 
   @Test
   public void testMoveFromDecommissioningServer()
   {
-    mockDruidServer(druidServer1, "1", "normal", 30L, 100L, segments);
-    mockDruidServer(druidServer2, "2", "normal", 0L, 100L, Collections.emptyList());
-
-    EasyMock.replay(druidServer3);
-    EasyMock.replay(druidServer4);
-
-    mockCoordinator(coordinator);
+    final ServerHolder holder1 = createHolder(server1, allSegments);
+    final ServerHolder holder2 = createHolder(server2);
 
-    ServerHolder holder2 = new ServerHolder(druidServer2, peon2, false);
     BalancerStrategy strategy = EasyMock.createMock(BalancerStrategy.class);
-    EasyMock.expect(strategy.pickSegmentsToMove(EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyInt()))
-            .andReturn(ImmutableList.of(new BalancerSegmentHolder(druidServer1, segment1)).iterator())
-            .once();
+    expectPickLoadingSegmentsAndReturnEmpty(strategy, 1);
+    expectPickLoadedSegmentsAndReturn(
+        strategy, 1,
+        new BalancerSegmentHolder(holder1, segment1)
+    );
     EasyMock.expect(strategy.findNewSegmentHomeBalancer(EasyMock.anyObject(), EasyMock.anyObject()))
             .andReturn(holder2)
             .once();
     EasyMock.replay(strategy);
 
-    DruidCoordinatorRuntimeParams params = defaultRuntimeParamsBuilder(
-        ImmutableList.of(druidServer1, druidServer2),
-        ImmutableList.of(peon1, peon2),
-        ImmutableList.of(true, false)
-    )
+    DruidCoordinatorRuntimeParams params = defaultRuntimeParamsBuilder(holder1, holder2)
         .withDynamicConfigs(CoordinatorDynamicConfig.builder().withMaxSegmentsToMove(1).build())
         .withBalancerStrategy(strategy)
         .withBroadcastDatasources(broadcastDatasources)
         .build();
 
-    params = new BalanceSegmentsTester(coordinator).run(params);
+    params = new BalanceSegments(stateManager).run(params);
     EasyMock.verify(strategy);
-    Assert.assertEquals(1, params.getCoordinatorStats().getTieredStat("movedCount", "normal"));
-    Assert.assertEquals(0, peon1.getNumberOfSegmentsInQueue());
-    Assert.assertEquals(1, peon2.getNumberOfSegmentsInQueue());
+    Assert.assertEquals(1, params.getCoordinatorStats().getTieredStat(Stats.Segments.MOVED, "normal"));
+    Assert.assertEquals(0, holder1.getPeon().getNumberOfSegmentsToLoad());
+    Assert.assertEquals(1, holder2.getPeon().getNumberOfSegmentsToLoad());
   }
 
   @Test
   public void testMoveMaxLoadQueueServerBalancer()
   {
-    mockDruidServer(druidServer1, "1", "normal", 30L, 100L, segments);
-    mockDruidServer(druidServer2, "2", "normal", 0L, 100L, Collections.emptyList());
-
-    EasyMock.replay(druidServer3);
-    EasyMock.replay(druidServer4);
-
-    // Mock stuff that the coordinator needs
-    mockCoordinator(coordinator);
-
-    BalancerStrategy predefinedPickOrderStrategy = new PredefinedPickOrderBalancerStrategy(
-        balancerStrategy,
-        ImmutableList.of(
-            new BalancerSegmentHolder(druidServer1, segment1),
-            new BalancerSegmentHolder(druidServer1, segment2),
-            new BalancerSegmentHolder(druidServer1, segment3),
-            new BalancerSegmentHolder(druidServer1, segment4)
+    final int maxSegmentsInQueue = 1;
+    final ServerHolder holder1 = createHolder(server1, maxSegmentsInQueue, allSegments);
+    final ServerHolder holder2 = createHolder(server2, maxSegmentsInQueue);
+
+    final CoordinatorDynamicConfig dynamicConfig = CoordinatorDynamicConfig
+        .builder()
+        .withMaxSegmentsToMove(MAX_SEGMENTS_TO_MOVE)
+        .withMaxSegmentsInNodeLoadingQueue(maxSegmentsInQueue)
+        .build();
+    DruidCoordinatorRuntimeParams params = CoordinatorRuntimeParamsTestHelpers
+        .newBuilder()
+        .withDruidCluster(
+            DruidClusterBuilder.newBuilder().addTier("normal", holder1, holder2).build()
         )
-    );
-
-    DruidCoordinatorRuntimeParams params = defaultRuntimeParamsBuilder(
-        ImmutableList.of(druidServer1, druidServer2),
-        ImmutableList.of(peon1, peon2)
-    )
-        .withBalancerStrategy(predefinedPickOrderStrategy)
-        .withBroadcastDatasources(broadcastDatasources)
-        .withDynamicConfigs(
-            CoordinatorDynamicConfig
-                .builder()
-                .withMaxSegmentsToMove(MAX_SEGMENTS_TO_MOVE)
-                .withMaxSegmentsInNodeLoadingQueue(1)
-                .build()
+        .withLoadManagementPeons(
+            ImmutableMap.of(server1.getName(), holder1.getPeon(), server2.getName(), holder2.getPeon())
         )
+        .withUsedSegmentsInTest(allSegments)
+        .withBalancerStrategy(balancerStrategy)
+        .withBroadcastDatasources(broadcastDatasources)
+        .withDynamicConfigs(dynamicConfig)
+        .withReplicationManager(createReplicationThrottler(dynamicConfig))
         .build();
 
-    params = new BalanceSegmentsTester(coordinator).run(params);
+    params = new BalanceSegments(stateManager).run(params);
 
     // max to move is 5, all segments on server 1, but only expect to move 1 to server 2 since max node load queue is 1
-    Assert.assertEquals(1, params.getCoordinatorStats().getTieredStat("movedCount", "normal"));
-  }
-
-  @Test
-  public void testMoveSameSegmentTwice()
-  {
-    mockDruidServer(druidServer1, "1", "normal", 30L, 100L, segments);
-    mockDruidServer(druidServer2, "2", "normal", 0L, 100L, Collections.emptyList());
-
-    EasyMock.replay(druidServer3);
-    EasyMock.replay(druidServer4);
-
-    // Mock stuff that the coordinator needs
-    mockCoordinator(coordinator);
-
-    BalancerStrategy predefinedPickOrderStrategy = new PredefinedPickOrderBalancerStrategy(
-        balancerStrategy,
-        ImmutableList.of(
-            new BalancerSegmentHolder(druidServer1, segment1)
-        )
-    );
-
-    DruidCoordinatorRuntimeParams params = defaultRuntimeParamsBuilder(
-        ImmutableList.of(druidServer1, druidServer2),
-        ImmutableList.of(peon1, peon2)
-    )
-        .withBalancerStrategy(predefinedPickOrderStrategy)
-        .withBroadcastDatasources(broadcastDatasources)
-        .withDynamicConfigs(
-            CoordinatorDynamicConfig.builder().withMaxSegmentsToMove(
-                2
-            ).build()
-        )
-        .build();
-
-    params = new BalanceSegmentsTester(coordinator).run(params);
-    Assert.assertEquals(1, params.getCoordinatorStats().getTieredStat("movedCount", "normal"));
+    Assert.assertEquals(1, params.getCoordinatorStats().getTieredStat(Stats.Segments.MOVED, "normal"));
   }
 
   @Test
   public void testRun1()
   {
     // Mock some servers of different usages
-    mockDruidServer(druidServer1, "1", "normal", 30L, 100L, segments);
-    mockDruidServer(druidServer2, "2", "normal", 0L, 100L, Collections.emptyList());
-
-    EasyMock.replay(druidServer3);
-    EasyMock.replay(druidServer4);
-
-    // Mock stuff that the coordinator needs
-    mockCoordinator(coordinator);
-
     DruidCoordinatorRuntimeParams params = defaultRuntimeParamsBuilder(
-        ImmutableList.of(druidServer1, druidServer2),
-        ImmutableList.of(peon1, peon2)
+        createHolder(server1, allSegments),
+        createHolder(server2)
     ).build();
 
-    params = new BalanceSegmentsTester(coordinator).run(params);
-    Assert.assertTrue(params.getCoordinatorStats().getTieredStat("movedCount", "normal") > 0);
+    params = new BalanceSegments(stateManager).run(params);
+    Assert.assertTrue(params.getCoordinatorStats().getTieredStat(Stats.Segments.MOVED, "normal") > 0);
   }
 
   @Test
   public void testRun2()
   {
     // Mock some servers of different usages
-    mockDruidServer(druidServer1, "1", "normal", 30L, 100L, segments);
-    mockDruidServer(druidServer2, "2", "normal", 0L, 100L, Collections.emptyList());
-    mockDruidServer(druidServer3, "3", "normal", 0L, 100L, Collections.emptyList());
-    mockDruidServer(druidServer4, "4", "normal", 0L, 100L, Collections.emptyList());
-
-    // Mock stuff that the coordinator needs
-    mockCoordinator(coordinator);
-
-    DruidCoordinatorRuntimeParams params = defaultRuntimeParamsBuilder(druidServers, peons).build();
+    DruidCoordinatorRuntimeParams params = defaultRuntimeParamsBuilder(
+        createHolder(server1, allSegments),
+        createHolder(server2),
+        createHolder(server3),
+        createHolder(server4)
+    ).build();
 
-    params = new BalanceSegmentsTester(coordinator).run(params);
-    Assert.assertTrue(params.getCoordinatorStats().getTieredStat("movedCount", "normal") > 0);
+    params = new BalanceSegments(stateManager).run(params);
+    Assert.assertTrue(params.getCoordinatorStats().getTieredStat(Stats.Segments.MOVED, "normal") > 0);
   }
 
-  /**
-   * Testing that the dynamic coordinator config value, percentOfSegmentsToConsiderPerMove, is honored when calling
-   * out to pickSegmentToMove. This config limits the number of segments that are considered when looking for a segment
-   * to move.
-   */
   @Test
-  public void testThatDynamicConfigIsHonoredWhenPickingSegmentToMove()
+  public void testThatMaxSegmentsToMoveIsHonored()
   {
-    mockDruidServer(druidServer1, "1", "normal", 50L, 100L, Arrays.asList(segment1, segment2));
-    mockDruidServer(druidServer2, "2", "normal", 30L, 100L, Arrays.asList(segment3, segment4));
-    mockDruidServer(druidServer3, "3", "normal", 0L, 100L, Collections.emptyList());
-
-    EasyMock.replay(druidServer4);
-
-    mockCoordinator(coordinator);
+    // Move from non-decomissioning servers
+    final ServerHolder holder1 = createHolder(server1, segment1, segment2);
+    final ServerHolder holder2 = createHolder(server2, segment3, segment4);
+    final ServerHolder holder3 = createHolder(server3);
 
     BalancerStrategy strategy = EasyMock.createMock(BalancerStrategy.class);
-
-    // Move from non-decomissioning servers
-    EasyMock.expect(
-        strategy.pickSegmentsToMove(
-            ImmutableList.of(
-                new ServerHolder(druidServer3, peon3, false),
-                new ServerHolder(druidServer2, peon2, false),
-                new ServerHolder(druidServer1, peon1, false)
-            ),
-            broadcastDatasources,
-            40.0
-        )
-    )
-            .andReturn(ImmutableList.of(new BalancerSegmentHolder(druidServer2, segment3)).iterator());
+    expectPickLoadingSegmentsAndReturnEmpty(strategy, 1);
+    expectPickLoadedSegmentsAndReturn(
+        strategy,
+        1,
+        new BalancerSegmentHolder(holder2, segment3)
+    );
 
     EasyMock.expect(strategy.findNewSegmentHomeBalancer(EasyMock.anyObject(), EasyMock.anyObject()))
-            .andReturn(new ServerHolder(druidServer3, peon3))
+            .andReturn(holder3)
             .anyTimes();
     EasyMock.replay(strategy);
 
-    DruidCoordinatorRuntimeParams params = defaultRuntimeParamsBuilder(
-        ImmutableList.of(druidServer1, druidServer2, druidServer3),
-        ImmutableList.of(peon1, peon2, peon3),
-        ImmutableList.of(false, false, false)
-    )
-        .withDynamicConfigs(
-            CoordinatorDynamicConfig.builder()
-                                    .withMaxSegmentsToMove(1)
-                                    .withUseBatchedSegmentSampler(false)
-                                    .withPercentOfSegmentsToConsiderPerMove(40)
-                                    .build()
-        )
-        .withBalancerStrategy(strategy)
-        .withBroadcastDatasources(broadcastDatasources)
-        .build();
-
-    params = new BalanceSegmentsTester(coordinator).run(params);
+    DruidCoordinatorRuntimeParams params =
+        defaultRuntimeParamsBuilder(holder1, holder2, holder3)
+            .withDynamicConfigs(
+                CoordinatorDynamicConfig.builder()
+                                        .withMaxSegmentsToMove(1)
+                                        .withUseBatchedSegmentSampler(true)
+                                        .withPercentOfSegmentsToConsiderPerMove(40)

Review Comment:
   ## Deprecated method or constructor invocation
   
   Invoking [Builder.withPercentOfSegmentsToConsiderPerMove](1) should be avoided because it has been deprecated.
   
   [Show more details](https://github.com/apache/druid/security/code-scanning/4744)



##########
server/src/test/java/org/apache/druid/server/coordinator/BalanceSegmentsTest.java:
##########
@@ -358,265 +257,172 @@
   @Test
   public void testMoveToDecommissioningServer()
   {
-    mockDruidServer(druidServer1, "1", "normal", 30L, 100L, segments);
-    mockDruidServer(druidServer2, "2", "normal", 0L, 100L, Collections.emptyList());
-
-    EasyMock.replay(druidServer3);
-    EasyMock.replay(druidServer4);
-
-    mockCoordinator(coordinator);
+    final ServerHolder serverHolder1 = createHolder(server1, false, allSegments);
+    final ServerHolder serverHolder2 = createHolder(server2, true);
 
     BalancerStrategy strategy = EasyMock.createMock(BalancerStrategy.class);
-    EasyMock.expect(strategy.pickSegmentsToMove(EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyInt()))
-            .andReturn(ImmutableList.of(new BalancerSegmentHolder(druidServer1, segment1)).iterator())
-            .anyTimes();
+    expectPickLoadingSegmentsAndReturnEmpty(strategy, 2);
+    expectPickLoadedSegmentsAndReturn(
+        strategy, 2,
+        new BalancerSegmentHolder(serverHolder1, segment1)
+    );
     EasyMock.expect(strategy.findNewSegmentHomeBalancer(EasyMock.anyObject(), EasyMock.anyObject())).andAnswer(() -> {
       List<ServerHolder> holders = (List<ServerHolder>) EasyMock.getCurrentArguments()[1];
       return holders.get(0);
     }).anyTimes();
     EasyMock.replay(strategy);
 
-    DruidCoordinatorRuntimeParams params = defaultRuntimeParamsBuilder(
-        ImmutableList.of(druidServer1, druidServer2),
-        ImmutableList.of(peon1, peon2),
-        ImmutableList.of(false, true)
-    )
-        .withBalancerStrategy(strategy)
-        .withBroadcastDatasources(broadcastDatasources)
-        .build();
+    DruidCoordinatorRuntimeParams params =
+        defaultRuntimeParamsBuilder(serverHolder1, serverHolder2)
+            .withBalancerStrategy(strategy)
+            .withBroadcastDatasources(broadcastDatasources)
+            .build();
 
-    params = new BalanceSegmentsTester(coordinator).run(params);
+    params = new BalanceSegments(stateManager).run(params);
     EasyMock.verify(strategy);
-    Assert.assertEquals(0, params.getCoordinatorStats().getTieredStat("movedCount", "normal"));
+    Assert.assertEquals(0, params.getCoordinatorStats().getTieredStat(Stats.Segments.MOVED, "normal"));
   }
 
   @Test
   public void testMoveFromDecommissioningServer()
   {
-    mockDruidServer(druidServer1, "1", "normal", 30L, 100L, segments);
-    mockDruidServer(druidServer2, "2", "normal", 0L, 100L, Collections.emptyList());
-
-    EasyMock.replay(druidServer3);
-    EasyMock.replay(druidServer4);
-
-    mockCoordinator(coordinator);
+    final ServerHolder holder1 = createHolder(server1, allSegments);
+    final ServerHolder holder2 = createHolder(server2);
 
-    ServerHolder holder2 = new ServerHolder(druidServer2, peon2, false);
     BalancerStrategy strategy = EasyMock.createMock(BalancerStrategy.class);
-    EasyMock.expect(strategy.pickSegmentsToMove(EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyInt()))
-            .andReturn(ImmutableList.of(new BalancerSegmentHolder(druidServer1, segment1)).iterator())
-            .once();
+    expectPickLoadingSegmentsAndReturnEmpty(strategy, 1);
+    expectPickLoadedSegmentsAndReturn(
+        strategy, 1,
+        new BalancerSegmentHolder(holder1, segment1)
+    );
     EasyMock.expect(strategy.findNewSegmentHomeBalancer(EasyMock.anyObject(), EasyMock.anyObject()))
             .andReturn(holder2)
             .once();
     EasyMock.replay(strategy);
 
-    DruidCoordinatorRuntimeParams params = defaultRuntimeParamsBuilder(
-        ImmutableList.of(druidServer1, druidServer2),
-        ImmutableList.of(peon1, peon2),
-        ImmutableList.of(true, false)
-    )
+    DruidCoordinatorRuntimeParams params = defaultRuntimeParamsBuilder(holder1, holder2)
         .withDynamicConfigs(CoordinatorDynamicConfig.builder().withMaxSegmentsToMove(1).build())
         .withBalancerStrategy(strategy)
         .withBroadcastDatasources(broadcastDatasources)
         .build();
 
-    params = new BalanceSegmentsTester(coordinator).run(params);
+    params = new BalanceSegments(stateManager).run(params);
     EasyMock.verify(strategy);
-    Assert.assertEquals(1, params.getCoordinatorStats().getTieredStat("movedCount", "normal"));
-    Assert.assertEquals(0, peon1.getNumberOfSegmentsInQueue());
-    Assert.assertEquals(1, peon2.getNumberOfSegmentsInQueue());
+    Assert.assertEquals(1, params.getCoordinatorStats().getTieredStat(Stats.Segments.MOVED, "normal"));
+    Assert.assertEquals(0, holder1.getPeon().getNumberOfSegmentsToLoad());
+    Assert.assertEquals(1, holder2.getPeon().getNumberOfSegmentsToLoad());
   }
 
   @Test
   public void testMoveMaxLoadQueueServerBalancer()
   {
-    mockDruidServer(druidServer1, "1", "normal", 30L, 100L, segments);
-    mockDruidServer(druidServer2, "2", "normal", 0L, 100L, Collections.emptyList());
-
-    EasyMock.replay(druidServer3);
-    EasyMock.replay(druidServer4);
-
-    // Mock stuff that the coordinator needs
-    mockCoordinator(coordinator);
-
-    BalancerStrategy predefinedPickOrderStrategy = new PredefinedPickOrderBalancerStrategy(
-        balancerStrategy,
-        ImmutableList.of(
-            new BalancerSegmentHolder(druidServer1, segment1),
-            new BalancerSegmentHolder(druidServer1, segment2),
-            new BalancerSegmentHolder(druidServer1, segment3),
-            new BalancerSegmentHolder(druidServer1, segment4)
+    final int maxSegmentsInQueue = 1;
+    final ServerHolder holder1 = createHolder(server1, maxSegmentsInQueue, allSegments);
+    final ServerHolder holder2 = createHolder(server2, maxSegmentsInQueue);
+
+    final CoordinatorDynamicConfig dynamicConfig = CoordinatorDynamicConfig
+        .builder()
+        .withMaxSegmentsToMove(MAX_SEGMENTS_TO_MOVE)
+        .withMaxSegmentsInNodeLoadingQueue(maxSegmentsInQueue)
+        .build();
+    DruidCoordinatorRuntimeParams params = CoordinatorRuntimeParamsTestHelpers
+        .newBuilder()
+        .withDruidCluster(
+            DruidClusterBuilder.newBuilder().addTier("normal", holder1, holder2).build()
         )
-    );
-
-    DruidCoordinatorRuntimeParams params = defaultRuntimeParamsBuilder(
-        ImmutableList.of(druidServer1, druidServer2),
-        ImmutableList.of(peon1, peon2)
-    )
-        .withBalancerStrategy(predefinedPickOrderStrategy)
-        .withBroadcastDatasources(broadcastDatasources)
-        .withDynamicConfigs(
-            CoordinatorDynamicConfig
-                .builder()
-                .withMaxSegmentsToMove(MAX_SEGMENTS_TO_MOVE)
-                .withMaxSegmentsInNodeLoadingQueue(1)
-                .build()
+        .withLoadManagementPeons(
+            ImmutableMap.of(server1.getName(), holder1.getPeon(), server2.getName(), holder2.getPeon())
         )
+        .withUsedSegmentsInTest(allSegments)
+        .withBalancerStrategy(balancerStrategy)
+        .withBroadcastDatasources(broadcastDatasources)
+        .withDynamicConfigs(dynamicConfig)
+        .withReplicationManager(createReplicationThrottler(dynamicConfig))
         .build();
 
-    params = new BalanceSegmentsTester(coordinator).run(params);
+    params = new BalanceSegments(stateManager).run(params);
 
     // max to move is 5, all segments on server 1, but only expect to move 1 to server 2 since max node load queue is 1
-    Assert.assertEquals(1, params.getCoordinatorStats().getTieredStat("movedCount", "normal"));
-  }
-
-  @Test
-  public void testMoveSameSegmentTwice()
-  {
-    mockDruidServer(druidServer1, "1", "normal", 30L, 100L, segments);
-    mockDruidServer(druidServer2, "2", "normal", 0L, 100L, Collections.emptyList());
-
-    EasyMock.replay(druidServer3);
-    EasyMock.replay(druidServer4);
-
-    // Mock stuff that the coordinator needs
-    mockCoordinator(coordinator);
-
-    BalancerStrategy predefinedPickOrderStrategy = new PredefinedPickOrderBalancerStrategy(
-        balancerStrategy,
-        ImmutableList.of(
-            new BalancerSegmentHolder(druidServer1, segment1)
-        )
-    );
-
-    DruidCoordinatorRuntimeParams params = defaultRuntimeParamsBuilder(
-        ImmutableList.of(druidServer1, druidServer2),
-        ImmutableList.of(peon1, peon2)
-    )
-        .withBalancerStrategy(predefinedPickOrderStrategy)
-        .withBroadcastDatasources(broadcastDatasources)
-        .withDynamicConfigs(
-            CoordinatorDynamicConfig.builder().withMaxSegmentsToMove(
-                2
-            ).build()
-        )
-        .build();
-
-    params = new BalanceSegmentsTester(coordinator).run(params);
-    Assert.assertEquals(1, params.getCoordinatorStats().getTieredStat("movedCount", "normal"));
+    Assert.assertEquals(1, params.getCoordinatorStats().getTieredStat(Stats.Segments.MOVED, "normal"));
   }
 
   @Test
   public void testRun1()
   {
     // Mock some servers of different usages
-    mockDruidServer(druidServer1, "1", "normal", 30L, 100L, segments);
-    mockDruidServer(druidServer2, "2", "normal", 0L, 100L, Collections.emptyList());
-
-    EasyMock.replay(druidServer3);
-    EasyMock.replay(druidServer4);
-
-    // Mock stuff that the coordinator needs
-    mockCoordinator(coordinator);
-
     DruidCoordinatorRuntimeParams params = defaultRuntimeParamsBuilder(
-        ImmutableList.of(druidServer1, druidServer2),
-        ImmutableList.of(peon1, peon2)
+        createHolder(server1, allSegments),
+        createHolder(server2)
     ).build();
 
-    params = new BalanceSegmentsTester(coordinator).run(params);
-    Assert.assertTrue(params.getCoordinatorStats().getTieredStat("movedCount", "normal") > 0);
+    params = new BalanceSegments(stateManager).run(params);
+    Assert.assertTrue(params.getCoordinatorStats().getTieredStat(Stats.Segments.MOVED, "normal") > 0);
   }
 
   @Test
   public void testRun2()
   {
     // Mock some servers of different usages
-    mockDruidServer(druidServer1, "1", "normal", 30L, 100L, segments);
-    mockDruidServer(druidServer2, "2", "normal", 0L, 100L, Collections.emptyList());
-    mockDruidServer(druidServer3, "3", "normal", 0L, 100L, Collections.emptyList());
-    mockDruidServer(druidServer4, "4", "normal", 0L, 100L, Collections.emptyList());
-
-    // Mock stuff that the coordinator needs
-    mockCoordinator(coordinator);
-
-    DruidCoordinatorRuntimeParams params = defaultRuntimeParamsBuilder(druidServers, peons).build();
+    DruidCoordinatorRuntimeParams params = defaultRuntimeParamsBuilder(
+        createHolder(server1, allSegments),
+        createHolder(server2),
+        createHolder(server3),
+        createHolder(server4)
+    ).build();
 
-    params = new BalanceSegmentsTester(coordinator).run(params);
-    Assert.assertTrue(params.getCoordinatorStats().getTieredStat("movedCount", "normal") > 0);
+    params = new BalanceSegments(stateManager).run(params);
+    Assert.assertTrue(params.getCoordinatorStats().getTieredStat(Stats.Segments.MOVED, "normal") > 0);
   }
 
-  /**
-   * Testing that the dynamic coordinator config value, percentOfSegmentsToConsiderPerMove, is honored when calling
-   * out to pickSegmentToMove. This config limits the number of segments that are considered when looking for a segment
-   * to move.
-   */
   @Test
-  public void testThatDynamicConfigIsHonoredWhenPickingSegmentToMove()
+  public void testThatMaxSegmentsToMoveIsHonored()
   {
-    mockDruidServer(druidServer1, "1", "normal", 50L, 100L, Arrays.asList(segment1, segment2));
-    mockDruidServer(druidServer2, "2", "normal", 30L, 100L, Arrays.asList(segment3, segment4));
-    mockDruidServer(druidServer3, "3", "normal", 0L, 100L, Collections.emptyList());
-
-    EasyMock.replay(druidServer4);
-
-    mockCoordinator(coordinator);
+    // Move from non-decomissioning servers
+    final ServerHolder holder1 = createHolder(server1, segment1, segment2);
+    final ServerHolder holder2 = createHolder(server2, segment3, segment4);
+    final ServerHolder holder3 = createHolder(server3);
 
     BalancerStrategy strategy = EasyMock.createMock(BalancerStrategy.class);
-
-    // Move from non-decomissioning servers
-    EasyMock.expect(
-        strategy.pickSegmentsToMove(
-            ImmutableList.of(
-                new ServerHolder(druidServer3, peon3, false),
-                new ServerHolder(druidServer2, peon2, false),
-                new ServerHolder(druidServer1, peon1, false)
-            ),
-            broadcastDatasources,
-            40.0
-        )
-    )
-            .andReturn(ImmutableList.of(new BalancerSegmentHolder(druidServer2, segment3)).iterator());
+    expectPickLoadingSegmentsAndReturnEmpty(strategy, 1);
+    expectPickLoadedSegmentsAndReturn(
+        strategy,
+        1,
+        new BalancerSegmentHolder(holder2, segment3)
+    );
 
     EasyMock.expect(strategy.findNewSegmentHomeBalancer(EasyMock.anyObject(), EasyMock.anyObject()))
-            .andReturn(new ServerHolder(druidServer3, peon3))
+            .andReturn(holder3)
             .anyTimes();
     EasyMock.replay(strategy);
 
-    DruidCoordinatorRuntimeParams params = defaultRuntimeParamsBuilder(
-        ImmutableList.of(druidServer1, druidServer2, druidServer3),
-        ImmutableList.of(peon1, peon2, peon3),
-        ImmutableList.of(false, false, false)
-    )
-        .withDynamicConfigs(
-            CoordinatorDynamicConfig.builder()
-                                    .withMaxSegmentsToMove(1)
-                                    .withUseBatchedSegmentSampler(false)
-                                    .withPercentOfSegmentsToConsiderPerMove(40)
-                                    .build()
-        )
-        .withBalancerStrategy(strategy)
-        .withBroadcastDatasources(broadcastDatasources)
-        .build();
-
-    params = new BalanceSegmentsTester(coordinator).run(params);
+    DruidCoordinatorRuntimeParams params =
+        defaultRuntimeParamsBuilder(holder1, holder2, holder3)
+            .withDynamicConfigs(
+                CoordinatorDynamicConfig.builder()
+                                        .withMaxSegmentsToMove(1)
+                                        .withUseBatchedSegmentSampler(true)

Review Comment:
   ## Deprecated method or constructor invocation
   
   Invoking [Builder.withUseBatchedSegmentSampler](1) should be avoided because it has been deprecated.
   
   [Show more details](https://github.com/apache/druid/security/code-scanning/4745)



-- 
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: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] github-code-scanning[bot] commented on a diff in pull request #13197: Segment loading: Allow cancellation and prioritization of load queue items

Posted by "github-code-scanning[bot] (via GitHub)" <gi...@apache.org>.
github-code-scanning[bot] commented on code in PR #13197:
URL: https://github.com/apache/druid/pull/13197#discussion_r1177684726


##########
server/src/main/java/org/apache/druid/server/coordinator/duty/RunRules.java:
##########
@@ -24,125 +24,96 @@
 import org.apache.druid.java.util.common.DateTimes;
 import org.apache.druid.java.util.emitter.EmittingLogger;
 import org.apache.druid.metadata.MetadataRuleManager;
-import org.apache.druid.server.coordinator.CoordinatorStats;
+import org.apache.druid.server.coordinator.CoordinatorDynamicConfig;
 import org.apache.druid.server.coordinator.DruidCluster;
-import org.apache.druid.server.coordinator.DruidCoordinator;
 import org.apache.druid.server.coordinator.DruidCoordinatorRuntimeParams;
 import org.apache.druid.server.coordinator.ReplicationThrottler;
+import org.apache.druid.server.coordinator.StrategicSegmentAssigner;
+import org.apache.druid.server.coordinator.loadqueue.SegmentLoadQueueManager;
 import org.apache.druid.server.coordinator.rules.BroadcastDistributionRule;
 import org.apache.druid.server.coordinator.rules.Rule;
+import org.apache.druid.server.coordinator.stats.CoordinatorRunStats;
 import org.apache.druid.timeline.DataSegment;
 import org.apache.druid.timeline.SegmentId;
 import org.joda.time.DateTime;
 
-import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
+import java.util.stream.Collectors;
 
 /**
+ * Duty to run retention rules.
+ * <p>
+ * The params returned from {@code run()} must have these fields initialized:
+ * <ul>
+ *   <li>{@link DruidCoordinatorRuntimeParams#getBroadcastDatasources()}</li>
+ *   <li>{@link DruidCoordinatorRuntimeParams#getReplicationManager()}</li>
+ * </ul>
+ * These fields are used by the downstream coordinator duty, {@link BalanceSegments}.
  */
 public class RunRules implements CoordinatorDuty
 {
   private static final EmittingLogger log = new EmittingLogger(RunRules.class);
   private static final int MAX_MISSING_RULES = 10;
 
-  private final ReplicationThrottler replicatorThrottler;
+  private final SegmentLoadQueueManager loadQueueManager;
 
-  private final DruidCoordinator coordinator;
-
-  public RunRules(DruidCoordinator coordinator)
-  {
-    this(
-        new ReplicationThrottler(
-            coordinator.getDynamicConfigs().getReplicationThrottleLimit(),
-            coordinator.getDynamicConfigs().getReplicantLifetime(),
-            false
-        ),
-        coordinator
-    );
-  }
-
-  public RunRules(ReplicationThrottler replicatorThrottler, DruidCoordinator coordinator)
+  public RunRules(SegmentLoadQueueManager loadQueueManager)
   {
-    this.replicatorThrottler = replicatorThrottler;
-    this.coordinator = coordinator;
+    this.loadQueueManager = loadQueueManager;
   }
 
   @Override
   public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params)
   {
-    replicatorThrottler.updateParams(
-        coordinator.getDynamicConfigs().getReplicationThrottleLimit(),
-        coordinator.getDynamicConfigs().getReplicantLifetime(),
-        false
-    );
-
-    CoordinatorStats stats = new CoordinatorStats();
-    DruidCluster cluster = params.getDruidCluster();
-
+    final DruidCluster cluster = params.getDruidCluster();
     if (cluster.isEmpty()) {
-      log.warn("Uh... I have no servers. Not assigning anything...");
+      log.warn("Cluster has no servers. Not running any rules.");
       return params;
     }
 
     // Get used segments which are overshadowed by other used segments. Those would not need to be loaded and
     // eventually will be unloaded from Historical servers. Segments overshadowed by *served* used segments are marked
     // as unused in MarkAsUnusedOvershadowedSegments, and then eventually Coordinator sends commands to Historical nodes
     // to unload such segments in UnloadUnusedSegments.
-    Set<DataSegment> overshadowed = params.getDataSourcesSnapshot().getOvershadowedSegments();
-
-    for (String tier : cluster.getTierNames()) {
-      replicatorThrottler.updateReplicationState(tier);
-    }
+    final Set<DataSegment> overshadowed = params.getDataSourcesSnapshot().getOvershadowedSegments();
+    final Set<DataSegment> usedSegments = params.getUsedSegments();
+    log.info(
+        "Running rules for [%d] used segments. Skipping [%d] overshadowed segments.",
+        usedSegments.size(), overshadowed.size()
+    );
 
-    DruidCoordinatorRuntimeParams paramsWithReplicationManager = params
-        .buildFromExistingWithoutSegmentsMetadata()
-        .withReplicationManager(replicatorThrottler)
-        .build();
+    final CoordinatorDynamicConfig dynamicConfig = params.getCoordinatorDynamicConfig();
+    final ReplicationThrottler replicationThrottler = new ReplicationThrottler(
+        dynamicConfig.getReplicationThrottleLimit(),
+        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/4898)



-- 
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: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] kfaraz commented on a diff in pull request #13197: Segment loading: Allow cancellation and prioritization of load queue items

Posted by "kfaraz (via GitHub)" <gi...@apache.org>.
kfaraz commented on code in PR #13197:
URL: https://github.com/apache/druid/pull/13197#discussion_r991367951


##########
server/src/test/java/org/apache/druid/server/coordinator/simulate/SegmentLoadingNegativeTest.java:
##########
@@ -91,130 +88,4 @@ public void testImmediateLoadingViolatesThrottleLimit()
     verifyDatasourceIsFullyLoaded(datasource);
   }
 
-  /**
-   * Correct behaviour: The first replica on any tier should not be throttled.
-   * <p>
-   * Fix Apache #12881 to fix this test.
-   */
-  @Test

Review Comment:
   Change summary: 
   The underlying behaviour for these tests is now fixed!
   The corresponding positive test cases can be found in `SegmentLoadingTest`.



-- 
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: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] kfaraz commented on a diff in pull request #13197: Segment loading: Allow cancellation and prioritization of load queue items

Posted by "kfaraz (via GitHub)" <gi...@apache.org>.
kfaraz commented on code in PR #13197:
URL: https://github.com/apache/druid/pull/13197#discussion_r1232981660


##########
server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java:
##########
@@ -888,26 +799,101 @@ public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params)
       List<ImmutableDruidServer> currentServers = prepareCurrentServers();
 
       startPeonsForNewServers(currentServers);
+      stopPeonsForDisappearedServers(currentServers);
 
-      cluster = prepareCluster(params, currentServers);
-      segmentReplicantLookup = SegmentReplicantLookup.make(cluster, getDynamicConfigs().getReplicateAfterLoadTimeout());
+      final DruidCluster cluster = prepareCluster(params.getCoordinatorDynamicConfig(), currentServers);
+      cancelLoadsOnDecommissioningServers(cluster);
 
-      stopPeonsForDisappearedServers(currentServers);
+      final CoordinatorDynamicConfig dynamicConfig = params.getCoordinatorDynamicConfig();
 
-      final RoundRobinServerSelector roundRobinServerSelector;
-      if (params.getCoordinatorDynamicConfig().isUseRoundRobinSegmentAssignment()) {
-        roundRobinServerSelector = new RoundRobinServerSelector(cluster);
-        log.info("Using round-robin segment assignment.");
-      } else {
-        roundRobinServerSelector = null;
+      initBalancerExecutor();
+      final BalancerStrategy balancerStrategy = balancerStrategyFactory.createBalancerStrategy(balancerExec);
+      log.info(
+          "Using balancer strategy [%s] with round-robin assignment [%s] and debug dimensions [%s].",
+          balancerStrategy.getClass().getSimpleName(),
+          dynamicConfig.isUseRoundRobinSegmentAssignment(), dynamicConfig.getDebugDimensions()
+      );
+
+      params = params.buildFromExisting()
+                     .withDruidCluster(cluster)
+                     .withDynamicConfigs(recomputeDynamicConfig(params))
+                     .withBalancerStrategy(balancerStrategy)
+                     .withSegmentAssignerUsing(loadQueueManager)
+                     .build();
+
+      segmentReplicantLookup = params.getSegmentReplicantLookup();
+
+      return params;
+    }
+
+    /**
+     * Recomputes dynamic config values if {@code smartLoadQueue} is enabled.
+     */
+    private CoordinatorDynamicConfig recomputeDynamicConfig(DruidCoordinatorRuntimeParams params)
+    {
+      final CoordinatorDynamicConfig dynamicConfig = params.getCoordinatorDynamicConfig();
+      if (!dynamicConfig.isSmartSegmentLoading()) {
+        return dynamicConfig;
       }
 
-      return params.buildFromExisting()
-                   .withDruidCluster(cluster)
-                   .withLoadManagementPeons(loadManagementPeons)
-                   .withSegmentReplicantLookup(segmentReplicantLookup)
-                   .withRoundRobinServerSelector(roundRobinServerSelector)
-                   .build();
+      // Impose a lower bound on both replicationThrottleLimit and maxSegmentsToMove
+      final int throttlePercentage = 2;
+      final int replicationThrottleLimit = Math.max(
+          100,
+          params.getUsedSegments().size() * throttlePercentage / 100
+      );
+
+      // Impose an upper bound on maxSegmentsToMove to ensure that coordinator
+      // run times are bounded. This limit can be relaxed as performance of
+      // the CostBalancerStrategy.computeCost() is improved.
+      final int maxSegmentsToMove = Math.min(1000, replicationThrottleLimit);
+
+      log.info(
+          "Smart segment loading is enabled. Recomputed replicationThrottleLimit"
+          + " [%d] (%d%% of used segments) and maxSegmentsToMove [%d].",
+          replicationThrottleLimit, throttlePercentage, maxSegmentsToMove
+      );
+
+      return CoordinatorDynamicConfig.builder()
+                                     .withMaxSegmentsInNodeLoadingQueue(0)
+                                     .withReplicationThrottleLimit(replicationThrottleLimit)
+                                     .withMaxSegmentsToMove(maxSegmentsToMove)
+                                     .withUseRoundRobinSegmentAssignment(true)
+                                     .withUseBatchedSegmentSampler(true)
+                                     .withEmitBalancingStats(false)

Review Comment:
   Added a new `SegmentLoadingConfig` which contains the recomputed values and is used by the coordinator duties.



-- 
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: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] kfaraz commented on a diff in pull request #13197: Segment loading: Allow cancellation and prioritization of load queue items

Posted by "kfaraz (via GitHub)" <gi...@apache.org>.
kfaraz commented on code in PR #13197:
URL: https://github.com/apache/druid/pull/13197#discussion_r1231753246


##########
server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java:
##########
@@ -888,26 +799,101 @@ public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params)
       List<ImmutableDruidServer> currentServers = prepareCurrentServers();
 
       startPeonsForNewServers(currentServers);
+      stopPeonsForDisappearedServers(currentServers);
 
-      cluster = prepareCluster(params, currentServers);
-      segmentReplicantLookup = SegmentReplicantLookup.make(cluster, getDynamicConfigs().getReplicateAfterLoadTimeout());
+      final DruidCluster cluster = prepareCluster(params.getCoordinatorDynamicConfig(), currentServers);
+      cancelLoadsOnDecommissioningServers(cluster);
 
-      stopPeonsForDisappearedServers(currentServers);
+      final CoordinatorDynamicConfig dynamicConfig = params.getCoordinatorDynamicConfig();
 
-      final RoundRobinServerSelector roundRobinServerSelector;
-      if (params.getCoordinatorDynamicConfig().isUseRoundRobinSegmentAssignment()) {
-        roundRobinServerSelector = new RoundRobinServerSelector(cluster);
-        log.info("Using round-robin segment assignment.");
-      } else {
-        roundRobinServerSelector = null;
+      initBalancerExecutor();
+      final BalancerStrategy balancerStrategy = balancerStrategyFactory.createBalancerStrategy(balancerExec);
+      log.info(
+          "Using balancer strategy [%s] with round-robin assignment [%s] and debug dimensions [%s].",
+          balancerStrategy.getClass().getSimpleName(),
+          dynamicConfig.isUseRoundRobinSegmentAssignment(), dynamicConfig.getDebugDimensions()
+      );
+
+      params = params.buildFromExisting()
+                     .withDruidCluster(cluster)
+                     .withDynamicConfigs(recomputeDynamicConfig(params))
+                     .withBalancerStrategy(balancerStrategy)
+                     .withSegmentAssignerUsing(loadQueueManager)
+                     .build();
+
+      segmentReplicantLookup = params.getSegmentReplicantLookup();
+
+      return params;
+    }
+
+    /**
+     * Recomputes dynamic config values if {@code smartLoadQueue} is enabled.
+     */
+    private CoordinatorDynamicConfig recomputeDynamicConfig(DruidCoordinatorRuntimeParams params)
+    {
+      final CoordinatorDynamicConfig dynamicConfig = params.getCoordinatorDynamicConfig();
+      if (!dynamicConfig.isSmartSegmentLoading()) {
+        return dynamicConfig;
       }
 
-      return params.buildFromExisting()
-                   .withDruidCluster(cluster)
-                   .withLoadManagementPeons(loadManagementPeons)
-                   .withSegmentReplicantLookup(segmentReplicantLookup)
-                   .withRoundRobinServerSelector(roundRobinServerSelector)
-                   .build();
+      // Impose a lower bound on both replicationThrottleLimit and maxSegmentsToMove
+      final int throttlePercentage = 2;
+      final int replicationThrottleLimit = Math.max(
+          100,
+          params.getUsedSegments().size() * throttlePercentage / 100
+      );
+
+      // Impose an upper bound on maxSegmentsToMove to ensure that coordinator
+      // run times are bounded. This limit can be relaxed as performance of
+      // the CostBalancerStrategy.computeCost() is improved.
+      final int maxSegmentsToMove = Math.min(1000, replicationThrottleLimit);
+
+      log.info(
+          "Smart segment loading is enabled. Recomputed replicationThrottleLimit"
+          + " [%d] (%d%% of used segments) and maxSegmentsToMove [%d].",
+          replicationThrottleLimit, throttlePercentage, maxSegmentsToMove
+      );
+
+      return CoordinatorDynamicConfig.builder()
+                                     .withMaxSegmentsInNodeLoadingQueue(0)
+                                     .withReplicationThrottleLimit(replicationThrottleLimit)
+                                     .withMaxSegmentsToMove(maxSegmentsToMove)
+                                     .withUseRoundRobinSegmentAssignment(true)
+                                     .withUseBatchedSegmentSampler(true)
+                                     .withEmitBalancingStats(false)
+                                     .build(dynamicConfig);
+    }
+
+    /**
+     * Cancels all load/move operations on decommissioning servers. This should
+     * be done before initializing the SegmentReplicantLookup so that
+     * under-replicated segments can be assigned in the current run itself.
+     */
+    private void cancelLoadsOnDecommissioningServers(DruidCluster cluster)
+    {
+      final AtomicInteger cancelledCount = new AtomicInteger(0);
+      final List<ServerHolder> decommissioningServers
+          = cluster.getAllServers().stream()
+                   .filter(ServerHolder::isDecommissioning)
+                   .collect(Collectors.toList());
+
+      for (ServerHolder server : decommissioningServers) {
+        server.getQueuedSegments().forEach(
+            (segment, action) -> {
+              // Cancel the operation if it is a type of load
+              if (action.isLoad() && server.cancelOperation(action, segment)) {
+                cancelledCount.incrementAndGet();
+              }
+            }
+        );
+      }
+
+      if (cancelledCount.get() > 0) {
+        log.info(
+            "Cancelled [%d] load/move operations on [%d] decommissioning servers.",
+            cancelledCount.get(), decommissioningServers.size()
+        );
+      }
     }

Review Comment:
   We are doing this cancellation in this duty (as opposed to `RunRules` or `BalanceSegments`) since the `SegmentReplicantLookup` is constructed right after this method and thus the Coordinator knows in this run itself that some segments are under-replicated and it needs to queue up some loads on active servers.



-- 
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: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] kfaraz commented on a diff in pull request #13197: Segment loading: Allow cancellation and prioritization of load queue items

Posted by "kfaraz (via GitHub)" <gi...@apache.org>.
kfaraz commented on code in PR #13197:
URL: https://github.com/apache/druid/pull/13197#discussion_r1231768921


##########
server/src/main/java/org/apache/druid/server/coordinator/loadqueue/HttpLoadQueuePeon.java:
##########
@@ -470,134 +517,127 @@
     return Collections.unmodifiableSet(segmentsMarkedToDrop);
   }
 
-  private abstract class SegmentHolder
+  /**
+   * A request is considered to have timed out if the time elapsed since it was
+   * first sent to the server is greater than the configured load timeout.
+   *
+   * @see DruidCoordinatorConfig#getLoadTimeoutDelay()
+   */
+  private boolean hasRequestTimedOut(SegmentHolder holder, long currentTimeMillis)
   {
-    private final DataSegment segment;
-    private final DataSegmentChangeRequest changeRequest;
-    private final List<LoadPeonCallback> callbacks = new ArrayList<>();
-
-    // Time when this request was sent to target server the first time.
-    private volatile long scheduleTime = -1;
-
-    private SegmentHolder(
-        DataSegment segment,
-        DataSegmentChangeRequest changeRequest,
-        LoadPeonCallback callback
-    )
-    {
-      this.segment = segment;
-      this.changeRequest = changeRequest;
+    return holder.isRequestSentToServer()
+           && currentTimeMillis - holder.getFirstRequestMillis()
+              > config.getLoadTimeoutDelay().getMillis();
+  }
 
-      if (callback != null) {
-        this.callbacks.add(callback);
-      }
-    }
+  private void onRequestFailed(SegmentHolder holder, String failureCause)
+  {
+    log.error(
+        "Server[%s] failed segment[%s] request[%s] with cause [%s].",
+        serverId, holder.getSegment().getId(), holder.getAction(), failureCause
+    );
+    onRequestCompleted(holder, QueueStatus.FAILED);
+  }
 
-    public void addCallback(LoadPeonCallback newCallback)
-    {
-      synchronized (callbacks) {
-        if (newCallback != null) {
-          callbacks.add(newCallback);
-        }
-      }
-    }
+  private void onRequestCompleted(SegmentHolder holder, QueueStatus status)
+  {
+    final SegmentAction action = holder.getAction();
+    log.trace(
+        "Server[%s] completed request[%s] on segment[%s] with status[%s].",
+        serverId, action, holder.getSegment().getId(), status
+    );
 
-    public DataSegment getSegment()
-    {
-      return segment;
+    if (holder.isLoad()) {
+      queuedSize.addAndGet(-holder.getSegment().getSize());
     }
+    incrementStat(holder, status);
+    executeCallbacks(holder, status == QueueStatus.SUCCESS);
+  }
 
-    public DataSegmentChangeRequest getChangeRequest()
-    {
-      return changeRequest;
+  private void incrementStat(SegmentHolder holder, QueueStatus status)
+  {
+    stats.add(status.getStatForAction(holder.getAction()), 1);
+    if (status.datasourceStat != null) {
+      stats.addToDatasourceStat(status.datasourceStat, holder.getSegment().getDataSource(), 1);
     }
+  }
 
-    public boolean hasTimedOut()
-    {
-      if (scheduleTime < 0) {
-        scheduleTime = System.currentTimeMillis();
-        return false;
-      } else if (System.currentTimeMillis() - scheduleTime > config.getLoadTimeoutDelay().getMillis()) {
-        return true;
-      } else {
+  private void executeCallbacks(SegmentHolder holder, boolean success)
+  {
+    callBackExecutor.execute(() -> {
+      for (LoadPeonCallback callback : holder.getCallbacks()) {
+        callback.execute(success);
+      }
+    });
+  }
+
+  /**
+   * Tries to cancel a load/drop operation. An load/drop request can be cancelled
+   * only if it has not already been sent to the corresponding server.
+   */
+  @Override
+  public boolean cancelOperation(DataSegment segment)
+  {
+    synchronized (lock) {
+      if (activeRequestSegments.contains(segment)) {
         return false;
       }
-    }
 
-    public void requestSucceeded()
-    {
-      log.trace(
-          "Server[%s] Successfully processed segment[%s] request[%s].",
-          serverId,
-          segment.getId(),
-          changeRequest.getClass().getSimpleName()
-      );
+      // Find the action on this segment, if any
+      final SegmentHolder holder = segmentsToLoad.containsKey(segment)
+                                   ? segmentsToLoad.remove(segment)
+                                   : segmentsToDrop.remove(segment);
+      if (holder == null) {
+        return false;
+      }
 
-      callBackExecutor.execute(() -> {
-        for (LoadPeonCallback callback : callbacks) {
-          if (callback != null) {
-            callback.execute(true);
-          }
-        }
-      });
+      queuedSegments.remove(holder);
+      onRequestCompleted(holder, QueueStatus.CANCELLED);
+      return true;
     }
+  }
 
-    public void requestFailed(String failureCause)
-    {
-      log.error(
-          "Server[%s] Failed segment[%s] request[%s] with cause [%s].",
-          serverId,
-          segment.getId(),
-          changeRequest.getClass().getSimpleName(),
-          failureCause
-      );
-
-      failedAssignCount.getAndIncrement();
+  private enum QueueStatus
+  {
+    ASSIGNED(Stats.SegmentQueue.ASSIGNED_ACTIONS),
+    SUCCESS(Stats.SegmentQueue.COMPLETED_ACTIONS),
+    FAILED(Stats.SegmentQueue.FAILED_ACTIONS),
+    CANCELLED(Stats.SegmentQueue.CANCELLED_ACTIONS);
 
-      callBackExecutor.execute(() -> {
-        for (LoadPeonCallback callback : callbacks) {
-          if (callback != null) {
-            callback.execute(false);
-          }
-        }
-      });
-    }
+    final CoordinatorStat loadStat;
+    final CoordinatorStat moveStat;
+    final CoordinatorStat dropStat;
+    final CoordinatorStat datasourceStat;
 
-    @Override
-    public String toString()
+    QueueStatus()
     {
-      return changeRequest.toString();
+      this(null);
     }
-  }
 
-  private class LoadSegmentHolder extends SegmentHolder
-  {
-    public LoadSegmentHolder(DataSegment segment, LoadPeonCallback callback)
+    QueueStatus(CoordinatorStat datasourceStat)
     {
-      super(segment, new SegmentChangeRequestLoad(segment), callback);
-      queuedSize.addAndGet(segment.getSize());
-    }
+      // These stats are not emitted and are tracked for debugging purposes only
+      final String prefix = StringUtils.toLowerCase(name());
+      this.loadStat = new CoordinatorStat(prefix + "Load");
+      this.moveStat = new CoordinatorStat(prefix + "Move");
+      this.dropStat = new CoordinatorStat(prefix + "Drop");
 
-    @Override
-    public void requestSucceeded()
-    {
-      queuedSize.addAndGet(-getSegment().getSize());
-      super.requestSucceeded();
+      this.datasourceStat = datasourceStat;
     }
 
-    @Override
-    public void requestFailed(String failureCause)
+    CoordinatorStat getStatForAction(SegmentAction action)
     {
-      queuedSize.addAndGet(-getSegment().getSize());
-      super.requestFailed(failureCause);
-    }
-  }
+      switch (action) {

Review Comment:
   No, it is okay, `MOVE_FROM` operations aren't supposed to come to the peon. So the next line will throw an IAE. I can just have the `default` case itself throw the exception so that `CodeQL` does not complain.



-- 
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: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] kfaraz commented on a diff in pull request #13197: Segment loading: Allow cancellation and prioritization of load queue items

Posted by "kfaraz (via GitHub)" <gi...@apache.org>.
kfaraz commented on code in PR #13197:
URL: https://github.com/apache/druid/pull/13197#discussion_r991354100


##########
server/src/main/java/org/apache/druid/server/coordinator/ServerHolder.java:
##########
@@ -20,31 +20,61 @@
 package org.apache.druid.server.coordinator;
 
 import org.apache.druid.client.ImmutableDruidServer;
-import org.apache.druid.java.util.common.logger.Logger;
 import org.apache.druid.timeline.DataSegment;
 import org.apache.druid.timeline.SegmentId;
 
 import java.util.Objects;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
 
 /**
+ *
  */
 public class ServerHolder implements Comparable<ServerHolder>

Review Comment:
   Change summary:
   Maintain a map containing
   - segments that were in load or drop queue when this run started
   - segments that were queued for load or drop during this run
   This helps the `SegmentLoader` determine the current state of a segment on a server and make load/drop decisions.



-- 
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: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] kfaraz commented on a diff in pull request #13197: Segment loading: Allow cancellation and prioritization of load queue items

Posted by "kfaraz (via GitHub)" <gi...@apache.org>.
kfaraz commented on code in PR #13197:
URL: https://github.com/apache/druid/pull/13197#discussion_r991336008


##########
server/src/main/java/org/apache/druid/server/coordinator/BalancerSegmentHolder.java:
##########
@@ -19,29 +19,28 @@
 
 package org.apache.druid.server.coordinator;
 
-import org.apache.druid.client.ImmutableDruidServer;
 import org.apache.druid.timeline.DataSegment;
 
 /**
  */
 public class BalancerSegmentHolder

Review Comment:
   Change summary:
   Added `ServerHolder` to avoid having to find the corresponding peon later in the flow.



##########
server/src/main/java/org/apache/druid/server/coordinator/BalancerStrategy.java:
##########
@@ -46,13 +46,18 @@
   ServerHolder findNewSegmentHomeBalancer(DataSegment proposalSegment, List<ServerHolder> serverHolders);
 
   /**
-   * Find the best server on which to place a {@link DataSegment} replica according to the balancing strategy
+   * Finds the best servers on which to place a replica of the {@code proposalSegment}
+   * according to the balancing strategy.
+   *
    * @param proposalSegment segment to replicate
-   * @param serverHolders servers to consider as replica holders
-   * @return The server to replicate to, or null if no suitable server is found
+   * @param serverHolders   servers to consider as replica holders
+   * @return Iterator over the best servers (in order) on which the replica(s)
+   * can be placed.
    */
-  @Nullable
-  ServerHolder findNewSegmentHomeReplicator(DataSegment proposalSegment, List<ServerHolder> serverHolders);
+  Iterator<ServerHolder> findNewSegmentHomeReplicator(

Review Comment:
   Change summary:
   Find all the best servers to place replicas in one go rather than multiple strategy computations.



##########
server/src/main/java/org/apache/druid/server/coordinator/CostBalancerStrategy.java:
##########
@@ -184,20 +193,17 @@ public CostBalancerStrategy(ListeningExecutorService exec)
   }
 
   @Override
-  public ServerHolder findNewSegmentHomeReplicator(DataSegment proposalSegment, List<ServerHolder> serverHolders)
+  public Iterator<ServerHolder> findNewSegmentHomeReplicator(DataSegment proposalSegment, List<ServerHolder> serverHolders)

Review Comment:
   Change summary:
   Find all the best servers to place replicas in one go.



-- 
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: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] kfaraz commented on a diff in pull request #13197: Segment loading: Allow cancellation and prioritization of load queue items

Posted by "kfaraz (via GitHub)" <gi...@apache.org>.
kfaraz commented on code in PR #13197:
URL: https://github.com/apache/druid/pull/13197#discussion_r991348309


##########
server/src/main/java/org/apache/druid/server/coordinator/ReplicationThrottler.java:
##########
@@ -36,74 +39,89 @@
 {
   private static final EmittingLogger log = new EmittingLogger(ReplicationThrottler.class);
 
-  private final Map<String, Boolean> replicatingLookup = new HashMap<>();

Review Comment:
   Change summary:
   Commoned out `ReplicationSegmentHolder` as `TierLoadingState` to be used for tracking both `currentlyReplicatingSegments` and `currentlyMovingSegments` (in `SegmentStateManager`).



-- 
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: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] kfaraz commented on a diff in pull request #13197: Segment loading: Allow cancellation and prioritization of load queue items

Posted by "kfaraz (via GitHub)" <gi...@apache.org>.
kfaraz commented on code in PR #13197:
URL: https://github.com/apache/druid/pull/13197#discussion_r1231759286


##########
server/src/main/java/org/apache/druid/server/coordinator/ServerHolder.java:
##########
@@ -155,128 +211,161 @@ public long getAvailableSize()
    * <p>
    * A load is possible only if the server meets all of the following criteria:
    * <ul>
-   *   <li>is not already serving or loading the segment</li>
    *   <li>is not being decommissioned</li>
+   *   <li>is not already serving the segment</li>
+   *   <li>is not performing any other action on the segment</li>
    *   <li>has not already exceeded the load queue limit in this run</li>
    *   <li>has available disk space</li>
    * </ul>
    */
   public boolean canLoadSegment(DataSegment segment)
   {
-    final SegmentState state = getSegmentState(segment);
     return !isDecommissioning
-           && (maxSegmentsInLoadQueue == 0 || maxSegmentsInLoadQueue > segmentsQueuedForLoad)
-           && getAvailableSize() >= segment.getSize()
-           && state == SegmentState.NONE;
+           && !hasSegmentLoaded(segment.getId())
+           && getActionOnSegment(segment) == null
+           && totalAssignmentsInRun < maxAssignmentsInRun
+           && getAvailableSize() >= segment.getSize();
   }
 
-  public SegmentState getSegmentState(DataSegment segment)
+  public SegmentAction getActionOnSegment(DataSegment segment)
   {
-    SegmentState state = queuedSegments.get(segment.getId());
-    if (state != null) {
-      return state;
-    }
-
-    return isServingSegment(segment) ? SegmentState.LOADED : SegmentState.NONE;
+    return queuedSegments.get(segment);
   }
 
   /**
    * Segments queued for load, drop or move on this server.
+   * <ul>
+   * <li>Contains segments present in the queue when the current coordinator run started.</li>
+   * <li>Contains segments added to the queue during the current run.</li>
+   * <li>Maps replicating segments to LOAD rather than REPLICATE for simplicity.</li>
+   * <li>Does not contain segments whose actions were cancelled.</li>
+   * </ul>
    */
-  public Map<SegmentId, SegmentState> getQueuedSegments()
+  public Map<DataSegment, SegmentAction> getQueuedSegments()
   {
-    return Collections.unmodifiableMap(queuedSegments);
+    return new HashMap<>(queuedSegments);
   }
 
+  /**
+   * Segments that are expected to be loaded on this server once all the
+   * operations in progress have completed.
+   */
+  public Set<DataSegment> getProjectedSegments()
+  {
+    return projectedSegments;
+  }
+
+  /**
+   * Segments that are currently in the queue for being loaded on this server.
+   * This does not include segments that are being moved to this server.
+   */
+  public List<DataSegment> getLoadingSegments()
+  {
+    final List<DataSegment> loadingSegments = new ArrayList<>();
+    queuedSegments.forEach((segment, action) -> {
+      if (action == SegmentAction.LOAD) {
+        loadingSegments.add(segment);
+      }
+    });
+
+    return loadingSegments;
+  }
+
+  /**
+   * Returns true if this server has the segment loaded and is not dropping it.
+   */
   public boolean isServingSegment(DataSegment segment)
   {
-    return isServingSegment(segment.getId());
+    return hasSegmentLoaded(segment.getId()) && getActionOnSegment(segment) == null;
   }
 
   public boolean isLoadingSegment(DataSegment segment)
   {
-    return getSegmentState(segment) == SegmentState.LOADING;
+    return getActionOnSegment(segment) == SegmentAction.LOAD;
   }
 
   public boolean isDroppingSegment(DataSegment segment)
   {
-    return getSegmentState(segment) == SegmentState.DROPPING;
+    return getActionOnSegment(segment) == SegmentAction.DROP;
   }
 
-  public boolean startOperation(DataSegment segment, SegmentState newState)
+  public int getNumMovingSegments()
   {
-    if (queuedSegments.containsKey(segment.getId())) {
-      return false;
-    }
+    return movingSegmentCount;
+  }
 
-    if (newState == SegmentState.LOADING || newState == SegmentState.MOVING_TO) {
-      ++segmentsQueuedForLoad;
-      sizeOfLoadingSegments += segment.getSize();
-    }
-    queuedSegments.put(segment.getId(), newState);
-    return true;
+  public int getNumLoadingReplicas()
+  {
+    return loadingReplicaCount;
   }
 
-  public boolean cancelOperation(DataSegment segment, SegmentState currentState)
+  public boolean startOperation(SegmentAction action, DataSegment segment)
   {
-    SegmentState observedState = queuedSegments.get(segment.getId());
-    if (observedState != currentState) {
+    if (queuedSegments.containsKey(segment)) {
       return false;
     }
 
-    if (currentState == SegmentState.LOADING || currentState == SegmentState.MOVING_TO) {
-      --segmentsQueuedForLoad;
-      sizeOfLoadingSegments -= segment.getSize();
+    if (action.isLoad()) {
+      ++totalAssignmentsInRun;
     }
-    queuedSegments.remove(segment.getId());
+
+    updateQueuedSegments(segment, simplify(action), true);
     return true;
   }
 
-  public boolean isServingSegment(SegmentId segmentId)
+  public boolean cancelOperation(SegmentAction action, DataSegment segment)
+  {
+    final SegmentAction queuedAction = queuedSegments.get(segment);
+    return queuedAction == simplify(action)
+           && (queuedAction == SegmentAction.MOVE_FROM || peon.cancelOperation(segment))
+           && updateQueuedSegments(segment, queuedAction, false);
+  }
+
+  public boolean hasSegmentLoaded(SegmentId segmentId)
   {
     return server.getSegment(segmentId) != null;
   }
 
-  /**
-   * Checks if the server can load the given segment.
-   * <p>
-   * A load is possible only if the server meets all of the following criteria:
-   * <ul>
-   *   <li>is not being decommissioned</li>
-   *   <li>is not already serving the segment</li>
-   *   <li>is not performing any other action on the segment</li>
-   *   <li>has not already exceeded the load queue limit in this run</li>
-   *   <li>has available disk space</li>
-   * </ul>
-   */
-  public boolean canLoadSegment(DataSegment segment)
+  public boolean isRealtimeServer()
   {
-    return !isDecommissioning
-           && !isServingSegment(segment.getId())
-           && !isLoadingSegment(segment)
-           && (maxSegmentsInLoadQueue == 0 || maxSegmentsInLoadQueue > peon.getNumberOfSegmentsInQueue())
-           && getAvailableSize() >= segment.getSize();
+    return server.getType() == ServerType.REALTIME
+           || server.getType() == ServerType.INDEXER_EXECUTOR;
   }
 
-  @Override
-  public int compareTo(ServerHolder serverHolder)
+  private SegmentAction simplify(SegmentAction action)
   {
-    int result = Long.compare(getAvailableSize(), serverHolder.getAvailableSize());
-    if (result != 0) {
-      return result;
+    return action == SegmentAction.REPLICATE ? SegmentAction.LOAD : action;
+  }
+
+  private boolean updateQueuedSegments(DataSegment segment, SegmentAction action, boolean addToQueue)
+  {
+    if (addToQueue) {
+      queuedSegments.put(segment, action);
+    } else {
+      queuedSegments.remove(segment);
     }
 
-    result = server.getHost().compareTo(serverHolder.server.getHost());
-    if (result != 0) {
-      return result;
+    final long sizeDelta = addToQueue ? segment.getSize() : -segment.getSize();
+    if (action.isLoad()) {
+      sizeOfLoadingSegments += sizeDelta;
+    } else if (action == SegmentAction.DROP) {
+      sizeOfDroppingSegments += sizeDelta;
     }
 
-    result = server.getTier().compareTo(serverHolder.server.getTier());
-    if (result != 0) {
-      return result;
+    // Remove from projected if load is cancelled or drop is started, add otherwise
+    if (addToQueue ^ action.isLoad()) {
+      projectedSegments.remove(segment);
+    } else {
+      projectedSegments.add(segment);
     }
 
-    return server.getType().compareTo(serverHolder.server.getType());
+    return true;
+  }

Review Comment:
   Yeah, the boolean here is stupid, it's always `true`, must have been doing some other things originally. With the recent changes, it would have become redundant. Can get rid of it without breaking up the method too.



-- 
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: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] github-code-scanning[bot] commented on a diff in pull request #13197: Segment loading: Allow cancellation and prioritization of load queue items

Posted by "github-code-scanning[bot] (via GitHub)" <gi...@apache.org>.
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: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] kfaraz merged pull request #13197: Segment loading: Allow cancellation and prioritization of load queue items

Posted by "kfaraz (via GitHub)" <gi...@apache.org>.
kfaraz merged PR #13197:
URL: https://github.com/apache/druid/pull/13197


-- 
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: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] kfaraz commented on a diff in pull request #13197: Segment loading: Allow cancellation and prioritization of load queue items

Posted by GitBox <gi...@apache.org>.
kfaraz commented on code in PR #13197:
URL: https://github.com/apache/druid/pull/13197#discussion_r993657773


##########
server/src/main/java/org/apache/druid/server/coordinator/TierLoadingState.java:
##########
@@ -0,0 +1,72 @@
+/*
+ * 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;
+
+import org.apache.druid.timeline.SegmentId;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * Contains segments being loaded or replicated in a tier.
+ */
+public class TierLoadingState
+{
+  private int lifetime;

Review Comment:
   That would be the ideal behaviour but as it turns out, this is the existing implementation. I have tried to keep it unchanged for now.
   
   Let's say, in a coordinator run, we queue some segments to move within a tier. All the segments queued would start with the same lifecycle. Only when all these queued segments have been processed will we consider this tier for moving again. So, at it turns out, in `BalanceSegments.currentlyMovingSegments` in the existing code, all segment holders for a given tier will always have the same lifecycle.
   Same principle applies to replication and `ReplicationThrottleLimit.currentlyReplicating`



-- 
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: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] kfaraz commented on a diff in pull request #13197: Segment loading: Allow cancellation and prioritization of load queue items

Posted by GitBox <gi...@apache.org>.
kfaraz commented on code in PR #13197:
URL: https://github.com/apache/druid/pull/13197#discussion_r993747007


##########
server/src/main/java/org/apache/druid/server/coordinator/TierLoadingState.java:
##########
@@ -0,0 +1,72 @@
+/*
+ * 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;
+
+import org.apache.druid.timeline.SegmentId;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * Contains segments being loaded or replicated in a tier.
+ */
+public class TierLoadingState
+{
+  private int lifetime;

Review Comment:
   Sorry, I must have missed it while testing. Do you mean that the behaviour is changing in this PR?



-- 
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: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] kfaraz commented on a diff in pull request #13197: Segment loading: Allow cancellation and prioritization of load queue items

Posted by GitBox <gi...@apache.org>.
kfaraz commented on code in PR #13197:
URL: https://github.com/apache/druid/pull/13197#discussion_r993141144


##########
server/src/main/java/org/apache/druid/server/coordinator/SegmentAction.java:
##########
@@ -0,0 +1,33 @@
+/*
+ * 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;
+
+/**
+ * Represents actions that can be performed on a server for a single segment.
+ * <p>
+ * The different action types can be used to prioritize items in a LoadQueuePeon.
+ */
+public enum SegmentAction
+{
+  DROP,
+  LOAD_AS_PRIMARY,

Review Comment:
   Rename to something like `ACTIVE_LOAD`, `LAZY_LOAD`.



-- 
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: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] kfaraz commented on a diff in pull request #13197: Segment loading: Allow cancellation and prioritization of load queue items

Posted by GitBox <gi...@apache.org>.
kfaraz commented on code in PR #13197:
URL: https://github.com/apache/druid/pull/13197#discussion_r993462870


##########
server/src/main/java/org/apache/druid/server/coordinator/CostBalancerStrategy.java:
##########
@@ -49,6 +50,14 @@ public class CostBalancerStrategy implements BalancerStrategy
   private static final double MILLIS_IN_HOUR = 3_600_000.0;
   private static final double MILLIS_FACTOR = MILLIS_IN_HOUR / LAMBDA;
 
+  /**
+   * Comparator that prioritizes servers by cost. Cheaper servers come before
+   * costlier servers. Servers with the same cost may appear in a random order.
+   */
+  private static final Comparator<Pair<Double, ServerHolder>> CHEAPEST_SERVERS_FIRST
+      = Comparator.<Pair<Double, ServerHolder>, Double>comparing(pair -> pair.lhs)
+      .thenComparing(pair -> ThreadLocalRandom.current().nextInt());

Review Comment:
   Sure, that would work too (but it would make more sense to use free size or free percentage rather than size used).
   
   This PR does not make any modifications to strategies, so it has not been included here. The only modification done here is reduction in the number of calls to strategy.



-- 
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: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] kfaraz commented on a diff in pull request #13197: Segment loading: Allow cancellation and prioritization of load queue items

Posted by GitBox <gi...@apache.org>.
kfaraz commented on code in PR #13197:
URL: https://github.com/apache/druid/pull/13197#discussion_r994060186


##########
server/src/main/java/org/apache/druid/server/coordinator/BalancerStrategy.java:
##########
@@ -46,13 +46,18 @@
   ServerHolder findNewSegmentHomeBalancer(DataSegment proposalSegment, List<ServerHolder> serverHolders);
 
   /**
-   * Find the best server on which to place a {@link DataSegment} replica according to the balancing strategy
+   * Finds the best servers on which to place a replica of the {@code proposalSegment}
+   * according to the balancing strategy.

Review Comment:
   Okay, I will try to update the javadoc to reflect this distinction. But, as it happens, the same method is used for both "new/unavailable" segments as well "replicating/available" segments. (this would go away if we do a round-robin assignment in the first cycle for every segment) 
   So I guess the method name should just be `findSegmentHomes`?



-- 
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: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] kfaraz commented on a diff in pull request #13197: Segment loading: Allow cancellation and prioritization of load queue items

Posted by GitBox <gi...@apache.org>.
kfaraz commented on code in PR #13197:
URL: https://github.com/apache/druid/pull/13197#discussion_r994080461


##########
server/src/main/java/org/apache/druid/server/coordinator/RandomBalancerStrategy.java:
##########
@@ -26,24 +26,20 @@
 import java.util.Iterator;
 import java.util.List;
 import java.util.NavigableSet;
-import java.util.concurrent.ThreadLocalRandom;
 import java.util.stream.Collectors;
 
 public class RandomBalancerStrategy implements BalancerStrategy
 {
   @Override
-  public ServerHolder findNewSegmentHomeReplicator(DataSegment proposalSegment, List<ServerHolder> serverHolders)
+  public Iterator<ServerHolder> findNewSegmentHomeReplicator(DataSegment proposalSegment, List<ServerHolder> serverHolders)

Review Comment:
   On second thought, I wonder if `Iterator` is actually better. A caller should always iterate over the eligible servers in order and we get the benefit of the simple `hasNext` and `next` methods, rather than having to use `isEmpty` and `get`.
   
   The one benefit I see with having a list is if we wanted to get the count of servers that were found to be eligible upfront (in case we wanted to report that).
   What do you advise?



-- 
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: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] AmatyaAvadhanula commented on a diff in pull request #13197: Segment loading: Allow cancellation and prioritization of load queue items

Posted by GitBox <gi...@apache.org>.
AmatyaAvadhanula commented on code in PR #13197:
URL: https://github.com/apache/druid/pull/13197#discussion_r994670349


##########
server/src/main/java/org/apache/druid/server/coordinator/TierLoadingState.java:
##########
@@ -0,0 +1,72 @@
+/*
+ * 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;
+
+import org.apache.druid.timeline.SegmentId;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * Contains segments being loaded or replicated in a tier.
+ */
+public class TierLoadingState
+{
+  private int lifetime;

Review Comment:
   Yes



-- 
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: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] AmatyaAvadhanula commented on a diff in pull request #13197: Segment loading: Allow cancellation and prioritization of load queue items

Posted by GitBox <gi...@apache.org>.
AmatyaAvadhanula commented on code in PR #13197:
URL: https://github.com/apache/druid/pull/13197#discussion_r994671386


##########
server/src/main/java/org/apache/druid/server/coordinator/TierLoadingState.java:
##########
@@ -0,0 +1,72 @@
+/*
+ * 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;
+
+import org.apache.druid.timeline.SegmentId;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * Contains segments being loaded or replicated in a tier.

Review Comment:
   Isn't it segments being moved or being replicated, but not as a primary replica?



-- 
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: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] kfaraz commented on a diff in pull request #13197: Segment loading: Allow cancellation and prioritization of load queue items

Posted by GitBox <gi...@apache.org>.
kfaraz commented on code in PR #13197:
URL: https://github.com/apache/druid/pull/13197#discussion_r994083255


##########
server/src/main/java/org/apache/druid/server/coordinator/duty/EmitClusterStatsAndMetrics.java:
##########
@@ -43,41 +42,31 @@ public class EmitClusterStatsAndMetrics implements CoordinatorDuty
 {
   private static final Logger log = new Logger(EmitClusterStatsAndMetrics.class);
 
-  public static final String TOTAL_CAPACITY = "totalCapacity";
   public static final String TOTAL_HISTORICAL_COUNT = "totalHistoricalCount";
   public static final String MAX_REPLICATION_FACTOR = "maxReplicationFactor";
 
   private final DruidCoordinator coordinator;
   private final String groupName;
   private final boolean isContainCompactSegmentDuty;
+  private final ServiceEmitter emitter;
 
-  public EmitClusterStatsAndMetrics(DruidCoordinator coordinator, String groupName, boolean isContainCompactSegmentDuty)
+  public EmitClusterStatsAndMetrics(
+      DruidCoordinator coordinator,
+      String groupName,
+      boolean isContainCompactSegmentDuty,
+      ServiceEmitter emitter

Review Comment:
   There is a single DI'd instance of `ServiceEmitter` that gets passed around. The emitter itself and the dimension map that it carries are immutable, so we should be good there.
   
   Unfortunately, we didn't have any existing test that verified (all) the dimensions that were being emitted in a metric event. Such tests passing would have helped us validate this fully. But at least looking at the code, we shouldn't be breaking anything.
   
   I will be doing some further cluster testing today and will verify the emitted metric events. We should be able to add some sim tests for this too.



-- 
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: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] kfaraz commented on a diff in pull request #13197: Segment loading: Allow cancellation and prioritization of load queue items

Posted by GitBox <gi...@apache.org>.
kfaraz commented on code in PR #13197:
URL: https://github.com/apache/druid/pull/13197#discussion_r994062342


##########
server/src/main/java/org/apache/druid/server/coordinator/SegmentLoader.java:
##########
@@ -0,0 +1,472 @@
+/*
+ * 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;
+
+import com.google.common.collect.Sets;
+import org.apache.druid.java.util.emitter.EmittingLogger;
+import org.apache.druid.timeline.DataSegment;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.EnumMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.stream.Collectors;
+
+/**
+ * Used by the coordinator in each run for segment loading, dropping, balancing
+ * and broadcasting.
+ * <p>
+ * An instance of this class is freshly created for each coordinator run.
+ */
+public class SegmentLoader
+{
+  private static final EmittingLogger log = new EmittingLogger(SegmentLoader.class);
+
+  private final SegmentStateManager stateManager;
+  private final DruidCluster cluster;
+  private final CoordinatorStats stats = new CoordinatorStats();
+  private final SegmentReplicantLookup replicantLookup;
+  private final BalancerStrategy strategy;
+  private final int maxLoadQueueSize;
+
+  public SegmentLoader(SegmentStateManager stateManager, DruidCoordinatorRuntimeParams runParams)
+  {
+    this.stateManager = stateManager;
+    this.strategy = runParams.getBalancerStrategy();
+    this.cluster = runParams.getDruidCluster();
+    this.replicantLookup = runParams.getSegmentReplicantLookup();
+    this.maxLoadQueueSize = runParams.getCoordinatorDynamicConfig()
+                                     .getMaxSegmentsInNodeLoadingQueue();
+  }
+
+  public CoordinatorStats getStats()
+  {
+    return stats;
+  }
+
+  /**
+   * Moves the given segment between two servers of the same tier.
+   * <p>
+   * See if we can move balancing here.
+   */
+  public boolean moveSegment(DataSegment segment, ServerHolder fromServer, ServerHolder toServer)
+  {
+    if (!fromServer.getServer().getTier().equals(toServer.getServer().getTier())) {
+      return false;
+    }
+
+    // fromServer must be loading or serving the segment
+    // and toServer must be able to load it
+    final SegmentState stateOnSrc = fromServer.getSegmentState(segment);
+    if ((stateOnSrc != SegmentState.LOADING && stateOnSrc != SegmentState.LOADED)
+        || !canLoadSegment(toServer, segment)) {
+      return false;
+    }
+
+    final boolean cancelSuccess = stateOnSrc == SegmentState.LOADING
+                                  && stateManager.cancelOperation(SegmentState.LOADING, segment, fromServer);
+
+    if (cancelSuccess) {
+      int loadedCountOnTier = replicantLookup
+          .getLoadedReplicants(segment.getId(), toServer.getServer().getTier());
+      stateManager.loadSegment(segment, toServer, loadedCountOnTier < 1);
+    } else {
+      return stateManager.moveSegment(segment, fromServer, toServer);
+    }
+
+    return true;
+  }
+
+  /**
+   * Queues load or drop of replicas of the given segment to achieve the
+   * target replication level in all the tiers.
+   */
+  public void updateReplicas(DataSegment segment, Map<String, Integer> tierToReplicaCount)
+  {
+    // Handle every target tier
+    tierToReplicaCount.forEach((tier, numReplicas) -> {
+      updateReplicasOnTier(segment, tier, tierToReplicaCount.get(tier));
+      stats.addToTieredStat(CoordinatorStats.REQUIRED_CAPACITY, tier, segment.getSize() * numReplicas);
+    });
+
+    // Find the minimum number of segments required for fault tolerance
+    final int totalTargetReplicas = tierToReplicaCount.values().stream()
+                                                      .reduce(0, Integer::sum);
+    final int minLoadedSegments = totalTargetReplicas > 1 ? 2 : 1;
+
+    // Drop segment from unneeded tiers if requirement is met across target tiers
+    int loadedTargetReplicas = 0;
+    final Set<String> targetTiers = tierToReplicaCount.keySet();
+    for (String tier : targetTiers) {
+      loadedTargetReplicas += replicantLookup.getLoadedReplicants(segment.getId(), tier);
+    }
+    if (loadedTargetReplicas < minLoadedSegments) {
+      return;
+    }
+
+    final Set<String> dropTiers = Sets.newHashSet(cluster.getTierNames());
+    dropTiers.removeAll(targetTiers);
+    for (String dropTier : dropTiers) {
+      updateReplicasOnTier(segment, dropTier, 0);
+    }
+  }
+
+  /**
+   * Broadcasts the given segment to all servers that are broadcast targets and
+   * queues a drop of the segment from decommissioning servers.
+   */
+  public void broadcastSegment(DataSegment segment)
+  {
+    int assignedCount = 0;
+    int droppedCount = 0;
+    for (ServerHolder server : cluster.getAllServers()) {
+      // Ignore servers which are not broadcast targets
+      if (!server.getServer().getType().isSegmentBroadcastTarget()) {
+        continue;
+      }
+
+      if (server.isDecommissioning()) {
+        droppedCount += dropBroadcastSegment(segment, server) ? 1 : 0;
+      } else {
+        assignedCount += loadBroadcastSegment(segment, server) ? 1 : 0;
+      }
+    }
+
+    if (assignedCount > 0) {
+      stats.addToDataSourceStat(CoordinatorStats.BROADCAST_LOADS, segment.getDataSource(), assignedCount);
+    }
+    if (droppedCount > 0) {
+      stats.addToDataSourceStat(CoordinatorStats.BROADCAST_DROPS, segment.getDataSource(), droppedCount);
+    }
+  }
+
+  /**
+   * Marks the given segment as unused.
+   */
+  public void deleteSegment(DataSegment segment)
+  {
+    stateManager.deleteSegment(segment);
+    stats.addToGlobalStat(CoordinatorStats.DELETED_SEGMENTS, 1);
+  }
+
+  /**
+   * Checks if the server can load the given segment.
+   * <p>
+   * A load is possible only if the server meets all of the following criteria:
+   * <ul>
+   *   <li>is not already serving or loading the segment</li>
+   *   <li>is not being decommissioned</li>
+   *   <li>has not already exceeded the load queue limit in this run</li>
+   *   <li>has available disk space</li>
+   * </ul>
+   */
+  public boolean canLoadSegment(ServerHolder server, DataSegment segment)
+  {
+    return server.canLoadSegment(segment)
+           && (maxLoadQueueSize == 0 || maxLoadQueueSize > server.getSegmentsQueuedForLoad());
+  }
+
+  /**
+   * Loads the broadcast segment if it is not loaded on the given server.
+   * Returns true only if the segment was successfully queued for load on the server.
+   */
+  private boolean loadBroadcastSegment(DataSegment segment, ServerHolder server)
+  {
+    final SegmentState state = server.getSegmentState(segment);
+    if (state == SegmentState.LOADED || state == SegmentState.LOADING) {
+      return false;
+    }
+
+    // Cancel drop if it is in progress
+    boolean dropCancelled = stateManager.cancelOperation(SegmentState.DROPPING, segment, server);
+    if (dropCancelled) {
+      return false;
+    }
+
+    if (canLoadSegment(server, segment)
+        && stateManager.loadSegment(segment, server, true)) {
+      return true;
+    } else {
+      log.makeAlert("Failed to broadcast segment for [%s]", segment.getDataSource())

Review Comment:
   Yes, I am going through all the log/alert messages to make sure they capture the right information.



-- 
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: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] kfaraz commented on a diff in pull request #13197: Segment loading: Allow cancellation and prioritization of load queue items

Posted by GitBox <gi...@apache.org>.
kfaraz commented on code in PR #13197:
URL: https://github.com/apache/druid/pull/13197#discussion_r993747007


##########
server/src/main/java/org/apache/druid/server/coordinator/TierLoadingState.java:
##########
@@ -0,0 +1,72 @@
+/*
+ * 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;
+
+import org.apache.druid.timeline.SegmentId;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * Contains segments being loaded or replicated in a tier.
+ */
+public class TierLoadingState
+{
+  private int lifetime;

Review Comment:
   Sorry, I must have missed it while testing. Do you mean that the behaviour seems to have changed in this PR?



-- 
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: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] kfaraz commented on a diff in pull request #13197: Segment loading: Allow cancellation and prioritization of load queue items

Posted by GitBox <gi...@apache.org>.
kfaraz commented on code in PR #13197:
URL: https://github.com/apache/druid/pull/13197#discussion_r993462870


##########
server/src/main/java/org/apache/druid/server/coordinator/CostBalancerStrategy.java:
##########
@@ -49,6 +50,14 @@ public class CostBalancerStrategy implements BalancerStrategy
   private static final double MILLIS_IN_HOUR = 3_600_000.0;
   private static final double MILLIS_FACTOR = MILLIS_IN_HOUR / LAMBDA;
 
+  /**
+   * Comparator that prioritizes servers by cost. Cheaper servers come before
+   * costlier servers. Servers with the same cost may appear in a random order.
+   */
+  private static final Comparator<Pair<Double, ServerHolder>> CHEAPEST_SERVERS_FIRST
+      = Comparator.<Pair<Double, ServerHolder>, Double>comparing(pair -> pair.lhs)
+      .thenComparing(pair -> ThreadLocalRandom.current().nextInt());

Review Comment:
   Sure, that would work too. 
   
   This PR does not make any modifications to strategies, so it has not been included here. The only modification done here is reduction in the number of calls to strategy.



-- 
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: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] kfaraz commented on a diff in pull request #13197: Segment loading: Allow cancellation and prioritization of load queue items

Posted by GitBox <gi...@apache.org>.
kfaraz commented on code in PR #13197:
URL: https://github.com/apache/druid/pull/13197#discussion_r993660988


##########
server/src/main/java/org/apache/druid/server/coordinator/TierLoadingState.java:
##########
@@ -0,0 +1,72 @@
+/*
+ * 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;
+
+import org.apache.druid.timeline.SegmentId;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * Contains segments being loaded or replicated in a tier.
+ */
+public class TierLoadingState
+{
+  private int lifetime;
+  private final ConcurrentHashMap<SegmentId, String> processingSegments = new ConcurrentHashMap<>();

Review Comment:
   > Does this PR add restrictions to simultaneous load / move of the same segment to multiple servers within a tier?
   
   Yes, that is pretty much the purpose of replication throttling.
   
   This class tries to retain the existing behaviour of `ReplicationThrottleLimit.currentlyReplicating`. 
   It will most likely go away when `replicationThrottleLimit` is relaxed.



-- 
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: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] AmatyaAvadhanula commented on a diff in pull request #13197: Segment loading: Allow cancellation and prioritization of load queue items

Posted by GitBox <gi...@apache.org>.
AmatyaAvadhanula commented on code in PR #13197:
URL: https://github.com/apache/druid/pull/13197#discussion_r993667010


##########
server/src/main/java/org/apache/druid/server/coordinator/TierLoadingState.java:
##########
@@ -0,0 +1,72 @@
+/*
+ * 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;
+
+import org.apache.druid.timeline.SegmentId;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * Contains segments being loaded or replicated in a tier.
+ */
+public class TierLoadingState
+{
+  private int lifetime;
+  private final ConcurrentHashMap<SegmentId, String> processingSegments = new ConcurrentHashMap<>();

Review Comment:
   A simple example is a LoadRule within a tier with a replication factor of 2. Both replicas can be loaded in the same cycle but the TierLoadingState would only keep track of the second load



-- 
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: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] kfaraz commented on a diff in pull request #13197: Segment loading: Allow cancellation and prioritization of load queue items

Posted by GitBox <gi...@apache.org>.
kfaraz commented on code in PR #13197:
URL: https://github.com/apache/druid/pull/13197#discussion_r998131365


##########
server/src/main/java/org/apache/druid/server/coordinator/SegmentStateManager.java:
##########
@@ -0,0 +1,261 @@
+/*
+ * 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;
+
+import com.google.inject.Inject;
+import org.apache.druid.client.ServerInventoryView;
+import org.apache.druid.java.util.emitter.EmittingLogger;
+import org.apache.druid.metadata.SegmentsMetadataManager;
+import org.apache.druid.timeline.DataSegment;
+
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * Manages state of segments being loaded.
+ */
+public class SegmentStateManager
+{
+  private static final EmittingLogger log = new EmittingLogger(SegmentStateManager.class);
+
+  private final LoadQueueTaskMaster taskMaster;
+  private final ServerInventoryView serverInventoryView;
+  private final SegmentsMetadataManager segmentsMetadataManager;
+
+  private final ConcurrentHashMap<String, TierLoadingState> currentlyMovingSegments =
+      new ConcurrentHashMap<>();
+  private final ConcurrentHashMap<String, TierLoadingState> currentlyReplicatingSegments
+      = new ConcurrentHashMap<>();
+
+  @Inject
+  public SegmentStateManager(
+      ServerInventoryView serverInventoryView,
+      SegmentsMetadataManager segmentsMetadataManager,
+      LoadQueueTaskMaster taskMaster
+  )
+  {
+    this.serverInventoryView = serverInventoryView;
+    this.segmentsMetadataManager = segmentsMetadataManager;
+    this.taskMaster = taskMaster;
+  }
+
+  /**
+   * Queues load of a replica of the segment on the given server.
+   */
+  public boolean loadSegment(
+      DataSegment segment,
+      ServerHolder server,
+      boolean isPrimary,
+      ReplicationThrottler throttler
+  )
+  {
+    final String tier = server.getServer().getTier();
+    final LoadPeonCallback callback;
+    if (isPrimary) {
+      // Primary replicas are not subject to throttling
+      callback = null;

Review Comment:
   Fixed.



-- 
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: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] AmatyaAvadhanula commented on a diff in pull request #13197: Segment loading: Allow cancellation and prioritization of load queue items

Posted by GitBox <gi...@apache.org>.
AmatyaAvadhanula commented on code in PR #13197:
URL: https://github.com/apache/druid/pull/13197#discussion_r993483143


##########
server/src/main/java/org/apache/druid/server/coordinator/TierLoadingState.java:
##########
@@ -0,0 +1,72 @@
+/*
+ * 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;
+
+import org.apache.druid.timeline.SegmentId;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * Contains segments being loaded or replicated in a tier.
+ */
+public class TierLoadingState
+{
+  private int lifetime;
+  private final ConcurrentHashMap<SegmentId, String> processingSegments = new ConcurrentHashMap<>();

Review Comment:
   Does this PR add restrictions to simultaneous load / move of the same segment to multiple servers within a tier?
   If not, this structure may not be able to capture all the required information



-- 
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: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] kfaraz commented on a diff in pull request #13197: Segment loading: Allow cancellation and prioritization of load queue items

Posted by GitBox <gi...@apache.org>.
kfaraz commented on code in PR #13197:
URL: https://github.com/apache/druid/pull/13197#discussion_r995348391


##########
server/src/main/java/org/apache/druid/server/coordinator/CuratorLoadQueuePeon.java:
##########
@@ -181,10 +183,10 @@ public int getNumberOfSegmentsInQueue()
   }
 
   @Override
-  public void loadSegment(final DataSegment segment, @Nullable final LoadPeonCallback callback)
+  public void loadSegment(final DataSegment segment, SegmentAction action, @Nullable final LoadPeonCallback callback)
   {
-    SegmentHolder segmentHolder = new SegmentHolder(segment, Action.LOAD, Collections.singletonList(callback));
-    final SegmentHolder existingHolder = segmentsToLoad.putIfAbsent(segment, segmentHolder);
+    QueuedSegment segmentHolder = new QueuedSegment(segment, action, callback);

Review Comment:
   > That said, I would've expected the queue itself to return a QueuedSegment rather than something creating one of those before it's actually queued...
   
   Do you mean like a `computeIfAbsent` rather than `putIfAbsent`?



-- 
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: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] kfaraz commented on a diff in pull request #13197: Segment loading: Allow cancellation and prioritization of load queue items

Posted by GitBox <gi...@apache.org>.
kfaraz commented on code in PR #13197:
URL: https://github.com/apache/druid/pull/13197#discussion_r991339473


##########
server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java:
##########
@@ -405,111 +405,6 @@ public String getCurrentLeader()
     return coordLeaderSelector.getCurrentLeader();
   }
 
-  public void moveSegment(

Review Comment:
   Change summary:
   Moved this method to `SegmentLoader`/`SegmentStateManager`.



##########
server/src/main/java/org/apache/druid/server/coordinator/LoadQueuePeon.java:
##########
@@ -21,37 +21,48 @@
 
 import org.apache.druid.timeline.DataSegment;
 
+import java.util.Map;
 import java.util.Set;
 
 /**
  * This interface exists only to support configurable load queue management via curator or http. Once HttpLoadQueuePeon
  * has been verified enough in production, CuratorLoadQueuePeon and this interface would be removed.
  */
 @Deprecated
-public abstract class LoadQueuePeon
+public interface LoadQueuePeon

Review Comment:
   Change summary:
   Added methods to `cancelLoad`, `cancelDrop`, and `loadSegment(segment, type)` (primary, replica or balance)



##########
server/src/main/java/org/apache/druid/server/coordinator/CostBalancerStrategy.java:
##########
@@ -184,20 +193,17 @@ public CostBalancerStrategy(ListeningExecutorService exec)
   }
 
   @Override
-  public ServerHolder findNewSegmentHomeReplicator(DataSegment proposalSegment, List<ServerHolder> serverHolders)
+  public Iterator<ServerHolder> findNewSegmentHomeReplicator(DataSegment proposalSegment, List<ServerHolder> serverHolders)

Review Comment:
   Change summary:
   Find all the best servers to place replicas in one go.



##########
server/src/main/java/org/apache/druid/server/coordinator/BalancerStrategy.java:
##########
@@ -46,13 +46,18 @@
   ServerHolder findNewSegmentHomeBalancer(DataSegment proposalSegment, List<ServerHolder> serverHolders);
 
   /**
-   * Find the best server on which to place a {@link DataSegment} replica according to the balancing strategy
+   * Finds the best servers on which to place a replica of the {@code proposalSegment}
+   * according to the balancing strategy.
+   *
    * @param proposalSegment segment to replicate
-   * @param serverHolders servers to consider as replica holders
-   * @return The server to replicate to, or null if no suitable server is found
+   * @param serverHolders   servers to consider as replica holders
+   * @return Iterator over the best servers (in order) on which the replica(s)
+   * can be placed.
    */
-  @Nullable
-  ServerHolder findNewSegmentHomeReplicator(DataSegment proposalSegment, List<ServerHolder> serverHolders);
+  Iterator<ServerHolder> findNewSegmentHomeReplicator(

Review Comment:
   Change summary:
   Find all the best servers to place replicas in one go rather than multiple strategy computations.



##########
server/src/main/java/org/apache/druid/server/coordinator/BalancerSegmentHolder.java:
##########
@@ -19,29 +19,28 @@
 
 package org.apache.druid.server.coordinator;
 
-import org.apache.druid.client.ImmutableDruidServer;
 import org.apache.druid.timeline.DataSegment;
 
 /**
  */
 public class BalancerSegmentHolder

Review Comment:
   Change summary:
   Added `ServerHolder` to avoid having to find the corresponding peon later in the flow.



##########
server/src/main/java/org/apache/druid/server/coordinator/duty/BalanceSegments.java:
##########
@@ -26,85 +26,65 @@
 import org.apache.druid.server.coordinator.BalancerSegmentHolder;
 import org.apache.druid.server.coordinator.BalancerStrategy;
 import org.apache.druid.server.coordinator.CoordinatorStats;
-import org.apache.druid.server.coordinator.DruidCoordinator;
 import org.apache.druid.server.coordinator.DruidCoordinatorRuntimeParams;
-import org.apache.druid.server.coordinator.LoadPeonCallback;
-import org.apache.druid.server.coordinator.LoadQueuePeon;
+import org.apache.druid.server.coordinator.SegmentLoader;
+import org.apache.druid.server.coordinator.SegmentStateManager;
 import org.apache.druid.server.coordinator.ServerHolder;
 import org.apache.druid.timeline.DataSegment;
-import org.apache.druid.timeline.SegmentId;
 
-import java.util.HashMap;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.NavigableSet;
 import java.util.SortedSet;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
 import java.util.stream.Collectors;
 
 /**
  */
 public class BalanceSegments implements CoordinatorDuty

Review Comment:
   Change summary:
   Move state of `currentlyMovingSegments` to `SegmentStateManager`.



##########
server/src/main/java/org/apache/druid/server/coordinator/duty/RunRules.java:
##########
@@ -46,38 +45,16 @@ public class RunRules implements CoordinatorDuty
   private static final EmittingLogger log = new EmittingLogger(RunRules.class);
   private static final int MAX_MISSING_RULES = 10;
 
-  private final ReplicationThrottler replicatorThrottler;
+  private final SegmentStateManager stateManager;
 
-  private final DruidCoordinator coordinator;
-
-  public RunRules(DruidCoordinator coordinator)

Review Comment:
   Change summary:
   Move replication throttling logic to `SegmentStateManager`.
   Move check on `maxNonPrimaryReplicantsToLoad` to `ReplicationThrottler` (requires revisit).



##########
server/src/main/java/org/apache/druid/server/coordinator/duty/EmitClusterStatsAndMetrics.java:
##########
@@ -322,18 +310,14 @@ private void emitStatsForHistoricalManagementDuties(DruidCluster cluster, Coordi
           );
         });
 
-    coordinator.computeNumsUnavailableUsedSegmentsPerDataSource().object2IntEntrySet().forEach(
-        (final Object2IntMap.Entry<String> entry) -> {
-          final String dataSource = entry.getKey();
-          final int numUnavailableUsedSegmentsInDataSource = entry.getIntValue();
-          emitter.emit(
-              new ServiceMetricEvent.Builder()
-                  .setDimension(DruidMetrics.DUTY_GROUP, groupName)
-                  .setDimension(DruidMetrics.DATASOURCE, dataSource).build(
-                  "segment/unavailable/count", numUnavailableUsedSegmentsInDataSource
-              )
-          );
-        }
+    coordinator.computeNumsUnavailableUsedSegmentsPerDataSource().forEach(

Review Comment:
   Style change: Use simpler iterator.



##########
server/src/test/java/org/apache/druid/server/coordinator/simulate/SegmentLoadingNegativeTest.java:
##########
@@ -91,130 +88,4 @@ public void testImmediateLoadingViolatesThrottleLimit()
     verifyDatasourceIsFullyLoaded(datasource);
   }
 
-  /**
-   * Correct behaviour: The first replica on any tier should not be throttled.
-   * <p>
-   * Fix Apache #12881 to fix this test.
-   */
-  @Test

Review Comment:
   Change summary: 
   The underlying behaviour for these tests is now fixed!
   The corresponding positive test cases can be found in `SegmentLoadingTest`.



##########
server/src/main/java/org/apache/druid/server/coordinator/duty/EmitClusterStatsAndMetrics.java:
##########
@@ -234,36 +218,40 @@ private void emitStatsForHistoricalManagementDuties(DruidCluster cluster, Coordi
         }
     );
 
+    // Log load queue status of all replication or broadcast targets
     log.info("Load Queues:");
-    for (Iterable<ServerHolder> serverHolders : cluster.getSortedHistoricalsByTier()) {

Review Comment:
   Change summary: Report queue status not just of historicals but all replication or broadcast targets.



##########
server/src/main/java/org/apache/druid/server/coordinator/CuratorLoadQueuePeon.java:
##########
@@ -63,7 +56,7 @@
  * of the same or different methods.
  */
 @Deprecated
-public class CuratorLoadQueuePeon extends LoadQueuePeon
+public class CuratorLoadQueuePeon implements LoadQueuePeon

Review Comment:
   Change summary:
   Commoned out class `SegmentHolder` as `QueuedSegment` to represent an item in a load queue and to be used by both `HttpLoadQueuePeon` and `CuratorLoadQueuePeon`.
   Implemented new methods in `LoadQueuePeon`.



##########
server/src/main/java/org/apache/druid/server/coordinator/ReplicationThrottler.java:
##########
@@ -36,74 +39,89 @@
 {
   private static final EmittingLogger log = new EmittingLogger(ReplicationThrottler.class);
 
-  private final Map<String, Boolean> replicatingLookup = new HashMap<>();

Review Comment:
   Change summary:
   Commoned out `ReplicationSegmentHolder` as `TierLoadingState` to be used for tracking both `currentlyReplicatingSegments` and `currentlyMovingSegments` (in `SegmentStateManager`).



##########
server/src/main/java/org/apache/druid/server/coordinator/HttpLoadQueuePeon.java:
##########
@@ -65,7 +66,7 @@
 
 /**
  */
-public class HttpLoadQueuePeon extends LoadQueuePeon
+public class HttpLoadQueuePeon implements LoadQueuePeon

Review Comment:
   Change summary:
   Used `QueuedSegment` instead of `SegmentHolder` and implemented new `LoadQueuePeon` methods for cancellation and loading with type.



##########
server/src/main/java/org/apache/druid/server/coordinator/ServerHolder.java:
##########
@@ -20,31 +20,61 @@
 package org.apache.druid.server.coordinator;
 
 import org.apache.druid.client.ImmutableDruidServer;
-import org.apache.druid.java.util.common.logger.Logger;
 import org.apache.druid.timeline.DataSegment;
 import org.apache.druid.timeline.SegmentId;
 
 import java.util.Objects;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
 
 /**
+ *
  */
 public class ServerHolder implements Comparable<ServerHolder>

Review Comment:
   Change summary:
   Maintain a map containing
   - segments that were in load or drop queue when this run started
   - segments that were queued for load or drop during this run
   This helps the `SegmentLoader` determine the current state of a segment on a server and make load/drop decisions.



-- 
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: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] kfaraz commented on a diff in pull request #13197: Segment loading: Allow cancellation and prioritization of load queue items

Posted by GitBox <gi...@apache.org>.
kfaraz commented on code in PR #13197:
URL: https://github.com/apache/druid/pull/13197#discussion_r1010207688


##########
server/src/test/java/org/apache/druid/server/coordinator/RunRulesTest.java:
##########
@@ -119,35 +118,39 @@ public void testOneTierTwoReplicantsWithStrictReplicantLimit()
         )).atLeastOnce();
     EasyMock.replay(databaseRuleManager);
 
-    DruidCluster druidCluster = DruidClusterBuilder
+    // server1 has all the segments already loaded
+    final DruidServer server1 =
+        new DruidServer("server1", "hostNorm", null, 1000, ServerType.HISTORICAL, "normal", 0);
+    usedSegments.forEach(server1::addDataSegment);
+
+    final DruidServer server2 =
+        new DruidServer("server2", "hostNorm", null, 1000, ServerType.HISTORICAL, "normal", 0);
+
+    final DruidCluster druidCluster = DruidClusterBuilder
         .newBuilder()
         .addTier(
             "normal",
-            new ServerHolder(
-                new DruidServer("serverNorm", "hostNorm", null, 1000, ServerType.HISTORICAL, "normal", 0)
-                    .toImmutableDruidServer(),
-                mockPeon
-            ),
-            new ServerHolder(
-                new DruidServer("serverNorm2", "hostNorm2", null, 1000, ServerType.HISTORICAL, "normal", 0)
-                    .toImmutableDruidServer(),
-                mockPeon
-            )
-        )
-        .build();
+            new ServerHolder(server1.toImmutableDruidServer(), mockPeon),
+            new ServerHolder(server2.toImmutableDruidServer(), mockPeon)
+        ).build();
 
     ListeningExecutorService exec = MoreExecutors.listeningDecorator(Executors.newFixedThreadPool(1));
     BalancerStrategy balancerStrategy = new CostBalancerStrategyFactory().createBalancerStrategy(exec);
 
     DruidCoordinatorRuntimeParams params = makeCoordinatorRuntimeParams(druidCluster, balancerStrategy)
-        .withDynamicConfigs(CoordinatorDynamicConfig.builder().withMaxSegmentsToMove(5).withMaxNonPrimaryReplicantsToLoad(10).build())
+        .withDynamicConfigs(
+            CoordinatorDynamicConfig
+                .builder()
+                .withMaxNonPrimaryReplicantsToLoad(10)
+                .build()
+        )
         .build();
 
     DruidCoordinatorRuntimeParams afterParams = ruleRunner.run(params);
     CoordinatorStats stats = afterParams.getCoordinatorStats();
 
-    Assert.assertEquals(34L, stats.getTieredStat("assignedCount", "normal"));
-    Assert.assertEquals(10L, stats.getGlobalStat("totalNonPrimaryReplicantsLoaded"));
+    // There are 24 under-replicated segments, but only 10 replicas are assigned

Review Comment:
   Added.



-- 
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: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] kfaraz commented on a diff in pull request #13197: Segment loading: Allow cancellation and prioritization of load queue items

Posted by "kfaraz (via GitHub)" <gi...@apache.org>.
kfaraz commented on code in PR #13197:
URL: https://github.com/apache/druid/pull/13197#discussion_r991340240


##########
server/src/main/java/org/apache/druid/server/coordinator/LoadQueuePeon.java:
##########
@@ -21,37 +21,48 @@
 
 import org.apache.druid.timeline.DataSegment;
 
+import java.util.Map;
 import java.util.Set;
 
 /**
  * This interface exists only to support configurable load queue management via curator or http. Once HttpLoadQueuePeon
  * has been verified enough in production, CuratorLoadQueuePeon and this interface would be removed.
  */
 @Deprecated
-public abstract class LoadQueuePeon
+public interface LoadQueuePeon

Review Comment:
   Change summary:
   Added methods to `cancelLoad`, `cancelDrop`, and `loadSegment(segment, type)` (primary, replica or balance)



-- 
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: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] github-code-scanning[bot] commented on a diff in pull request #13197: Segment loading: Allow cancellation and prioritization of load queue items

Posted by "github-code-scanning[bot] (via GitHub)" <gi...@apache.org>.
github-code-scanning[bot] commented on code in PR #13197:
URL: https://github.com/apache/druid/pull/13197#discussion_r1167765355


##########
server/src/test/java/org/apache/druid/server/coordinator/rules/BroadcastDistributionRuleTest.java:
##########
@@ -402,18 +411,41 @@
             largeSegments.get(2),
             largeSegments2.get(0),
             largeSegments2.get(1)
-        ),
-        smallSegment
+        )
     );
 
-    Assert.assertEquals(5L, stats.getGlobalStat(LoadRule.ASSIGNED_COUNT));
-    Assert.assertFalse(stats.hasPerTierStats());
+    Assert.assertEquals(2L, stats.getSegmentStat(Stats.Segments.ASSIGNED_BROADCAST, TIER_1, DS_SMALL));
+    Assert.assertEquals(3L, stats.getSegmentStat(Stats.Segments.ASSIGNED_BROADCAST, TIER_2, DS_SMALL));
 
     Assert.assertTrue(
-        druidCluster
-            .getAllServers()
-            .stream()
-            .allMatch(holder -> holder.isLoadingSegment(smallSegment) || holder.isServingSegment(smallSegment))
+        druidCluster.getAllServers().stream().allMatch(
+            holder -> holder.isLoadingSegment(smallSegment) || holder.isServingSegment(smallSegment)
+        )
+    );
+  }
+
+  private CoordinatorRunStats runRuleAndGetStats(
+      Rule rule,
+      DataSegment segment,
+      DruidCoordinatorRuntimeParams params
+  )
+  {
+    final CoordinatorDynamicConfig dynamicConfig = params.getCoordinatorDynamicConfig();
+    ReplicationThrottler throttler = new ReplicationThrottler(
+        Sets.newHashSet(params.getDruidCluster().getTierNames()),
+        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/4830)



##########
server/src/test/java/org/apache/druid/server/coordinator/CuratorDruidCoordinatorTest.java:
##########
@@ -497,4 +439,27 @@
                       .size(0)
                       .build();
   }
+
+  private StrategicSegmentAssigner createSegmentAssigner(
+      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/4828)



##########
server/src/test/java/org/apache/druid/server/coordinator/rules/LoadRuleTest.java:
##########
@@ -117,397 +101,290 @@
   @Before
   public void setUp()
   {
-    EmittingLogger.registerEmitter(EMITTER);
-    EMITTER.start();
-    throttler = EasyMock.createMock(ReplicationThrottler.class);
-
     exec = MoreExecutors.listeningDecorator(Execs.multiThreaded(1, "LoadRuleTest-%d"));
     balancerStrategy = new CostBalancerStrategyFactory().createBalancerStrategy(exec);
     cachingCostBalancerStrategy = new CachingCostBalancerStrategy(ClusterCostCache.builder().build(), exec);
 
     mockBalancerStrategy = EasyMock.createMock(BalancerStrategy.class);
+    stateManager = new SegmentLoadQueueManager(null, null, null);
   }
 
   @After
-  public void tearDown() throws Exception
+  public void tearDown()
   {
     exec.shutdown();
-    EMITTER.close();
   }
 
   @Test
   public void testLoad()
   {
-    EasyMock.expect(throttler.canCreateReplicant(EasyMock.anyString())).andReturn(true).anyTimes();
-
     final LoadQueuePeon mockPeon = createEmptyPeon();
-    mockPeon.loadSegment(EasyMock.anyObject(), EasyMock.anyObject());
+    mockPeon.loadSegment(EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyObject());
     EasyMock.expectLastCall().atLeastOnce();
 
-    LoadRule rule = createLoadRule(ImmutableMap.of(
-        "hot", 1,
-        DruidServer.DEFAULT_TIER, 2
-    ));
-
-    final DataSegment segment = createDataSegment("foo");
-
-    throttler.registerReplicantCreation(DruidServer.DEFAULT_TIER, segment.getId(), "hostNorm");
-    EasyMock.expectLastCall().once();
-
     if (!useRoundRobinAssignment) {
       EasyMock.expect(mockBalancerStrategy.findNewSegmentHomeReplicator(EasyMock.anyObject(), EasyMock.anyObject()))
               .andDelegateTo(balancerStrategy)
-              .times(3);
+              .times(2);
     }
+    EasyMock.replay(mockPeon, mockBalancerStrategy);
 
-    EasyMock.replay(throttler, mockPeon, mockBalancerStrategy);
-
-    DruidCluster druidCluster = DruidClusterBuilder
-        .newBuilder()
-        .addTier(
-            "hot",
-            new ServerHolder(
-                new DruidServer("serverHot", "hostHot", null, 1000, ServerType.HISTORICAL, "hot", 1)
-                    .toImmutableDruidServer(),
-                mockPeon
-            )
-        )
-        .addTier(
-            DruidServer.DEFAULT_TIER,
-            new ServerHolder(
-                new DruidServer(
-                    "serverNorm",
-                    "hostNorm",
-                    null,
-                    1000,
-                    ServerType.HISTORICAL,
-                    DruidServer.DEFAULT_TIER,
-                    0
-                ).toImmutableDruidServer(),
-                mockPeon
-            )
-        )
+    DruidCluster druidCluster = DruidCluster
+        .builder()
+        .addTier(Tier.T1, createServerHolder(Tier.T1, mockPeon, false))
+        .addTier(Tier.T2, createServerHolder(Tier.T2, mockPeon, false))
         .build();
 
-    CoordinatorStats stats = rule.run(null, makeCoordinatorRuntimeParams(druidCluster, segment), segment);
+    final DataSegment segment = createDataSegment(DS_WIKI);
+    LoadRule rule = loadForever(ImmutableMap.of(Tier.T1, 1, Tier.T2, 2));
+    CoordinatorRunStats stats = runRuleAndGetStats(rule, segment, druidCluster);
+
+    Assert.assertEquals(1L, stats.getSegmentStat(Stats.Segments.ASSIGNED, Tier.T1, DS_WIKI));
+    Assert.assertEquals(1L, stats.getSegmentStat(Stats.Segments.ASSIGNED, Tier.T2, DS_WIKI));
 
-    Assert.assertEquals(1L, stats.getTieredStat(LoadRule.ASSIGNED_COUNT, "hot"));
-    Assert.assertEquals(1L, stats.getTieredStat(LoadRule.ASSIGNED_COUNT, DruidServer.DEFAULT_TIER));
+    EasyMock.verify(mockPeon, mockBalancerStrategy);
+  }
 
-    EasyMock.verify(throttler, mockPeon, mockBalancerStrategy);
+  private CoordinatorRunStats runRuleAndGetStats(
+      LoadRule rule,
+      DataSegment segment,
+      DruidCluster cluster
+  )
+  {
+    return runRuleAndGetStats(rule, segment, makeCoordinatorRuntimeParams(cluster, segment));
+  }
+
+  private CoordinatorRunStats runRuleAndGetStats(
+      LoadRule rule,
+      DataSegment segment,
+      DruidCoordinatorRuntimeParams params
+  )
+  {
+    final CoordinatorDynamicConfig dynamicConfig = params.getCoordinatorDynamicConfig();
+    ReplicationThrottler throttler = new ReplicationThrottler(
+        Sets.newHashSet(params.getDruidCluster().getTierNames()),
+        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/4829)



-- 
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: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] kfaraz commented on a diff in pull request #13197: Segment loading: Allow cancellation and prioritization of load queue items

Posted by "kfaraz (via GitHub)" <gi...@apache.org>.
kfaraz commented on code in PR #13197:
URL: https://github.com/apache/druid/pull/13197#discussion_r1231758154


##########
server/src/main/java/org/apache/druid/server/coordinator/CoordinatorDynamicConfig.java:
##########
@@ -194,21 +211,46 @@ public CoordinatorDynamicConfig(
           + "reflect this configuration being added to Druid in a recent release. Druid is defaulting the value "
           + "to the Druid default of %d. It is recommended that you re-submit your dynamic config with your "
           + "desired value for maxNonPrimaryReplicantsToLoad",
-          Builder.DEFAULT_MAX_NON_PRIMARY_REPLICANTS_TO_LOAD
+          Defaults.MAX_NON_PRIMARY_REPLICANTS_TO_LOAD
       );
-      maxNonPrimaryReplicantsToLoad = Builder.DEFAULT_MAX_NON_PRIMARY_REPLICANTS_TO_LOAD;
+      maxNonPrimaryReplicantsToLoad = Defaults.MAX_NON_PRIMARY_REPLICANTS_TO_LOAD;
     }
     Preconditions.checkArgument(
         maxNonPrimaryReplicantsToLoad >= 0,
         "maxNonPrimaryReplicantsToLoad must be greater than or equal to 0."
     );
     this.maxNonPrimaryReplicantsToLoad = maxNonPrimaryReplicantsToLoad;
 
-    if (useRoundRobinSegmentAssignment == null) {
-      this.useRoundRobinSegmentAssignment = Builder.DEFAULT_USE_ROUND_ROBIN_ASSIGNMENT;
-    } else {
-      this.useRoundRobinSegmentAssignment = useRoundRobinSegmentAssignment;
+    this.useRoundRobinSegmentAssignment = Builder.valueOrDefault(
+        useRoundRobinSegmentAssignment,
+        Defaults.USE_ROUND_ROBIN_ASSIGNMENT
+    );
+    this.debugDimensions = debugDimensions;
+    this.validDebugDimensions = validateDebugDimensions(debugDimensions);
+  }
+
+  private Map<Dimension, String> validateDebugDimensions(Map<String, String> debugDimensions)
+  {
+    final Map<Dimension, String> validDebugDimensions = new EnumMap<>(Dimension.class);
+    if (debugDimensions == null || debugDimensions.isEmpty()) {
+      return validDebugDimensions;
+    }
+
+    final Map<String, Dimension> nameToDimension = new HashMap<>();
+    for (Dimension dimension : Dimension.values()) {
+      nameToDimension.put(dimension.reportedName(), dimension);
     }
+
+    debugDimensions.forEach(
+        (dimensionName, value) -> {
+          Dimension dimension = nameToDimension.get(dimensionName);
+          if (dimension != null && value != null) {
+            validDebugDimensions.put(dimension, value);
+          }
+        }
+    );

Review Comment:
   Don't know what I had done it in such a round about way in the first place 😅 , thanks for pointing this out!



-- 
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: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] kfaraz commented on a diff in pull request #13197: Segment loading: Allow cancellation and prioritization of load queue items

Posted by "kfaraz (via GitHub)" <gi...@apache.org>.
kfaraz commented on code in PR #13197:
URL: https://github.com/apache/druid/pull/13197#discussion_r1231752804


##########
server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java:
##########
@@ -888,26 +799,101 @@ public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params)
       List<ImmutableDruidServer> currentServers = prepareCurrentServers();
 
       startPeonsForNewServers(currentServers);
+      stopPeonsForDisappearedServers(currentServers);
 
-      cluster = prepareCluster(params, currentServers);
-      segmentReplicantLookup = SegmentReplicantLookup.make(cluster, getDynamicConfigs().getReplicateAfterLoadTimeout());
+      final DruidCluster cluster = prepareCluster(params.getCoordinatorDynamicConfig(), currentServers);
+      cancelLoadsOnDecommissioningServers(cluster);
 
-      stopPeonsForDisappearedServers(currentServers);
+      final CoordinatorDynamicConfig dynamicConfig = params.getCoordinatorDynamicConfig();
 
-      final RoundRobinServerSelector roundRobinServerSelector;
-      if (params.getCoordinatorDynamicConfig().isUseRoundRobinSegmentAssignment()) {
-        roundRobinServerSelector = new RoundRobinServerSelector(cluster);
-        log.info("Using round-robin segment assignment.");
-      } else {
-        roundRobinServerSelector = null;
+      initBalancerExecutor();
+      final BalancerStrategy balancerStrategy = balancerStrategyFactory.createBalancerStrategy(balancerExec);
+      log.info(
+          "Using balancer strategy [%s] with round-robin assignment [%s] and debug dimensions [%s].",
+          balancerStrategy.getClass().getSimpleName(),
+          dynamicConfig.isUseRoundRobinSegmentAssignment(), dynamicConfig.getDebugDimensions()
+      );
+
+      params = params.buildFromExisting()
+                     .withDruidCluster(cluster)
+                     .withDynamicConfigs(recomputeDynamicConfig(params))
+                     .withBalancerStrategy(balancerStrategy)
+                     .withSegmentAssignerUsing(loadQueueManager)
+                     .build();
+
+      segmentReplicantLookup = params.getSegmentReplicantLookup();
+
+      return params;
+    }
+
+    /**
+     * Recomputes dynamic config values if {@code smartLoadQueue} is enabled.
+     */
+    private CoordinatorDynamicConfig recomputeDynamicConfig(DruidCoordinatorRuntimeParams params)
+    {
+      final CoordinatorDynamicConfig dynamicConfig = params.getCoordinatorDynamicConfig();
+      if (!dynamicConfig.isSmartSegmentLoading()) {
+        return dynamicConfig;
       }
 
-      return params.buildFromExisting()
-                   .withDruidCluster(cluster)
-                   .withLoadManagementPeons(loadManagementPeons)
-                   .withSegmentReplicantLookup(segmentReplicantLookup)
-                   .withRoundRobinServerSelector(roundRobinServerSelector)
-                   .build();
+      // Impose a lower bound on both replicationThrottleLimit and maxSegmentsToMove
+      final int throttlePercentage = 2;
+      final int replicationThrottleLimit = Math.max(
+          100,
+          params.getUsedSegments().size() * throttlePercentage / 100
+      );
+
+      // Impose an upper bound on maxSegmentsToMove to ensure that coordinator
+      // run times are bounded. This limit can be relaxed as performance of
+      // the CostBalancerStrategy.computeCost() is improved.
+      final int maxSegmentsToMove = Math.min(1000, replicationThrottleLimit);
+
+      log.info(
+          "Smart segment loading is enabled. Recomputed replicationThrottleLimit"
+          + " [%d] (%d%% of used segments) and maxSegmentsToMove [%d].",
+          replicationThrottleLimit, throttlePercentage, maxSegmentsToMove
+      );
+
+      return CoordinatorDynamicConfig.builder()
+                                     .withMaxSegmentsInNodeLoadingQueue(0)
+                                     .withReplicationThrottleLimit(replicationThrottleLimit)
+                                     .withMaxSegmentsToMove(maxSegmentsToMove)
+                                     .withUseRoundRobinSegmentAssignment(true)
+                                     .withUseBatchedSegmentSampler(true)
+                                     .withEmitBalancingStats(false)
+                                     .build(dynamicConfig);
+    }
+
+    /**
+     * Cancels all load/move operations on decommissioning servers. This should
+     * be done before initializing the SegmentReplicantLookup so that
+     * under-replicated segments can be assigned in the current run itself.
+     */
+    private void cancelLoadsOnDecommissioningServers(DruidCluster cluster)
+    {
+      final AtomicInteger cancelledCount = new AtomicInteger(0);
+      final List<ServerHolder> decommissioningServers
+          = cluster.getAllServers().stream()
+                   .filter(ServerHolder::isDecommissioning)
+                   .collect(Collectors.toList());
+
+      for (ServerHolder server : decommissioningServers) {
+        server.getQueuedSegments().forEach(
+            (segment, action) -> {
+              // Cancel the operation if it is a type of load
+              if (action.isLoad() && server.cancelOperation(action, segment)) {
+                cancelledCount.incrementAndGet();
+              }
+            }
+        );
+      }
+
+      if (cancelledCount.get() > 0) {
+        log.info(
+            "Cancelled [%d] load/move operations on [%d] decommissioning servers.",
+            cancelledCount.get(), decommissioningServers.size()
+        );
+      }
     }

Review Comment:
   Cancelling all loads on decomissioning servers is faster than moving the segments away.
   
   Balancing moves are subject to the cost computation performance in `BalancerStrategy` and thus are typically limited (`maxSegmentsToMove <= 1000`). With unlimited load queues, there can potentially be many more segments in the load queue of decommissioning servers.
   
   With cancellation of loads on decommissioning servers, these segments would immediately be (round-robin) assigned to active servers in this run itself.



-- 
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: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] kfaraz commented on a diff in pull request #13197: Segment loading: Allow cancellation and prioritization of load queue items

Posted by "kfaraz (via GitHub)" <gi...@apache.org>.
kfaraz commented on code in PR #13197:
URL: https://github.com/apache/druid/pull/13197#discussion_r1231750050


##########
server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java:
##########
@@ -888,26 +799,101 @@ public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params)
       List<ImmutableDruidServer> currentServers = prepareCurrentServers();
 
       startPeonsForNewServers(currentServers);
+      stopPeonsForDisappearedServers(currentServers);
 
-      cluster = prepareCluster(params, currentServers);
-      segmentReplicantLookup = SegmentReplicantLookup.make(cluster, getDynamicConfigs().getReplicateAfterLoadTimeout());
+      final DruidCluster cluster = prepareCluster(params.getCoordinatorDynamicConfig(), currentServers);
+      cancelLoadsOnDecommissioningServers(cluster);
 
-      stopPeonsForDisappearedServers(currentServers);
+      final CoordinatorDynamicConfig dynamicConfig = params.getCoordinatorDynamicConfig();
 
-      final RoundRobinServerSelector roundRobinServerSelector;
-      if (params.getCoordinatorDynamicConfig().isUseRoundRobinSegmentAssignment()) {
-        roundRobinServerSelector = new RoundRobinServerSelector(cluster);
-        log.info("Using round-robin segment assignment.");
-      } else {
-        roundRobinServerSelector = null;
+      initBalancerExecutor();
+      final BalancerStrategy balancerStrategy = balancerStrategyFactory.createBalancerStrategy(balancerExec);
+      log.info(
+          "Using balancer strategy [%s] with round-robin assignment [%s] and debug dimensions [%s].",
+          balancerStrategy.getClass().getSimpleName(),
+          dynamicConfig.isUseRoundRobinSegmentAssignment(), dynamicConfig.getDebugDimensions()
+      );
+
+      params = params.buildFromExisting()
+                     .withDruidCluster(cluster)
+                     .withDynamicConfigs(recomputeDynamicConfig(params))
+                     .withBalancerStrategy(balancerStrategy)
+                     .withSegmentAssignerUsing(loadQueueManager)
+                     .build();
+
+      segmentReplicantLookup = params.getSegmentReplicantLookup();
+
+      return params;
+    }
+
+    /**
+     * Recomputes dynamic config values if {@code smartLoadQueue} is enabled.
+     */
+    private CoordinatorDynamicConfig recomputeDynamicConfig(DruidCoordinatorRuntimeParams params)
+    {
+      final CoordinatorDynamicConfig dynamicConfig = params.getCoordinatorDynamicConfig();
+      if (!dynamicConfig.isSmartSegmentLoading()) {
+        return dynamicConfig;
       }
 
-      return params.buildFromExisting()
-                   .withDruidCluster(cluster)
-                   .withLoadManagementPeons(loadManagementPeons)
-                   .withSegmentReplicantLookup(segmentReplicantLookup)
-                   .withRoundRobinServerSelector(roundRobinServerSelector)
-                   .build();
+      // Impose a lower bound on both replicationThrottleLimit and maxSegmentsToMove
+      final int throttlePercentage = 2;
+      final int replicationThrottleLimit = Math.max(
+          100,
+          params.getUsedSegments().size() * throttlePercentage / 100
+      );
+
+      // Impose an upper bound on maxSegmentsToMove to ensure that coordinator
+      // run times are bounded. This limit can be relaxed as performance of
+      // the CostBalancerStrategy.computeCost() is improved.
+      final int maxSegmentsToMove = Math.min(1000, replicationThrottleLimit);
+
+      log.info(
+          "Smart segment loading is enabled. Recomputed replicationThrottleLimit"
+          + " [%d] (%d%% of used segments) and maxSegmentsToMove [%d].",
+          replicationThrottleLimit, throttlePercentage, maxSegmentsToMove
+      );
+
+      return CoordinatorDynamicConfig.builder()
+                                     .withMaxSegmentsInNodeLoadingQueue(0)
+                                     .withReplicationThrottleLimit(replicationThrottleLimit)
+                                     .withMaxSegmentsToMove(maxSegmentsToMove)
+                                     .withUseRoundRobinSegmentAssignment(true)
+                                     .withUseBatchedSegmentSampler(true)
+                                     .withEmitBalancingStats(false)

Review Comment:
   Yeah, I was not too keen on this myself. I considered putting this logic inside `DruidCoordinatorRuntimeParams` itself but I guess the best thing to do is just have a separate config object, as you suggest.



-- 
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: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] imply-cheddar commented on a diff in pull request #13197: Segment loading: Allow cancellation and prioritization of load queue items

Posted by "imply-cheddar (via GitHub)" <gi...@apache.org>.
imply-cheddar commented on code in PR #13197:
URL: https://github.com/apache/druid/pull/13197#discussion_r1231715441


##########
server/src/main/java/org/apache/druid/server/coordinator/balancer/BalancerStrategy.java:
##########
@@ -36,77 +39,65 @@
  */
 public interface BalancerStrategy
 {
+
   /**
-   * Find the best server to move a {@link DataSegment} to according the balancing strategy.
-   * @param proposalSegment segment to move
-   * @param serverHolders servers to consider as move destinations
+   * Finds the best server to move a segment to according to the balancing strategy.
+   *
+   * @param proposalSegment    segment to move
+   * @param sourceServer       Server the segment is currently placed on.
+   * @param destinationServers servers to consider as move destinations
    * @return The server to move to, or null if no move should be made or no server is suitable
    */
   @Nullable
-  ServerHolder findNewSegmentHomeBalancer(DataSegment proposalSegment, List<ServerHolder> serverHolders);
+  ServerHolder findDestinationServerToMoveSegment(
+      DataSegment proposalSegment,
+      ServerHolder sourceServer,
+      List<ServerHolder> destinationServers
+  );
 
   /**
-   * Finds the best servers on which to place a replica of the {@code proposalSegment}
-   * according to the balancing strategy.
+   * Finds the best servers on which to place the {@code proposalSegment}.
+   * This method can be used both for placing the first copy of a segment
+   * in the tier or a replica of the segment.
    *
-   * @param proposalSegment segment to replicate
-   * @param serverHolders   servers to consider as replica holders
-   * @return Iterator over the best servers (in order) on which the replica(s)
+   * @param proposalSegment segment to place on servers
+   * @param serverHolders   servers to consider as segment homes
+   * @return Iterator over the best servers (in order) on which the segment
    * can be placed.
    */
-  Iterator<ServerHolder> findNewSegmentHomeReplicator(
+  Iterator<ServerHolder> findServersToLoadSegment(
       DataSegment proposalSegment,
       List<ServerHolder> serverHolders
   );
 
   /**
-   * Pick the best segments to move from one of the supplied set of servers according to the balancing strategy.
+   * Picks segments from the given set of servers based on the balancing strategy.
+   * Default behaviour is to pick segments using reservoir sampling.
    *
-   * @param serverHolders set of historicals to consider for moving segments
-   * @param broadcastDatasources Datasources that contain segments which were loaded via broadcast rules.
-   *                             Balancing strategies should avoid rebalancing segments for such datasources, since
-   *                             they should be loaded on all servers anyway.
-   *                             NOTE: this should really be handled on a per-segment basis, to properly support
-   *                                   the interval or period-based broadcast rules. For simplicity of the initial
-   *                                   implementation, only forever broadcast rules are supported.
-   * @param reservoirSize the reservoir size maintained by the Reservoir Sampling algorithm.
-   * @return Iterator for set of {@link BalancerSegmentHolder} containing segment to move and server they currently
-   * reside on, or empty if there are no segments to pick from (i. e. all provided serverHolders are empty).
+   * @param serverHolders        Set of historicals to consider for picking segments
+   * @param broadcastDatasources Segments belonging to these datasources will not
+   *                             be picked for balancing, since they should be
+   *                             loaded on all servers anyway.
+   * @param maxSegmentsToPick    Maximum number of segments to pick
+   * @param pickLoadingSegments  If true, picks only segments currently being
+   *                             loaded on a server. If false, picks segments
+   *                             already loaded on a server.
+   * @return Iterator over {@link BalancerSegmentHolder}s, each of which contains
+   * a segment picked for moving and the server currently serving/loading it.
    */
   default Iterator<BalancerSegmentHolder> pickSegmentsToMove(
       List<ServerHolder> serverHolders,
       Set<String> broadcastDatasources,
-      int reservoirSize
+      int maxSegmentsToPick,
+      boolean pickLoadingSegments

Review Comment:
   This API definition with a boolean that says to pick loading segments or not is not really a good structure.  I initially thought it meant "also include loading segments", but it turns out that it means "either pick only loading or pick only non-loading".  If we want those semantics, we should have 2 different methods, one for picking loading segments and one for picking non loading.
   
   From what I can tell, at least with what is committed to the Druid code base, both of the "pickSegmentsToMove" methods are never overridden.  I don't know that this method actually makes sense as an extension point (it would likely be better to just replace the CoordinatorDuty entirely), so I think that the best thing to do in this PR is to actually completely eliminate this method *and* the other `pickSegmentsToMove` method and just inline the code.
   
   That will then make it the whole decision between loading segments versus not much easier to ascertain (you can basically grab the list that you care about and only use that, no need to push down the boolean).



##########
server/src/main/java/org/apache/druid/server/coordinator/CoordinatorDynamicConfig.java:
##########
@@ -194,21 +211,46 @@ public CoordinatorDynamicConfig(
           + "reflect this configuration being added to Druid in a recent release. Druid is defaulting the value "
           + "to the Druid default of %d. It is recommended that you re-submit your dynamic config with your "
           + "desired value for maxNonPrimaryReplicantsToLoad",
-          Builder.DEFAULT_MAX_NON_PRIMARY_REPLICANTS_TO_LOAD
+          Defaults.MAX_NON_PRIMARY_REPLICANTS_TO_LOAD
       );
-      maxNonPrimaryReplicantsToLoad = Builder.DEFAULT_MAX_NON_PRIMARY_REPLICANTS_TO_LOAD;
+      maxNonPrimaryReplicantsToLoad = Defaults.MAX_NON_PRIMARY_REPLICANTS_TO_LOAD;
     }
     Preconditions.checkArgument(
         maxNonPrimaryReplicantsToLoad >= 0,
         "maxNonPrimaryReplicantsToLoad must be greater than or equal to 0."
     );
     this.maxNonPrimaryReplicantsToLoad = maxNonPrimaryReplicantsToLoad;
 
-    if (useRoundRobinSegmentAssignment == null) {
-      this.useRoundRobinSegmentAssignment = Builder.DEFAULT_USE_ROUND_ROBIN_ASSIGNMENT;
-    } else {
-      this.useRoundRobinSegmentAssignment = useRoundRobinSegmentAssignment;
+    this.useRoundRobinSegmentAssignment = Builder.valueOrDefault(
+        useRoundRobinSegmentAssignment,
+        Defaults.USE_ROUND_ROBIN_ASSIGNMENT
+    );
+    this.debugDimensions = debugDimensions;
+    this.validDebugDimensions = validateDebugDimensions(debugDimensions);
+  }
+
+  private Map<Dimension, String> validateDebugDimensions(Map<String, String> debugDimensions)
+  {
+    final Map<Dimension, String> validDebugDimensions = new EnumMap<>(Dimension.class);
+    if (debugDimensions == null || debugDimensions.isEmpty()) {
+      return validDebugDimensions;
+    }
+
+    final Map<String, Dimension> nameToDimension = new HashMap<>();
+    for (Dimension dimension : Dimension.values()) {
+      nameToDimension.put(dimension.reportedName(), dimension);
     }
+
+    debugDimensions.forEach(
+        (dimensionName, value) -> {
+          Dimension dimension = nameToDimension.get(dimensionName);
+          if (dimension != null && value != null) {
+            validDebugDimensions.put(dimension, value);
+          }
+        }
+    );

Review Comment:
   This `forEach` seems egregious to me.  Can't you just use the name from the for loop above to lookup whether there's an entry in `debugDimensions` and then add it if it exists?



##########
server/src/main/java/org/apache/druid/server/coordinator/ServerHolder.java:
##########
@@ -155,128 +211,161 @@ public long getAvailableSize()
    * <p>
    * A load is possible only if the server meets all of the following criteria:
    * <ul>
-   *   <li>is not already serving or loading the segment</li>
    *   <li>is not being decommissioned</li>
+   *   <li>is not already serving the segment</li>
+   *   <li>is not performing any other action on the segment</li>
    *   <li>has not already exceeded the load queue limit in this run</li>
    *   <li>has available disk space</li>
    * </ul>
    */
   public boolean canLoadSegment(DataSegment segment)
   {
-    final SegmentState state = getSegmentState(segment);
     return !isDecommissioning
-           && (maxSegmentsInLoadQueue == 0 || maxSegmentsInLoadQueue > segmentsQueuedForLoad)
-           && getAvailableSize() >= segment.getSize()
-           && state == SegmentState.NONE;
+           && !hasSegmentLoaded(segment.getId())
+           && getActionOnSegment(segment) == null
+           && totalAssignmentsInRun < maxAssignmentsInRun
+           && getAvailableSize() >= segment.getSize();
   }
 
-  public SegmentState getSegmentState(DataSegment segment)
+  public SegmentAction getActionOnSegment(DataSegment segment)
   {
-    SegmentState state = queuedSegments.get(segment.getId());
-    if (state != null) {
-      return state;
-    }
-
-    return isServingSegment(segment) ? SegmentState.LOADED : SegmentState.NONE;
+    return queuedSegments.get(segment);
   }
 
   /**
    * Segments queued for load, drop or move on this server.
+   * <ul>
+   * <li>Contains segments present in the queue when the current coordinator run started.</li>
+   * <li>Contains segments added to the queue during the current run.</li>
+   * <li>Maps replicating segments to LOAD rather than REPLICATE for simplicity.</li>
+   * <li>Does not contain segments whose actions were cancelled.</li>
+   * </ul>
    */
-  public Map<SegmentId, SegmentState> getQueuedSegments()
+  public Map<DataSegment, SegmentAction> getQueuedSegments()
   {
-    return Collections.unmodifiableMap(queuedSegments);
+    return new HashMap<>(queuedSegments);
   }
 
+  /**
+   * Segments that are expected to be loaded on this server once all the
+   * operations in progress have completed.
+   */
+  public Set<DataSegment> getProjectedSegments()
+  {
+    return projectedSegments;
+  }
+
+  /**
+   * Segments that are currently in the queue for being loaded on this server.
+   * This does not include segments that are being moved to this server.
+   */
+  public List<DataSegment> getLoadingSegments()
+  {
+    final List<DataSegment> loadingSegments = new ArrayList<>();
+    queuedSegments.forEach((segment, action) -> {
+      if (action == SegmentAction.LOAD) {
+        loadingSegments.add(segment);
+      }
+    });
+
+    return loadingSegments;
+  }
+
+  /**
+   * Returns true if this server has the segment loaded and is not dropping it.
+   */
   public boolean isServingSegment(DataSegment segment)
   {
-    return isServingSegment(segment.getId());
+    return hasSegmentLoaded(segment.getId()) && getActionOnSegment(segment) == null;
   }
 
   public boolean isLoadingSegment(DataSegment segment)
   {
-    return getSegmentState(segment) == SegmentState.LOADING;
+    return getActionOnSegment(segment) == SegmentAction.LOAD;
   }
 
   public boolean isDroppingSegment(DataSegment segment)
   {
-    return getSegmentState(segment) == SegmentState.DROPPING;
+    return getActionOnSegment(segment) == SegmentAction.DROP;
   }
 
-  public boolean startOperation(DataSegment segment, SegmentState newState)
+  public int getNumMovingSegments()
   {
-    if (queuedSegments.containsKey(segment.getId())) {
-      return false;
-    }
+    return movingSegmentCount;
+  }
 
-    if (newState == SegmentState.LOADING || newState == SegmentState.MOVING_TO) {
-      ++segmentsQueuedForLoad;
-      sizeOfLoadingSegments += segment.getSize();
-    }
-    queuedSegments.put(segment.getId(), newState);
-    return true;
+  public int getNumLoadingReplicas()
+  {
+    return loadingReplicaCount;
   }
 
-  public boolean cancelOperation(DataSegment segment, SegmentState currentState)
+  public boolean startOperation(SegmentAction action, DataSegment segment)
   {
-    SegmentState observedState = queuedSegments.get(segment.getId());
-    if (observedState != currentState) {
+    if (queuedSegments.containsKey(segment)) {
       return false;
     }
 
-    if (currentState == SegmentState.LOADING || currentState == SegmentState.MOVING_TO) {
-      --segmentsQueuedForLoad;
-      sizeOfLoadingSegments -= segment.getSize();
+    if (action.isLoad()) {
+      ++totalAssignmentsInRun;
     }
-    queuedSegments.remove(segment.getId());
+
+    updateQueuedSegments(segment, simplify(action), true);
     return true;
   }
 
-  public boolean isServingSegment(SegmentId segmentId)
+  public boolean cancelOperation(SegmentAction action, DataSegment segment)
+  {
+    final SegmentAction queuedAction = queuedSegments.get(segment);
+    return queuedAction == simplify(action)
+           && (queuedAction == SegmentAction.MOVE_FROM || peon.cancelOperation(segment))
+           && updateQueuedSegments(segment, queuedAction, false);
+  }
+
+  public boolean hasSegmentLoaded(SegmentId segmentId)
   {
     return server.getSegment(segmentId) != null;
   }
 
-  /**
-   * Checks if the server can load the given segment.
-   * <p>
-   * A load is possible only if the server meets all of the following criteria:
-   * <ul>
-   *   <li>is not being decommissioned</li>
-   *   <li>is not already serving the segment</li>
-   *   <li>is not performing any other action on the segment</li>
-   *   <li>has not already exceeded the load queue limit in this run</li>
-   *   <li>has available disk space</li>
-   * </ul>
-   */
-  public boolean canLoadSegment(DataSegment segment)
+  public boolean isRealtimeServer()
   {
-    return !isDecommissioning
-           && !isServingSegment(segment.getId())
-           && !isLoadingSegment(segment)
-           && (maxSegmentsInLoadQueue == 0 || maxSegmentsInLoadQueue > peon.getNumberOfSegmentsInQueue())
-           && getAvailableSize() >= segment.getSize();
+    return server.getType() == ServerType.REALTIME
+           || server.getType() == ServerType.INDEXER_EXECUTOR;
   }
 
-  @Override
-  public int compareTo(ServerHolder serverHolder)
+  private SegmentAction simplify(SegmentAction action)
   {
-    int result = Long.compare(getAvailableSize(), serverHolder.getAvailableSize());
-    if (result != 0) {
-      return result;
+    return action == SegmentAction.REPLICATE ? SegmentAction.LOAD : action;
+  }
+
+  private boolean updateQueuedSegments(DataSegment segment, SegmentAction action, boolean addToQueue)
+  {
+    if (addToQueue) {
+      queuedSegments.put(segment, action);
+    } else {
+      queuedSegments.remove(segment);
     }
 
-    result = server.getHost().compareTo(serverHolder.server.getHost());
-    if (result != 0) {
-      return result;
+    final long sizeDelta = addToQueue ? segment.getSize() : -segment.getSize();
+    if (action.isLoad()) {
+      sizeOfLoadingSegments += sizeDelta;
+    } else if (action == SegmentAction.DROP) {
+      sizeOfDroppingSegments += sizeDelta;
     }
 
-    result = server.getTier().compareTo(serverHolder.server.getTier());
-    if (result != 0) {
-      return result;
+    // Remove from projected if load is cancelled or drop is started, add otherwise
+    if (addToQueue ^ action.isLoad()) {
+      projectedSegments.remove(segment);
+    } else {
+      projectedSegments.add(segment);
     }
 
-    return server.getType().compareTo(serverHolder.server.getType());
+    return true;
+  }

Review Comment:
   nit: the boolean parameter here feels really weird.  Why not just have 2 methods:
   
   ```
   addQueuedSegment()
   removeQueuedSegment()
   ```



##########
server/src/main/java/org/apache/druid/server/coordinator/loadqueue/HttpLoadQueuePeon.java:
##########
@@ -470,134 +517,127 @@
     return Collections.unmodifiableSet(segmentsMarkedToDrop);
   }
 
-  private abstract class SegmentHolder
+  /**
+   * A request is considered to have timed out if the time elapsed since it was
+   * first sent to the server is greater than the configured load timeout.
+   *
+   * @see DruidCoordinatorConfig#getLoadTimeoutDelay()
+   */
+  private boolean hasRequestTimedOut(SegmentHolder holder, long currentTimeMillis)
   {
-    private final DataSegment segment;
-    private final DataSegmentChangeRequest changeRequest;
-    private final List<LoadPeonCallback> callbacks = new ArrayList<>();
-
-    // Time when this request was sent to target server the first time.
-    private volatile long scheduleTime = -1;
-
-    private SegmentHolder(
-        DataSegment segment,
-        DataSegmentChangeRequest changeRequest,
-        LoadPeonCallback callback
-    )
-    {
-      this.segment = segment;
-      this.changeRequest = changeRequest;
+    return holder.isRequestSentToServer()
+           && currentTimeMillis - holder.getFirstRequestMillis()
+              > config.getLoadTimeoutDelay().getMillis();
+  }
 
-      if (callback != null) {
-        this.callbacks.add(callback);
-      }
-    }
+  private void onRequestFailed(SegmentHolder holder, String failureCause)
+  {
+    log.error(
+        "Server[%s] failed segment[%s] request[%s] with cause [%s].",
+        serverId, holder.getSegment().getId(), holder.getAction(), failureCause
+    );
+    onRequestCompleted(holder, QueueStatus.FAILED);
+  }
 
-    public void addCallback(LoadPeonCallback newCallback)
-    {
-      synchronized (callbacks) {
-        if (newCallback != null) {
-          callbacks.add(newCallback);
-        }
-      }
-    }
+  private void onRequestCompleted(SegmentHolder holder, QueueStatus status)
+  {
+    final SegmentAction action = holder.getAction();
+    log.trace(
+        "Server[%s] completed request[%s] on segment[%s] with status[%s].",
+        serverId, action, holder.getSegment().getId(), status
+    );
 
-    public DataSegment getSegment()
-    {
-      return segment;
+    if (holder.isLoad()) {
+      queuedSize.addAndGet(-holder.getSegment().getSize());
     }
+    incrementStat(holder, status);
+    executeCallbacks(holder, status == QueueStatus.SUCCESS);
+  }
 
-    public DataSegmentChangeRequest getChangeRequest()
-    {
-      return changeRequest;
+  private void incrementStat(SegmentHolder holder, QueueStatus status)
+  {
+    stats.add(status.getStatForAction(holder.getAction()), 1);
+    if (status.datasourceStat != null) {
+      stats.addToDatasourceStat(status.datasourceStat, holder.getSegment().getDataSource(), 1);
     }
+  }
 
-    public boolean hasTimedOut()
-    {
-      if (scheduleTime < 0) {
-        scheduleTime = System.currentTimeMillis();
-        return false;
-      } else if (System.currentTimeMillis() - scheduleTime > config.getLoadTimeoutDelay().getMillis()) {
-        return true;
-      } else {
+  private void executeCallbacks(SegmentHolder holder, boolean success)
+  {
+    callBackExecutor.execute(() -> {
+      for (LoadPeonCallback callback : holder.getCallbacks()) {
+        callback.execute(success);
+      }
+    });
+  }
+
+  /**
+   * Tries to cancel a load/drop operation. An load/drop request can be cancelled
+   * only if it has not already been sent to the corresponding server.
+   */
+  @Override
+  public boolean cancelOperation(DataSegment segment)
+  {
+    synchronized (lock) {
+      if (activeRequestSegments.contains(segment)) {
         return false;
       }
-    }
 
-    public void requestSucceeded()
-    {
-      log.trace(
-          "Server[%s] Successfully processed segment[%s] request[%s].",
-          serverId,
-          segment.getId(),
-          changeRequest.getClass().getSimpleName()
-      );
+      // Find the action on this segment, if any
+      final SegmentHolder holder = segmentsToLoad.containsKey(segment)
+                                   ? segmentsToLoad.remove(segment)
+                                   : segmentsToDrop.remove(segment);
+      if (holder == null) {
+        return false;
+      }
 
-      callBackExecutor.execute(() -> {
-        for (LoadPeonCallback callback : callbacks) {
-          if (callback != null) {
-            callback.execute(true);
-          }
-        }
-      });
+      queuedSegments.remove(holder);
+      onRequestCompleted(holder, QueueStatus.CANCELLED);
+      return true;
     }
+  }
 
-    public void requestFailed(String failureCause)
-    {
-      log.error(
-          "Server[%s] Failed segment[%s] request[%s] with cause [%s].",
-          serverId,
-          segment.getId(),
-          changeRequest.getClass().getSimpleName(),
-          failureCause
-      );
-
-      failedAssignCount.getAndIncrement();
+  private enum QueueStatus
+  {
+    ASSIGNED(Stats.SegmentQueue.ASSIGNED_ACTIONS),
+    SUCCESS(Stats.SegmentQueue.COMPLETED_ACTIONS),
+    FAILED(Stats.SegmentQueue.FAILED_ACTIONS),
+    CANCELLED(Stats.SegmentQueue.CANCELLED_ACTIONS);
 
-      callBackExecutor.execute(() -> {
-        for (LoadPeonCallback callback : callbacks) {
-          if (callback != null) {
-            callback.execute(false);
-          }
-        }
-      });
-    }
+    final CoordinatorStat loadStat;
+    final CoordinatorStat moveStat;
+    final CoordinatorStat dropStat;
+    final CoordinatorStat datasourceStat;
 
-    @Override
-    public String toString()
+    QueueStatus()
     {
-      return changeRequest.toString();
+      this(null);
     }
-  }
 
-  private class LoadSegmentHolder extends SegmentHolder
-  {
-    public LoadSegmentHolder(DataSegment segment, LoadPeonCallback callback)
+    QueueStatus(CoordinatorStat datasourceStat)
     {
-      super(segment, new SegmentChangeRequestLoad(segment), callback);
-      queuedSize.addAndGet(segment.getSize());
-    }
+      // These stats are not emitted and are tracked for debugging purposes only
+      final String prefix = StringUtils.toLowerCase(name());
+      this.loadStat = new CoordinatorStat(prefix + "Load");
+      this.moveStat = new CoordinatorStat(prefix + "Move");
+      this.dropStat = new CoordinatorStat(prefix + "Drop");
 
-    @Override
-    public void requestSucceeded()
-    {
-      queuedSize.addAndGet(-getSegment().getSize());
-      super.requestSucceeded();
+      this.datasourceStat = datasourceStat;
     }
 
-    @Override
-    public void requestFailed(String failureCause)
+    CoordinatorStat getStatForAction(SegmentAction action)
     {
-      queuedSize.addAndGet(-getSegment().getSize());
-      super.requestFailed(failureCause);
-    }
-  }
+      switch (action) {

Review Comment:
   This codeQL seems legitimate?



##########
server/src/main/java/org/apache/druid/server/coordinator/SegmentReplicantLookup.java:
##########
@@ -125,49 +117,85 @@ public int getMovingReplicas(SegmentId segmentId, String tier)
    * Number of replicas of the segment which are safely loaded on the given tier
    * and are not being dropped.
    */
-  public int getLoadedReplicas(SegmentId segmentId, String tier)
+  public int getLoadedNotDroppingReplicas(SegmentId segmentId, String tier)
+  {
+    ReplicaCount count = replicaCounts.get(segmentId, tier);
+    return (count == null) ? 0 : count.loadedNotDropping();
+  }
+
+  public int getLoadingReplicas(SegmentId segmentId, String tier)
   {
     ReplicaCount count = replicaCounts.get(segmentId, tier);
-    return (count == null) ? 0 : count.safelyLoaded();
+    return count == null ? 0 : count.loading;
   }
 
   /**
-   * Number of replicas of the segment which are safely loaded on the cluster
-   * and are not being dropped.
+   * Number of replicas of the segment which are loaded on the cluster.
+   *
+   * @param includeDropping Whether segments which are being dropped should be
+   *                        included in the total count.
    */
-  public int getTotalLoadedReplicas(SegmentId segmentId)
+  public int getLoadedReplicas(SegmentId segmentId, boolean includeDropping)
   {
     final Map<String, ReplicaCount> allTiers = replicaCounts.row(segmentId);
     int totalLoaded = 0;
     for (ReplicaCount count : allTiers.values()) {
-      totalLoaded += count.safelyLoaded();
+      totalLoaded += includeDropping ? count.loaded : count.loadedNotDropping();
     }
     return totalLoaded;
   }

Review Comment:
   I wonder if it doesn't make sense to switch the signature of this to
   
   ```
   public ReplicaCount getReplicationStatus(SegmentId segmentId) {}
   ```
   
   And then in the method, create a `ReplicateCount` object that you aggregate all of the counts into.  This allows the caller to determine what they want to do with that information and the added overhead of adding the extra fields that aren't currently added right now is relatively minimal, I think.



##########
server/src/main/java/org/apache/druid/server/coordinator/SegmentReplicantLookup.java:
##########
@@ -125,49 +117,85 @@ public int getMovingReplicas(SegmentId segmentId, String tier)
    * Number of replicas of the segment which are safely loaded on the given tier
    * and are not being dropped.
    */
-  public int getLoadedReplicas(SegmentId segmentId, String tier)
+  public int getLoadedNotDroppingReplicas(SegmentId segmentId, String tier)
+  {
+    ReplicaCount count = replicaCounts.get(segmentId, tier);
+    return (count == null) ? 0 : count.loadedNotDropping();
+  }

Review Comment:
   There's a number of methods that are basically just getting the `ReplicaCount` and then reading a single field from it.  We can collapse them together and just return the `ReplicaCount` object.  Let the caller decide which thing they want to read from it.



##########
server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java:
##########
@@ -888,26 +799,101 @@ public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params)
       List<ImmutableDruidServer> currentServers = prepareCurrentServers();
 
       startPeonsForNewServers(currentServers);
+      stopPeonsForDisappearedServers(currentServers);
 
-      cluster = prepareCluster(params, currentServers);
-      segmentReplicantLookup = SegmentReplicantLookup.make(cluster, getDynamicConfigs().getReplicateAfterLoadTimeout());
+      final DruidCluster cluster = prepareCluster(params.getCoordinatorDynamicConfig(), currentServers);
+      cancelLoadsOnDecommissioningServers(cluster);
 
-      stopPeonsForDisappearedServers(currentServers);
+      final CoordinatorDynamicConfig dynamicConfig = params.getCoordinatorDynamicConfig();
 
-      final RoundRobinServerSelector roundRobinServerSelector;
-      if (params.getCoordinatorDynamicConfig().isUseRoundRobinSegmentAssignment()) {
-        roundRobinServerSelector = new RoundRobinServerSelector(cluster);
-        log.info("Using round-robin segment assignment.");
-      } else {
-        roundRobinServerSelector = null;
+      initBalancerExecutor();
+      final BalancerStrategy balancerStrategy = balancerStrategyFactory.createBalancerStrategy(balancerExec);
+      log.info(
+          "Using balancer strategy [%s] with round-robin assignment [%s] and debug dimensions [%s].",
+          balancerStrategy.getClass().getSimpleName(),
+          dynamicConfig.isUseRoundRobinSegmentAssignment(), dynamicConfig.getDebugDimensions()
+      );
+
+      params = params.buildFromExisting()
+                     .withDruidCluster(cluster)
+                     .withDynamicConfigs(recomputeDynamicConfig(params))
+                     .withBalancerStrategy(balancerStrategy)
+                     .withSegmentAssignerUsing(loadQueueManager)
+                     .build();
+
+      segmentReplicantLookup = params.getSegmentReplicantLookup();
+
+      return params;
+    }
+
+    /**
+     * Recomputes dynamic config values if {@code smartLoadQueue} is enabled.
+     */
+    private CoordinatorDynamicConfig recomputeDynamicConfig(DruidCoordinatorRuntimeParams params)
+    {
+      final CoordinatorDynamicConfig dynamicConfig = params.getCoordinatorDynamicConfig();
+      if (!dynamicConfig.isSmartSegmentLoading()) {
+        return dynamicConfig;
       }
 
-      return params.buildFromExisting()
-                   .withDruidCluster(cluster)
-                   .withLoadManagementPeons(loadManagementPeons)
-                   .withSegmentReplicantLookup(segmentReplicantLookup)
-                   .withRoundRobinServerSelector(roundRobinServerSelector)
-                   .build();
+      // Impose a lower bound on both replicationThrottleLimit and maxSegmentsToMove
+      final int throttlePercentage = 2;
+      final int replicationThrottleLimit = Math.max(
+          100,
+          params.getUsedSegments().size() * throttlePercentage / 100
+      );
+
+      // Impose an upper bound on maxSegmentsToMove to ensure that coordinator
+      // run times are bounded. This limit can be relaxed as performance of
+      // the CostBalancerStrategy.computeCost() is improved.
+      final int maxSegmentsToMove = Math.min(1000, replicationThrottleLimit);
+
+      log.info(
+          "Smart segment loading is enabled. Recomputed replicationThrottleLimit"
+          + " [%d] (%d%% of used segments) and maxSegmentsToMove [%d].",
+          replicationThrottleLimit, throttlePercentage, maxSegmentsToMove
+      );
+
+      return CoordinatorDynamicConfig.builder()
+                                     .withMaxSegmentsInNodeLoadingQueue(0)
+                                     .withReplicationThrottleLimit(replicationThrottleLimit)
+                                     .withMaxSegmentsToMove(maxSegmentsToMove)
+                                     .withUseRoundRobinSegmentAssignment(true)
+                                     .withUseBatchedSegmentSampler(true)
+                                     .withEmitBalancingStats(false)

Review Comment:
   Instead of recomputing the dynamic config, I would've preferred that there be a separate config which houses all of the things that are recomputed and that reference gets updated/changed.  The `DynamicConfig` object itself should really only be mutated by users changing config values.



##########
server/src/main/java/org/apache/druid/server/coordinator/CoordinatorDynamicConfig.java:
##########
@@ -428,66 +489,36 @@ public boolean equals(Object o)
 
     CoordinatorDynamicConfig that = (CoordinatorDynamicConfig) o;
 
-    if (leadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments !=
-        that.leadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments) {
-      return false;
-    }
-    if (mergeBytesLimit != that.mergeBytesLimit) {
-      return false;
-    }
-    if (mergeSegmentsLimit != that.mergeSegmentsLimit) {
-      return false;
-    }
-    if (maxSegmentsToMove != that.maxSegmentsToMove) {
-      return false;
-    }
-    if (percentOfSegmentsToConsiderPerMove != that.percentOfSegmentsToConsiderPerMove) {
-      return false;
-    }
-    if (useBatchedSegmentSampler != that.useBatchedSegmentSampler) {
-      return false;
-    }
-    if (replicantLifetime != that.replicantLifetime) {
-      return false;
-    }
-    if (replicationThrottleLimit != that.replicationThrottleLimit) {
-      return false;
-    }
-    if (balancerComputeThreads != that.balancerComputeThreads) {
-      return false;
-    }
-    if (emitBalancingStats != that.emitBalancingStats) {
-      return false;
-    }
-    if (maxSegmentsInNodeLoadingQueue != that.maxSegmentsInNodeLoadingQueue) {
-      return false;
-    }
-    if (!Objects.equals(specificDataSourcesToKillUnusedSegmentsIn, that.specificDataSourcesToKillUnusedSegmentsIn)) {
-      return false;
-    }
-    if (!Objects.equals(dataSourcesToNotKillStalePendingSegmentsIn, that.dataSourcesToNotKillStalePendingSegmentsIn)) {
-      return false;
-    }
-    if (!Objects.equals(decommissioningNodes, that.decommissioningNodes)) {
-      return false;
-    }
-    if (pauseCoordination != that.pauseCoordination) {
-      return false;
-    }
-    if (replicateAfterLoadTimeout != that.replicateAfterLoadTimeout) {
-      return false;
-    }
-    if (maxNonPrimaryReplicantsToLoad != that.maxNonPrimaryReplicantsToLoad) {
-      return false;
-    }
-    return decommissioningMaxPercentOfMaxSegmentsToMove == that.decommissioningMaxPercentOfMaxSegmentsToMove;
+    return markSegmentAsUnusedDelayMillis == that.markSegmentAsUnusedDelayMillis
+           && mergeBytesLimit == that.mergeBytesLimit
+           && mergeSegmentsLimit == that.mergeSegmentsLimit
+           && maxSegmentsToMove == that.maxSegmentsToMove
+           && percentOfSegmentsToConsiderPerMove == that.percentOfSegmentsToConsiderPerMove
+           && decommissioningMaxPercentOfMaxSegmentsToMove == that.decommissioningMaxPercentOfMaxSegmentsToMove
+           && useBatchedSegmentSampler == that.useBatchedSegmentSampler
+           && balancerComputeThreads == that.balancerComputeThreads
+           && emitBalancingStats == that.emitBalancingStats
+           && replicantLifetime == that.replicantLifetime
+           && replicationThrottleLimit == that.replicationThrottleLimit
+           && replicateAfterLoadTimeout == that.replicateAfterLoadTimeout
+           && maxSegmentsInNodeLoadingQueue == that.maxSegmentsInNodeLoadingQueue
+           && maxNonPrimaryReplicantsToLoad == that.maxNonPrimaryReplicantsToLoad
+           && useRoundRobinSegmentAssignment == that.useRoundRobinSegmentAssignment
+           && pauseCoordination == that.pauseCoordination
+           && Objects.equals(
+               specificDataSourcesToKillUnusedSegmentsIn,
+               that.specificDataSourcesToKillUnusedSegmentsIn)
+           && Objects.equals(
+               dataSourcesToNotKillStalePendingSegmentsIn,
+               that.dataSourcesToNotKillStalePendingSegmentsIn)
+           && Objects.equals(decommissioningNodes, that.decommissioningNodes);

Review Comment:
   This (and hashcode) appear to be ignoring the various debugDimensions things, is that intentional?



##########
server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java:
##########
@@ -888,26 +799,101 @@ public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params)
       List<ImmutableDruidServer> currentServers = prepareCurrentServers();
 
       startPeonsForNewServers(currentServers);
+      stopPeonsForDisappearedServers(currentServers);
 
-      cluster = prepareCluster(params, currentServers);
-      segmentReplicantLookup = SegmentReplicantLookup.make(cluster, getDynamicConfigs().getReplicateAfterLoadTimeout());
+      final DruidCluster cluster = prepareCluster(params.getCoordinatorDynamicConfig(), currentServers);
+      cancelLoadsOnDecommissioningServers(cluster);
 
-      stopPeonsForDisappearedServers(currentServers);
+      final CoordinatorDynamicConfig dynamicConfig = params.getCoordinatorDynamicConfig();
 
-      final RoundRobinServerSelector roundRobinServerSelector;
-      if (params.getCoordinatorDynamicConfig().isUseRoundRobinSegmentAssignment()) {
-        roundRobinServerSelector = new RoundRobinServerSelector(cluster);
-        log.info("Using round-robin segment assignment.");
-      } else {
-        roundRobinServerSelector = null;
+      initBalancerExecutor();
+      final BalancerStrategy balancerStrategy = balancerStrategyFactory.createBalancerStrategy(balancerExec);
+      log.info(
+          "Using balancer strategy [%s] with round-robin assignment [%s] and debug dimensions [%s].",
+          balancerStrategy.getClass().getSimpleName(),
+          dynamicConfig.isUseRoundRobinSegmentAssignment(), dynamicConfig.getDebugDimensions()
+      );
+
+      params = params.buildFromExisting()
+                     .withDruidCluster(cluster)
+                     .withDynamicConfigs(recomputeDynamicConfig(params))
+                     .withBalancerStrategy(balancerStrategy)
+                     .withSegmentAssignerUsing(loadQueueManager)
+                     .build();
+
+      segmentReplicantLookup = params.getSegmentReplicantLookup();
+
+      return params;
+    }
+
+    /**
+     * Recomputes dynamic config values if {@code smartLoadQueue} is enabled.
+     */
+    private CoordinatorDynamicConfig recomputeDynamicConfig(DruidCoordinatorRuntimeParams params)
+    {
+      final CoordinatorDynamicConfig dynamicConfig = params.getCoordinatorDynamicConfig();
+      if (!dynamicConfig.isSmartSegmentLoading()) {
+        return dynamicConfig;
       }
 
-      return params.buildFromExisting()
-                   .withDruidCluster(cluster)
-                   .withLoadManagementPeons(loadManagementPeons)
-                   .withSegmentReplicantLookup(segmentReplicantLookup)
-                   .withRoundRobinServerSelector(roundRobinServerSelector)
-                   .build();
+      // Impose a lower bound on both replicationThrottleLimit and maxSegmentsToMove
+      final int throttlePercentage = 2;
+      final int replicationThrottleLimit = Math.max(
+          100,
+          params.getUsedSegments().size() * throttlePercentage / 100
+      );
+
+      // Impose an upper bound on maxSegmentsToMove to ensure that coordinator
+      // run times are bounded. This limit can be relaxed as performance of
+      // the CostBalancerStrategy.computeCost() is improved.
+      final int maxSegmentsToMove = Math.min(1000, replicationThrottleLimit);
+
+      log.info(
+          "Smart segment loading is enabled. Recomputed replicationThrottleLimit"
+          + " [%d] (%d%% of used segments) and maxSegmentsToMove [%d].",
+          replicationThrottleLimit, throttlePercentage, maxSegmentsToMove
+      );
+
+      return CoordinatorDynamicConfig.builder()
+                                     .withMaxSegmentsInNodeLoadingQueue(0)
+                                     .withReplicationThrottleLimit(replicationThrottleLimit)
+                                     .withMaxSegmentsToMove(maxSegmentsToMove)
+                                     .withUseRoundRobinSegmentAssignment(true)
+                                     .withUseBatchedSegmentSampler(true)
+                                     .withEmitBalancingStats(false)
+                                     .build(dynamicConfig);
+    }
+
+    /**
+     * Cancels all load/move operations on decommissioning servers. This should
+     * be done before initializing the SegmentReplicantLookup so that
+     * under-replicated segments can be assigned in the current run itself.
+     */
+    private void cancelLoadsOnDecommissioningServers(DruidCluster cluster)
+    {
+      final AtomicInteger cancelledCount = new AtomicInteger(0);
+      final List<ServerHolder> decommissioningServers
+          = cluster.getAllServers().stream()
+                   .filter(ServerHolder::isDecommissioning)
+                   .collect(Collectors.toList());
+
+      for (ServerHolder server : decommissioningServers) {
+        server.getQueuedSegments().forEach(
+            (segment, action) -> {
+              // Cancel the operation if it is a type of load
+              if (action.isLoad() && server.cancelOperation(action, segment)) {
+                cancelledCount.incrementAndGet();
+              }
+            }
+        );
+      }
+
+      if (cancelledCount.get() > 0) {
+        log.info(
+            "Cancelled [%d] load/move operations on [%d] decommissioning servers.",
+            cancelledCount.get(), decommissioningServers.size()
+        );
+      }
     }

Review Comment:
   I would've expected that the act of decommissioning would put the queue into a state where it will only accept DROP requests and then dropped the load queue immediately.  It seems weird to me that this is logic handled in this class instead.  The only thing I'd expect a CoordinatorDuty to do about it is to look for decomissioning servers and move the segments away.



##########
server/src/main/java/org/apache/druid/server/coordinator/SegmentReplicantLookup.java:
##########
@@ -125,49 +117,85 @@ public int getMovingReplicas(SegmentId segmentId, String tier)
    * Number of replicas of the segment which are safely loaded on the given tier
    * and are not being dropped.
    */
-  public int getLoadedReplicas(SegmentId segmentId, String tier)
+  public int getLoadedNotDroppingReplicas(SegmentId segmentId, String tier)
+  {
+    ReplicaCount count = replicaCounts.get(segmentId, tier);
+    return (count == null) ? 0 : count.loadedNotDropping();
+  }
+
+  public int getLoadingReplicas(SegmentId segmentId, String tier)
   {
     ReplicaCount count = replicaCounts.get(segmentId, tier);
-    return (count == null) ? 0 : count.safelyLoaded();
+    return count == null ? 0 : count.loading;
   }
 
   /**
-   * Number of replicas of the segment which are safely loaded on the cluster
-   * and are not being dropped.
+   * Number of replicas of the segment which are loaded on the cluster.
+   *
+   * @param includeDropping Whether segments which are being dropped should be
+   *                        included in the total count.
    */
-  public int getTotalLoadedReplicas(SegmentId segmentId)
+  public int getLoadedReplicas(SegmentId segmentId, boolean includeDropping)
   {
     final Map<String, ReplicaCount> allTiers = replicaCounts.row(segmentId);
     int totalLoaded = 0;
     for (ReplicaCount count : allTiers.values()) {
-      totalLoaded += count.safelyLoaded();
+      totalLoaded += includeDropping ? count.loaded : count.loadedNotDropping();
     }
     return totalLoaded;
   }
 
-  public Object2LongMap<String> getBroadcastUnderReplication(SegmentId segmentId)
+  /**
+   * Sets the number of replicas required for the specified segment in the tier.
+   * In a given coordinator run, this method must be called atleast once for
+   * every segment every tier.
+   */
+  public void setRequiredReplicas(SegmentId segmentId, boolean isBroadcast, String tier, int requiredReplicas)
   {
-    Object2LongOpenHashMap<String> perTier = new Object2LongOpenHashMap<>();
-    for (ServerHolder holder : cluster.getAllServers()) {
-      // Only record tier entry for server that is segment broadcast target
-      if (holder.getServer().getType().isSegmentBroadcastTarget()) {
-        // Every broadcast target server should be serving 1 replica of the segment
-        if (!holder.isServingSegment(segmentId)) {
-          perTier.addTo(holder.getServer().getTier(), 1L);
-        } else {
-          perTier.putIfAbsent(holder.getServer().getTier(), 0);
-        }
+    ReplicaCount counts = computeIfAbsent(replicaCounts, segmentId, tier);
+    counts.required = requiredReplicas;
+    if (isBroadcast) {
+      counts.possible = requiredReplicas;
+    } else {
+      counts.possible = tierToHistoricalCount.getOrDefault(tier, 0);
+    }
+  }
+
+  public Map<String, Object2LongMap<String>> getTierToDatasourceToUnderReplicated(
+      Iterable<DataSegment> usedSegments,
+      boolean ignoreMissingServers
+  )
+  {
+    final Map<String, Object2LongMap<String>> tierToUnderReplicated = new HashMap<>();
+
+    for (DataSegment segment : usedSegments) {
+      final Map<String, ReplicaCount> tierToReplicaCount = replicaCounts.row(segment.getId());
+      if (tierToReplicaCount == null) {
+        continue;
       }
+
+      tierToReplicaCount.forEach((tier, counts) -> {
+        final int underReplicated = counts.underReplicated(ignoreMissingServers);
+        if (underReplicated >= 0) {
+          Object2LongOpenHashMap<String> datasourceToUnderReplicated = (Object2LongOpenHashMap<String>)
+              tierToUnderReplicated.computeIfAbsent(tier, ds -> new Object2LongOpenHashMap<>());
+          datasourceToUnderReplicated.addTo(segment.getDataSource(), underReplicated);
+        }
+      });
     }
-    return perTier;
+
+    return tierToUnderReplicated;
   }

Review Comment:
   You might've noticed by now, but any time I see a `boolean` parameter in a function signature, I go and try to figure out why it exists and what to do to remove it.  Most of the time, it's a signal to some code that needs to be re-designed.
   
   This case is no different.  That said, I'm a little less confident about this suggestion, but I *think* that if this method were to compute and return a `Map<String, Map<String, ReplicaCount>>` that was 
   
   ```
   tier -> datasource -> total replication status of that datasource across that tier
   ```
   
   Then that would be enough to effectively eliminate the boolean property from this method.  I think it might also allow for you to compute the value once and reuse it for other purposes as well.
   
   Given that I'm a bit less confident about this one, treat it as just a suggestion.



##########
server/src/main/java/org/apache/druid/server/coordinator/StrategicSegmentAssigner.java:
##########
@@ -0,0 +1,577 @@
+/*
+ * 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;
+
+import com.google.common.collect.Sets;
+import it.unimi.dsi.fastutil.objects.Object2IntOpenHashMap;
+import org.apache.druid.java.util.emitter.EmittingLogger;
+import org.apache.druid.server.coordinator.balancer.BalancerStrategy;
+import org.apache.druid.server.coordinator.loadqueue.SegmentAction;
+import org.apache.druid.server.coordinator.loadqueue.SegmentLoadQueueManager;
+import org.apache.druid.server.coordinator.rules.SegmentActionHandler;
+import org.apache.druid.server.coordinator.stats.CoordinatorRunStats;
+import org.apache.druid.server.coordinator.stats.CoordinatorStat;
+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 java.util.Comparator;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+
+/**
+ * Used by the coordinator in each run for segment loading, dropping, balancing
+ * and broadcasting.
+ * <p>
+ * An instance of this class is freshly created for each coordinator run.
+ */
+public class StrategicSegmentAssigner implements SegmentActionHandler
+{
+  private static final EmittingLogger log = new EmittingLogger(StrategicSegmentAssigner.class);
+
+  private final SegmentLoadQueueManager loadQueueManager;
+  private final DruidCluster cluster;
+  private final CoordinatorRunStats stats;
+  private final SegmentReplicantLookup replicantLookup;
+  private final ReplicationThrottler replicationThrottler;
+  private final RoundRobinServerSelector serverSelector;
+  private final BalancerStrategy strategy;
+
+  private final boolean useRoundRobinAssignment;
+
+  private final Set<String> tiersWithNoServer = new HashSet<>();
+
+  public StrategicSegmentAssigner(
+      SegmentLoadQueueManager loadQueueManager,
+      DruidCluster cluster,
+      BalancerStrategy strategy,
+      CoordinatorDynamicConfig dynamicConfig,
+      CoordinatorRunStats stats
+  )
+  {
+    this.stats = stats;
+    this.cluster = cluster;
+    this.strategy = strategy;
+    this.loadQueueManager = loadQueueManager;
+    this.replicantLookup = SegmentReplicantLookup.make(cluster);
+    this.replicationThrottler = createReplicationThrottler(dynamicConfig);
+    this.useRoundRobinAssignment = dynamicConfig.isUseRoundRobinSegmentAssignment();
+    this.serverSelector = useRoundRobinAssignment ? new RoundRobinServerSelector(cluster) : null;
+  }
+
+  public CoordinatorRunStats getStats()
+  {
+    return stats;
+  }
+
+  public SegmentReplicantLookup getReplicantLookup()
+  {
+    return replicantLookup;
+  }
+
+  public void makeAlerts()
+  {
+    if (!tiersWithNoServer.isEmpty()) {
+      log.makeAlert("Tiers [%s] have no servers! Check your cluster configuration.", tiersWithNoServer).emit();
+    }
+  }
+
+  /**
+   * Moves the given segment from the source server to an eligible destination
+   * server.
+   * <p>
+   * An eligible destination server must:
+   * <ul>
+   *   <li>be present in the given list of destination servers</li>
+   *   <li>belong to the same tier as the source server</li>
+   *   <li>not already be serving or loading a replica of the segment</li>
+   *   <li>have enough space to load the segment</li>
+   * </ul>
+   * <p>
+   * The segment is not moved if:
+   * <ul>
+   *   <li>there is no eligible destination server, or</li>
+   *   <li>skipIfOptimallyPlaced is true and segment is already optimally placed, or</li>
+   *   <li>some other error occurs</li>
+   * </ul>
+   */
+  public boolean moveSegment(
+      DataSegment segment,
+      ServerHolder sourceServer,
+      List<ServerHolder> destinationServers
+  )
+  {
+    final String tier = sourceServer.getServer().getTier();
+    final List<ServerHolder> eligibleDestinationServers =
+        destinationServers.stream()
+                          .filter(s -> s.getServer().getTier().equals(tier))
+                          .filter(s -> s.canLoadSegment(segment))
+                          .collect(Collectors.toList());
+
+    if (eligibleDestinationServers.isEmpty()) {
+      incrementSkipStat(Stats.Segments.MOVE_SKIPPED, "No eligible server", segment, tier);
+      return false;
+    }
+
+    // If the source server is not decommissioning, move can be skipped if the
+    // segment is already optimally placed
+    if (!sourceServer.isDecommissioning()) {
+      eligibleDestinationServers.add(sourceServer);
+    }
+
+    final ServerHolder destination =
+        strategy.findDestinationServerToMoveSegment(segment, sourceServer, eligibleDestinationServers);
+
+    if (destination == null || destination.getServer().equals(sourceServer.getServer())) {
+      incrementSkipStat(Stats.Segments.MOVE_SKIPPED, "Optimally placed", segment, tier);
+      return false;
+    } else if (moveSegment(segment, sourceServer, destination)) {
+      incrementStat(Stats.Segments.MOVED, segment, tier, 1);
+      return true;
+    } else {
+      incrementSkipStat(Stats.Segments.MOVE_SKIPPED, "Encountered error", segment, tier);
+      return false;
+    }
+  }
+
+  /**
+   * Moves the given segment from serverA to serverB.
+   */
+  private boolean moveSegment(DataSegment segment, ServerHolder serverA, ServerHolder serverB)
+  {
+    final String tier = serverA.getServer().getTier();
+    if (serverA.isLoadingSegment(segment)) {
+      // Cancel the load on serverA and load on serverB instead
+      if (serverA.cancelOperation(SegmentAction.LOAD, segment)) {
+        int loadedCountOnTier = replicantLookup.getLoadedNotDroppingReplicas(segment.getId(), tier);
+        return loadSegment(segment, serverB, loadedCountOnTier >= 1);
+      }
+
+      // Could not cancel load, let the segment load on serverA and count it as unmoved
+      return false;
+    } else if (serverA.isServingSegment(segment)) {
+      return loadQueueManager.moveSegment(segment, serverA, serverB);
+    } else {
+      return false;
+    }
+  }
+
+  @Override
+  public void updateSegmentReplicasInTiers(DataSegment segment, Map<String, Integer> tierToReplicaCount)
+  {
+    // Identify empty tiers and determine total required replicas
+    final AtomicInteger requiredTotalReplicas = new AtomicInteger(0);
+    final Set<String> allTiers = Sets.newHashSet(cluster.getTierNames());
+    tierToReplicaCount.forEach((tier, requiredReplicas) -> {
+      reportTierCapacityStats(segment, requiredReplicas, tier);
+      replicantLookup.setRequiredReplicas(segment.getId(), false, tier, requiredReplicas);
+      if (allTiers.contains(tier)) {
+        requiredTotalReplicas.addAndGet(requiredReplicas);
+      } else {
+        tiersWithNoServer.add(tier);
+      }
+    });
+
+    final int loadedNotDroppingReplicas = replicantLookup.getLoadedReplicas(segment.getId(), false);
+    final int replicaSurplus = loadedNotDroppingReplicas - requiredTotalReplicas.get();
+
+    // Update replicas in every tier
+    int dropsQueued = 0;
+    for (String tier : allTiers) {
+      dropsQueued += updateReplicasInTier(
+          segment,
+          tier,
+          tierToReplicaCount.getOrDefault(tier, 0),
+          replicaSurplus - dropsQueued
+      );
+    }
+  }
+
+  /**
+   * Queues load or drop operations on this tier based on the required
+   * number of replicas and the current state.
+   * <p>
+   * The {@code maxReplicasToDrop} helps to maintain the required level of
+   * replication in the cluster. This ensures that segment read concurrency does
+   * not suffer during a tier shift or load rule change.
+   * <p>
+   * Returns the number of new drop operations queued on this tier.
+   */
+  private int updateReplicasInTier(
+      DataSegment segment,
+      String tier,
+      int requiredReplicas,
+      int maxReplicasToDrop
+  )
+  {
+    final int loadedNotDroppingReplicas =
+        replicantLookup.getLoadedNotDroppingReplicas(segment.getId(), tier);
+    final int loadingReplicas = replicantLookup.getLoadingReplicas(segment.getId(), tier);
+    final int projectedReplicas = loadedNotDroppingReplicas + loadingReplicas;
+
+    final int movingReplicas = replicantLookup.getMovingReplicas(segment.getId(), tier);
+    final boolean shouldCancelMoves = requiredReplicas == 0 && movingReplicas > 0;
+
+    // Check if there is any action required on this tier
+    if (projectedReplicas == requiredReplicas && !shouldCancelMoves) {
+      return 0;
+    }
+
+    final SegmentStatusInTier segmentStatus =
+        new SegmentStatusInTier(segment, cluster.getHistoricalsByTier(tier));
+
+    // Cancel all moves in this tier if it does not need to have replicas
+    if (shouldCancelMoves) {
+      cancelOperations(SegmentAction.MOVE_TO, movingReplicas, segment, segmentStatus);
+      cancelOperations(SegmentAction.MOVE_FROM, movingReplicas, segment, segmentStatus);
+    }
+
+    // Cancel drops and queue loads if the projected count is below the requirement
+    if (projectedReplicas < requiredReplicas) {
+      int replicaDeficit = requiredReplicas - projectedReplicas;
+      int cancelledDrops =
+          cancelOperations(SegmentAction.DROP, replicaDeficit, segment, segmentStatus);
+
+      // Cancelled drops can be counted as loaded replicas, thus reducing deficit
+      int numReplicasToLoad = replicaDeficit - cancelledDrops;
+      if (numReplicasToLoad > 0) {
+        boolean isAlreadyLoadedOnTier = loadedNotDroppingReplicas + cancelledDrops >= 1;
+        int numLoadsQueued = loadReplicas(numReplicasToLoad, segment, tier, segmentStatus, isAlreadyLoadedOnTier);
+        incrementStat(Stats.Segments.ASSIGNED, segment, tier, numLoadsQueued);
+      }
+    }
+
+    // Cancel loads and queue drops if the projected count exceeds the requirement
+    if (projectedReplicas > requiredReplicas) {
+      int replicaSurplus = projectedReplicas - requiredReplicas;
+      int cancelledLoads =
+          cancelOperations(SegmentAction.LOAD, replicaSurplus, segment, segmentStatus);
+
+      int numReplicasToDrop = Math.min(replicaSurplus - cancelledLoads, maxReplicasToDrop);
+      if (numReplicasToDrop > 0) {
+        int dropsQueuedOnTier = dropReplicas(numReplicasToDrop, segment, tier, segmentStatus);
+        incrementStat(Stats.Segments.DROPPED, segment, tier, dropsQueuedOnTier);
+        return dropsQueuedOnTier;
+      }
+    }
+
+    return 0;
+  }
+
+  private void reportTierCapacityStats(DataSegment segment, int requiredReplicas, String tier)
+  {
+    final RowKey rowKey = RowKey.forTier(tier);
+    stats.updateMax(Stats.Tier.REPLICATION_FACTOR, rowKey, requiredReplicas);
+    stats.add(Stats.Tier.REQUIRED_CAPACITY, rowKey, segment.getSize() * requiredReplicas);
+  }
+
+  @Override
+  public void broadcastSegment(DataSegment segment)
+  {
+    final Object2IntOpenHashMap<String> tierToRequiredReplicas = new Object2IntOpenHashMap<>();
+    for (ServerHolder server : cluster.getAllServers()) {
+      // Ignore servers which are not broadcast targets
+      if (!server.getServer().getType().isSegmentBroadcastTarget()) {
+        continue;
+      }
+
+      final String tier = server.getServer().getTier();
+
+      // Drop from decommissioning servers and load on active servers
+      int numDropsQueued = 0;
+      int numLoadsQueued = 0;
+      if (server.isDecommissioning()) {
+        numDropsQueued += dropBroadcastSegment(segment, server) ? 1 : 0;
+      } else {
+        tierToRequiredReplicas.addTo(tier, 1);
+        numLoadsQueued += loadBroadcastSegment(segment, server) ? 1 : 0;
+      }
+
+      if (numLoadsQueued > 0) {
+        incrementStat(Stats.Segments.ASSIGNED, segment, tier, numLoadsQueued);
+      }
+      if (numDropsQueued > 0) {
+        incrementStat(Stats.Segments.DROPPED, segment, tier, numDropsQueued);
+      }
+    }
+
+    // Update required replica counts
+    tierToRequiredReplicas.object2IntEntrySet().fastForEach(
+        entry -> replicantLookup
+            .setRequiredReplicas(segment.getId(), true, entry.getKey(), entry.getIntValue())
+    );
+  }
+
+  @Override
+  public void deleteSegment(DataSegment segment)
+  {
+    loadQueueManager.deleteSegment(segment);
+    stats.addToDatasourceStat(Stats.Segments.DELETED, segment.getDataSource(), 1);
+  }
+
+  /**
+   * Loads the broadcast segment if it is not loaded on the given server.
+   * Returns true only if the segment was successfully queued for load on the server.
+   */
+  private boolean loadBroadcastSegment(DataSegment segment, ServerHolder server)
+  {
+    if (server.isServingSegment(segment) || server.isLoadingSegment(segment)) {
+      return false;
+    } else if (server.isDroppingSegment(segment)) {
+      return server.cancelOperation(SegmentAction.DROP, segment);
+    }
+
+    if (server.canLoadSegment(segment) && loadSegment(segment, server, false)) {
+      return true;
+    } else {
+      log.makeAlert("Could not assign broadcast segment for datasource [%s]", segment.getDataSource())
+         .addData("segmentId", segment.getId())
+         .addData("segmentSize", segment.getSize())
+         .addData("hostName", server.getServer().getHost())
+         .addData("availableSize", server.getAvailableSize())
+         .emit();
+      return false;
+    }
+  }
+
+  /**
+   * Drops the broadcast segment if it is loaded on the given server.
+   * Returns true only if the segment was successfully queued for drop on the server.
+   */
+  private boolean dropBroadcastSegment(DataSegment segment, ServerHolder server)
+  {
+    if (server.isLoadingSegment(segment)) {
+      return server.cancelOperation(SegmentAction.LOAD, segment);
+    } else if (server.isServingSegment(segment)) {
+      return loadQueueManager.dropSegment(segment, server);
+    } else {
+      return false;
+    }
+  }
+
+  /**
+   * Queues drop of {@code numToDrop} replicas of the segment from a tier.
+   * Tries to drop replicas first from decommissioning servers and then from
+   * active servers.
+   * <p>
+   * Returns the number of successfully queued drop operations.
+   */
+  private int dropReplicas(
+      final int numToDrop,
+      DataSegment segment,
+      String tier,
+      SegmentStatusInTier segmentStatus
+  )
+  {
+    if (numToDrop <= 0) {
+      return 0;
+    }
+
+    final List<ServerHolder> eligibleServers = segmentStatus.getServersEligibleToDrop();
+    if (eligibleServers.isEmpty()) {
+      incrementSkipStat(Stats.Segments.DROP_SKIPPED, "No eligible server", segment, tier);
+      return 0;
+    }
+
+    // Keep eligible servers sorted by most full first
+    final TreeSet<ServerHolder> eligibleLiveServers = new TreeSet<>(Comparator.reverseOrder());
+    final TreeSet<ServerHolder> eligibleDyingServers = new TreeSet<>(Comparator.reverseOrder());
+    for (ServerHolder server : eligibleServers) {
+      if (server.isDecommissioning()) {
+        eligibleDyingServers.add(server);
+      } else {
+        eligibleLiveServers.add(server);
+      }
+    }
+
+    // Drop as many replicas as possible from decommissioning servers
+    int remainingNumToDrop = numToDrop;
+    int numDropsQueued =
+        dropReplicasFromServers(remainingNumToDrop, segment, eligibleDyingServers.iterator(), tier);
+
+    // Drop replicas from active servers if required
+    if (numToDrop > numDropsQueued) {
+      remainingNumToDrop = numToDrop - numDropsQueued;
+      Iterator<ServerHolder> serverIterator =
+          (useRoundRobinAssignment || eligibleLiveServers.size() >= remainingNumToDrop)
+          ? eligibleLiveServers.iterator()
+          : strategy.pickServersToDropSegment(segment, eligibleLiveServers);
+      numDropsQueued += dropReplicasFromServers(remainingNumToDrop, segment, serverIterator, tier);
+    }
+
+    return numDropsQueued;
+  }
+
+  /**
+   * Queues drop of {@code numToDrop} replicas of the segment from the servers.
+   * Returns the number of successfully queued drop operations.
+   */
+  private int dropReplicasFromServers(
+      int numToDrop,
+      DataSegment segment,
+      Iterator<ServerHolder> serverIterator,
+      String tier
+  )
+  {
+    int numDropsQueued = 0;
+    while (numToDrop > numDropsQueued && serverIterator.hasNext()) {
+      ServerHolder holder = serverIterator.next();
+      boolean dropped = loadQueueManager.dropSegment(segment, holder);
+
+      if (dropped) {
+        ++numDropsQueued;
+      } else {
+        incrementSkipStat(Stats.Segments.DROP_SKIPPED, "Encountered error", segment, tier);
+      }
+    }
+
+    return numDropsQueued;
+  }
+
+  /**
+   * Queues load of {@code numToLoad} replicas of the segment on a tier.
+   */
+  private int loadReplicas(
+      int numToLoad,
+      DataSegment segment,
+      String tier,
+      SegmentStatusInTier segmentStatus,
+      boolean isAlreadyLoadedOnTier
+  )
+  {
+    // Do not assign replicas if tier is already busy loading some
+    if (isAlreadyLoadedOnTier && replicationThrottler.isTierLoadingReplicas(tier)) {
+      return 0;
+    }
+
+    final List<ServerHolder> eligibleServers = segmentStatus.getServersEligibleToLoad();
+    if (eligibleServers.isEmpty()) {
+      incrementSkipStat(Stats.Segments.ASSIGN_SKIPPED, "No eligible server", segment, tier);
+      return 0;
+    }
+
+    final Iterator<ServerHolder> serverIterator =
+        useRoundRobinAssignment
+        ? serverSelector.getServersInTierToLoadSegment(tier, segment)
+        : strategy.findServersToLoadSegment(segment, eligibleServers);
+    if (!serverIterator.hasNext()) {
+      incrementSkipStat(Stats.Segments.ASSIGN_SKIPPED, "No server chosen by strategy", segment, tier);
+      return 0;
+    }
+
+    // Load the replicas on this tier
+    int numLoadsQueued = 0;
+    while (numLoadsQueued < numToLoad && serverIterator.hasNext()) {
+      numLoadsQueued += loadSegment(segment, serverIterator.next(), isAlreadyLoadedOnTier)
+                        ? 1 : 0;
+    }
+
+    return numLoadsQueued;
+  }
+
+  private boolean loadSegment(DataSegment segment, ServerHolder server, boolean isAlreadyLoadedOnTier)
+  {
+    final String tier = server.getServer().getTier();
+    if (isAlreadyLoadedOnTier && !replicationThrottler.canAssignReplica(tier)) {
+      incrementSkipStat(Stats.Segments.ASSIGN_SKIPPED, "Throttled replication", segment, tier);
+      return false;
+    }
+
+    final SegmentAction action = isAlreadyLoadedOnTier ? SegmentAction.REPLICATE : SegmentAction.LOAD;
+    final boolean assigned = loadQueueManager.loadSegment(segment, server, action);
+
+    if (!assigned) {
+      incrementSkipStat(Stats.Segments.ASSIGN_SKIPPED, "Encountered error", segment, tier);
+    } else if (isAlreadyLoadedOnTier) {
+      replicationThrottler.incrementAssignedReplicas(tier);
+    }
+
+    return assigned;
+  }

Review Comment:
   boolean police again: is it really better to have this boolean and a bunch of checks strewn through this code than to just have a `replicateSegment` method and a `loadSegment` method?



##########
server/src/main/java/org/apache/druid/server/coordinator/StrategicSegmentAssigner.java:
##########
@@ -0,0 +1,577 @@
+/*
+ * 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;
+
+import com.google.common.collect.Sets;
+import it.unimi.dsi.fastutil.objects.Object2IntOpenHashMap;
+import org.apache.druid.java.util.emitter.EmittingLogger;
+import org.apache.druid.server.coordinator.balancer.BalancerStrategy;
+import org.apache.druid.server.coordinator.loadqueue.SegmentAction;
+import org.apache.druid.server.coordinator.loadqueue.SegmentLoadQueueManager;
+import org.apache.druid.server.coordinator.rules.SegmentActionHandler;
+import org.apache.druid.server.coordinator.stats.CoordinatorRunStats;
+import org.apache.druid.server.coordinator.stats.CoordinatorStat;
+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 java.util.Comparator;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+
+/**
+ * Used by the coordinator in each run for segment loading, dropping, balancing
+ * and broadcasting.
+ * <p>
+ * An instance of this class is freshly created for each coordinator run.
+ */
+public class StrategicSegmentAssigner implements SegmentActionHandler
+{
+  private static final EmittingLogger log = new EmittingLogger(StrategicSegmentAssigner.class);
+
+  private final SegmentLoadQueueManager loadQueueManager;
+  private final DruidCluster cluster;
+  private final CoordinatorRunStats stats;
+  private final SegmentReplicantLookup replicantLookup;
+  private final ReplicationThrottler replicationThrottler;
+  private final RoundRobinServerSelector serverSelector;
+  private final BalancerStrategy strategy;
+
+  private final boolean useRoundRobinAssignment;
+
+  private final Set<String> tiersWithNoServer = new HashSet<>();
+
+  public StrategicSegmentAssigner(
+      SegmentLoadQueueManager loadQueueManager,
+      DruidCluster cluster,
+      BalancerStrategy strategy,
+      CoordinatorDynamicConfig dynamicConfig,
+      CoordinatorRunStats stats
+  )
+  {
+    this.stats = stats;
+    this.cluster = cluster;
+    this.strategy = strategy;
+    this.loadQueueManager = loadQueueManager;
+    this.replicantLookup = SegmentReplicantLookup.make(cluster);
+    this.replicationThrottler = createReplicationThrottler(dynamicConfig);
+    this.useRoundRobinAssignment = dynamicConfig.isUseRoundRobinSegmentAssignment();
+    this.serverSelector = useRoundRobinAssignment ? new RoundRobinServerSelector(cluster) : null;
+  }
+
+  public CoordinatorRunStats getStats()
+  {
+    return stats;
+  }
+
+  public SegmentReplicantLookup getReplicantLookup()
+  {
+    return replicantLookup;
+  }
+
+  public void makeAlerts()
+  {
+    if (!tiersWithNoServer.isEmpty()) {
+      log.makeAlert("Tiers [%s] have no servers! Check your cluster configuration.", tiersWithNoServer).emit();
+    }
+  }
+
+  /**
+   * Moves the given segment from the source server to an eligible destination
+   * server.
+   * <p>
+   * An eligible destination server must:
+   * <ul>
+   *   <li>be present in the given list of destination servers</li>
+   *   <li>belong to the same tier as the source server</li>
+   *   <li>not already be serving or loading a replica of the segment</li>
+   *   <li>have enough space to load the segment</li>
+   * </ul>
+   * <p>
+   * The segment is not moved if:
+   * <ul>
+   *   <li>there is no eligible destination server, or</li>
+   *   <li>skipIfOptimallyPlaced is true and segment is already optimally placed, or</li>
+   *   <li>some other error occurs</li>
+   * </ul>
+   */
+  public boolean moveSegment(
+      DataSegment segment,
+      ServerHolder sourceServer,
+      List<ServerHolder> destinationServers
+  )
+  {
+    final String tier = sourceServer.getServer().getTier();
+    final List<ServerHolder> eligibleDestinationServers =
+        destinationServers.stream()
+                          .filter(s -> s.getServer().getTier().equals(tier))
+                          .filter(s -> s.canLoadSegment(segment))
+                          .collect(Collectors.toList());
+
+    if (eligibleDestinationServers.isEmpty()) {
+      incrementSkipStat(Stats.Segments.MOVE_SKIPPED, "No eligible server", segment, tier);
+      return false;
+    }
+
+    // If the source server is not decommissioning, move can be skipped if the
+    // segment is already optimally placed
+    if (!sourceServer.isDecommissioning()) {
+      eligibleDestinationServers.add(sourceServer);
+    }
+
+    final ServerHolder destination =
+        strategy.findDestinationServerToMoveSegment(segment, sourceServer, eligibleDestinationServers);
+
+    if (destination == null || destination.getServer().equals(sourceServer.getServer())) {
+      incrementSkipStat(Stats.Segments.MOVE_SKIPPED, "Optimally placed", segment, tier);
+      return false;
+    } else if (moveSegment(segment, sourceServer, destination)) {
+      incrementStat(Stats.Segments.MOVED, segment, tier, 1);
+      return true;
+    } else {
+      incrementSkipStat(Stats.Segments.MOVE_SKIPPED, "Encountered error", segment, tier);
+      return false;
+    }
+  }
+
+  /**
+   * Moves the given segment from serverA to serverB.
+   */
+  private boolean moveSegment(DataSegment segment, ServerHolder serverA, ServerHolder serverB)
+  {
+    final String tier = serverA.getServer().getTier();
+    if (serverA.isLoadingSegment(segment)) {
+      // Cancel the load on serverA and load on serverB instead
+      if (serverA.cancelOperation(SegmentAction.LOAD, segment)) {
+        int loadedCountOnTier = replicantLookup.getLoadedNotDroppingReplicas(segment.getId(), tier);
+        return loadSegment(segment, serverB, loadedCountOnTier >= 1);
+      }
+
+      // Could not cancel load, let the segment load on serverA and count it as unmoved
+      return false;
+    } else if (serverA.isServingSegment(segment)) {
+      return loadQueueManager.moveSegment(segment, serverA, serverB);
+    } else {
+      return false;
+    }
+  }
+
+  @Override
+  public void updateSegmentReplicasInTiers(DataSegment segment, Map<String, Integer> tierToReplicaCount)
+  {
+    // Identify empty tiers and determine total required replicas
+    final AtomicInteger requiredTotalReplicas = new AtomicInteger(0);
+    final Set<String> allTiers = Sets.newHashSet(cluster.getTierNames());
+    tierToReplicaCount.forEach((tier, requiredReplicas) -> {
+      reportTierCapacityStats(segment, requiredReplicas, tier);
+      replicantLookup.setRequiredReplicas(segment.getId(), false, tier, requiredReplicas);
+      if (allTiers.contains(tier)) {
+        requiredTotalReplicas.addAndGet(requiredReplicas);
+      } else {
+        tiersWithNoServer.add(tier);
+      }
+    });
+
+    final int loadedNotDroppingReplicas = replicantLookup.getLoadedReplicas(segment.getId(), false);
+    final int replicaSurplus = loadedNotDroppingReplicas - requiredTotalReplicas.get();
+
+    // Update replicas in every tier
+    int dropsQueued = 0;
+    for (String tier : allTiers) {
+      dropsQueued += updateReplicasInTier(
+          segment,
+          tier,
+          tierToReplicaCount.getOrDefault(tier, 0),
+          replicaSurplus - dropsQueued
+      );
+    }
+  }
+
+  /**
+   * Queues load or drop operations on this tier based on the required
+   * number of replicas and the current state.
+   * <p>
+   * The {@code maxReplicasToDrop} helps to maintain the required level of
+   * replication in the cluster. This ensures that segment read concurrency does
+   * not suffer during a tier shift or load rule change.
+   * <p>
+   * Returns the number of new drop operations queued on this tier.
+   */
+  private int updateReplicasInTier(
+      DataSegment segment,
+      String tier,
+      int requiredReplicas,
+      int maxReplicasToDrop
+  )
+  {
+    final int loadedNotDroppingReplicas =
+        replicantLookup.getLoadedNotDroppingReplicas(segment.getId(), tier);
+    final int loadingReplicas = replicantLookup.getLoadingReplicas(segment.getId(), tier);
+    final int projectedReplicas = loadedNotDroppingReplicas + loadingReplicas;
+
+    final int movingReplicas = replicantLookup.getMovingReplicas(segment.getId(), tier);
+    final boolean shouldCancelMoves = requiredReplicas == 0 && movingReplicas > 0;
+
+    // Check if there is any action required on this tier
+    if (projectedReplicas == requiredReplicas && !shouldCancelMoves) {
+      return 0;
+    }
+
+    final SegmentStatusInTier segmentStatus =
+        new SegmentStatusInTier(segment, cluster.getHistoricalsByTier(tier));
+
+    // Cancel all moves in this tier if it does not need to have replicas
+    if (shouldCancelMoves) {
+      cancelOperations(SegmentAction.MOVE_TO, movingReplicas, segment, segmentStatus);
+      cancelOperations(SegmentAction.MOVE_FROM, movingReplicas, segment, segmentStatus);
+    }
+
+    // Cancel drops and queue loads if the projected count is below the requirement
+    if (projectedReplicas < requiredReplicas) {
+      int replicaDeficit = requiredReplicas - projectedReplicas;
+      int cancelledDrops =
+          cancelOperations(SegmentAction.DROP, replicaDeficit, segment, segmentStatus);
+
+      // Cancelled drops can be counted as loaded replicas, thus reducing deficit
+      int numReplicasToLoad = replicaDeficit - cancelledDrops;
+      if (numReplicasToLoad > 0) {
+        boolean isAlreadyLoadedOnTier = loadedNotDroppingReplicas + cancelledDrops >= 1;
+        int numLoadsQueued = loadReplicas(numReplicasToLoad, segment, tier, segmentStatus, isAlreadyLoadedOnTier);
+        incrementStat(Stats.Segments.ASSIGNED, segment, tier, numLoadsQueued);
+      }
+    }
+
+    // Cancel loads and queue drops if the projected count exceeds the requirement
+    if (projectedReplicas > requiredReplicas) {
+      int replicaSurplus = projectedReplicas - requiredReplicas;
+      int cancelledLoads =
+          cancelOperations(SegmentAction.LOAD, replicaSurplus, segment, segmentStatus);
+
+      int numReplicasToDrop = Math.min(replicaSurplus - cancelledLoads, maxReplicasToDrop);
+      if (numReplicasToDrop > 0) {
+        int dropsQueuedOnTier = dropReplicas(numReplicasToDrop, segment, tier, segmentStatus);
+        incrementStat(Stats.Segments.DROPPED, segment, tier, dropsQueuedOnTier);
+        return dropsQueuedOnTier;
+      }
+    }
+
+    return 0;
+  }
+
+  private void reportTierCapacityStats(DataSegment segment, int requiredReplicas, String tier)
+  {
+    final RowKey rowKey = RowKey.forTier(tier);
+    stats.updateMax(Stats.Tier.REPLICATION_FACTOR, rowKey, requiredReplicas);
+    stats.add(Stats.Tier.REQUIRED_CAPACITY, rowKey, segment.getSize() * requiredReplicas);
+  }
+
+  @Override
+  public void broadcastSegment(DataSegment segment)
+  {
+    final Object2IntOpenHashMap<String> tierToRequiredReplicas = new Object2IntOpenHashMap<>();
+    for (ServerHolder server : cluster.getAllServers()) {
+      // Ignore servers which are not broadcast targets
+      if (!server.getServer().getType().isSegmentBroadcastTarget()) {
+        continue;
+      }
+
+      final String tier = server.getServer().getTier();
+
+      // Drop from decommissioning servers and load on active servers
+      int numDropsQueued = 0;
+      int numLoadsQueued = 0;
+      if (server.isDecommissioning()) {
+        numDropsQueued += dropBroadcastSegment(segment, server) ? 1 : 0;
+      } else {
+        tierToRequiredReplicas.addTo(tier, 1);
+        numLoadsQueued += loadBroadcastSegment(segment, server) ? 1 : 0;
+      }
+
+      if (numLoadsQueued > 0) {
+        incrementStat(Stats.Segments.ASSIGNED, segment, tier, numLoadsQueued);
+      }
+      if (numDropsQueued > 0) {
+        incrementStat(Stats.Segments.DROPPED, segment, tier, numDropsQueued);
+      }
+    }
+
+    // Update required replica counts
+    tierToRequiredReplicas.object2IntEntrySet().fastForEach(
+        entry -> replicantLookup
+            .setRequiredReplicas(segment.getId(), true, entry.getKey(), entry.getIntValue())
+    );
+  }
+
+  @Override
+  public void deleteSegment(DataSegment segment)
+  {
+    loadQueueManager.deleteSegment(segment);
+    stats.addToDatasourceStat(Stats.Segments.DELETED, segment.getDataSource(), 1);
+  }
+
+  /**
+   * Loads the broadcast segment if it is not loaded on the given server.
+   * Returns true only if the segment was successfully queued for load on the server.
+   */
+  private boolean loadBroadcastSegment(DataSegment segment, ServerHolder server)
+  {
+    if (server.isServingSegment(segment) || server.isLoadingSegment(segment)) {
+      return false;
+    } else if (server.isDroppingSegment(segment)) {
+      return server.cancelOperation(SegmentAction.DROP, segment);
+    }
+
+    if (server.canLoadSegment(segment) && loadSegment(segment, server, false)) {
+      return true;
+    } else {
+      log.makeAlert("Could not assign broadcast segment for datasource [%s]", segment.getDataSource())
+         .addData("segmentId", segment.getId())
+         .addData("segmentSize", segment.getSize())
+         .addData("hostName", server.getServer().getHost())
+         .addData("availableSize", server.getAvailableSize())
+         .emit();
+      return false;
+    }
+  }
+
+  /**
+   * Drops the broadcast segment if it is loaded on the given server.
+   * Returns true only if the segment was successfully queued for drop on the server.
+   */
+  private boolean dropBroadcastSegment(DataSegment segment, ServerHolder server)
+  {
+    if (server.isLoadingSegment(segment)) {
+      return server.cancelOperation(SegmentAction.LOAD, segment);
+    } else if (server.isServingSegment(segment)) {
+      return loadQueueManager.dropSegment(segment, server);
+    } else {
+      return false;
+    }
+  }
+
+  /**
+   * Queues drop of {@code numToDrop} replicas of the segment from a tier.
+   * Tries to drop replicas first from decommissioning servers and then from
+   * active servers.
+   * <p>
+   * Returns the number of successfully queued drop operations.
+   */
+  private int dropReplicas(
+      final int numToDrop,
+      DataSegment segment,
+      String tier,
+      SegmentStatusInTier segmentStatus
+  )
+  {
+    if (numToDrop <= 0) {
+      return 0;
+    }
+
+    final List<ServerHolder> eligibleServers = segmentStatus.getServersEligibleToDrop();
+    if (eligibleServers.isEmpty()) {
+      incrementSkipStat(Stats.Segments.DROP_SKIPPED, "No eligible server", segment, tier);
+      return 0;
+    }
+
+    // Keep eligible servers sorted by most full first
+    final TreeSet<ServerHolder> eligibleLiveServers = new TreeSet<>(Comparator.reverseOrder());
+    final TreeSet<ServerHolder> eligibleDyingServers = new TreeSet<>(Comparator.reverseOrder());
+    for (ServerHolder server : eligibleServers) {
+      if (server.isDecommissioning()) {
+        eligibleDyingServers.add(server);
+      } else {
+        eligibleLiveServers.add(server);
+      }
+    }
+
+    // Drop as many replicas as possible from decommissioning servers
+    int remainingNumToDrop = numToDrop;
+    int numDropsQueued =
+        dropReplicasFromServers(remainingNumToDrop, segment, eligibleDyingServers.iterator(), tier);
+
+    // Drop replicas from active servers if required
+    if (numToDrop > numDropsQueued) {
+      remainingNumToDrop = numToDrop - numDropsQueued;
+      Iterator<ServerHolder> serverIterator =
+          (useRoundRobinAssignment || eligibleLiveServers.size() >= remainingNumToDrop)
+          ? eligibleLiveServers.iterator()
+          : strategy.pickServersToDropSegment(segment, eligibleLiveServers);
+      numDropsQueued += dropReplicasFromServers(remainingNumToDrop, segment, serverIterator, tier);
+    }
+
+    return numDropsQueued;
+  }
+
+  /**
+   * Queues drop of {@code numToDrop} replicas of the segment from the servers.
+   * Returns the number of successfully queued drop operations.
+   */
+  private int dropReplicasFromServers(
+      int numToDrop,
+      DataSegment segment,
+      Iterator<ServerHolder> serverIterator,
+      String tier
+  )
+  {
+    int numDropsQueued = 0;
+    while (numToDrop > numDropsQueued && serverIterator.hasNext()) {
+      ServerHolder holder = serverIterator.next();
+      boolean dropped = loadQueueManager.dropSegment(segment, holder);
+
+      if (dropped) {
+        ++numDropsQueued;
+      } else {
+        incrementSkipStat(Stats.Segments.DROP_SKIPPED, "Encountered error", segment, tier);
+      }
+    }
+
+    return numDropsQueued;
+  }
+
+  /**
+   * Queues load of {@code numToLoad} replicas of the segment on a tier.
+   */
+  private int loadReplicas(
+      int numToLoad,
+      DataSegment segment,
+      String tier,
+      SegmentStatusInTier segmentStatus,
+      boolean isAlreadyLoadedOnTier
+  )
+  {
+    // Do not assign replicas if tier is already busy loading some
+    if (isAlreadyLoadedOnTier && replicationThrottler.isTierLoadingReplicas(tier)) {
+      return 0;
+    }
+
+    final List<ServerHolder> eligibleServers = segmentStatus.getServersEligibleToLoad();
+    if (eligibleServers.isEmpty()) {
+      incrementSkipStat(Stats.Segments.ASSIGN_SKIPPED, "No eligible server", segment, tier);
+      return 0;
+    }
+
+    final Iterator<ServerHolder> serverIterator =
+        useRoundRobinAssignment
+        ? serverSelector.getServersInTierToLoadSegment(tier, segment)
+        : strategy.findServersToLoadSegment(segment, eligibleServers);
+    if (!serverIterator.hasNext()) {
+      incrementSkipStat(Stats.Segments.ASSIGN_SKIPPED, "No server chosen by strategy", segment, tier);
+      return 0;
+    }
+
+    // Load the replicas on this tier
+    int numLoadsQueued = 0;
+    while (numLoadsQueued < numToLoad && serverIterator.hasNext()) {
+      numLoadsQueued += loadSegment(segment, serverIterator.next(), isAlreadyLoadedOnTier)
+                        ? 1 : 0;
+    }
+
+    return numLoadsQueued;
+  }
+
+  private boolean loadSegment(DataSegment segment, ServerHolder server, boolean isAlreadyLoadedOnTier)
+  {
+    final String tier = server.getServer().getTier();
+    if (isAlreadyLoadedOnTier && !replicationThrottler.canAssignReplica(tier)) {
+      incrementSkipStat(Stats.Segments.ASSIGN_SKIPPED, "Throttled replication", segment, tier);
+      return false;
+    }
+
+    final SegmentAction action = isAlreadyLoadedOnTier ? SegmentAction.REPLICATE : SegmentAction.LOAD;
+    final boolean assigned = loadQueueManager.loadSegment(segment, server, action);
+
+    if (!assigned) {
+      incrementSkipStat(Stats.Segments.ASSIGN_SKIPPED, "Encountered error", segment, tier);
+    } else if (isAlreadyLoadedOnTier) {
+      replicationThrottler.incrementAssignedReplicas(tier);
+    }
+
+    return assigned;
+  }
+
+  private ReplicationThrottler createReplicationThrottler(CoordinatorDynamicConfig dynamicConfig)

Review Comment:
   Can this be static?  I think it can and maybe should be.  If it cannot be static, please rename to `initializeReplicationThrottler()` to help indicate that it is intended for initialization and not as a factory to be called multiple times from the same object.



-- 
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: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] kfaraz commented on a diff in pull request #13197: Segment loading: Allow cancellation and prioritization of load queue items

Posted by "kfaraz (via GitHub)" <gi...@apache.org>.
kfaraz commented on code in PR #13197:
URL: https://github.com/apache/druid/pull/13197#discussion_r991357698


##########
server/src/main/java/org/apache/druid/server/coordinator/duty/EmitClusterStatsAndMetrics.java:
##########
@@ -234,36 +218,40 @@ private void emitStatsForHistoricalManagementDuties(DruidCluster cluster, Coordi
         }
     );
 
+    // Log load queue status of all replication or broadcast targets
     log.info("Load Queues:");
-    for (Iterable<ServerHolder> serverHolders : cluster.getSortedHistoricalsByTier()) {

Review Comment:
   Change summary: Report queue status not just of historicals but all replication or broadcast targets.



##########
server/src/main/java/org/apache/druid/server/coordinator/SegmentStateManager.java:
##########
@@ -0,0 +1,235 @@
+/*
+ * 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;
+
+import org.apache.druid.client.ServerInventoryView;
+import org.apache.druid.java.util.emitter.EmittingLogger;
+import org.apache.druid.metadata.SegmentsMetadataManager;
+import org.apache.druid.timeline.DataSegment;
+
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * Manages state of segments being loaded.
+ */
+public class SegmentStateManager

Review Comment:
   Rename to `LoadQueueManager` as this class really acts as an interface between the duties and the load queues.



-- 
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: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org