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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to