Enable smartSegmentLoading on the Coordinator (#13197)

This commit does a complete revamp of the coordinator to address problem areas:
- Stability: Fix several bugs, add capabilities to prioritize and cancel load queue items
- Visibility: Add new metrics, improve logs, revamp `CoordinatorRunStats`
- Configuration: Add dynamic config `smartSegmentLoading` to automatically set
optimal values for all segment loading configs such as `maxSegmentsToMove`,
`replicationThrottleLimit` and `maxSegmentsInNodeLoadingQueue`.

Changed classes:
- Add `StrategicSegmentAssigner` to make assignment decisions for load, replicate and move
- Add `SegmentAction` to distinguish between load, replicate, drop and move operations
- Add `SegmentReplicationStatus` to capture current state of replication of all used segments
- Add `SegmentLoadingConfig` to contain recomputed dynamic config values
- Simplify classes `LoadRule`, `BroadcastRule`
- Simplify the `BalancerStrategy` and `CostBalancerStrategy`
- Add several new methods to `ServerHolder` to track loaded and queued segments
- Refactor `DruidCoordinator`

Impact:
- Enable `smartSegmentLoading` by default. With this enabled, none of the following
dynamic configs need to be set: `maxSegmentsToMove`, `replicationThrottleLimit`,
`maxSegmentsInNodeLoadingQueue`, `useRoundRobinSegmentAssignment`,
`emitBalancingStats` and `replicantLifetime`.
- Coordinator reports richer metrics and produces cleaner and more informative logs
- Coordinator uses an unlimited load queue for all serves, and makes better assignment decisions
This commit is contained in:
Kashif Faraz 2023-06-19 14:27:35 +05:30 committed by GitHub
parent cfd07a95b7
commit 50461c3bd5
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
123 changed files with 8898 additions and 9584 deletions

View File

@ -20,13 +20,14 @@
package org.apache.druid.server.coordinator;
import com.google.common.collect.ImmutableMap;
import com.google.common.util.concurrent.MoreExecutors;
import org.apache.druid.client.ImmutableDruidDataSource;
import org.apache.druid.client.ImmutableDruidServer;
import org.apache.druid.java.util.common.Intervals;
import org.apache.druid.java.util.common.concurrent.Execs;
import org.apache.druid.server.coordination.DruidServerMetadata;
import org.apache.druid.server.coordination.ServerType;
import org.apache.druid.server.coordinator.balancer.BalancerSegmentHolder;
import org.apache.druid.server.coordinator.balancer.ReservoirSegmentSampler;
import org.apache.druid.server.coordinator.loading.LoadQueuePeonTester;
import org.apache.druid.timeline.DataSegment;
import org.joda.time.Interval;
import org.openjdk.jmh.annotations.Benchmark;
@ -63,7 +64,7 @@ public class BalancerStrategyBenchmark
private static final Interval TEST_SEGMENT_INTERVAL = Intervals.of("2012-03-15T00:00:00.000/2012-03-16T00:00:00.000");
private static final int NUMBER_OF_SERVERS = 20;
@Param({"default", "50percentOfSegmentsToConsiderPerMove", "useBatchedSegmentSampler"})
@Param({"default", "useBatchedSegmentSampler"})
private String mode;
@Param({"10000", "100000", "1000000"})
@ -73,26 +74,13 @@ public class BalancerStrategyBenchmark
private int maxSegmentsToMove;
private final List<ServerHolder> serverHolders = new ArrayList<>();
private boolean useBatchedSegmentSampler;
private int reservoirSize = 1;
private double percentOfSegmentsToConsider = 100;
private final BalancerStrategy balancerStrategy = new CostBalancerStrategy(
MoreExecutors.listeningDecorator(Execs.multiThreaded(1, "BalancerStrategyBenchmark-%d"))
);
@Setup(Level.Trial)
public void setup()
{
switch (mode) {
case "50percentOfSegmentsToConsiderPerMove":
percentOfSegmentsToConsider = 50;
useBatchedSegmentSampler = false;
break;
case "useBatchedSegmentSampler":
reservoirSize = maxSegmentsToMove;
useBatchedSegmentSampler = true;
break;
default:
if ("useBatchedSegmentSampler".equals(mode)) {
reservoirSize = maxSegmentsToMove;
}
List<List<DataSegment>> segmentList = new ArrayList<>(NUMBER_OF_SERVERS);
@ -131,20 +119,9 @@ public class BalancerStrategyBenchmark
@Benchmark
public void pickSegmentsToMove(Blackhole blackhole)
{
Iterator<BalancerSegmentHolder> iterator;
if (useBatchedSegmentSampler) {
iterator = balancerStrategy.pickSegmentsToMove(
serverHolders,
Collections.emptySet(),
reservoirSize
);
} else {
iterator = balancerStrategy.pickSegmentsToMove(
serverHolders,
Collections.emptySet(),
percentOfSegmentsToConsider
);
}
Iterator<BalancerSegmentHolder> iterator = ReservoirSegmentSampler
.pickMovableSegmentsFrom(serverHolders, reservoirSize, ServerHolder::getServedSegments, Collections.emptySet())
.iterator();
for (int i = 0; i < maxSegmentsToMove && iterator.hasNext(); i++) {
blackhole.consume(iterator.next());

View File

@ -21,7 +21,8 @@ package org.apache.druid.server.coordinator;
import org.apache.druid.java.util.common.DateTimes;
import org.apache.druid.java.util.common.logger.Logger;
import org.apache.druid.server.coordinator.cost.SegmentsCostCache;
import org.apache.druid.server.coordinator.balancer.CostBalancerStrategy;
import org.apache.druid.server.coordinator.balancer.SegmentsCostCache;
import org.apache.druid.timeline.DataSegment;
import org.joda.time.DateTime;
import org.joda.time.Interval;

View File

@ -20,6 +20,7 @@
package org.apache.druid.server.coordinator;
import org.apache.druid.java.util.common.DateTimes;
import org.apache.druid.server.coordinator.balancer.CostBalancerStrategy;
import org.apache.druid.timeline.DataSegment;
import org.joda.time.DateTime;
import org.joda.time.Interval;

View File

@ -124,7 +124,7 @@ public class DruidClusterAdminClient
{
waitUntilInstanceReady(config.getCoordinatorUrl());
postDynamicConfig(CoordinatorDynamicConfig.builder()
.withLeadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments(1)
.withMarkSegmentAsUnusedDelayMillis(1)
.build());
}
@ -132,7 +132,7 @@ public class DruidClusterAdminClient
{
waitUntilInstanceReady(config.getCoordinatorTwoUrl());
postDynamicConfig(CoordinatorDynamicConfig.builder()
.withLeadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments(1)
.withMarkSegmentAsUnusedDelayMillis(1)
.build());
}

View File

@ -123,7 +123,7 @@ public class DruidClusterAdminClient
{
waitUntilInstanceReady(config.getCoordinatorUrl());
postDynamicConfig(CoordinatorDynamicConfig.builder()
.withLeadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments(1)
.withMarkSegmentAsUnusedDelayMillis(1)
.build());
}
@ -131,7 +131,7 @@ public class DruidClusterAdminClient
{
waitUntilInstanceReady(config.getCoordinatorTwoUrl());
postDynamicConfig(CoordinatorDynamicConfig.builder()
.withLeadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments(1)
.withMarkSegmentAsUnusedDelayMillis(1)
.build());
}

View File

@ -43,12 +43,6 @@ public class DruidMetrics
public static final String TASK_TYPE = "taskType";
public static final String TASK_STATUS = "taskStatus";
public static final String SERVER = "server";
public static final String TIER = "tier";
public static final String DUTY = "duty";
public static final String DUTY_GROUP = "dutyGroup";
public static final String STREAM = "stream";
public static final String PARTITION = "partition";

View File

@ -558,8 +558,7 @@ public class SqlSegmentsMetadataManager implements SegmentsMetadataManager
private int doMarkAsUsedNonOvershadowedSegments(String dataSourceName, @Nullable Interval interval)
{
final List<DataSegment> unusedSegments = new ArrayList<>();
final SegmentTimeline timeline =
SegmentTimeline.forSegments(Collections.emptyIterator());
final SegmentTimeline timeline = new SegmentTimeline();
connector.inReadOnlyTransaction(
(handle, status) -> {

View File

@ -1,186 +0,0 @@
/*
* 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.server.coordinator.duty.BalanceSegments;
import org.apache.druid.timeline.DataSegment;
import javax.annotation.Nullable;
import java.util.Iterator;
import java.util.List;
import java.util.NavigableSet;
import java.util.Set;
/**
* This interface describes the coordinator balancing strategy, which is responsible for making decisions on where
* to place {@link DataSegment}s on historical servers (described by {@link ServerHolder}). The balancing strategy
* is used by {@link org.apache.druid.server.coordinator.rules.LoadRule} to assign and drop segments, and by
* {@link BalanceSegments} to migrate segments between historicals.
*/
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
* @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);
/**
* Find the best server on which to place a {@link DataSegment} replica 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
*/
@Nullable
ServerHolder findNewSegmentHomeReplicator(DataSegment proposalSegment, List<ServerHolder> serverHolders);
/**
* Pick the best segments to move from one of the supplied set of servers according to the balancing strategy.
*
* @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).
*/
default Iterator<BalancerSegmentHolder> pickSegmentsToMove(
List<ServerHolder> serverHolders,
Set<String> broadcastDatasources,
int reservoirSize
)
{
return new Iterator<BalancerSegmentHolder>()
{
private Iterator<BalancerSegmentHolder> it = sample();
private Iterator<BalancerSegmentHolder> sample()
{
return ReservoirSegmentSampler.getRandomBalancerSegmentHolders(
serverHolders,
broadcastDatasources,
reservoirSize
).iterator();
}
@Override
public boolean hasNext()
{
if (it.hasNext()) {
return true;
}
it = sample();
return it.hasNext();
}
@Override
public BalancerSegmentHolder next()
{
return it.next();
}
};
}
/**
* Pick the best segments to move from one of the supplied set of servers according to the balancing strategy.
*
* @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 percentOfSegmentsToConsider The percentage of the total number of segments that we will consider when
* choosing which segment to move. {@link CoordinatorDynamicConfig} defines a
* config percentOfSegmentsToConsiderPerMove that will be used as an argument
* for implementations of this method.
* @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).
*
* @deprecated Use {@link #pickSegmentsToMove(List, Set, int)} instead as it is
* a much more performant sampling method which does not allow duplicates. This
* method will be removed in future releases.
*/
@Deprecated
default Iterator<BalancerSegmentHolder> pickSegmentsToMove(
List<ServerHolder> serverHolders,
Set<String> broadcastDatasources,
double percentOfSegmentsToConsider
)
{
return new Iterator<BalancerSegmentHolder>()
{
private BalancerSegmentHolder next = sample();
private BalancerSegmentHolder sample()
{
return ReservoirSegmentSampler.getRandomBalancerSegmentHolder(
serverHolders,
broadcastDatasources,
percentOfSegmentsToConsider
);
}
@Override
public boolean hasNext()
{
return next != null;
}
@Override
public BalancerSegmentHolder next()
{
BalancerSegmentHolder ret = next;
next = sample();
return ret;
}
};
}
/**
* Returns an iterator for a set of servers to drop from, ordered by preference of which server to drop from first
* for a given drop strategy. One or more segments may be dropped, depending on how much the segment is
* over-replicated.
* @param toDropSegment segment to drop from one or more servers
* @param serverHolders set of historicals to consider dropping from
* @return Iterator for set of historicals, ordered by drop preference
*/
default Iterator<ServerHolder> pickServersToDrop(DataSegment toDropSegment, NavigableSet<ServerHolder> serverHolders)
{
// By default, use the reverse order to get the holders with least available size first.
return serverHolders.descendingIterator();
}
/**
* Add balancing strategy stats during the 'balanceTier' operation of
* {@link BalanceSegments} to be included
* @param tier historical tier being balanced
* @param stats stats object to add balancing strategy stats to
* @param serverHolderList servers in tier being balanced
*/
void emitStats(String tier, CoordinatorStats stats, List<ServerHolder> serverHolderList);
}

View File

@ -27,13 +27,18 @@ import com.google.common.collect.ImmutableSet;
import org.apache.druid.common.config.JacksonConfigManager;
import org.apache.druid.java.util.common.logger.Logger;
import org.apache.druid.server.coordinator.duty.KillUnusedSegments;
import org.apache.druid.server.coordinator.loading.LoadQueuePeon;
import org.apache.druid.server.coordinator.stats.Dimension;
import javax.annotation.Nonnull;
import javax.annotation.Nullable;
import javax.validation.constraints.Max;
import javax.validation.constraints.Min;
import javax.validation.constraints.NotNull;
import java.util.Collection;
import java.util.EnumMap;
import java.util.HashSet;
import java.util.Map;
import java.util.Objects;
import java.util.Set;
import java.util.concurrent.TimeUnit;
@ -50,7 +55,7 @@ public class CoordinatorDynamicConfig
{
public static final String CONFIG_KEY = "coordinator.config";
private final long leadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments;
private final long markSegmentAsUnusedDelayMillis;
private final long mergeBytesLimit;
private final int mergeSegmentsLimit;
private final int maxSegmentsToMove;
@ -63,12 +68,18 @@ public class CoordinatorDynamicConfig
private final int balancerComputeThreads;
private final boolean emitBalancingStats;
private final boolean useRoundRobinSegmentAssignment;
private final boolean smartSegmentLoading;
/**
* List of specific data sources for which kill tasks are sent in {@link KillUnusedSegments}.
*/
private final Set<String> specificDataSourcesToKillUnusedSegmentsIn;
private final Set<String> decommissioningNodes;
private final Map<String, String> debugDimensions;
private final Map<Dimension, String> validDebugDimensions;
@Deprecated
private final int decommissioningMaxPercentOfMaxSegmentsToMove;
/**
@ -102,6 +113,7 @@ public class CoordinatorDynamicConfig
* be set to put a hard upper limit on the number of replicants loaded. It is a tool that can help prevent
* long delays in new data loads after events such as a Historical server leaving the cluster.
*/
@Deprecated
private final int maxNonPrimaryReplicantsToLoad;
private static final Logger log = new Logger(CoordinatorDynamicConfig.class);
@ -111,7 +123,7 @@ public class CoordinatorDynamicConfig
// Keeping the legacy 'millisToWaitBeforeDeleting' property name for backward compatibility. When the project is
// updated to Jackson 2.9 it could be changed, see https://github.com/apache/druid/issues/7152
@JsonProperty("millisToWaitBeforeDeleting")
long leadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments,
long markSegmentAsUnusedDelayMillis,
@JsonProperty("mergeBytesLimit") long mergeBytesLimit,
@JsonProperty("mergeSegmentsLimit") int mergeSegmentsLimit,
@JsonProperty("maxSegmentsToMove") int maxSegmentsToMove,
@ -137,14 +149,17 @@ public class CoordinatorDynamicConfig
@JsonProperty("pauseCoordination") boolean pauseCoordination,
@JsonProperty("replicateAfterLoadTimeout") boolean replicateAfterLoadTimeout,
@JsonProperty("maxNonPrimaryReplicantsToLoad") @Nullable Integer maxNonPrimaryReplicantsToLoad,
@JsonProperty("useRoundRobinSegmentAssignment") @Nullable Boolean useRoundRobinSegmentAssignment
@JsonProperty("useRoundRobinSegmentAssignment") @Nullable Boolean useRoundRobinSegmentAssignment,
@JsonProperty("smartSegmentLoading") @Nullable Boolean smartSegmentLoading,
@JsonProperty("debugDimensions") @Nullable Map<String, String> debugDimensions
)
{
this.leadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments =
leadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments;
this.markSegmentAsUnusedDelayMillis =
markSegmentAsUnusedDelayMillis;
this.mergeBytesLimit = mergeBytesLimit;
this.mergeSegmentsLimit = mergeSegmentsLimit;
this.maxSegmentsToMove = maxSegmentsToMove;
this.smartSegmentLoading = Builder.valueOrDefault(smartSegmentLoading, Defaults.SMART_SEGMENT_LOADING);
if (percentOfSegmentsToConsiderPerMove == null) {
log.debug(
@ -152,36 +167,37 @@ public class CoordinatorDynamicConfig
+ "reflect this configuration being added to Druid in a recent release. Druid is defaulting the value "
+ "to the Druid default of %f. It is recommended that you re-submit your dynamic config with your "
+ "desired value for percentOfSegmentsToConsideredPerMove",
Builder.DEFAULT_PERCENT_OF_SEGMENTS_TO_CONSIDER_PER_MOVE
Defaults.PERCENT_OF_SEGMENTS_TO_CONSIDER_PER_MOVE
);
percentOfSegmentsToConsiderPerMove = Builder.DEFAULT_PERCENT_OF_SEGMENTS_TO_CONSIDER_PER_MOVE;
percentOfSegmentsToConsiderPerMove = Defaults.PERCENT_OF_SEGMENTS_TO_CONSIDER_PER_MOVE;
}
Preconditions.checkArgument(
percentOfSegmentsToConsiderPerMove > 0 && percentOfSegmentsToConsiderPerMove <= 100,
"percentOfSegmentsToConsiderPerMove should be between 1 and 100!"
"'percentOfSegmentsToConsiderPerMove' should be between 1 and 100"
);
this.percentOfSegmentsToConsiderPerMove = percentOfSegmentsToConsiderPerMove;
if (useBatchedSegmentSampler == null) {
this.useBatchedSegmentSampler = Builder.DEFAULT_USE_BATCHED_SEGMENT_SAMPLER;
} else {
this.useBatchedSegmentSampler = useBatchedSegmentSampler;
}
this.useBatchedSegmentSampler = Builder.valueOrDefault(
useBatchedSegmentSampler,
Defaults.USE_BATCHED_SEGMENT_SAMPLER
);
this.replicantLifetime = replicantLifetime;
this.replicationThrottleLimit = replicationThrottleLimit;
this.balancerComputeThreads = Math.max(balancerComputeThreads, 1);
this.emitBalancingStats = emitBalancingStats;
this.specificDataSourcesToKillUnusedSegmentsIn = parseJsonStringOrArray(specificDataSourcesToKillUnusedSegmentsIn);
this.dataSourcesToNotKillStalePendingSegmentsIn =
parseJsonStringOrArray(dataSourcesToNotKillStalePendingSegmentsIn);
this.maxSegmentsInNodeLoadingQueue = maxSegmentsInNodeLoadingQueue == null
? Builder.DEFAULT_MAX_SEGMENTS_IN_NODE_LOADING_QUEUE
: maxSegmentsInNodeLoadingQueue;
this.specificDataSourcesToKillUnusedSegmentsIn
= parseJsonStringOrArray(specificDataSourcesToKillUnusedSegmentsIn);
this.dataSourcesToNotKillStalePendingSegmentsIn
= parseJsonStringOrArray(dataSourcesToNotKillStalePendingSegmentsIn);
this.maxSegmentsInNodeLoadingQueue = Builder.valueOrDefault(
maxSegmentsInNodeLoadingQueue,
Defaults.MAX_SEGMENTS_IN_NODE_LOADING_QUEUE
);
this.decommissioningNodes = parseJsonStringOrArray(decommissioningNodes);
Preconditions.checkArgument(
decommissioningMaxPercentOfMaxSegmentsToMove >= 0 && decommissioningMaxPercentOfMaxSegmentsToMove <= 100,
"decommissioningMaxPercentOfMaxSegmentsToMove should be in range [0, 100]"
"'decommissioningMaxPercentOfMaxSegmentsToMove' should be in range [0, 100]"
);
this.decommissioningMaxPercentOfMaxSegmentsToMove = decommissioningMaxPercentOfMaxSegmentsToMove;
@ -194,9 +210,9 @@ public class 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,
@ -204,11 +220,29 @@ public class CoordinatorDynamicConfig
);
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;
}
for (Dimension dimension : Dimension.values()) {
final String dimensionValue = debugDimensions.get(dimension.reportedName());
if (dimensionValue != null) {
validDebugDimensions.put(dimension, dimensionValue);
}
}
return validDebugDimensions;
}
private static Set<String> parseJsonStringOrArray(Object jsonStringOrArray)
@ -224,7 +258,7 @@ public class CoordinatorDynamicConfig
}
return result;
} else if (jsonStringOrArray instanceof Collection) {
return ImmutableSet.copyOf(((Collection) jsonStringOrArray));
return ImmutableSet.copyOf((Collection) jsonStringOrArray);
} else {
return ImmutableSet.of();
}
@ -246,9 +280,9 @@ public class CoordinatorDynamicConfig
}
@JsonProperty("millisToWaitBeforeDeleting")
public long getLeadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments()
public long getMarkSegmentAsUnusedDelayMillis()
{
return leadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments;
return markSegmentAsUnusedDelayMillis;
}
@JsonProperty
@ -337,6 +371,12 @@ public class CoordinatorDynamicConfig
return useRoundRobinSegmentAssignment;
}
@JsonProperty
public boolean isSmartSegmentLoading()
{
return smartSegmentLoading;
}
/**
* List of historical servers to 'decommission'. Coordinator will not assign new segments to 'decommissioning'
* servers, and segments will be moved away from them to be placed on non-decommissioning servers at the maximum rate
@ -348,7 +388,18 @@ public class CoordinatorDynamicConfig
public Set<String> getDecommissioningNodes()
{
return decommissioningNodes;
}
@JsonProperty
public Map<String, String> getDebugDimensions()
{
return debugDimensions;
}
@JsonIgnore
public Map<Dimension, String> getValidatedDebugDimensions()
{
return validDebugDimensions;
}
/**
@ -365,6 +416,7 @@ public class CoordinatorDynamicConfig
*/
@Min(0)
@Max(100)
@Deprecated
@JsonProperty
public int getDecommissioningMaxPercentOfMaxSegmentsToMove()
{
@ -384,6 +436,7 @@ public class CoordinatorDynamicConfig
}
@Min(0)
@Deprecated
@JsonProperty
public int getMaxNonPrimaryReplicantsToLoad()
{
@ -395,7 +448,7 @@ public class CoordinatorDynamicConfig
{
return "CoordinatorDynamicConfig{" +
"leadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments="
+ leadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments +
+ markSegmentAsUnusedDelayMillis +
", mergeBytesLimit=" + mergeBytesLimit +
", mergeSegmentsLimit=" + mergeSegmentsLimit +
", maxSegmentsToMove=" + maxSegmentsToMove +
@ -428,66 +481,37 @@ public class CoordinatorDynamicConfig
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)
&& Objects.equals(debugDimensions, that.debugDimensions);
}
@Override
public int hashCode()
{
return Objects.hash(
leadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments,
markSegmentAsUnusedDelayMillis,
mergeBytesLimit,
mergeSegmentsLimit,
maxSegmentsToMove,
@ -503,7 +527,8 @@ public class CoordinatorDynamicConfig
decommissioningNodes,
decommissioningMaxPercentOfMaxSegmentsToMove,
pauseCoordination,
maxNonPrimaryReplicantsToLoad
maxNonPrimaryReplicantsToLoad,
debugDimensions
);
}
@ -512,27 +537,30 @@ public class CoordinatorDynamicConfig
return new Builder();
}
private static class Defaults
{
static final long LEADING_MILLIS_BEFORE_MARK_UNUSED = TimeUnit.MINUTES.toMillis(15);
static final long MERGE_BYTES_LIMIT = 524_288_000L;
static final int MERGE_SEGMENTS_LIMIT = 100;
static final int MAX_SEGMENTS_TO_MOVE = 100;
static final double PERCENT_OF_SEGMENTS_TO_CONSIDER_PER_MOVE = 100;
static final int REPLICANT_LIFETIME = 15;
static final int REPLICATION_THROTTLE_LIMIT = 500;
static final int BALANCER_COMPUTE_THREADS = 1;
static final boolean EMIT_BALANCING_STATS = false;
static final boolean USE_BATCHED_SEGMENT_SAMPLER = true;
static final int MAX_SEGMENTS_IN_NODE_LOADING_QUEUE = 500;
static final int DECOMMISSIONING_MAX_SEGMENTS_TO_MOVE_PERCENT = 70;
static final boolean PAUSE_COORDINATION = false;
static final boolean REPLICATE_AFTER_LOAD_TIMEOUT = false;
static final int MAX_NON_PRIMARY_REPLICANTS_TO_LOAD = Integer.MAX_VALUE;
static final boolean USE_ROUND_ROBIN_ASSIGNMENT = true;
static final boolean SMART_SEGMENT_LOADING = true;
}
public static class Builder
{
private static final long DEFAULT_LEADING_TIME_MILLIS_BEFORE_CAN_MARK_AS_UNUSED_OVERSHADOWED_SEGMENTS =
TimeUnit.MINUTES.toMillis(15);
private static final long DEFAULT_MERGE_BYTES_LIMIT = 524_288_000L;
private static final int DEFAULT_MERGE_SEGMENTS_LIMIT = 100;
private static final int DEFAULT_MAX_SEGMENTS_TO_MOVE = 100;
private static final double DEFAULT_PERCENT_OF_SEGMENTS_TO_CONSIDER_PER_MOVE = 100;
private static final int DEFAULT_REPLICANT_LIFETIME = 15;
private static final int DEFAULT_REPLICATION_THROTTLE_LIMIT = 500;
private static final int DEFAULT_BALANCER_COMPUTE_THREADS = 1;
private static final boolean DEFAULT_EMIT_BALANCING_STATS = false;
private static final boolean DEFAULT_USE_BATCHED_SEGMENT_SAMPLER = true;
private static final int DEFAULT_MAX_SEGMENTS_IN_NODE_LOADING_QUEUE = 500;
private static final int DEFAULT_DECOMMISSIONING_MAX_SEGMENTS_TO_MOVE_PERCENT = 70;
private static final boolean DEFAULT_PAUSE_COORDINATION = false;
private static final boolean DEFAULT_REPLICATE_AFTER_LOAD_TIMEOUT = false;
private static final int DEFAULT_MAX_NON_PRIMARY_REPLICANTS_TO_LOAD = Integer.MAX_VALUE;
private static final boolean DEFAULT_USE_ROUND_ROBIN_ASSIGNMENT = true;
private Long leadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments;
private Long markSegmentAsUnusedDelayMillis;
private Long mergeBytesLimit;
private Integer mergeSegmentsLimit;
private Integer maxSegmentsToMove;
@ -546,11 +574,13 @@ public class CoordinatorDynamicConfig
private Object dataSourcesToNotKillStalePendingSegmentsIn;
private Integer maxSegmentsInNodeLoadingQueue;
private Object decommissioningNodes;
private Map<String, String> debugDimensions;
private Integer decommissioningMaxPercentOfMaxSegmentsToMove;
private Boolean pauseCoordination;
private Boolean replicateAfterLoadTimeout;
private Integer maxNonPrimaryReplicantsToLoad;
private Boolean useRoundRobinSegmentAssignment;
private Boolean smartSegmentLoading;
public Builder()
{
@ -558,12 +588,12 @@ public class CoordinatorDynamicConfig
@JsonCreator
public Builder(
@JsonProperty("millisToWaitBeforeDeleting")
@Nullable Long leadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments,
@JsonProperty("millisToWaitBeforeDeleting") @Nullable Long markSegmentAsUnusedDelayMillis,
@JsonProperty("mergeBytesLimit") @Nullable Long mergeBytesLimit,
@JsonProperty("mergeSegmentsLimit") @Nullable Integer mergeSegmentsLimit,
@JsonProperty("maxSegmentsToMove") @Nullable Integer maxSegmentsToMove,
@Deprecated @JsonProperty("percentOfSegmentsToConsiderPerMove") @Nullable Double percentOfSegmentsToConsiderPerMove,
@Deprecated @JsonProperty("percentOfSegmentsToConsiderPerMove")
@Nullable Double percentOfSegmentsToConsiderPerMove,
@Deprecated @JsonProperty("useBatchedSegmentSampler") Boolean useBatchedSegmentSampler,
@JsonProperty("replicantLifetime") @Nullable Integer replicantLifetime,
@JsonProperty("replicationThrottleLimit") @Nullable Integer replicationThrottleLimit,
@ -578,11 +608,12 @@ public class CoordinatorDynamicConfig
@JsonProperty("pauseCoordination") @Nullable Boolean pauseCoordination,
@JsonProperty("replicateAfterLoadTimeout") @Nullable Boolean replicateAfterLoadTimeout,
@JsonProperty("maxNonPrimaryReplicantsToLoad") @Nullable Integer maxNonPrimaryReplicantsToLoad,
@JsonProperty("useRoundRobinSegmentAssignment") @Nullable Boolean useRoundRobinSegmentAssignment
@JsonProperty("useRoundRobinSegmentAssignment") @Nullable Boolean useRoundRobinSegmentAssignment,
@JsonProperty("smartSegmentLoading") @Nullable Boolean smartSegmentLoading,
@JsonProperty("debugDimensions") @Nullable Map<String, String> debugDimensions
)
{
this.leadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments =
leadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments;
this.markSegmentAsUnusedDelayMillis = markSegmentAsUnusedDelayMillis;
this.mergeBytesLimit = mergeBytesLimit;
this.mergeSegmentsLimit = mergeSegmentsLimit;
this.maxSegmentsToMove = maxSegmentsToMove;
@ -601,11 +632,13 @@ public class CoordinatorDynamicConfig
this.replicateAfterLoadTimeout = replicateAfterLoadTimeout;
this.maxNonPrimaryReplicantsToLoad = maxNonPrimaryReplicantsToLoad;
this.useRoundRobinSegmentAssignment = useRoundRobinSegmentAssignment;
this.smartSegmentLoading = smartSegmentLoading;
this.debugDimensions = debugDimensions;
}
public Builder withLeadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments(long leadingTimeMillis)
public Builder withMarkSegmentAsUnusedDelayMillis(long leadingTimeMillis)
{
this.leadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments = leadingTimeMillis;
this.markSegmentAsUnusedDelayMillis = leadingTimeMillis;
return this;
}
@ -641,6 +674,12 @@ public class CoordinatorDynamicConfig
return this;
}
public Builder withSmartSegmentLoading(boolean smartSegmentLoading)
{
this.smartSegmentLoading = smartSegmentLoading;
return this;
}
public Builder withReplicantLifetime(int replicantLifetime)
{
this.replicantLifetime = replicantLifetime;
@ -653,6 +692,12 @@ public class CoordinatorDynamicConfig
return this;
}
public Builder withDebugDimensions(Map<String, String> debugDimensions)
{
this.debugDimensions = debugDimensions;
return this;
}
public Builder withBalancerComputeThreads(int balancerComputeThreads)
{
this.balancerComputeThreads = balancerComputeThreads;
@ -713,75 +758,78 @@ public class CoordinatorDynamicConfig
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(
markSegmentAsUnusedDelayMillis,
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),
valueOrDefault(smartSegmentLoading, Defaults.SMART_SEGMENT_LOADING),
debugDimensions
);
}
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(
markSegmentAsUnusedDelayMillis,
defaults.getMarkSegmentAsUnusedDelayMillis()
),
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()),
valueOrDefault(useRoundRobinSegmentAssignment, defaults.isUseRoundRobinSegmentAssignment()),
valueOrDefault(smartSegmentLoading, defaults.isSmartSegmentLoading()),
valueOrDefault(debugDimensions, defaults.getDebugDimensions())
);
}
}

View File

@ -1,226 +0,0 @@
/*
* 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 it.unimi.dsi.fastutil.objects.Object2LongMap;
import it.unimi.dsi.fastutil.objects.Object2LongMap.Entry;
import it.unimi.dsi.fastutil.objects.Object2LongOpenHashMap;
import java.util.Collections;
import java.util.HashMap;
import java.util.Map;
import java.util.Set;
import java.util.function.ObjLongConsumer;
/**
*/
public class CoordinatorStats
{
private final Map<String, Object2LongOpenHashMap<String>> perTierStats;
private final Map<String, Object2LongOpenHashMap<String>> perDataSourceStats;
private final Map<String, Object2LongOpenHashMap<String>> perDutyStats;
private final Object2LongOpenHashMap<String> globalStats;
public CoordinatorStats()
{
perTierStats = new HashMap<>();
perDataSourceStats = new HashMap<>();
perDutyStats = new HashMap<>();
globalStats = new Object2LongOpenHashMap<>();
}
public boolean hasPerTierStats()
{
return !perTierStats.isEmpty();
}
public boolean hasPerDataSourceStats()
{
return !perDataSourceStats.isEmpty();
}
public boolean hasPerDutyStats()
{
return !perDutyStats.isEmpty();
}
public Set<String> getTiers(final String statName)
{
final Object2LongOpenHashMap<String> theStat = perTierStats.get(statName);
if (theStat == null) {
return Collections.emptySet();
}
return Collections.unmodifiableSet(theStat.keySet());
}
public Set<String> getDataSources(String statName)
{
final Object2LongOpenHashMap<String> stat = perDataSourceStats.get(statName);
if (stat == null) {
return Collections.emptySet();
}
return Collections.unmodifiableSet(stat.keySet());
}
public Set<String> getDuties(String statName)
{
final Object2LongOpenHashMap<String> stat = perDutyStats.get(statName);
if (stat == null) {
return Collections.emptySet();
}
return Collections.unmodifiableSet(stat.keySet());
}
/**
*
* @param statName the name of the statistics
* @param tier the tier
* @return the value for the statistics {@code statName} under {@code tier} tier
* @throws NullPointerException if {@code statName} is not found
*/
public long getTieredStat(final String statName, final String tier)
{
return perTierStats.get(statName).getLong(tier);
}
public void forEachTieredStat(final String statName, final ObjLongConsumer<String> consumer)
{
final Object2LongOpenHashMap<String> theStat = perTierStats.get(statName);
if (theStat != null) {
for (final Object2LongMap.Entry<String> entry : theStat.object2LongEntrySet()) {
consumer.accept(entry.getKey(), entry.getLongValue());
}
}
}
public long getDataSourceStat(String statName, String dataSource)
{
return perDataSourceStats.get(statName).getLong(dataSource);
}
public void forEachDataSourceStat(String statName, ObjLongConsumer<String> consumer)
{
final Object2LongOpenHashMap<String> stat = perDataSourceStats.get(statName);
if (stat != null) {
for (Entry<String> entry : stat.object2LongEntrySet()) {
consumer.accept(entry.getKey(), entry.getLongValue());
}
}
}
public long getDutyStat(String statName, String duty)
{
return perDutyStats.get(statName).getLong(duty);
}
public void forEachDutyStat(String statName, ObjLongConsumer<String> consumer)
{
final Object2LongOpenHashMap<String> stat = perDutyStats.get(statName);
if (stat != null) {
for (Entry<String> entry : stat.object2LongEntrySet()) {
consumer.accept(entry.getKey(), entry.getLongValue());
}
}
}
public long getGlobalStat(final String statName)
{
return globalStats.getLong(statName);
}
public void addToTieredStat(final String statName, final String tier, final long value)
{
perTierStats.computeIfAbsent(statName, ignored -> new Object2LongOpenHashMap<>())
.addTo(tier, value);
}
public void accumulateMaxTieredStat(final String statName, final String tier, final long value)
{
perTierStats.computeIfAbsent(statName, ignored -> new Object2LongOpenHashMap<>())
.mergeLong(tier, value, Math::max);
}
public void addToDataSourceStat(String statName, String dataSource, long value)
{
perDataSourceStats.computeIfAbsent(statName, k -> new Object2LongOpenHashMap<>())
.addTo(dataSource, value);
}
public void addToDutyStat(String statName, String duty, long value)
{
perDutyStats.computeIfAbsent(statName, k -> new Object2LongOpenHashMap<>())
.addTo(duty, value);
}
public void addToGlobalStat(final String statName, final long value)
{
globalStats.addTo(statName, value);
}
public CoordinatorStats accumulate(final CoordinatorStats stats)
{
stats.perTierStats.forEach(
(final String statName, final Object2LongOpenHashMap<String> urStat) -> {
final Object2LongOpenHashMap<String> myStat = perTierStats.computeIfAbsent(
statName,
name -> new Object2LongOpenHashMap<>()
);
for (final Object2LongMap.Entry<String> entry : urStat.object2LongEntrySet()) {
myStat.addTo(entry.getKey(), entry.getLongValue());
}
}
);
stats.perDataSourceStats.forEach(
(statName, urStat) -> {
final Object2LongOpenHashMap<String> myStat = perDataSourceStats.computeIfAbsent(
statName,
k -> new Object2LongOpenHashMap<>()
);
for (Entry<String> entry : urStat.object2LongEntrySet()) {
myStat.addTo(entry.getKey(), entry.getLongValue());
}
}
);
stats.perDutyStats.forEach(
(statName, urStat) -> {
final Object2LongOpenHashMap<String> myStat = perDutyStats.computeIfAbsent(
statName,
k -> new Object2LongOpenHashMap<>()
);
for (Entry<String> entry : urStat.object2LongEntrySet()) {
myStat.addTo(entry.getKey(), entry.getLongValue());
}
}
);
for (final Object2LongMap.Entry<String> entry : stats.globalStats.object2LongEntrySet()) {
globalStats.addTo(entry.getKey(), entry.getLongValue());
}
return this;
}
}

View File

@ -19,13 +19,11 @@
package org.apache.druid.server.coordinator;
import com.google.common.annotations.VisibleForTesting;
import org.apache.druid.client.ImmutableDruidServer;
import org.apache.druid.java.util.common.IAE;
import org.apache.druid.utils.CollectionUtils;
import javax.annotation.Nullable;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.Comparator;
@ -35,7 +33,6 @@ import java.util.List;
import java.util.Map;
import java.util.NavigableSet;
import java.util.Set;
import java.util.TreeSet;
/**
* Contains a representation of the current state of the cluster by tier.
@ -43,83 +40,24 @@ import java.util.TreeSet;
*/
public class DruidCluster
{
/** This static factory method must be called only from inside DruidClusterBuilder in tests. */
@VisibleForTesting
static DruidCluster createDruidClusterFromBuilderInTest(
@Nullable Set<ServerHolder> realtimes,
Map<String, Iterable<ServerHolder>> historicals,
@Nullable Set<ServerHolder> brokers
)
{
return new DruidCluster(realtimes, historicals, brokers);
}
public static final DruidCluster EMPTY = builder().build();
private final Set<ServerHolder> realtimes;
private final Map<String, NavigableSet<ServerHolder>> historicals;
private final Set<ServerHolder> brokers;
public DruidCluster()
{
this.realtimes = new HashSet<>();
this.historicals = new HashMap<>();
this.brokers = new HashSet<>();
}
private DruidCluster(
@Nullable Set<ServerHolder> realtimes,
Map<String, Iterable<ServerHolder>> historicals,
@Nullable Set<ServerHolder> brokers
Set<ServerHolder> realtimes,
Map<String, Set<ServerHolder>> historicals,
Set<ServerHolder> brokers
)
{
this.realtimes = realtimes == null ? new HashSet<>() : new HashSet<>(realtimes);
this.realtimes = Collections.unmodifiableSet(realtimes);
this.historicals = CollectionUtils.mapValues(
historicals,
holders -> CollectionUtils.newTreeSet(Comparator.reverseOrder(), holders)
holders -> CollectionUtils.newTreeSet(Comparator.naturalOrder(), holders)
);
this.brokers = brokers == null ? new HashSet<>() : new HashSet<>(brokers);
}
public void add(ServerHolder serverHolder)
{
switch (serverHolder.getServer().getType()) {
case HISTORICAL:
addHistorical(serverHolder);
break;
case REALTIME:
addRealtime(serverHolder);
break;
case BRIDGE:
addHistorical(serverHolder);
break;
case INDEXER_EXECUTOR:
addRealtime(serverHolder);
break;
case BROKER:
addBroker(serverHolder);
break;
default:
throw new IAE("unknown server type[%s]", serverHolder.getServer().getType());
}
}
private void addRealtime(ServerHolder serverHolder)
{
realtimes.add(serverHolder);
}
private void addHistorical(ServerHolder serverHolder)
{
final ImmutableDruidServer server = serverHolder.getServer();
final NavigableSet<ServerHolder> tierServers = historicals.computeIfAbsent(
server.getTier(),
k -> new TreeSet<>(Collections.reverseOrder())
);
tierServers.add(serverHolder);
}
private void addBroker(ServerHolder serverHolder)
{
brokers.add(serverHolder);
this.brokers = Collections.unmodifiableSet(brokers);
}
public Set<ServerHolder> getRealtimes()
@ -132,7 +70,6 @@ public class DruidCluster
return historicals;
}
public Set<ServerHolder> getBrokers()
{
return brokers;
@ -160,39 +97,71 @@ public class DruidCluster
return allServers;
}
public Iterable<NavigableSet<ServerHolder>> getSortedHistoricalsByTier()
{
return historicals.values();
}
public boolean isEmpty()
{
return historicals.isEmpty() && realtimes.isEmpty() && brokers.isEmpty();
}
public boolean hasHistoricals()
public static Builder builder()
{
return !historicals.isEmpty();
return new Builder();
}
public boolean hasRealtimes()
public static class Builder
{
return !realtimes.isEmpty();
}
private final Set<ServerHolder> realtimes = new HashSet<>();
private final Map<String, Set<ServerHolder>> historicals = new HashMap<>();
private final Set<ServerHolder> brokers = new HashSet<>();
public boolean hasBrokers()
{
return !brokers.isEmpty();
}
public boolean hasTier(String tier)
{
NavigableSet<ServerHolder> historicalServers = historicals.get(tier);
boolean historicalsHasTier = (historicalServers != null) && !historicalServers.isEmpty();
if (historicalsHasTier) {
return true;
public Builder add(ServerHolder serverHolder)
{
switch (serverHolder.getServer().getType()) {
case BRIDGE:
case HISTORICAL:
addHistorical(serverHolder);
break;
case REALTIME:
case INDEXER_EXECUTOR:
realtimes.add(serverHolder);
break;
case BROKER:
brokers.add(serverHolder);
break;
default:
throw new IAE("unknown server type[%s]", serverHolder.getServer().getType());
}
return this;
}
return false;
public Builder addRealtimes(ServerHolder... realtimeServers)
{
realtimes.addAll(Arrays.asList(realtimeServers));
return this;
}
public Builder addBrokers(ServerHolder... brokers)
{
this.brokers.addAll(Arrays.asList(brokers));
return this;
}
public Builder addTier(String tier, ServerHolder... historicals)
{
this.historicals.computeIfAbsent(tier, t -> new HashSet<>())
.addAll(Arrays.asList(historicals));
return this;
}
private void addHistorical(ServerHolder serverHolder)
{
final String tier = serverHolder.getServer().getTier();
historicals.computeIfAbsent(tier, t -> new HashSet<>()).add(serverHolder);
}
public DruidCluster build()
{
return new DruidCluster(realtimes, historicals, brokers);
}
}
}

View File

@ -25,18 +25,24 @@ import com.google.common.collect.ImmutableMap;
import org.apache.druid.client.DataSourcesSnapshot;
import org.apache.druid.java.util.emitter.service.ServiceEmitter;
import org.apache.druid.metadata.MetadataRuleManager;
import org.apache.druid.server.coordinator.balancer.BalancerStrategy;
import org.apache.druid.server.coordinator.loading.SegmentLoadQueueManager;
import org.apache.druid.server.coordinator.loading.SegmentLoadingConfig;
import org.apache.druid.server.coordinator.loading.SegmentReplicationStatus;
import org.apache.druid.server.coordinator.loading.StrategicSegmentAssigner;
import org.apache.druid.server.coordinator.stats.CoordinatorRunStats;
import org.apache.druid.server.coordinator.stats.Dimension;
import org.apache.druid.timeline.DataSegment;
import org.apache.druid.timeline.SegmentTimeline;
import org.joda.time.DateTime;
import javax.annotation.Nullable;
import java.util.Arrays;
import java.util.Collection;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Collections;
import java.util.Map;
import java.util.Set;
import java.util.TreeSet;
import java.util.concurrent.TimeUnit;
/**
*/
@ -56,60 +62,54 @@ public class DruidCoordinatorRuntimeParams
return segmentsSet;
}
private final long startTimeNanos;
private final DateTime coordinatorStartTime;
private final DruidCluster druidCluster;
private final MetadataRuleManager databaseRuleManager;
private final SegmentReplicantLookup segmentReplicantLookup;
private final StrategicSegmentAssigner segmentAssigner;
private final @Nullable TreeSet<DataSegment> usedSegments;
private final @Nullable DataSourcesSnapshot dataSourcesSnapshot;
private final Map<String, LoadQueuePeon> loadManagementPeons;
private final ReplicationThrottler replicationManager;
private final ServiceEmitter emitter;
private final CoordinatorDynamicConfig coordinatorDynamicConfig;
private final CoordinatorCompactionConfig coordinatorCompactionConfig;
private final CoordinatorStats stats;
private final SegmentLoadingConfig segmentLoadingConfig;
private final CoordinatorRunStats stats;
private final BalancerStrategy balancerStrategy;
private final Set<String> broadcastDatasources;
private final @Nullable RoundRobinServerSelector roundRobinServerSelector;
private DruidCoordinatorRuntimeParams(
long startTimeNanos,
DateTime coordinatorStartTime,
DruidCluster druidCluster,
MetadataRuleManager databaseRuleManager,
SegmentReplicantLookup segmentReplicantLookup,
StrategicSegmentAssigner segmentAssigner,
@Nullable TreeSet<DataSegment> usedSegments,
@Nullable DataSourcesSnapshot dataSourcesSnapshot,
Map<String, LoadQueuePeon> loadManagementPeons,
ReplicationThrottler replicationManager,
@Nullable RoundRobinServerSelector roundRobinServerSelector,
ServiceEmitter emitter,
CoordinatorDynamicConfig coordinatorDynamicConfig,
CoordinatorCompactionConfig coordinatorCompactionConfig,
CoordinatorStats stats,
SegmentLoadingConfig segmentLoadingConfig,
CoordinatorRunStats stats,
BalancerStrategy balancerStrategy,
Set<String> broadcastDatasources
)
{
this.startTimeNanos = startTimeNanos;
this.coordinatorStartTime = coordinatorStartTime;
this.druidCluster = druidCluster;
this.databaseRuleManager = databaseRuleManager;
this.segmentReplicantLookup = segmentReplicantLookup;
this.segmentAssigner = segmentAssigner;
this.usedSegments = usedSegments;
this.dataSourcesSnapshot = dataSourcesSnapshot;
this.loadManagementPeons = loadManagementPeons;
this.replicationManager = replicationManager;
this.roundRobinServerSelector = roundRobinServerSelector;
this.emitter = emitter;
this.coordinatorDynamicConfig = coordinatorDynamicConfig;
this.coordinatorCompactionConfig = coordinatorCompactionConfig;
this.segmentLoadingConfig = segmentLoadingConfig;
this.stats = stats;
this.balancerStrategy = balancerStrategy;
this.broadcastDatasources = broadcastDatasources;
}
public long getStartTimeNanos()
public DateTime getCoordinatorStartTime()
{
return startTimeNanos;
return coordinatorStartTime;
}
public DruidCluster getDruidCluster()
@ -122,15 +122,17 @@ public class DruidCoordinatorRuntimeParams
return databaseRuleManager;
}
public SegmentReplicantLookup getSegmentReplicantLookup()
@Nullable
public SegmentReplicationStatus getSegmentReplicationStatus()
{
return segmentReplicantLookup;
return segmentAssigner == null ? null : segmentAssigner.getReplicationStatus();
}
public StrategicSegmentAssigner getSegmentAssigner()
{
return segmentAssigner;
}
/**
* Creates and returns a "dataSource -> VersionedIntervalTimeline[version String, DataSegment]" map with "used"
* segments.
*/
public Map<String, SegmentTimeline> getUsedSegmentsTimelinesPerDataSource()
{
Preconditions.checkState(dataSourcesSnapshot != null, "dataSourcesSnapshot or usedSegments must be set");
@ -143,22 +145,6 @@ public class DruidCoordinatorRuntimeParams
return usedSegments;
}
public Map<String, LoadQueuePeon> getLoadManagementPeons()
{
return loadManagementPeons;
}
public ReplicationThrottler getReplicationManager()
{
return replicationManager;
}
@Nullable
public RoundRobinServerSelector getRoundRobinServerSelector()
{
return roundRobinServerSelector;
}
public ServiceEmitter getEmitter()
{
return emitter;
@ -174,7 +160,12 @@ public class DruidCoordinatorRuntimeParams
return coordinatorCompactionConfig;
}
public CoordinatorStats getCoordinatorStats()
public SegmentLoadingConfig getSegmentLoadingConfig()
{
return segmentLoadingConfig;
}
public CoordinatorRunStats getCoordinatorStats()
{
return stats;
}
@ -189,62 +180,30 @@ public class DruidCoordinatorRuntimeParams
return broadcastDatasources;
}
public boolean coordinatorIsLeadingEnoughTimeToMarkAsUnusedOvershadowedSegements()
{
long nanosElapsedSinceCoordinatorStart = System.nanoTime() - getStartTimeNanos();
long lagNanos = TimeUnit.MILLISECONDS.toNanos(
coordinatorDynamicConfig.getLeadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments()
);
return nanosElapsedSinceCoordinatorStart > lagNanos;
}
public DataSourcesSnapshot getDataSourcesSnapshot()
{
Preconditions.checkState(dataSourcesSnapshot != null, "usedSegments or dataSourcesSnapshot must be set");
return dataSourcesSnapshot;
}
public static Builder newBuilder()
public static Builder newBuilder(DateTime coordinatorStartTime)
{
return new Builder();
return new Builder(coordinatorStartTime);
}
public Builder buildFromExisting()
{
return new Builder(
startTimeNanos,
coordinatorStartTime,
druidCluster,
databaseRuleManager,
segmentReplicantLookup,
segmentAssigner,
usedSegments,
dataSourcesSnapshot,
loadManagementPeons,
replicationManager,
roundRobinServerSelector,
emitter,
coordinatorDynamicConfig,
coordinatorCompactionConfig,
stats,
balancerStrategy,
broadcastDatasources
);
}
public Builder buildFromExistingWithoutSegmentsMetadata()
{
return new Builder(
startTimeNanos,
druidCluster,
databaseRuleManager,
segmentReplicantLookup,
null, // usedSegments
null, // dataSourcesSnapshot
loadManagementPeons,
replicationManager,
roundRobinServerSelector,
emitter,
coordinatorDynamicConfig,
coordinatorCompactionConfig,
segmentLoadingConfig,
stats,
balancerStrategy,
broadcastDatasources
@ -253,70 +212,55 @@ public class DruidCoordinatorRuntimeParams
public static class Builder
{
private @Nullable Long startTimeNanos;
private final DateTime coordinatorStartTime;
private DruidCluster druidCluster;
private MetadataRuleManager databaseRuleManager;
private SegmentReplicantLookup segmentReplicantLookup;
private SegmentLoadQueueManager loadQueueManager;
private StrategicSegmentAssigner segmentAssigner;
private @Nullable TreeSet<DataSegment> usedSegments;
private @Nullable DataSourcesSnapshot dataSourcesSnapshot;
private final Map<String, LoadQueuePeon> loadManagementPeons;
private ReplicationThrottler replicationManager;
private @Nullable RoundRobinServerSelector roundRobinServerSelector;
private ServiceEmitter emitter;
private CoordinatorDynamicConfig coordinatorDynamicConfig;
private CoordinatorCompactionConfig coordinatorCompactionConfig;
private CoordinatorStats stats;
private SegmentLoadingConfig segmentLoadingConfig;
private CoordinatorRunStats stats;
private BalancerStrategy balancerStrategy;
private Set<String> broadcastDatasources;
private Builder()
private Builder(DateTime coordinatorStartTime)
{
this.startTimeNanos = null;
this.druidCluster = null;
this.databaseRuleManager = null;
this.segmentReplicantLookup = null;
this.usedSegments = null;
this.dataSourcesSnapshot = null;
this.loadManagementPeons = new HashMap<>();
this.replicationManager = null;
this.roundRobinServerSelector = null;
this.emitter = null;
this.stats = new CoordinatorStats();
this.coordinatorStartTime = coordinatorStartTime;
this.coordinatorDynamicConfig = CoordinatorDynamicConfig.builder().build();
this.coordinatorCompactionConfig = CoordinatorCompactionConfig.empty();
this.broadcastDatasources = new HashSet<>();
this.broadcastDatasources = Collections.emptySet();
}
Builder(
long startTimeNanos,
private Builder(
DateTime coordinatorStartTime,
DruidCluster cluster,
MetadataRuleManager databaseRuleManager,
SegmentReplicantLookup segmentReplicantLookup,
StrategicSegmentAssigner segmentAssigner,
@Nullable TreeSet<DataSegment> usedSegments,
@Nullable DataSourcesSnapshot dataSourcesSnapshot,
Map<String, LoadQueuePeon> loadManagementPeons,
ReplicationThrottler replicationManager,
@Nullable RoundRobinServerSelector roundRobinServerSelector,
ServiceEmitter emitter,
CoordinatorDynamicConfig coordinatorDynamicConfig,
CoordinatorCompactionConfig coordinatorCompactionConfig,
CoordinatorStats stats,
SegmentLoadingConfig segmentLoadingConfig,
CoordinatorRunStats stats,
BalancerStrategy balancerStrategy,
Set<String> broadcastDatasources
)
{
this.startTimeNanos = startTimeNanos;
this.coordinatorStartTime = coordinatorStartTime;
this.druidCluster = cluster;
this.databaseRuleManager = databaseRuleManager;
this.segmentReplicantLookup = segmentReplicantLookup;
this.segmentAssigner = segmentAssigner;
this.usedSegments = usedSegments;
this.dataSourcesSnapshot = dataSourcesSnapshot;
this.loadManagementPeons = loadManagementPeons;
this.replicationManager = replicationManager;
this.roundRobinServerSelector = roundRobinServerSelector;
this.emitter = emitter;
this.coordinatorDynamicConfig = coordinatorDynamicConfig;
this.coordinatorCompactionConfig = coordinatorCompactionConfig;
this.segmentLoadingConfig = segmentLoadingConfig;
this.stats = stats;
this.balancerStrategy = balancerStrategy;
this.broadcastDatasources = broadcastDatasources;
@ -324,42 +268,60 @@ public class DruidCoordinatorRuntimeParams
public DruidCoordinatorRuntimeParams build()
{
Preconditions.checkNotNull(startTimeNanos, "startTime must be set");
initStatsIfRequired();
initSegmentLoadingConfigIfRequired();
initSegmentAssignerIfRequired();
return new DruidCoordinatorRuntimeParams(
startTimeNanos,
coordinatorStartTime,
druidCluster,
databaseRuleManager,
segmentReplicantLookup,
segmentAssigner,
usedSegments,
dataSourcesSnapshot,
loadManagementPeons,
replicationManager,
getOrCreateRoundRobinServerSelector(),
emitter,
coordinatorDynamicConfig,
coordinatorCompactionConfig,
segmentLoadingConfig,
stats,
balancerStrategy,
broadcastDatasources
);
}
private RoundRobinServerSelector getOrCreateRoundRobinServerSelector()
private void initStatsIfRequired()
{
if (druidCluster == null || coordinatorDynamicConfig == null
|| !coordinatorDynamicConfig.isUseRoundRobinSegmentAssignment()) {
return null;
} else if (roundRobinServerSelector == null) {
return new RoundRobinServerSelector(druidCluster);
} else {
return roundRobinServerSelector;
Map<Dimension, String> debugDimensions =
coordinatorDynamicConfig == null ? null : coordinatorDynamicConfig.getValidatedDebugDimensions();
stats = stats == null ? new CoordinatorRunStats(debugDimensions) : stats;
}
private void initSegmentLoadingConfigIfRequired()
{
if (segmentLoadingConfig == null
&& coordinatorDynamicConfig != null
&& usedSegments != null) {
segmentLoadingConfig = SegmentLoadingConfig.create(coordinatorDynamicConfig, usedSegments.size());
}
}
public Builder withStartTimeNanos(long startTimeNanos)
private void initSegmentAssignerIfRequired()
{
this.startTimeNanos = startTimeNanos;
return this;
if (segmentAssigner != null || loadQueueManager == null) {
return;
}
Preconditions.checkNotNull(druidCluster);
Preconditions.checkNotNull(balancerStrategy);
Preconditions.checkNotNull(segmentLoadingConfig);
Preconditions.checkNotNull(stats);
segmentAssigner = new StrategicSegmentAssigner(
loadQueueManager,
druidCluster,
balancerStrategy,
segmentLoadingConfig,
stats
);
}
public Builder withDruidCluster(DruidCluster cluster)
@ -374,9 +336,13 @@ public class DruidCoordinatorRuntimeParams
return this;
}
public Builder withSegmentReplicantLookup(SegmentReplicantLookup lookup)
/**
* Sets the {@link SegmentLoadQueueManager} which is used to construct the
* {@link StrategicSegmentAssigner} for this run.
*/
public Builder withSegmentAssignerUsing(SegmentLoadQueueManager loadQueueManager)
{
this.segmentReplicantLookup = lookup;
this.loadQueueManager = loadQueueManager;
return this;
}
@ -403,50 +369,12 @@ public class DruidCoordinatorRuntimeParams
return this;
}
/** This method must be used in test code only. */
@VisibleForTesting
public Builder withUsedSegmentsTimelinesPerDataSourceInTest(
Map<String, SegmentTimeline> usedSegmentsTimelinesPerDataSource
)
{
this.dataSourcesSnapshot = DataSourcesSnapshot.fromUsedSegmentsTimelines(
usedSegmentsTimelinesPerDataSource,
ImmutableMap.of()
);
usedSegments = createUsedSegmentsSet(dataSourcesSnapshot.iterateAllUsedSegmentsInSnapshot());
return this;
}
public Builder withLoadManagementPeons(Map<String, LoadQueuePeon> loadManagementPeonsCollection)
{
loadManagementPeons.putAll(loadManagementPeonsCollection);
return this;
}
public Builder withReplicationManager(ReplicationThrottler replicationManager)
{
this.replicationManager = replicationManager;
return this;
}
public Builder withRoundRobinServerSelector(RoundRobinServerSelector roundRobinServerSelector)
{
this.roundRobinServerSelector = roundRobinServerSelector;
return this;
}
public Builder withEmitter(ServiceEmitter emitter)
{
this.emitter = emitter;
return this;
}
public Builder withCoordinatorStats(CoordinatorStats stats)
{
this.stats.accumulate(stats);
return this;
}
public Builder withDynamicConfigs(CoordinatorDynamicConfig configs)
{
this.coordinatorDynamicConfig = configs;

View File

@ -1,182 +0,0 @@
/*
* 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.java.util.emitter.EmittingLogger;
import org.apache.druid.timeline.SegmentId;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
/**
* The ReplicationThrottler is used to throttle the number of replicants that are created.
*/
public class ReplicationThrottler
{
private static final EmittingLogger log = new EmittingLogger(ReplicationThrottler.class);
private final Map<String, Boolean> replicatingLookup = new HashMap<>();
private final ReplicatorSegmentHolder currentlyReplicating = new ReplicatorSegmentHolder();
private volatile int maxReplicants;
private volatile int maxLifetime;
private volatile boolean loadPrimaryReplicantsOnly;
public ReplicationThrottler(int maxReplicants, int maxLifetime, boolean loadPrimaryReplicantsOnly)
{
updateParams(maxReplicants, maxLifetime, loadPrimaryReplicantsOnly);
}
public void updateParams(int maxReplicants, int maxLifetime, boolean loadPrimaryReplicantsOnly)
{
this.maxReplicants = maxReplicants;
this.maxLifetime = maxLifetime;
this.loadPrimaryReplicantsOnly = loadPrimaryReplicantsOnly;
}
public void updateReplicationState(String tier)
{
update(tier, currentlyReplicating, replicatingLookup, "create");
}
public boolean isLoadPrimaryReplicantsOnly()
{
return loadPrimaryReplicantsOnly;
}
public void setLoadPrimaryReplicantsOnly(boolean loadPrimaryReplicantsOnly)
{
this.loadPrimaryReplicantsOnly = loadPrimaryReplicantsOnly;
}
private void update(String tier, ReplicatorSegmentHolder holder, Map<String, Boolean> lookup, String type)
{
int size = holder.getNumProcessing(tier);
if (size != 0) {
log.info(
"[%s]: Replicant %s queue still has %d segments. Lifetime[%d]. Segments %s",
tier,
type,
size,
holder.getLifetime(tier),
holder.getCurrentlyProcessingSegmentsAndHosts(tier)
);
holder.reduceLifetime(tier);
lookup.put(tier, false);
if (holder.getLifetime(tier) < 0) {
log.makeAlert("[%s]: Replicant %s queue stuck after %d+ runs!", tier, type, maxLifetime)
.addData("segments", holder.getCurrentlyProcessingSegmentsAndHosts(tier))
.emit();
}
} else {
log.info("[%s]: Replicant %s queue is empty.", tier, type);
lookup.put(tier, true);
holder.resetLifetime(tier);
}
}
public boolean canCreateReplicant(String tier)
{
return !loadPrimaryReplicantsOnly && replicatingLookup.get(tier) && !currentlyReplicating.isAtMaxReplicants(tier);
}
public void registerReplicantCreation(String tier, SegmentId segmentId, String serverId)
{
currentlyReplicating.addSegment(tier, segmentId, serverId);
}
public void unregisterReplicantCreation(String tier, SegmentId segmentId)
{
currentlyReplicating.removeSegment(tier, segmentId);
}
private class ReplicatorSegmentHolder
{
private final Map<String, ConcurrentHashMap<SegmentId, String>> currentlyProcessingSegments = new HashMap<>();
private final Map<String, Integer> lifetimes = new HashMap<>();
public boolean isAtMaxReplicants(String tier)
{
final ConcurrentHashMap<SegmentId, String> segments = currentlyProcessingSegments.get(tier);
return (segments != null && segments.size() >= maxReplicants);
}
public void addSegment(String tier, SegmentId segmentId, String serverId)
{
ConcurrentHashMap<SegmentId, String> segments =
currentlyProcessingSegments.computeIfAbsent(tier, t -> new ConcurrentHashMap<>());
if (!isAtMaxReplicants(tier)) {
segments.put(segmentId, serverId);
}
}
public void removeSegment(String tier, SegmentId segmentId)
{
ConcurrentMap<SegmentId, String> segments = currentlyProcessingSegments.get(tier);
if (segments != null) {
segments.remove(segmentId);
}
}
public int getNumProcessing(String tier)
{
ConcurrentMap<SegmentId, String> segments = currentlyProcessingSegments.get(tier);
return (segments == null) ? 0 : segments.size();
}
public int getLifetime(String tier)
{
Integer lifetime = lifetimes.putIfAbsent(tier, maxLifetime);
return lifetime != null ? lifetime : maxLifetime;
}
public void reduceLifetime(String tier)
{
lifetimes.compute(
tier,
(t, lifetime) -> {
if (lifetime == null) {
return maxLifetime - 1;
}
return lifetime - 1;
}
);
}
public void resetLifetime(String tier)
{
lifetimes.put(tier, maxLifetime);
}
public List<String> getCurrentlyProcessingSegmentsAndHosts(String tier)
{
ConcurrentMap<SegmentId, String> segments = currentlyProcessingSegments.get(tier);
List<String> segmentsAndHosts = new ArrayList<>();
segments.forEach((segmentId, serverId) -> segmentsAndHosts.add(segmentId + " ON " + serverId));
return segmentsAndHosts;
}
}
}

View File

@ -1,174 +0,0 @@
/*
* 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.HashBasedTable;
import com.google.common.collect.Table;
import it.unimi.dsi.fastutil.objects.Object2IntMap;
import it.unimi.dsi.fastutil.objects.Object2IntOpenHashMap;
import it.unimi.dsi.fastutil.objects.Object2LongMap;
import it.unimi.dsi.fastutil.objects.Object2LongOpenHashMap;
import org.apache.druid.client.ImmutableDruidServer;
import org.apache.druid.timeline.DataSegment;
import org.apache.druid.timeline.SegmentId;
import java.util.HashMap;
import java.util.Map;
import java.util.SortedSet;
/**
* A lookup for the number of replicants of a given segment for a certain tier.
*/
public class SegmentReplicantLookup
{
public static SegmentReplicantLookup make(DruidCluster cluster, boolean replicateAfterLoadTimeout)
{
final Table<SegmentId, String, Integer> segmentsInCluster = HashBasedTable.create();
/**
* For each tier, this stores the number of replicants for all the segments presently queued to load in {@link cluster}.
* Segments that have failed to load due to the load timeout may not be present in this table if {@link replicateAfterLoadTimeout} is true.
* This is to enable additional replication of the timed out segments for improved availability.
*/
final Table<SegmentId, String, Integer> loadingSegments = HashBasedTable.create();
for (SortedSet<ServerHolder> serversByType : cluster.getSortedHistoricalsByTier()) {
for (ServerHolder serverHolder : serversByType) {
ImmutableDruidServer server = serverHolder.getServer();
for (DataSegment segment : server.iterateAllSegments()) {
Integer numReplicants = segmentsInCluster.get(segment.getId(), server.getTier());
if (numReplicants == null) {
numReplicants = 0;
}
segmentsInCluster.put(segment.getId(), server.getTier(), numReplicants + 1);
}
// Also account for queued segments
for (DataSegment segment : serverHolder.getPeon().getSegmentsToLoad()) {
Integer numReplicants = loadingSegments.get(segment.getId(), server.getTier());
if (numReplicants == null) {
numReplicants = 0;
}
// Timed out segments need to be replicated in another server for faster availability.
// Therefore we skip incrementing numReplicants for timed out segments if replicateAfterLoadTimeout is enabled.
if (!replicateAfterLoadTimeout || !serverHolder.getPeon().getTimedOutSegments().contains(segment)) {
loadingSegments.put(segment.getId(), server.getTier(), numReplicants + 1);
}
}
}
}
return new SegmentReplicantLookup(segmentsInCluster, loadingSegments, cluster);
}
private final Table<SegmentId, String, Integer> segmentsInCluster;
private final Table<SegmentId, String, Integer> loadingSegments;
private final Map<SegmentId, Integer> segmentIdToReplicationFactor = new HashMap<>();
private final DruidCluster cluster;
private SegmentReplicantLookup(
Table<SegmentId, String, Integer> segmentsInCluster,
Table<SegmentId, String, Integer> loadingSegments,
DruidCluster cluster
)
{
this.segmentsInCluster = segmentsInCluster;
this.loadingSegments = loadingSegments;
this.cluster = cluster;
}
public Map<String, Integer> getClusterTiers(SegmentId segmentId)
{
Map<String, Integer> retVal = segmentsInCluster.row(segmentId);
return (retVal == null) ? new HashMap<>() : retVal;
}
int getLoadedReplicants(SegmentId segmentId)
{
Map<String, Integer> allTiers = segmentsInCluster.row(segmentId);
int retVal = 0;
for (Integer replicants : allTiers.values()) {
retVal += replicants;
}
return retVal;
}
public int getLoadedReplicants(SegmentId segmentId, String tier)
{
Integer retVal = segmentsInCluster.get(segmentId, tier);
return (retVal == null) ? 0 : retVal;
}
// TODO: Refactor this setter, as this class is following a singleton pattern with only getters, and this breaks convention.
// This would be revamped in https://github.com/apache/druid/pull/13197
public void setReplicationFactor(SegmentId segmentId, Integer requiredReplicas)
{
segmentIdToReplicationFactor.put(segmentId, requiredReplicas);
}
public Object2IntMap<SegmentId> getSegmentIdToReplicationFactor()
{
return new Object2IntOpenHashMap<>(segmentIdToReplicationFactor);
}
private int getLoadingReplicants(SegmentId segmentId, String tier)
{
Integer retVal = loadingSegments.get(segmentId, tier);
return (retVal == null) ? 0 : retVal;
}
private int getLoadingReplicants(SegmentId segmentId)
{
Map<String, Integer> allTiers = loadingSegments.row(segmentId);
int retVal = 0;
for (Integer replicants : allTiers.values()) {
retVal += replicants;
}
return retVal;
}
public int getTotalReplicants(SegmentId segmentId)
{
return getLoadedReplicants(segmentId) + getLoadingReplicants(segmentId);
}
public int getTotalReplicants(SegmentId segmentId, String tier)
{
return getLoadedReplicants(segmentId, tier) + getLoadingReplicants(segmentId, tier);
}
public Object2LongMap<String> getBroadcastUnderReplication(SegmentId segmentId)
{
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);
}
}
}
return perTier;
}
}

View File

@ -20,43 +20,156 @@
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.java.util.emitter.EmittingLogger;
import org.apache.druid.server.coordination.ServerType;
import org.apache.druid.server.coordinator.loading.LoadQueuePeon;
import org.apache.druid.server.coordinator.loading.SegmentAction;
import org.apache.druid.server.coordinator.loading.SegmentHolder;
import org.apache.druid.timeline.DataSegment;
import org.apache.druid.timeline.SegmentId;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Comparator;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.Set;
import java.util.concurrent.atomic.AtomicInteger;
/**
* Encapsulates the state of a DruidServer during a single coordinator run.
* <p>
* ServerHolders are naturally ordered by available size, servers with more
* available size first.
*/
public class ServerHolder implements Comparable<ServerHolder>
{
private static final Logger log = new Logger(ServerHolder.class);
private static final Comparator<ServerHolder> MORE_AVAILABLE_SIZE_SERVER_FIRST =
Comparator.comparing(ServerHolder::getAvailableSize)
.thenComparing(holder -> holder.getServer().getHost())
.thenComparing(holder -> holder.getServer().getTier())
.thenComparing(holder -> holder.getServer().getType())
.reversed();
private static final EmittingLogger log = new EmittingLogger(ServerHolder.class);
private final ImmutableDruidServer server;
private final LoadQueuePeon peon;
private final boolean isDecommissioning;
private final int maxSegmentsInLoadQueue;
private final int maxAssignmentsInRun;
private final int maxLifetimeInQueue;
private final int movingSegmentCount;
private final int loadingReplicaCount;
private int totalAssignmentsInRun;
private long sizeOfLoadingSegments;
private long sizeOfDroppingSegments;
/**
* Remove entries from this map only if the operation is cancelled.
* Do not remove entries on load/drop success or failure during the run.
*/
private final Map<DataSegment, SegmentAction> queuedSegments = new HashMap<>();
/**
* Segments that are expected to be loaded on this server once all the
* operations in progress have completed.
*/
private final Set<DataSegment> projectedSegments = new HashSet<>();
public ServerHolder(ImmutableDruidServer server, LoadQueuePeon peon)
{
this(server, peon, false);
this(server, peon, false, 0, 1);
}
public ServerHolder(ImmutableDruidServer server, LoadQueuePeon peon, boolean isDecommissioning)
{
this(server, peon, isDecommissioning, 0);
this(server, peon, isDecommissioning, 0, 1);
}
/**
* Creates a new ServerHolder valid for a single coordinator run.
*
* @param server Underlying Druid server
* @param peon Load queue peon for this server
* @param isDecommissioning Whether the server is decommissioning
* @param maxSegmentsInLoadQueue Max number of segments that can be present in
* the load queue at any point. If this is 0, the
* load queue can have an unlimited number of segments.
* @param maxLifetimeInQueue Number of coordinator runs after a which a segment
* in load/drop queue is considered to be stuck.
*/
public ServerHolder(
ImmutableDruidServer server,
LoadQueuePeon peon,
boolean isDecommissioning,
int maxSegmentsInNodeLoadingQueue
int maxSegmentsInLoadQueue,
int maxLifetimeInQueue
)
{
this.server = server;
this.peon = peon;
this.isDecommissioning = isDecommissioning;
this.maxSegmentsInLoadQueue = maxSegmentsInNodeLoadingQueue;
this.maxAssignmentsInRun = maxSegmentsInLoadQueue == 0
? Integer.MAX_VALUE
: maxSegmentsInLoadQueue - peon.getSegmentsToLoad().size();
this.maxLifetimeInQueue = maxLifetimeInQueue;
final AtomicInteger movingSegmentCount = new AtomicInteger();
final AtomicInteger loadingReplicaCount = new AtomicInteger();
initializeQueuedSegments(movingSegmentCount, loadingReplicaCount);
this.movingSegmentCount = movingSegmentCount.get();
this.loadingReplicaCount = loadingReplicaCount.get();
}
private void initializeQueuedSegments(
AtomicInteger movingSegmentCount,
AtomicInteger loadingReplicaCount
)
{
projectedSegments.addAll(server.iterateAllSegments());
final List<SegmentHolder> expiredSegments = new ArrayList<>();
peon.getSegmentsInQueue().forEach(
(holder) -> {
int runsInQueue = holder.incrementAndGetRunsInQueue();
if (runsInQueue > maxLifetimeInQueue) {
expiredSegments.add(holder);
}
final SegmentAction action = holder.getAction();
addToQueuedSegments(holder.getSegment(), simplify(action));
if (action == SegmentAction.MOVE_TO) {
movingSegmentCount.incrementAndGet();
}
if (action == SegmentAction.REPLICATE) {
loadingReplicaCount.incrementAndGet();
}
}
);
peon.getSegmentsMarkedToDrop().forEach(
segment -> addToQueuedSegments(segment, SegmentAction.MOVE_FROM)
);
if (!expiredSegments.isEmpty()) {
List<SegmentHolder> expiredSegmentsSubList =
expiredSegments.size() > 10 ? expiredSegments.subList(0, 10) : expiredSegments;
log.makeAlert(
"Load queue for server [%s], tier [%s] has [%d] segments stuck.",
server.getName(), server.getTier(), expiredSegments.size()
)
.addData("segments", expiredSegmentsSubList.toString())
.addData("maxLifetime", maxLifetimeInQueue).emit();
}
}
public ImmutableDruidServer getServer()
@ -74,19 +187,9 @@ public class ServerHolder implements Comparable<ServerHolder>
return server.getMaxSize();
}
public long getCurrServerSize()
{
return server.getCurrSize();
}
public long getLoadQueueSize()
{
return peon.getLoadQueueSize();
}
public long getSizeUsed()
{
return getCurrServerSize() + getLoadQueueSize();
return server.getCurrSize() + sizeOfLoadingSegments - sizeOfDroppingSegments;
}
public double getPercentUsed()
@ -94,13 +197,6 @@ public class ServerHolder implements Comparable<ServerHolder>
return (100.0 * getSizeUsed()) / getMaxSize();
}
/**
* Historical nodes can be 'decommissioned', which instructs Coordinator to move segments from them according to
* the percent of move operations diverted from normal balancer moves for this purpose by
* {@link CoordinatorDynamicConfig#getDecommissioningMaxPercentOfMaxSegmentsToMove()}. The mechanism allows draining
* segments from nodes which are planned for replacement.
* @return true if the node is decommissioning
*/
public boolean isDecommissioning()
{
return isDecommissioning;
@ -108,46 +204,7 @@ public class ServerHolder implements Comparable<ServerHolder>
public long getAvailableSize()
{
long maxSize = getMaxSize();
long sizeUsed = getSizeUsed();
long availableSize = maxSize - sizeUsed;
log.debug(
"Server[%s], MaxSize[%,d], CurrSize[%,d], QueueSize[%,d], SizeUsed[%,d], AvailableSize[%,d]",
server.getName(),
maxSize,
getCurrServerSize(),
getLoadQueueSize(),
sizeUsed,
availableSize
);
return availableSize;
}
public boolean isServingSegment(DataSegment segment)
{
return isServingSegment(segment.getId());
}
public boolean isLoadingSegment(DataSegment segment)
{
return peon.getSegmentsToLoad().contains(segment);
}
public boolean isDroppingSegment(DataSegment segment)
{
return peon.getSegmentsToDrop().contains(segment);
}
public int getNumberOfSegmentsInQueue()
{
return peon.getNumberOfSegmentsInQueue();
}
public boolean isServingSegment(SegmentId segmentId)
{
return server.getSegment(segmentId) != null;
return getMaxSize() - getSizeUsed();
}
/**
@ -165,31 +222,177 @@ public class ServerHolder implements Comparable<ServerHolder>
public boolean canLoadSegment(DataSegment segment)
{
return !isDecommissioning
&& !isServingSegment(segment.getId())
&& !isLoadingSegment(segment)
&& (maxSegmentsInLoadQueue == 0 || maxSegmentsInLoadQueue > peon.getNumberOfSegmentsInQueue())
&& !hasSegmentLoaded(segment.getId())
&& getActionOnSegment(segment) == null
&& totalAssignmentsInRun < maxAssignmentsInRun
&& getAvailableSize() >= segment.getSize();
}
public SegmentAction getActionOnSegment(DataSegment segment)
{
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<DataSegment, SegmentAction> getQueuedSegments()
{
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;
}
/**
* Segments that are currently loaded on this server.
*/
public Collection<DataSegment> getServedSegments()
{
return server.iterateAllSegments();
}
/**
* Returns true if this server has the segment loaded and is not dropping it.
*/
public boolean isServingSegment(DataSegment segment)
{
return hasSegmentLoaded(segment.getId()) && getActionOnSegment(segment) == null;
}
public boolean isLoadingSegment(DataSegment segment)
{
return getActionOnSegment(segment) == SegmentAction.LOAD;
}
public boolean isDroppingSegment(DataSegment segment)
{
return getActionOnSegment(segment) == SegmentAction.DROP;
}
public int getNumMovingSegments()
{
return movingSegmentCount;
}
public int getNumLoadingReplicas()
{
return loadingReplicaCount;
}
public boolean startOperation(SegmentAction action, DataSegment segment)
{
if (queuedSegments.containsKey(segment)) {
return false;
}
if (action.isLoad()) {
++totalAssignmentsInRun;
}
addToQueuedSegments(segment, simplify(action));
return true;
}
public boolean cancelOperation(SegmentAction action, DataSegment segment)
{
// Cancel only if the action is currently in queue
final SegmentAction queuedAction = queuedSegments.get(segment);
if (queuedAction != simplify(action)) {
return false;
}
// Try cancelling the operation on the peon
// MOVE_FROM operations are not sent to the peon, so they can be considered cancelled
if (queuedAction == SegmentAction.MOVE_FROM || peon.cancelOperation(segment)) {
removeFromQueuedSegments(segment, queuedAction);
return true;
} else {
return false;
}
}
public boolean hasSegmentLoaded(SegmentId segmentId)
{
return server.getSegment(segmentId) != null;
}
public boolean isRealtimeServer()
{
return server.getType() == ServerType.REALTIME
|| server.getType() == ServerType.INDEXER_EXECUTOR;
}
private SegmentAction simplify(SegmentAction action)
{
return action == SegmentAction.REPLICATE ? SegmentAction.LOAD : action;
}
private void addToQueuedSegments(DataSegment segment, SegmentAction 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);
sizeOfLoadingSegments += segment.getSize();
} else {
projectedSegments.remove(segment);
if (action == SegmentAction.DROP) {
sizeOfDroppingSegments += segment.getSize();
}
// 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);
if (action.isLoad()) {
projectedSegments.remove(segment);
sizeOfLoadingSegments -= segment.getSize();
} else {
projectedSegments.add(segment);
if (action == SegmentAction.DROP) {
sizeOfDroppingSegments -= segment.getSize();
}
}
}
@Override
public int compareTo(ServerHolder serverHolder)
{
int result = Long.compare(getAvailableSize(), serverHolder.getAvailableSize());
if (result != 0) {
return result;
}
result = server.getHost().compareTo(serverHolder.server.getHost());
if (result != 0) {
return result;
}
result = server.getTier().compareTo(serverHolder.server.getTier());
if (result != 0) {
return result;
}
return server.getType().compareTo(serverHolder.server.getType());
return MORE_AVAILABLE_SIZE_SERVER_FIRST.compare(this, serverHolder);
}
@Override
@ -203,16 +406,9 @@ public class ServerHolder implements Comparable<ServerHolder>
}
ServerHolder that = (ServerHolder) o;
if (!this.server.getHost().equals(that.server.getHost())) {
return false;
}
if (!this.server.getTier().equals(that.getServer().getTier())) {
return false;
}
return this.server.getType().equals(that.getServer().getType());
return Objects.equals(server.getHost(), that.server.getHost())
&& Objects.equals(server.getTier(), that.server.getTier())
&& Objects.equals(server.getType(), that.server.getType());
}
@Override
@ -220,4 +416,10 @@ public class ServerHolder implements Comparable<ServerHolder>
{
return Objects.hash(server.getHost(), server.getTier(), server.getType());
}
@Override
public String toString()
{
return "ServerHolder{" + server.getHost() + "}";
}
}

View File

@ -17,33 +17,28 @@
* under the License.
*/
package org.apache.druid.server.coordinator;
package org.apache.druid.server.coordinator.balancer;
import org.apache.druid.client.ImmutableDruidServer;
import org.apache.druid.server.coordinator.ServerHolder;
import org.apache.druid.timeline.DataSegment;
/**
* Represents a segment picked for moving by a balancer strategy.
*/
public class BalancerSegmentHolder
{
private final ImmutableDruidServer fromServer;
private final ServerHolder server;
private final DataSegment segment;
// This is a pretty fugly hard coding of the maximum lifetime
private int lifetime = 15;
public BalancerSegmentHolder(
ImmutableDruidServer fromServer,
DataSegment segment
)
public BalancerSegmentHolder(ServerHolder server, DataSegment segment)
{
this.fromServer = fromServer;
this.server = server;
this.segment = segment;
}
public ImmutableDruidServer getFromServer()
public ServerHolder getServer()
{
return fromServer;
return server;
}
public DataSegment getSegment()
@ -51,13 +46,4 @@ public class BalancerSegmentHolder
return segment;
}
public int getLifetime()
{
return lifetime;
}
public void reduceLifetime()
{
lifetime--;
}
}

View File

@ -0,0 +1,89 @@
/*
* 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 org.apache.druid.server.coordinator.ServerHolder;
import org.apache.druid.server.coordinator.duty.BalanceSegments;
import org.apache.druid.server.coordinator.stats.CoordinatorRunStats;
import org.apache.druid.timeline.DataSegment;
import javax.annotation.Nullable;
import java.util.Iterator;
import java.util.List;
import java.util.NavigableSet;
/**
* This interface describes the coordinator balancing strategy, which is responsible for making decisions on where
* to place {@link DataSegment}s on historical servers (described by {@link ServerHolder}). The balancing strategy
* is used by {@link org.apache.druid.server.coordinator.rules.LoadRule} to assign and drop segments, and by
* {@link BalanceSegments} to migrate segments between historicals.
*/
public interface BalancerStrategy
{
/**
* 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 findDestinationServerToMoveSegment(
DataSegment proposalSegment,
ServerHolder sourceServer,
List<ServerHolder> destinationServers
);
/**
* 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 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> findServersToLoadSegment(
DataSegment proposalSegment,
List<ServerHolder> serverHolders
);
/**
* Returns an iterator for a set of servers to drop from, ordered by preference of which server to drop from first
* for a given drop strategy. One or more segments may be dropped, depending on how much the segment is
* over-replicated.
* @param toDropSegment segment to drop from one or more servers
* @param serverHolders set of historicals to consider dropping from
* @return Iterator for set of historicals, ordered by drop preference
*/
Iterator<ServerHolder> pickServersToDropSegment(DataSegment toDropSegment, NavigableSet<ServerHolder> serverHolders);
/**
* Add balancing strategy stats during the 'balanceTier' operation of
* {@link BalanceSegments} to be included
* @param tier historical tier being balanced
* @param stats stats object to add balancing strategy stats to
* @param serverHolderList servers in tier being balanced
*/
void emitStats(String tier, CoordinatorRunStats stats, List<ServerHolder> serverHolderList);
}

View File

@ -17,7 +17,7 @@
* under the License.
*/
package org.apache.druid.server.coordinator;
package org.apache.druid.server.coordinator.balancer;
import com.fasterxml.jackson.annotation.JsonSubTypes;
import com.fasterxml.jackson.annotation.JsonTypeInfo;
@ -25,10 +25,10 @@ import com.google.common.util.concurrent.ListeningExecutorService;
@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "strategy", defaultImpl = CostBalancerStrategyFactory.class)
@JsonSubTypes(value = {
@JsonSubTypes.Type(name = "diskNormalized", value = DiskNormalizedCostBalancerStrategyFactory.class),
@JsonSubTypes.Type(name = "cost", value = CostBalancerStrategyFactory.class),
@JsonSubTypes.Type(name = "cachingCost", value = CachingCostBalancerStrategyFactory.class),
@JsonSubTypes.Type(name = "random", value = RandomBalancerStrategyFactory.class),
@JsonSubTypes.Type(name = "cost", value = CostBalancerStrategyFactory.class),
@JsonSubTypes.Type(name = "cachingCost", value = CachingCostBalancerStrategyFactory.class),
@JsonSubTypes.Type(name = "diskNormalized", value = DiskNormalizedCostBalancerStrategyFactory.class),
@JsonSubTypes.Type(name = "random", value = RandomBalancerStrategyFactory.class)
})
public interface BalancerStrategyFactory
{

View File

@ -17,12 +17,12 @@
* under the License.
*/
package org.apache.druid.server.coordinator;
package org.apache.druid.server.coordinator.balancer;
import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableMap;
import com.google.common.util.concurrent.ListeningExecutorService;
import org.apache.druid.server.coordinator.cost.ClusterCostCache;
import org.apache.druid.server.coordinator.ServerHolder;
import org.apache.druid.timeline.DataSegment;
import java.util.Collections;
@ -43,15 +43,8 @@ public class CachingCostBalancerStrategy extends CostBalancerStrategy
@Override
protected double computeCost(DataSegment proposalSegment, ServerHolder server, boolean includeCurrentServer)
{
final long proposalSegmentSize = proposalSegment.getSize();
// (optional) Don't include server if it is already serving segment
if (!includeCurrentServer && server.isServingSegment(proposalSegment)) {
return Double.POSITIVE_INFINITY;
}
// Don't calculate cost if the server doesn't have enough space or is loading the segment
if (proposalSegmentSize > server.getAvailableSize() || server.isLoadingSegment(proposalSegment)) {
// (optional) Don't include server if it cannot load the segment
if (!includeCurrentServer && !server.canLoadSegment(proposalSegment)) {
return Double.POSITIVE_INFINITY;
}
@ -63,11 +56,15 @@ public class CachingCostBalancerStrategy extends CostBalancerStrategy
cost += costCacheForLoadingSegments(server).computeCost(serverName, proposalSegment);
// minus the cost of the segment itself
if (server.isServingSegment(proposalSegment)) {
if (server.isServingSegment(proposalSegment) || server.isLoadingSegment(proposalSegment)) {
cost -= costCacheForSegments(server, Collections.singleton(proposalSegment))
.computeCost(serverName, proposalSegment);
}
// minus the costs of segments that are being dropped
cost -= costCacheForSegments(server, server.getPeon().getSegmentsToDrop())
.computeCost(serverName, proposalSegment);
// minus the costs of segments that are marked to be dropped
cost -= costCacheForSegments(server, server.getPeon().getSegmentsMarkedToDrop())
.computeCost(serverName, proposalSegment);

View File

@ -17,7 +17,7 @@
* under the License.
*/
package org.apache.druid.server.coordinator;
package org.apache.druid.server.coordinator.balancer;
import com.fasterxml.jackson.annotation.JsonProperty;

View File

@ -17,7 +17,7 @@
* under the License.
*/
package org.apache.druid.server.coordinator;
package org.apache.druid.server.coordinator.balancer;
import com.fasterxml.jackson.annotation.JacksonInject;
import com.fasterxml.jackson.annotation.JsonCreator;
@ -30,7 +30,6 @@ import org.apache.druid.java.util.common.lifecycle.LifecycleStart;
import org.apache.druid.java.util.common.lifecycle.LifecycleStop;
import org.apache.druid.java.util.emitter.EmittingLogger;
import org.apache.druid.server.coordination.DruidServerMetadata;
import org.apache.druid.server.coordinator.cost.ClusterCostCache;
import org.apache.druid.timeline.DataSegment;
import java.util.concurrent.CancellationException;

View File

@ -17,7 +17,7 @@
* under the License.
*/
package org.apache.druid.server.coordinator.cost;
package org.apache.druid.server.coordinator.balancer;
import com.google.common.base.Preconditions;
import org.apache.druid.timeline.DataSegment;

View File

@ -17,9 +17,9 @@
* under the License.
*/
package org.apache.druid.server.coordinator;
package org.apache.druid.server.coordinator.balancer;
import com.google.common.collect.Iterables;
import com.google.common.collect.Lists;
import com.google.common.util.concurrent.Futures;
import com.google.common.util.concurrent.ListenableFuture;
import com.google.common.util.concurrent.ListeningExecutorService;
@ -27,19 +27,22 @@ import org.apache.commons.math3.util.FastMath;
import org.apache.druid.java.util.common.Pair;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.java.util.emitter.EmittingLogger;
import org.apache.druid.server.coordinator.ServerHolder;
import org.apache.druid.server.coordinator.stats.CoordinatorRunStats;
import org.apache.druid.server.coordinator.stats.RowKey;
import org.apache.druid.server.coordinator.stats.Stats;
import org.apache.druid.timeline.DataSegment;
import org.joda.time.Interval;
import java.util.ArrayList;
import java.util.Collections;
import java.util.Comparator;
import java.util.Iterator;
import java.util.List;
import java.util.NavigableSet;
import java.util.concurrent.ThreadLocalRandom;
import java.util.PriorityQueue;
import java.util.Set;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import java.util.stream.Collectors;
public class CostBalancerStrategy implements BalancerStrategy
{
@ -52,6 +55,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 -> pair.rhs);
/**
* This defines the unnormalized cost function between two segments.
*
@ -187,23 +198,28 @@ public class CostBalancerStrategy implements BalancerStrategy
}
@Override
public ServerHolder findNewSegmentHomeReplicator(DataSegment proposalSegment, List<ServerHolder> serverHolders)
public Iterator<ServerHolder> findServersToLoadSegment(
DataSegment proposalSegment,
List<ServerHolder> serverHolders
)
{
ServerHolder holder = chooseBestServer(proposalSegment, serverHolders, false).rhs;
if (holder != null && !holder.isServingSegment(proposalSegment)) {
return holder;
}
return null;
return getServersByPlacementCost(proposalSegment, serverHolders, false, "findServersToLoadSegment");
}
@Override
public ServerHolder findNewSegmentHomeBalancer(DataSegment proposalSegment, List<ServerHolder> serverHolders)
public ServerHolder findDestinationServerToMoveSegment(
DataSegment proposalSegment,
ServerHolder sourceServer,
List<ServerHolder> serverHolders
)
{
return chooseBestServer(proposalSegment, serverHolders, true).rhs;
Iterator<ServerHolder> servers =
getServersByPlacementCost(proposalSegment, serverHolders, true, "findServerToMoveSegment");
return servers.hasNext() ? servers.next() : null;
}
static double computeJointSegmentsCost(final DataSegment segment, final Iterable<DataSegment> segmentSet)
public static double computeJointSegmentsCost(DataSegment segment, Iterable<DataSegment> segmentSet)
{
double totalCost = 0;
for (DataSegment s : segmentSet) {
@ -213,34 +229,17 @@ public class CostBalancerStrategy implements BalancerStrategy
}
@Override
public Iterator<ServerHolder> pickServersToDrop(DataSegment toDrop, NavigableSet<ServerHolder> serverHolders)
public Iterator<ServerHolder> pickServersToDropSegment(
DataSegment segmentToDrop,
NavigableSet<ServerHolder> serverHolders
)
{
List<ListenableFuture<Pair<Double, ServerHolder>>> futures = new ArrayList<>();
List<ServerHolder> serversByCost = Lists.newArrayList(
getServersByPlacementCost(segmentToDrop, serverHolders, true, "pickServersToDropSegment")
);
for (final ServerHolder server : serverHolders) {
futures.add(
exec.submit(
() -> Pair.of(computeCost(toDrop, server, true), server)
)
);
}
final ListenableFuture<List<Pair<Double, ServerHolder>>> resultsFuture = Futures.allAsList(futures);
try {
// results is an un-ordered list of a pair consisting of the 'cost' of a segment being on a server and the server
List<Pair<Double, ServerHolder>> results = resultsFuture.get(1, TimeUnit.MINUTES);
return results.stream()
// Comparator.comapringDouble will order by lowest cost...
// reverse it because we want to drop from the highest cost servers first
.sorted(Comparator.comparingDouble((Pair<Double, ServerHolder> o) -> o.lhs).reversed())
.map(x -> x.rhs).collect(Collectors.toList())
.iterator();
}
catch (Exception e) {
alertOnFailure(e, "pick drop server");
}
return Collections.emptyIterator();
// Prioritize drop from highest cost servers
return Lists.reverse(serversByCost).iterator();
}
/**
@ -281,7 +280,7 @@ public class CostBalancerStrategy implements BalancerStrategy
{
double cost = 0;
for (ServerHolder server : serverHolders) {
for (DataSegment segment : server.getServer().iterateAllSegments()) {
for (DataSegment segment : server.getServedSegments()) {
cost += computeJointSegmentsCost(segment, segment);
}
}
@ -289,15 +288,16 @@ public class CostBalancerStrategy implements BalancerStrategy
}
@Override
public void emitStats(String tier, CoordinatorStats stats, List<ServerHolder> serverHolderList)
public void emitStats(String tier, CoordinatorRunStats stats, List<ServerHolder> serverHolderList)
{
final double initialTotalCost = calculateInitialTotalCost(serverHolderList);
final double normalization = calculateNormalization(serverHolderList);
final double normalizedInitialCost = initialTotalCost / normalization;
stats.addToTieredStat("initialCost", tier, (long) initialTotalCost);
stats.addToTieredStat("normalization", tier, (long) normalization);
stats.addToTieredStat("normalizedInitialCostTimesOneThousand", tier, (long) (normalizedInitialCost * 1000));
final RowKey rowKey = RowKey.forTier(tier);
stats.add(Stats.Balancer.RAW_COST, rowKey, (long) initialTotalCost);
stats.add(Stats.Balancer.NORMALIZATION_COST, rowKey, (long) normalization);
stats.add(Stats.Balancer.NORMALIZED_COST_X_1000, rowKey, (long) (normalizedInitialCost * 1000));
log.info(
"[%s]: Initial Total Cost: [%f], Normalization: [%f], Initial Normalized Cost: [%f]",
@ -311,57 +311,42 @@ public class CostBalancerStrategy implements BalancerStrategy
final boolean includeCurrentServer
)
{
final long proposalSegmentSize = proposalSegment.getSize();
// (optional) Don't include server if it is already serving segment
if (!includeCurrentServer && server.isServingSegment(proposalSegment)) {
return Double.POSITIVE_INFINITY;
}
// Don't calculate cost if the server doesn't have enough space or is loading the segment
if (proposalSegmentSize > server.getAvailableSize() || server.isLoadingSegment(proposalSegment)) {
// (optional) Don't include server if it cannot load the segment
if (!includeCurrentServer && !server.canLoadSegment(proposalSegment)) {
return Double.POSITIVE_INFINITY;
}
// The contribution to the total cost of a given server by proposing to move the segment to that server is...
double cost = 0d;
// the sum of the costs of other (exclusive of the proposalSegment) segments on the server
cost += computeJointSegmentsCost(
proposalSegment,
Iterables.filter(server.getServer().iterateAllSegments(), segment -> !proposalSegment.equals(segment))
);
// the sum of the costs of segments expected to be on the server (loaded + loading - dropping)
Set<DataSegment> projectedSegments = server.getProjectedSegments();
cost += computeJointSegmentsCost(proposalSegment, projectedSegments);
// plus the costs of segments that will be loaded
cost += computeJointSegmentsCost(proposalSegment, server.getPeon().getSegmentsToLoad());
// minus the costs of segments that are marked to be dropped
cost -= computeJointSegmentsCost(proposalSegment, server.getPeon().getSegmentsMarkedToDrop());
// minus the self cost of the segment
if (projectedSegments.contains(proposalSegment)) {
cost -= computeJointSegmentsCost(proposalSegment, proposalSegment);
}
return cost;
}
/**
* For assignment, we want to move to the lowest cost server that isn't already serving the segment.
* Returns an iterator over the servers, ordered by increasing cost for
* placing the given segment on that server.
*
* @param proposalSegment A DataSegment that we are proposing to move.
* @param serverHolders An iterable of ServerHolders for a particular tier.
*
* @return A ServerHolder with the new home for a segment.
* @param includeCurrentServer true if the server already serving a replica
* of this segment should be included in the results
*/
protected Pair<Double, ServerHolder> chooseBestServer(
final DataSegment proposalSegment,
final Iterable<ServerHolder> serverHolders,
final boolean includeCurrentServer
private Iterator<ServerHolder> getServersByPlacementCost(
DataSegment proposalSegment,
Iterable<ServerHolder> serverHolders,
boolean includeCurrentServer,
String action
)
{
final Pair<Double, ServerHolder> noServer = Pair.of(Double.POSITIVE_INFINITY, null);
Pair<Double, ServerHolder> bestServer = noServer;
List<ListenableFuture<Pair<Double, ServerHolder>>> futures = new ArrayList<>();
for (final ServerHolder server : serverHolders) {
final List<ListenableFuture<Pair<Double, ServerHolder>>> futures = new ArrayList<>();
for (ServerHolder server : serverHolders) {
futures.add(
exec.submit(
() -> Pair.of(computeCost(proposalSegment, server, includeCurrentServer), server)
@ -369,30 +354,23 @@ public class CostBalancerStrategy implements BalancerStrategy
);
}
final ListenableFuture<List<Pair<Double, ServerHolder>>> resultsFuture = Futures.allAsList(futures);
final List<Pair<Double, ServerHolder>> bestServers = new ArrayList<>();
bestServers.add(bestServer);
final PriorityQueue<Pair<Double, ServerHolder>> costPrioritizedServers =
new PriorityQueue<>(CHEAPEST_SERVERS_FIRST);
try {
for (Pair<Double, ServerHolder> server : resultsFuture.get(1, TimeUnit.MINUTES)) {
if (server.lhs <= bestServers.get(0).lhs) {
if (server.lhs < bestServers.get(0).lhs) {
bestServers.clear();
}
bestServers.add(server);
}
}
// If the best server list contains server whose cost of serving the segment is INFINITE then this means
// no usable servers are found so return a null server so that segment assignment does not happen
if (bestServers.get(0).lhs.isInfinite()) {
return noServer;
}
// Randomly choose a server from the best servers
bestServer = bestServers.get(ThreadLocalRandom.current().nextInt(bestServers.size()));
// 1 minute is the typical time for a full run of all historical management duties
// and is more than enough time for the cost computation of a single segment
costPrioritizedServers.addAll(
Futures.allAsList(futures).get(1, TimeUnit.MINUTES)
);
}
catch (Exception e) {
alertOnFailure(e, "choose best load server");
alertOnFailure(e, action);
}
return bestServer;
// Include current server only if specified
return costPrioritizedServers.stream()
.filter(pair -> includeCurrentServer || pair.rhs.canLoadSegment(proposalSegment))
.map(pair -> pair.rhs).iterator();
}
private void alertOnFailure(Exception e, String action)
@ -404,7 +382,6 @@ public class CostBalancerStrategy implements BalancerStrategy
}
final boolean hasTimedOut = e instanceof TimeoutException;
final String message = StringUtils.format(
"Cost balancer strategy %s in action [%s].%s",
hasTimedOut ? "timed out" : "failed", action,

View File

@ -17,7 +17,7 @@
* under the License.
*/
package org.apache.druid.server.coordinator;
package org.apache.druid.server.coordinator.balancer;
import com.google.common.util.concurrent.ListeningExecutorService;

View File

@ -17,9 +17,10 @@
* under the License.
*/
package org.apache.druid.server.coordinator;
package org.apache.druid.server.coordinator.balancer;
import com.google.common.util.concurrent.ListeningExecutorService;
import org.apache.druid.server.coordinator.ServerHolder;
import org.apache.druid.timeline.DataSegment;
public class DiskNormalizedCostBalancerStrategy extends CostBalancerStrategy

View File

@ -17,7 +17,7 @@
* under the License.
*/
package org.apache.druid.server.coordinator;
package org.apache.druid.server.coordinator.balancer;
import com.google.common.util.concurrent.ListeningExecutorService;

View File

@ -17,8 +17,10 @@
* under the License.
*/
package org.apache.druid.server.coordinator;
package org.apache.druid.server.coordinator.balancer;
import org.apache.druid.server.coordinator.ServerHolder;
import org.apache.druid.server.coordinator.stats.CoordinatorRunStats;
import org.apache.druid.timeline.DataSegment;
import java.util.ArrayList;
@ -26,34 +28,45 @@ import java.util.Collections;
import java.util.Iterator;
import java.util.List;
import java.util.NavigableSet;
import java.util.concurrent.ThreadLocalRandom;
import java.util.stream.Collectors;
/**
* A simple {@link BalancerStrategy} that
* <ul>
* <li>assigns segments randomly amongst eligible servers</li>
* <li>performs no balancing</li>
* </ul>
*/
public class RandomBalancerStrategy implements BalancerStrategy
{
@Override
public ServerHolder findNewSegmentHomeReplicator(DataSegment proposalSegment, List<ServerHolder> serverHolders)
public Iterator<ServerHolder> findServersToLoadSegment(
DataSegment proposalSegment,
List<ServerHolder> serverHolders
)
{
// filter out servers whose avaialable size is less than required for this segment and those already serving this segment
final List<ServerHolder> usableServerHolders = serverHolders.stream().filter(
serverHolder -> serverHolder.getAvailableSize() >= proposalSegment.getSize() && !serverHolder.isServingSegment(
proposalSegment)
).collect(Collectors.toList());
if (usableServerHolders.size() == 0) {
return null;
} else {
return usableServerHolders.get(ThreadLocalRandom.current().nextInt(usableServerHolders.size()));
}
// Filter out servers which cannot load this segment
final List<ServerHolder> usableServerHolders =
serverHolders.stream()
.filter(server -> server.canLoadSegment(proposalSegment))
.collect(Collectors.toList());
Collections.shuffle(usableServerHolders);
return usableServerHolders.iterator();
}
@Override
public ServerHolder findNewSegmentHomeBalancer(DataSegment proposalSegment, List<ServerHolder> serverHolders)
public ServerHolder findDestinationServerToMoveSegment(
DataSegment proposalSegment,
ServerHolder sourceServer,
List<ServerHolder> serverHolders
)
{
return null; //To change body of implemented methods use File | Settings | File Templates.
// This strategy does not do any balancing
return null;
}
@Override
public Iterator<ServerHolder> pickServersToDrop(DataSegment toDropSegment, NavigableSet<ServerHolder> serverHolders)
public Iterator<ServerHolder> pickServersToDropSegment(DataSegment toDropSegment, NavigableSet<ServerHolder> serverHolders)
{
List<ServerHolder> serverList = new ArrayList<>(serverHolders);
Collections.shuffle(serverList);
@ -61,7 +74,7 @@ public class RandomBalancerStrategy implements BalancerStrategy
}
@Override
public void emitStats(String tier, CoordinatorStats stats, List<ServerHolder> serverHolderList)
public void emitStats(String tier, CoordinatorRunStats stats, List<ServerHolder> serverHolderList)
{
}
}

View File

@ -17,7 +17,7 @@
* under the License.
*/
package org.apache.druid.server.coordinator;
package org.apache.druid.server.coordinator.balancer;
import com.google.common.util.concurrent.ListeningExecutorService;

View File

@ -17,28 +17,50 @@
* under the License.
*/
package org.apache.druid.server.coordinator;
package org.apache.druid.server.coordinator.balancer;
import org.apache.druid.java.util.emitter.EmittingLogger;
import org.apache.druid.server.coordinator.ServerHolder;
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(
final List<ServerHolder> serverHolders,
Set<String> broadcastDatasources,
int k
/**
* Picks segments from 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 from which it was picked.
*/
public static List<BalancerSegmentHolder> pickMovableSegmentsFrom(
List<ServerHolder> serverHolders,
int maxSegmentsToPick,
Function<ServerHolder, Collection<DataSegment>> segmentProvider,
Set<String> broadcastDatasources
)
{
List<BalancerSegmentHolder> holders = new ArrayList<>(k);
if (maxSegmentsToPick == 0 || serverHolders.isEmpty()) {
return Collections.emptyList();
}
final List<BalancerSegmentHolder> pickedSegments = new ArrayList<>(maxSegmentsToPick);
int numSoFar = 0;
for (ServerHolder server : serverHolders) {
@ -47,25 +69,29 @@ final class ReservoirSegmentSampler
continue;
}
for (DataSegment segment : server.getServer().iterateAllSegments()) {
final Collection<DataSegment> movableSegments = segmentProvider.apply(server);
for (DataSegment segment : movableSegments) {
if (broadcastDatasources.contains(segment.getDataSource())) {
// we don't need to rebalance segments that were assigned via broadcast rules
continue;
} else if (server.getActionOnSegment(segment) == SegmentAction.MOVE_FROM) {
// Do not pick a segment which is already being moved
continue;
}
if (numSoFar < k) {
holders.add(new BalancerSegmentHolder(server.getServer(), segment));
if (numSoFar < maxSegmentsToPick) {
pickedSegments.add(new BalancerSegmentHolder(server, segment));
numSoFar++;
continue;
}
int randNum = ThreadLocalRandom.current().nextInt(numSoFar + 1);
if (randNum < k) {
holders.set(randNum, new BalancerSegmentHolder(server.getServer(), segment));
if (randNum < maxSegmentsToPick) {
pickedSegments.set(randNum, new BalancerSegmentHolder(server, segment));
}
numSoFar++;
}
}
return holders;
return pickedSegments;
}
/**
@ -150,7 +176,7 @@ final class ReservoirSegmentSampler
}
}
if (fromServerHolder != null) {
return new BalancerSegmentHolder(fromServerHolder.getServer(), proposalSegment);
return new BalancerSegmentHolder(fromServerHolder, proposalSegment);
} else {
return null;
}

View File

@ -17,7 +17,7 @@
* under the License.
*/
package org.apache.druid.server.coordinator.cost;
package org.apache.druid.server.coordinator.balancer;
import com.google.common.base.Preconditions;
import com.google.common.collect.Ordering;
@ -26,7 +26,6 @@ import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.common.Intervals;
import org.apache.druid.java.util.common.granularity.DurationGranularity;
import org.apache.druid.java.util.common.guava.Comparators;
import org.apache.druid.server.coordinator.CostBalancerStrategy;
import org.apache.druid.timeline.DataSegment;
import org.joda.time.Interval;

View File

@ -17,7 +17,7 @@
* under the License.
*/
package org.apache.druid.server.coordinator.cost;
package org.apache.druid.server.coordinator.balancer;
import com.google.common.base.Preconditions;
import org.apache.druid.timeline.DataSegment;

View File

@ -0,0 +1,237 @@
/*
* 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.DruidCoordinatorRuntimeParams;
import org.apache.druid.server.coordinator.ServerHolder;
import org.apache.druid.server.coordinator.loading.SegmentLoadingConfig;
import org.apache.druid.server.coordinator.loading.StrategicSegmentAssigner;
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 javax.annotation.Nullable;
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 SegmentLoadingConfig loadingConfig;
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.loadingConfig = params.getSegmentLoadingConfig();
this.totalMaxSegmentsToMove = loadingConfig.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 = loadingConfig.getPercentDecommSegmentsToMove();
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 (loadingConfig.isEmitBalancingStats()) {
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;
}
final Set<String> broadcastDatasources = params.getBroadcastDatasources();
// Always move loading segments first as it is a cheaper operation
List<BalancerSegmentHolder> pickedSegments = ReservoirSegmentSampler.pickMovableSegmentsFrom(
sourceServers,
maxSegmentsToMove,
ServerHolder::getLoadingSegments,
broadcastDatasources
);
int movedCount = moveSegmentsTo(destServers, pickedSegments, maxSegmentsToMove);
// Move loaded segments only if tier is not already busy moving segments
if (movingSegmentCount <= 0) {
maxSegmentsToMove -= movedCount;
pickedSegments = ReservoirSegmentSampler.pickMovableSegmentsFrom(
sourceServers,
maxSegmentsToMove,
server -> server.getServer().iterateAllSegments(),
broadcastDatasources
);
movedCount += moveSegmentsTo(destServers, pickedSegments, maxSegmentsToMove);
}
return movedCount;
}
private int moveSegmentsTo(
List<ServerHolder> destinationServers,
List<BalancerSegmentHolder> movableSegments,
int maxSegmentsToMove
)
{
int processed = 0;
int movedCount = 0;
final Iterator<BalancerSegmentHolder> segmentIterator = movableSegments.iterator();
while (segmentIterator.hasNext() && processed < maxSegmentsToMove) {
++processed;
final BalancerSegmentHolder segmentHolder = segmentIterator.next();
DataSegment segmentToMove = getLoadableSegment(segmentHolder.getSegment());
if (segmentToMove != null &&
segmentAssigner.moveSegment(segmentToMove, segmentHolder.getServer(), destinationServers)) {
++movedCount;
}
}
return movedCount;
}
/**
* Returns a DataSegment with the correct value of loadSpec (as obtained from
* metadata store). This method may return null if there is no snapshot available
* for the underlying datasource or if the segment is unused.
*/
@Nullable
private DataSegment getLoadableSegment(DataSegment segmentToMove)
{
if (!params.getUsedSegments().contains(segmentToMove)) {
markUnmoved("Segment is unused", segmentToMove);
return null;
}
ImmutableDruidDataSource datasource = params.getDataSourcesSnapshot()
.getDataSource(segmentToMove.getDataSource());
if (datasource == null) {
markUnmoved("Invalid datasource", segmentToMove);
return null;
}
DataSegment loadableSegment = datasource.getSegment(segmentToMove.getId());
if (loadableSegment == null) {
markUnmoved("Invalid segment ID", segmentToMove);
return null;
}
return loadableSegment;
}
private void markUnmoved(String reason, DataSegment segment)
{
final RowKey key
= RowKey.builder()
.add(Dimension.TIER, tier)
.add(Dimension.DATASOURCE, segment.getDataSource())
.add(Dimension.DESCRIPTION, reason)
.build();
runStats.add(Stats.Segments.MOVE_SKIPPED, key, 1);
}
}

View File

@ -18,6 +18,6 @@
*/
@ParametersAreNonnullByDefault
package org.apache.druid.server.coordinator.cost;
package org.apache.druid.server.coordinator.balancer;
import javax.annotation.ParametersAreNonnullByDefault;

View File

@ -19,293 +19,45 @@
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.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.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;
import org.apache.druid.server.coordinator.balancer.TierSegmentBalancer;
import org.apache.druid.server.coordinator.loading.SegmentLoadingConfig;
/**
*
*/
public class BalanceSegments implements CoordinatorDuty
{
protected static final EmittingLogger log = new EmittingLogger(BalanceSegments.class);
protected final DruidCoordinator coordinator;
protected final Map<String, ConcurrentHashMap<SegmentId, BalancerSegmentHolder>> currentlyMovingSegments =
new HashMap<>();
public BalanceSegments(DruidCoordinator coordinator)
{
this.coordinator = coordinator;
}
protected void reduceLifetimes(String tier)
{
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();
}
}
}
private static final EmittingLogger log = new EmittingLogger(BalanceSegments.class);
@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);
});
return params.buildFromExisting().withCoordinatorStats(stats).build();
}
private void balanceTier(
DruidCoordinatorRuntimeParams params,
String tier,
SortedSet<ServerHolder> servers,
CoordinatorStats stats
)
{
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;
if (params.getUsedSegments().isEmpty()) {
log.info("Skipping balance as there are no used segments.");
return params;
}
/*
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()
);
if ((decommissioningServers.isEmpty() && activeServers.size() <= 1) || activeServers.isEmpty()) {
log.warn("[%s]: insufficient active servers. Cannot balance.", tier);
// suppress emit zero stats
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
return;
}
final int maxSegmentsToMove = Math.min(params.getCoordinatorDynamicConfig().getMaxSegmentsToMove(), numSegments);
// Prioritize moving segments from decomissioning servers.
int decommissioningMaxPercentOfMaxSegmentsToMove =
params.getCoordinatorDynamicConfig().getDecommissioningMaxPercentOfMaxSegmentsToMove();
int maxSegmentsToMoveFromDecommissioningNodes =
(int) Math.ceil(maxSegmentsToMove * (decommissioningMaxPercentOfMaxSegmentsToMove / 100.0));
log.info(
"Processing %d segments for moving from decommissioning servers",
maxSegmentsToMoveFromDecommissioningNodes
);
Pair<Integer, Integer> decommissioningResult =
balanceServers(params, decommissioningServers, activeServers, maxSegmentsToMoveFromDecommissioningNodes);
// After moving segments from decomissioning servers, move the remaining segments from the rest of the servers.
int maxGeneralSegmentsToMove = maxSegmentsToMove - decommissioningResult.lhs;
log.info("Processing %d segments for balancing between active servers", maxGeneralSegmentsToMove);
Pair<Integer, Integer> generalResult =
balanceServers(params, activeServers, activeServers, maxGeneralSegmentsToMove);
int moved = generalResult.lhs + decommissioningResult.lhs;
int unmoved = generalResult.rhs + decommissioningResult.rhs;
if (unmoved == maxSegmentsToMove) {
// Cluster should be alive and constantly adjusting
log.info("No good moves found in tier [%s]", tier);
}
stats.addToTieredStat("unmovedCount", tier, unmoved);
stats.addToTieredStat("movedCount", tier, moved);
if (params.getCoordinatorDynamicConfig().emitBalancingStats()) {
final BalancerStrategy strategy = params.getBalancerStrategy();
strategy.emitStats(tier, stats, Lists.newArrayList(servers));
}
log.info("[%s]: Segments Moved: [%d] Segments Let Alone: [%d]", tier, moved, unmoved);
}
private Pair<Integer, Integer> balanceServers(
DruidCoordinatorRuntimeParams params,
List<ServerHolder> toMoveFrom,
List<ServerHolder> toMoveTo,
int maxSegmentsToMove
)
{
final DruidCluster cluster = params.getDruidCluster();
final SegmentLoadingConfig loadingConfig = params.getSegmentLoadingConfig();
final int maxSegmentsToMove = loadingConfig.getMaxSegmentsToMove();
if (maxSegmentsToMove <= 0) {
log.debug("maxSegmentsToMove is 0; no balancing work can be performed.");
return new Pair<>(0, 0);
} else if (toMoveFrom.isEmpty()) {
log.debug("toMoveFrom is empty; no balancing work can be performed.");
return new Pair<>(0, 0);
} else if (toMoveTo.isEmpty()) {
log.debug("toMoveTo is empty; no balancing work can be peformed.");
return new Pair<>(0, 0);
}
final BalancerStrategy strategy = params.getBalancerStrategy();
final int maxIterations = 2 * maxSegmentsToMove;
final int maxToLoad = params.getCoordinatorDynamicConfig().getMaxSegmentsInNodeLoadingQueue();
int moved = 0, unmoved = 0;
Iterator<BalancerSegmentHolder> segmentsToMove;
// The pick method depends on if the operator has enabled batched segment sampling in the Coorinator dynamic config.
if (params.getCoordinatorDynamicConfig().useBatchedSegmentSampler()) {
segmentsToMove = strategy.pickSegmentsToMove(
toMoveFrom,
params.getBroadcastDatasources(),
maxSegmentsToMove
);
log.info("Skipping balance as maxSegmentsToMove is [%d].", maxSegmentsToMove);
return params;
} else {
segmentsToMove = strategy.pickSegmentsToMove(
toMoveFrom,
params.getBroadcastDatasources(),
params.getCoordinatorDynamicConfig().getPercentOfSegmentsToConsiderPerMove()
log.info(
"Balancing segments in tiers [%s] with maxSegmentsToMove=[%d], maxLifetime=[%d].",
cluster.getTierNames(), maxSegmentsToMove, loadingConfig.getMaxLifetimeInLoadQueue()
);
}
//noinspection ForLoopThatDoesntUseLoopVariable
for (int iter = 0; (moved + unmoved) < maxSegmentsToMove; ++iter) {
if (!segmentsToMove.hasNext()) {
log.info("All servers to move segments from are empty, ending run.");
break;
}
final BalancerSegmentHolder segmentToMoveHolder = segmentsToMove.next();
cluster.getHistoricals().forEach(
(tier, servers) -> new TierSegmentBalancer(tier, servers, params).run()
);
// DruidCoordinatorRuntimeParams.getUsedSegments originate from SegmentsMetadataManager, i. e. that's a set of segments
// that *should* be loaded. segmentToMoveHolder.getSegment originates from ServerInventoryView, i. e. that may be
// any segment that happens to be loaded on some server, even if it is not used. (Coordinator closes such
// discrepancies eventually via UnloadUnusedSegments). Therefore the picked segmentToMoveHolder's segment may not
// need to be balanced.
boolean needToBalancePickedSegment = params.getUsedSegments().contains(segmentToMoveHolder.getSegment());
if (needToBalancePickedSegment) {
final DataSegment segmentToMove = segmentToMoveHolder.getSegment();
final ImmutableDruidServer fromServer = segmentToMoveHolder.getFromServer();
// we want to leave the server the segment is currently on in the list...
// but filter out replicas that are already serving the segment, and servers with a full load queue
final List<ServerHolder> toMoveToWithLoadQueueCapacityAndNotServingSegment =
toMoveTo.stream()
.filter(s -> s.getServer().equals(fromServer) ||
(!s.isServingSegment(segmentToMove) &&
(maxToLoad <= 0 || s.getNumberOfSegmentsInQueue() < maxToLoad)))
.collect(Collectors.toList());
if (toMoveToWithLoadQueueCapacityAndNotServingSegment.size() > 0) {
final ServerHolder destinationHolder =
strategy.findNewSegmentHomeBalancer(segmentToMove, toMoveToWithLoadQueueCapacityAndNotServingSegment);
if (destinationHolder != null && !destinationHolder.getServer().equals(fromServer)) {
if (moveSegment(segmentToMoveHolder, destinationHolder.getServer(), params)) {
moved++;
} else {
unmoved++;
}
} else {
log.debug("Segment [%s] is 'optimally' placed.", segmentToMove.getId());
unmoved++;
}
} else {
log.debug("No valid movement destinations for segment [%s].", segmentToMove.getId());
unmoved++;
}
}
if (iter >= maxIterations) {
log.info(
"Unable to select %d remaining candidate segments out of %d total to balance "
+ "after %d iterations, ending run.",
(maxSegmentsToMove - moved - unmoved),
maxSegmentsToMove,
iter
);
break;
}
}
return new Pair<>(moved, unmoved);
return params;
}
protected boolean moveSegment(
final BalancerSegmentHolder segment,
final ImmutableDruidServer toServer,
final DruidCoordinatorRuntimeParams params
)
{
final LoadQueuePeon toPeon = params.getLoadManagementPeons().get(toServer.getName());
final ImmutableDruidServer fromServer = segment.getFromServer();
final DataSegment segmentToMove = segment.getSegment();
final SegmentId segmentId = segmentToMove.getId();
if (!toPeon.getSegmentsToLoad().contains(segmentToMove) &&
(toServer.getSegment(segmentId) == null) &&
new ServerHolder(toServer, toPeon).getAvailableSize() > segmentToMove.getSize()) {
log.debug("Moving [%s] from [%s] to [%s]", segmentId, fromServer.getName(), toServer.getName());
ConcurrentMap<SegmentId, BalancerSegmentHolder> movingSegments =
currentlyMovingSegments.get(toServer.getTier());
movingSegments.put(segmentId, segment);
final LoadPeonCallback callback = moveSuccess -> movingSegments.remove(segmentId);
try {
coordinator
.moveSegment(params, fromServer, toServer, segmentToMove, callback);
return true;
}
catch (Exception e) {
log.makeAlert(e, "[%s] : Moving exception", segmentId).emit();
callback.execute(false);
}
}
return false;
}
}

View File

@ -0,0 +1,154 @@
/*
* 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.AtomicDouble;
import org.apache.druid.client.ImmutableDruidServer;
import org.apache.druid.java.util.common.logger.Logger;
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.ServerHolder;
import org.apache.druid.server.coordinator.loading.LoadQueuePeon;
import org.apache.druid.server.coordinator.loading.StrategicSegmentAssigner;
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 java.util.Map;
import java.util.Set;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong;
/**
* Collects stats pertaining to segment availability on different servers.
*/
public class CollectSegmentAndServerStats implements CoordinatorDuty
{
private static final Logger log = new Logger(CollectSegmentAndServerStats.class);
private final DruidCoordinator coordinator;
public CollectSegmentAndServerStats(DruidCoordinator coordinator)
{
this.coordinator = coordinator;
}
@Override
public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params)
{
params.getDruidCluster().getHistoricals()
.forEach(this::logHistoricalTierStats);
collectSegmentStats(params);
StrategicSegmentAssigner segmentAssigner = params.getSegmentAssigner();
segmentAssigner.makeAlerts();
return params;
}
private void collectSegmentStats(DruidCoordinatorRuntimeParams params)
{
final CoordinatorRunStats stats = params.getCoordinatorStats();
final DruidCluster cluster = params.getDruidCluster();
cluster.getHistoricals().forEach((tier, historicals) -> {
final RowKey rowKey = RowKey.forTier(tier);
stats.add(Stats.Tier.HISTORICAL_COUNT, rowKey, historicals.size());
long totalCapacity = historicals.stream().map(ServerHolder::getMaxSize).reduce(0L, Long::sum);
stats.add(Stats.Tier.TOTAL_CAPACITY, rowKey, totalCapacity);
});
// Collect load queue stats
coordinator.getLoadManagementPeons().forEach((serverName, queuePeon) -> {
final RowKey rowKey = RowKey.builder().add(Dimension.SERVER, serverName).build();
stats.add(Stats.SegmentQueue.BYTES_TO_LOAD, rowKey, queuePeon.getSizeOfSegmentsToLoad());
stats.add(Stats.SegmentQueue.NUM_TO_LOAD, rowKey, queuePeon.getSegmentsToLoad().size());
stats.add(Stats.SegmentQueue.NUM_TO_DROP, rowKey, queuePeon.getSegmentsToDrop().size());
queuePeon.getAndResetStats().forEachStat(
(dimValues, stat, statValue) ->
stats.add(stat, createRowKeyForServer(serverName, dimValues), statValue)
);
});
coordinator.getDatasourceToUnavailableSegmentCount().forEach(
(dataSource, numUnavailable) ->
stats.addToDatasourceStat(Stats.Segments.UNAVAILABLE, dataSource, numUnavailable)
);
coordinator.getTierToDatasourceToUnderReplicatedCount(false).forEach(
(tier, countsPerDatasource) -> countsPerDatasource.forEach(
(dataSource, underReplicatedCount) ->
stats.addToSegmentStat(Stats.Segments.UNDER_REPLICATED, tier, dataSource, underReplicatedCount)
)
);
// Collect total segment stats
params.getUsedSegmentsTimelinesPerDataSource().forEach(
(dataSource, timeline) -> {
long totalSizeOfUsedSegments = timeline.iterateAllObjects().stream()
.mapToLong(DataSegment::getSize).sum();
stats.addToDatasourceStat(Stats.Segments.USED_BYTES, dataSource, totalSizeOfUsedSegments);
stats.addToDatasourceStat(Stats.Segments.USED, dataSource, timeline.getNumObjects());
}
);
}
private RowKey createRowKeyForServer(String serverName, Map<Dimension, String> dimensionValues)
{
final RowKey.Builder builder = RowKey.builder();
dimensionValues.forEach(builder::add);
builder.add(Dimension.SERVER, serverName);
return builder.build();
}
private void logHistoricalTierStats(String tier, Set<ServerHolder> historicals)
{
final AtomicInteger servedCount = new AtomicInteger();
final AtomicInteger loadingCount = new AtomicInteger();
final AtomicInteger droppingCount = new AtomicInteger();
final AtomicDouble usageSum = new AtomicDouble();
final AtomicLong currentBytesSum = new AtomicLong();
historicals.forEach(serverHolder -> {
final ImmutableDruidServer server = serverHolder.getServer();
servedCount.addAndGet(server.getNumSegments());
currentBytesSum.addAndGet(server.getCurrSize());
usageSum.addAndGet(100.0f * server.getCurrSize() / server.getMaxSize());
final LoadQueuePeon queuePeon = serverHolder.getPeon();
loadingCount.addAndGet(queuePeon.getSegmentsToLoad().size());
droppingCount.addAndGet(queuePeon.getSegmentsToDrop().size());
});
final int numHistoricals = historicals.size();
log.info(
"Tier [%s] is serving [%,d], loading [%,d] and dropping [%,d] segments"
+ " across [%d] historicals with average usage [%d GBs], [%.1f%%].",
tier, servedCount.get(), loadingCount.get(), droppingCount.get(), numHistoricals,
(currentBytesSum.get() >> 30) / numHistoricals, usageSum.get() / numHistoricals
);
}
}

View File

@ -39,16 +39,19 @@ import org.apache.druid.java.util.common.logger.Logger;
import org.apache.druid.server.coordinator.AutoCompactionSnapshot;
import org.apache.druid.server.coordinator.CompactionStatistics;
import org.apache.druid.server.coordinator.CoordinatorCompactionConfig;
import org.apache.druid.server.coordinator.CoordinatorStats;
import org.apache.druid.server.coordinator.DataSourceCompactionConfig;
import org.apache.druid.server.coordinator.DruidCoordinatorConfig;
import org.apache.druid.server.coordinator.DruidCoordinatorRuntimeParams;
import org.apache.druid.server.coordinator.stats.CoordinatorRunStats;
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.SegmentTimeline;
import org.joda.time.Interval;
import javax.annotation.Nullable;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
@ -58,25 +61,9 @@ import java.util.stream.Collectors;
public class CompactSegments implements CoordinatorCustomDuty
{
static final String COMPACTION_TASK_COUNT = "compactTaskCount";
static final String AVAILABLE_COMPACTION_TASK_SLOT = "availableCompactionTaskSlot";
static final String MAX_COMPACTION_TASK_SLOT = "maxCompactionTaskSlot";
static final String TOTAL_SIZE_OF_SEGMENTS_SKIPPED = "segmentSizeSkippedCompact";
static final String TOTAL_COUNT_OF_SEGMENTS_SKIPPED = "segmentCountSkippedCompact";
static final String TOTAL_INTERVAL_OF_SEGMENTS_SKIPPED = "segmentIntervalSkippedCompact";
static final String TOTAL_SIZE_OF_SEGMENTS_AWAITING = "segmentSizeWaitCompact";
static final String TOTAL_COUNT_OF_SEGMENTS_AWAITING = "segmentCountWaitCompact";
static final String TOTAL_INTERVAL_OF_SEGMENTS_AWAITING = "segmentIntervalWaitCompact";
static final String TOTAL_SIZE_OF_SEGMENTS_COMPACTED = "segmentSizeCompacted";
static final String TOTAL_COUNT_OF_SEGMENTS_COMPACTED = "segmentCountCompacted";
static final String TOTAL_INTERVAL_OF_SEGMENTS_COMPACTED = "segmentIntervalCompacted";
/** Must be synced with org.apache.druid.indexing.common.task.CompactionTask.TYPE. */
/** Must be the same as org.apache.druid.indexing.common.task.CompactionTask.TYPE. */
public static final String COMPACTION_TASK_TYPE = "compact";
/** Must be synced with org.apache.druid.indexing.common.task.Tasks.STORE_COMPACTION_STATE_KEY */
/** Must be the same as org.apache.druid.indexing.common.task.Tasks.STORE_COMPACTION_STATE_KEY */
public static final String STORE_COMPACTION_STATE_KEY = "storeCompactionState";
private static final Logger LOG = new Logger(CompactSegments.class);
@ -100,7 +87,7 @@ public class CompactSegments implements CoordinatorCustomDuty
this.policy = policy;
this.indexingServiceClient = indexingServiceClient;
this.skipLockedIntervals = config.getCompactionSkipLockedIntervals();
autoCompactionSnapshotPerDataSource.set(new HashMap<>());
resetCompactionSnapshot();
LOG.info("Scheduling compaction with skipLockedIntervals [%s]", skipLockedIntervals);
}
@ -111,144 +98,134 @@ public class CompactSegments implements CoordinatorCustomDuty
return skipLockedIntervals;
}
@VisibleForTesting
IndexingServiceClient getIndexingServiceClient()
{
return indexingServiceClient;
}
@Override
public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params)
{
LOG.info("Compact segments");
LOG.info("Running CompactSegments duty");
final CoordinatorCompactionConfig dynamicConfig = params.getCoordinatorCompactionConfig();
final CoordinatorStats stats = new CoordinatorStats();
List<DataSourceCompactionConfig> compactionConfigList = dynamicConfig.getCompactionConfigs();
if (dynamicConfig.getMaxCompactionTaskSlots() > 0) {
Map<String, SegmentTimeline> dataSources =
params.getUsedSegmentsTimelinesPerDataSource();
if (compactionConfigList != null && !compactionConfigList.isEmpty()) {
Map<String, DataSourceCompactionConfig> compactionConfigs = compactionConfigList
.stream()
.collect(Collectors.toMap(DataSourceCompactionConfig::getDataSource, Function.identity()));
final List<TaskStatusPlus> compactionTasks = filterNonCompactionTasks(indexingServiceClient.getActiveTasks());
// dataSource -> list of intervals for which compaction will be skipped in this run
final Map<String, List<Interval>> intervalsToSkipCompaction = new HashMap<>();
int numEstimatedNonCompleteCompactionTasks = 0;
for (TaskStatusPlus status : compactionTasks) {
final TaskPayloadResponse response = indexingServiceClient.getTaskPayload(status.getId());
if (response == null) {
throw new ISE("Got a null paylord from overlord for task[%s]", status.getId());
}
if (COMPACTION_TASK_TYPE.equals(response.getPayload().getType())) {
final ClientCompactionTaskQuery compactionTaskQuery = (ClientCompactionTaskQuery) response.getPayload();
DataSourceCompactionConfig dataSourceCompactionConfig = compactionConfigs.get(status.getDataSource());
if (dataSourceCompactionConfig != null && dataSourceCompactionConfig.getGranularitySpec() != null) {
Granularity configuredSegmentGranularity = dataSourceCompactionConfig.getGranularitySpec().getSegmentGranularity();
if (configuredSegmentGranularity != null
&& compactionTaskQuery.getGranularitySpec() != null
&& !configuredSegmentGranularity.equals(compactionTaskQuery.getGranularitySpec().getSegmentGranularity())) {
// We will cancel active compaction task if segmentGranularity changes and we will need to
// re-compact the interval
LOG.info("Canceled task[%s] as task segmentGranularity is [%s] but compaction config "
+ "segmentGranularity is [%s]",
status.getId(),
compactionTaskQuery.getGranularitySpec().getSegmentGranularity(),
configuredSegmentGranularity);
indexingServiceClient.cancelTask(status.getId());
continue;
}
}
// Skip interval as the current active compaction task is good
final Interval interval = compactionTaskQuery.getIoConfig().getInputSpec().getInterval();
intervalsToSkipCompaction.computeIfAbsent(status.getDataSource(), k -> new ArrayList<>()).add(interval);
// Since we keep the current active compaction task running, we count the active task slots
numEstimatedNonCompleteCompactionTasks += findMaxNumTaskSlotsUsedByOneCompactionTask(
compactionTaskQuery.getTuningConfig()
);
} else {
throw new ISE("task[%s] is not a compactionTask", status.getId());
}
}
// Skip all the intervals locked by higher priority tasks for each datasource
// This must be done after the invalid compaction tasks are cancelled
// in the loop above so that their intervals are not considered locked
getLockedIntervalsToSkip(compactionConfigList).forEach(
(dataSource, intervals) ->
intervalsToSkipCompaction
.computeIfAbsent(dataSource, ds -> new ArrayList<>())
.addAll(intervals)
);
final CompactionSegmentIterator iterator =
policy.reset(compactionConfigs, dataSources, intervalsToSkipCompaction);
int totalCapacity;
if (dynamicConfig.isUseAutoScaleSlots()) {
try {
totalCapacity = indexingServiceClient.getTotalWorkerCapacityWithAutoScale();
}
catch (Exception e) {
LOG.warn("Failed to get total worker capacity with auto scale slots. Falling back to current capacity count");
totalCapacity = indexingServiceClient.getTotalWorkerCapacity();
}
} else {
totalCapacity = indexingServiceClient.getTotalWorkerCapacity();
}
final int compactionTaskCapacity = (int) Math.min(
totalCapacity * dynamicConfig.getCompactionTaskSlotRatio(),
dynamicConfig.getMaxCompactionTaskSlots()
);
final int numAvailableCompactionTaskSlots;
if (numEstimatedNonCompleteCompactionTasks > 0) {
numAvailableCompactionTaskSlots = Math.max(
0,
compactionTaskCapacity - numEstimatedNonCompleteCompactionTasks
);
} else {
// compactionTaskCapacity might be 0 if totalWorkerCapacity is low.
// This guarantees that at least one slot is available if
// compaction is enabled and numEstimatedNonCompleteCompactionTasks is 0.
numAvailableCompactionTaskSlots = Math.max(1, compactionTaskCapacity);
}
LOG.info(
"Found [%d] available task slots for compaction out of [%d] max compaction task capacity",
numAvailableCompactionTaskSlots,
compactionTaskCapacity
);
stats.addToGlobalStat(AVAILABLE_COMPACTION_TASK_SLOT, numAvailableCompactionTaskSlots);
stats.addToGlobalStat(MAX_COMPACTION_TASK_SLOT, compactionTaskCapacity);
final Map<String, AutoCompactionSnapshot.Builder> currentRunAutoCompactionSnapshotBuilders = new HashMap<>();
if (numAvailableCompactionTaskSlots > 0) {
stats.accumulate(
doRun(
compactionConfigs,
currentRunAutoCompactionSnapshotBuilders,
numAvailableCompactionTaskSlots,
iterator
)
);
} else {
stats.accumulate(makeStats(currentRunAutoCompactionSnapshotBuilders, 0, iterator));
}
} else {
LOG.info("compactionConfig is empty. Skip.");
autoCompactionSnapshotPerDataSource.set(new HashMap<>());
}
} else {
LOG.info("maxCompactionTaskSlots was set to 0. Skip compaction");
autoCompactionSnapshotPerDataSource.set(new HashMap<>());
final int maxCompactionTaskSlots = dynamicConfig.getMaxCompactionTaskSlots();
if (maxCompactionTaskSlots <= 0) {
LOG.info("Skipping compaction as maxCompactionTaskSlots is [%d].", maxCompactionTaskSlots);
resetCompactionSnapshot();
return params;
}
return params.buildFromExisting()
.withCoordinatorStats(stats)
.build();
List<DataSourceCompactionConfig> compactionConfigList = dynamicConfig.getCompactionConfigs();
if (compactionConfigList == null || compactionConfigList.isEmpty()) {
LOG.info("Skipping compaction as compaction config list is empty.");
resetCompactionSnapshot();
return params;
}
Map<String, DataSourceCompactionConfig> compactionConfigs = compactionConfigList
.stream()
.collect(Collectors.toMap(DataSourceCompactionConfig::getDataSource, Function.identity()));
// Map from dataSource to list of intervals for which compaction will be skipped in this run
final Map<String, List<Interval>> intervalsToSkipCompaction = new HashMap<>();
// Fetch currently running compaction tasks
int busyCompactionTaskSlots = 0;
final List<TaskStatusPlus> compactionTasks = filterNonCompactionTasks(indexingServiceClient.getActiveTasks());
for (TaskStatusPlus status : compactionTasks) {
final TaskPayloadResponse response = indexingServiceClient.getTaskPayload(status.getId());
if (response == null) {
throw new ISE("Could not find payload for active compaction task[%s]", status.getId());
} else if (!COMPACTION_TASK_TYPE.equals(response.getPayload().getType())) {
throw new ISE(
"Payload of active compaction task[%s] is of invalid type[%s]",
status.getId(), response.getPayload().getType()
);
}
final ClientCompactionTaskQuery compactionTaskQuery = (ClientCompactionTaskQuery) response.getPayload();
DataSourceCompactionConfig dataSourceCompactionConfig = compactionConfigs.get(status.getDataSource());
if (cancelTaskIfGranularityChanged(compactionTaskQuery, dataSourceCompactionConfig)) {
continue;
}
// Skip this interval as the current active compaction task is good
final Interval interval = compactionTaskQuery.getIoConfig().getInputSpec().getInterval();
intervalsToSkipCompaction.computeIfAbsent(status.getDataSource(), k -> new ArrayList<>())
.add(interval);
busyCompactionTaskSlots += findMaxNumTaskSlotsUsedByOneCompactionTask(
compactionTaskQuery.getTuningConfig()
);
}
// Skip all the intervals locked by higher priority tasks for each datasource
// This must be done after the invalid compaction tasks are cancelled
// in the loop above so that their intervals are not considered locked
getLockedIntervalsToSkip(compactionConfigList).forEach(
(dataSource, intervals) ->
intervalsToSkipCompaction
.computeIfAbsent(dataSource, ds -> new ArrayList<>())
.addAll(intervals)
);
// Get iterator over segments to compact and submit compaction tasks
Map<String, SegmentTimeline> dataSources = params.getUsedSegmentsTimelinesPerDataSource();
final CompactionSegmentIterator iterator =
policy.reset(compactionConfigs, dataSources, intervalsToSkipCompaction);
final int compactionTaskCapacity = getCompactionTaskCapacity(dynamicConfig);
final int availableCompactionTaskSlots
= getAvailableCompactionTaskSlots(compactionTaskCapacity, busyCompactionTaskSlots);
final Map<String, AutoCompactionSnapshot.Builder> currentRunAutoCompactionSnapshotBuilders = new HashMap<>();
final int numSubmittedCompactionTasks = submitCompactionTasks(
compactionConfigs,
currentRunAutoCompactionSnapshotBuilders,
availableCompactionTaskSlots,
iterator
);
final CoordinatorRunStats stats = params.getCoordinatorStats();
stats.add(Stats.Compaction.MAX_SLOTS, compactionTaskCapacity);
stats.add(Stats.Compaction.AVAILABLE_SLOTS, availableCompactionTaskSlots);
stats.add(Stats.Compaction.SUBMITTED_TASKS, numSubmittedCompactionTasks);
addCompactionSnapshotStats(currentRunAutoCompactionSnapshotBuilders, iterator, stats);
return params;
}
private void resetCompactionSnapshot()
{
autoCompactionSnapshotPerDataSource.set(Collections.emptyMap());
}
/**
* Cancels a currently running compaction task if the segment granularity
* for this datasource has changed in the compaction config.
*
* @return true if the task was canceled, false otherwise.
*/
private boolean cancelTaskIfGranularityChanged(
ClientCompactionTaskQuery compactionTaskQuery,
DataSourceCompactionConfig dataSourceCompactionConfig
)
{
if (dataSourceCompactionConfig == null
|| dataSourceCompactionConfig.getGranularitySpec() == null
|| compactionTaskQuery.getGranularitySpec() == null) {
return false;
}
Granularity configuredSegmentGranularity = dataSourceCompactionConfig.getGranularitySpec()
.getSegmentGranularity();
Granularity taskSegmentGranularity = compactionTaskQuery.getGranularitySpec().getSegmentGranularity();
if (configuredSegmentGranularity.equals(taskSegmentGranularity)) {
return false;
}
LOG.info(
"Cancelling task [%s] as task segmentGranularity is [%s] but compaction config segmentGranularity is [%s]",
compactionTaskQuery.getId(), taskSegmentGranularity, configuredSegmentGranularity
);
indexingServiceClient.cancelTask(compactionTaskQuery.getId());
return true;
}
/**
@ -345,135 +322,184 @@ public class CompactSegments implements CoordinatorCustomDuty
.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()) {
final String dataSourceName = segmentsToCompact.get(0).getDataSource();
// 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 {
LOG.warn("segmentsToCompact does not have the same interval. Fallback to not setting segmentGranularity for auto compaction task");
}
} 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 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();
}
// 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);
}
}
// 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())
);
LOG.info(
"Submitted a compactionTask[%s] for %s segments",
taskId,
segmentsToCompact.size()
);
LOG.infoSegments(segmentsToCompact, "Compacting segments");
// Count the compaction task itself + its sub tasks
numSubmittedTasks++;
numCompactionTasksAndSubtasks += findMaxNumTaskSlotsUsedByOneCompactionTask(config.getTuningConfig());
} else {
if (segmentsToCompact.isEmpty()) {
throw new ISE("segmentsToCompact is empty?");
}
final String dataSourceName = segmentsToCompact.get(0).getDataSource();
// 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 {
LOG.warn(
"segmentsToCompact does not have the same interval. Fallback to not setting segmentGranularity for auto compaction task");
}
} 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 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();
}
// 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.");
}
}
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())
);
LOG.info("Submitted a compactionTask[%s] for [%d] segments", taskId, segmentsToCompact.size());
LOG.infoSegments(segmentsToCompact, "Compacting segments");
// Count the compaction task itself + its sub tasks
numSubmittedTasks++;
numCompactionTasksAndSubtasks += findMaxNumTaskSlotsUsedByOneCompactionTask(config.getTuningConfig());
}
return makeStats(currentRunAutoCompactionSnapshotBuilders, numSubmittedTasks, iterator);
return numSubmittedTasks;
}
private Map<String, Object> newAutoCompactionContext(@Nullable Map<String, Object> configuredContext)
@ -485,19 +511,13 @@ public class CompactSegments implements CoordinatorCustomDuty
return newContext;
}
private CoordinatorStats makeStats(
private void addCompactionSnapshotStats(
Map<String, AutoCompactionSnapshot.Builder> currentRunAutoCompactionSnapshotBuilders,
int numCompactionTasks,
CompactionSegmentIterator iterator
CompactionSegmentIterator iterator,
CoordinatorRunStats stats
)
{
final Map<String, AutoCompactionSnapshot> currentAutoCompactionSnapshotPerDataSource = new HashMap<>();
final CoordinatorStats stats = new CoordinatorStats();
stats.addToGlobalStat(COMPACTION_TASK_COUNT, numCompactionTasks);
// Iterate through all the remaining segments in the iterator.
// As these segments could be compacted but were not compacted due to lack of task slot, we will aggregates
// the statistic to the AwaitingCompaction statistics
// Mark all the segments remaining in the iterator as "awaiting compaction"
while (iterator.hasNext()) {
final List<DataSegment> segmentsToCompact = iterator.next();
if (!segmentsToCompact.isEmpty()) {
@ -549,7 +569,9 @@ public class CompactSegments implements CoordinatorCustomDuty
builder.incrementIntervalCountSkipped(dataSourceSkippedStatistics.getSegmentIntervalCountSum());
}
for (Map.Entry<String, AutoCompactionSnapshot.Builder> autoCompactionSnapshotBuilderEntry : currentRunAutoCompactionSnapshotBuilders.entrySet()) {
final Map<String, AutoCompactionSnapshot> currentAutoCompactionSnapshotPerDataSource = new HashMap<>();
for (Map.Entry<String, AutoCompactionSnapshot.Builder> autoCompactionSnapshotBuilderEntry
: currentRunAutoCompactionSnapshotBuilders.entrySet()) {
final String dataSource = autoCompactionSnapshotBuilderEntry.getKey();
final AutoCompactionSnapshot.Builder builder = autoCompactionSnapshotBuilderEntry.getValue();
@ -557,58 +579,31 @@ public class CompactSegments implements CoordinatorCustomDuty
AutoCompactionSnapshot autoCompactionSnapshot = builder.build();
currentAutoCompactionSnapshotPerDataSource.put(dataSource, autoCompactionSnapshot);
// Use the complete snapshot to emits metrics
stats.addToDataSourceStat(
TOTAL_SIZE_OF_SEGMENTS_AWAITING,
dataSource,
autoCompactionSnapshot.getBytesAwaitingCompaction()
);
stats.addToDataSourceStat(
TOTAL_COUNT_OF_SEGMENTS_AWAITING,
dataSource,
autoCompactionSnapshot.getSegmentCountAwaitingCompaction()
);
stats.addToDataSourceStat(
TOTAL_INTERVAL_OF_SEGMENTS_AWAITING,
dataSource,
autoCompactionSnapshot.getIntervalCountAwaitingCompaction()
);
stats.addToDataSourceStat(
TOTAL_SIZE_OF_SEGMENTS_COMPACTED,
dataSource,
autoCompactionSnapshot.getBytesCompacted()
);
stats.addToDataSourceStat(
TOTAL_COUNT_OF_SEGMENTS_COMPACTED,
dataSource,
autoCompactionSnapshot.getSegmentCountCompacted()
);
stats.addToDataSourceStat(
TOTAL_INTERVAL_OF_SEGMENTS_COMPACTED,
dataSource,
autoCompactionSnapshot.getIntervalCountCompacted()
);
stats.addToDataSourceStat(
TOTAL_SIZE_OF_SEGMENTS_SKIPPED,
dataSource,
autoCompactionSnapshot.getBytesSkipped()
);
stats.addToDataSourceStat(
TOTAL_COUNT_OF_SEGMENTS_SKIPPED,
dataSource,
autoCompactionSnapshot.getSegmentCountSkipped()
);
stats.addToDataSourceStat(
TOTAL_INTERVAL_OF_SEGMENTS_SKIPPED,
dataSource,
autoCompactionSnapshot.getIntervalCountSkipped()
);
// Use the complete snapshot to emit metrics
addStatsForDatasource(dataSource, autoCompactionSnapshot, stats);
}
// Atomic update of autoCompactionSnapshotPerDataSource with the latest from this coordinator run
autoCompactionSnapshotPerDataSource.set(currentAutoCompactionSnapshotPerDataSource);
}
return stats;
private void addStatsForDatasource(
String dataSource,
AutoCompactionSnapshot autoCompactionSnapshot,
CoordinatorRunStats stats
)
{
final RowKey rowKey = RowKey.forDatasource(dataSource);
stats.add(Stats.Compaction.PENDING_BYTES, rowKey, autoCompactionSnapshot.getBytesAwaitingCompaction());
stats.add(Stats.Compaction.PENDING_SEGMENTS, rowKey, autoCompactionSnapshot.getSegmentCountAwaitingCompaction());
stats.add(Stats.Compaction.PENDING_INTERVALS, rowKey, autoCompactionSnapshot.getIntervalCountAwaitingCompaction());
stats.add(Stats.Compaction.COMPACTED_BYTES, rowKey, autoCompactionSnapshot.getBytesCompacted());
stats.add(Stats.Compaction.COMPACTED_SEGMENTS, rowKey, autoCompactionSnapshot.getSegmentCountCompacted());
stats.add(Stats.Compaction.COMPACTED_INTERVALS, rowKey, autoCompactionSnapshot.getIntervalCountCompacted());
stats.add(Stats.Compaction.SKIPPED_BYTES, rowKey, autoCompactionSnapshot.getBytesSkipped());
stats.add(Stats.Compaction.SKIPPED_SEGMENTS, rowKey, autoCompactionSnapshot.getSegmentCountSkipped());
stats.add(Stats.Compaction.SKIPPED_INTERVALS, rowKey, autoCompactionSnapshot.getIntervalCountSkipped());
}
@Nullable

View File

@ -1,525 +0,0 @@
/*
* 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 it.unimi.dsi.fastutil.objects.Object2IntMap;
import it.unimi.dsi.fastutil.objects.Object2LongMap;
import org.apache.druid.client.ImmutableDruidServer;
import org.apache.druid.java.util.common.logger.Logger;
import org.apache.druid.java.util.emitter.service.ServiceEmitter;
import org.apache.druid.java.util.emitter.service.ServiceMetricEvent;
import org.apache.druid.query.DruidMetrics;
import org.apache.druid.server.coordinator.CoordinatorStats;
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.LoadQueuePeon;
import org.apache.druid.server.coordinator.ServerHolder;
import org.apache.druid.server.coordinator.rules.LoadRule;
import org.apache.druid.timeline.DataSegment;
import org.apache.druid.timeline.VersionedIntervalTimeline;
/**
* Emits stats of the cluster and metrics of the coordination (including segment balancing) process.
*/
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;
public EmitClusterStatsAndMetrics(DruidCoordinator coordinator, String groupName, boolean isContainCompactSegmentDuty)
{
this.coordinator = coordinator;
this.groupName = groupName;
this.isContainCompactSegmentDuty = isContainCompactSegmentDuty;
}
private void emitTieredStat(
final ServiceEmitter emitter,
final String metricName,
final String tier,
final double value
)
{
emitter.emit(
new ServiceMetricEvent.Builder()
.setDimension(DruidMetrics.TIER, tier)
.setDimension(DruidMetrics.DUTY_GROUP, groupName)
.build(metricName, value)
);
}
private void emitTieredStat(
final ServiceEmitter emitter,
final String metricName,
final String tier,
final long value
)
{
emitter.emit(
new ServiceMetricEvent.Builder()
.setDimension(DruidMetrics.TIER, tier)
.setDimension(DruidMetrics.DUTY_GROUP, groupName)
.build(metricName, value)
);
}
private void emitTieredStats(
final ServiceEmitter emitter,
final String metricName,
final CoordinatorStats stats,
final String statName
)
{
stats.forEachTieredStat(
statName,
(final String tier, final long count) -> {
emitTieredStat(emitter, metricName, tier, count);
}
);
}
private void emitDutyStat(
final ServiceEmitter emitter,
final String metricName,
final String duty,
final long value
)
{
emitter.emit(
new ServiceMetricEvent.Builder()
.setDimension(DruidMetrics.DUTY, duty)
.setDimension(DruidMetrics.DUTY_GROUP, groupName)
.build(metricName, value)
);
}
private void emitDutyStats(
final ServiceEmitter emitter,
final String metricName,
final CoordinatorStats stats,
final String statName
)
{
stats.forEachDutyStat(
statName,
(final String duty, final long count) -> {
emitDutyStat(emitter, metricName, duty, count);
}
);
}
@Override
public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params)
{
DruidCluster cluster = params.getDruidCluster();
CoordinatorStats stats = params.getCoordinatorStats();
ServiceEmitter emitter = params.getEmitter();
if (DruidCoordinator.HISTORICAL_MANAGEMENT_DUTIES_DUTY_GROUP.equals(groupName)) {
emitStatsForHistoricalManagementDuties(cluster, stats, emitter, params);
}
if (isContainCompactSegmentDuty) {
emitStatsForCompactSegments(cluster, stats, emitter);
}
// Emit coordinator runtime stats
emitDutyStats(emitter, "coordinator/time", stats, "runtime");
return params;
}
private void emitStatsForHistoricalManagementDuties(DruidCluster cluster, CoordinatorStats stats, ServiceEmitter emitter, DruidCoordinatorRuntimeParams params)
{
stats.forEachTieredStat(
"assignedCount",
(final String tier, final long count) -> {
log.info(
"[%s] : Assigned %s segments among %,d servers",
tier,
count,
cluster.getHistoricalsByTier(tier).size()
);
emitTieredStat(emitter, "segment/assigned/count", tier, count);
}
);
stats.forEachTieredStat(
"droppedCount",
(final String tier, final long count) -> {
log.info(
"[%s] : Dropped %s segments among %,d servers",
tier,
count,
cluster.getHistoricalsByTier(tier).size()
);
emitTieredStat(emitter, "segment/dropped/count", tier, count);
}
);
emitTieredStats(emitter, "segment/cost/raw", stats, "initialCost");
emitTieredStats(emitter, "segment/cost/normalization", stats, "normalization");
emitTieredStats(emitter, "segment/moved/count", stats, "movedCount");
emitTieredStats(emitter, "segment/unmoved/count", stats, "unmovedCount");
emitter.emit(
new ServiceMetricEvent.Builder()
.build(
"segment/deleted/count",
stats.getGlobalStat("deletedCount")
)
);
stats.forEachTieredStat(
"normalizedInitialCostTimesOneThousand",
(final String tier, final long count) -> {
emitTieredStat(emitter, "segment/cost/normalized", tier, count / 1000d);
}
);
stats.forEachTieredStat(
"unneededCount",
(final String tier, final long count) -> {
log.info(
"[%s] : Removed %s unneeded segments among %,d servers",
tier,
count,
cluster.getHistoricalsByTier(tier).size()
);
emitTieredStat(emitter, "segment/unneeded/count", tier, count);
}
);
emitter.emit(
new ServiceMetricEvent.Builder()
.setDimension(DruidMetrics.DUTY_GROUP, groupName)
.build(
"segment/overShadowed/count",
stats.getGlobalStat("overShadowedCount")
)
);
stats.forEachTieredStat(
"movedCount",
(final String tier, final long count) -> {
log.info("[%s] : Moved %,d segment(s)", tier, count);
}
);
stats.forEachTieredStat(
"unmovedCount",
(final String tier, final long count) -> {
log.info("[%s] : Let alone %,d segment(s)", tier, count);
}
);
log.info("Load Queues:");
for (Iterable<ServerHolder> serverHolders : cluster.getSortedHistoricalsByTier()) {
for (ServerHolder serverHolder : serverHolders) {
ImmutableDruidServer server = serverHolder.getServer();
LoadQueuePeon queuePeon = serverHolder.getPeon();
log.info(
"Server[%s, %s, %s] has %,d left to load, %,d left to drop, %,d served, %,d bytes queued, %,d bytes served.",
server.getName(),
server.getType().toString(),
server.getTier(),
queuePeon.getSegmentsToLoad().size(),
queuePeon.getSegmentsToDrop().size(),
server.getNumSegments(),
queuePeon.getLoadQueueSize(),
server.getCurrSize()
);
if (log.isDebugEnabled()) {
for (DataSegment segment : queuePeon.getSegmentsToLoad()) {
log.debug("Segment to load[%s]", segment);
}
for (DataSegment segment : queuePeon.getSegmentsToDrop()) {
log.debug("Segment to drop[%s]", segment);
}
}
stats.addToTieredStat(TOTAL_CAPACITY, server.getTier(), server.getMaxSize());
stats.addToTieredStat(TOTAL_HISTORICAL_COUNT, server.getTier(), 1);
}
}
params.getDatabaseRuleManager()
.getAllRules()
.values()
.forEach(
rules -> rules.forEach(
rule -> {
if (rule instanceof LoadRule) {
((LoadRule) rule).getTieredReplicants()
.forEach(
(tier, replica) -> stats.accumulateMaxTieredStat(
MAX_REPLICATION_FACTOR,
tier,
replica
));
}
}
));
emitTieredStats(emitter, "tier/required/capacity", stats, LoadRule.REQUIRED_CAPACITY);
emitTieredStats(emitter, "tier/total/capacity", stats, TOTAL_CAPACITY);
emitTieredStats(emitter, "tier/replication/factor", stats, MAX_REPLICATION_FACTOR);
emitTieredStats(emitter, "tier/historical/count", stats, TOTAL_HISTORICAL_COUNT);
// Emit coordinator metrics
params
.getLoadManagementPeons()
.forEach((final String serverName, final LoadQueuePeon queuePeon) -> {
emitter.emit(
new ServiceMetricEvent.Builder()
.setDimension(DruidMetrics.DUTY_GROUP, groupName)
.setDimension(DruidMetrics.SERVER, serverName).build(
"segment/loadQueue/size", queuePeon.getLoadQueueSize()
)
);
emitter.emit(
new ServiceMetricEvent.Builder()
.setDimension(DruidMetrics.DUTY_GROUP, groupName)
.setDimension(DruidMetrics.SERVER, serverName).build(
"segment/loadQueue/failed", queuePeon.getAndResetFailedAssignCount()
)
);
emitter.emit(
new ServiceMetricEvent.Builder()
.setDimension(DruidMetrics.DUTY_GROUP, groupName)
.setDimension(DruidMetrics.SERVER, serverName).build(
"segment/loadQueue/count", queuePeon.getSegmentsToLoad().size()
)
);
emitter.emit(
new ServiceMetricEvent.Builder()
.setDimension(DruidMetrics.DUTY_GROUP, groupName)
.setDimension(DruidMetrics.SERVER, serverName).build(
"segment/dropQueue/count", queuePeon.getSegmentsToDrop().size()
)
);
});
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.computeUnderReplicationCountsPerDataSourcePerTier().forEach(
(final String tier, final Object2LongMap<String> underReplicationCountsPerDataSource) -> {
for (final Object2LongMap.Entry<String> entry : underReplicationCountsPerDataSource.object2LongEntrySet()) {
final String dataSource = entry.getKey();
final long underReplicationCount = entry.getLongValue();
emitter.emit(
new ServiceMetricEvent.Builder()
.setDimension(DruidMetrics.DUTY_GROUP, groupName)
.setDimension(DruidMetrics.TIER, tier)
.setDimension(DruidMetrics.DATASOURCE, dataSource).build(
"segment/underReplicated/count", underReplicationCount
)
);
}
}
);
// Emit segment metrics
params.getUsedSegmentsTimelinesPerDataSource().forEach(
(String dataSource, VersionedIntervalTimeline<String, DataSegment> dataSourceWithUsedSegments) -> {
long totalSizeOfUsedSegments = dataSourceWithUsedSegments
.iterateAllObjects()
.stream()
.mapToLong(DataSegment::getSize)
.sum();
emitter.emit(
new ServiceMetricEvent.Builder()
.setDimension(DruidMetrics.DUTY_GROUP, groupName)
.setDimension(DruidMetrics.DATASOURCE, dataSource)
.build("segment/size", totalSizeOfUsedSegments)
);
emitter.emit(
new ServiceMetricEvent.Builder()
.setDimension(DruidMetrics.DUTY_GROUP, groupName)
.setDimension(DruidMetrics.DATASOURCE, dataSource)
.build("segment/count", dataSourceWithUsedSegments.getNumObjects())
);
}
);
}
private void emitStatsForCompactSegments(DruidCluster cluster, CoordinatorStats stats, ServiceEmitter emitter)
{
emitter.emit(
new ServiceMetricEvent.Builder()
.setDimension(DruidMetrics.DUTY_GROUP, groupName)
.build(
"compact/task/count",
stats.getGlobalStat(CompactSegments.COMPACTION_TASK_COUNT)
)
);
emitter.emit(
new ServiceMetricEvent.Builder()
.setDimension(DruidMetrics.DUTY_GROUP, groupName)
.build(
"compactTask/maxSlot/count",
stats.getGlobalStat(CompactSegments.MAX_COMPACTION_TASK_SLOT)
)
);
emitter.emit(
new ServiceMetricEvent.Builder()
.setDimension(DruidMetrics.DUTY_GROUP, groupName)
.build(
"compactTask/availableSlot/count",
stats.getGlobalStat(CompactSegments.AVAILABLE_COMPACTION_TASK_SLOT)
)
);
stats.forEachDataSourceStat(
CompactSegments.TOTAL_SIZE_OF_SEGMENTS_AWAITING,
(final String dataSource, final long count) -> {
emitter.emit(
new ServiceMetricEvent.Builder()
.setDimension(DruidMetrics.DUTY_GROUP, groupName)
.setDimension(DruidMetrics.DATASOURCE, dataSource)
.build("segment/waitCompact/bytes", count)
);
}
);
stats.forEachDataSourceStat(
CompactSegments.TOTAL_COUNT_OF_SEGMENTS_AWAITING,
(final String dataSource, final long count) -> {
emitter.emit(
new ServiceMetricEvent.Builder()
.setDimension(DruidMetrics.DUTY_GROUP, groupName)
.setDimension(DruidMetrics.DATASOURCE, dataSource)
.build("segment/waitCompact/count", count)
);
}
);
stats.forEachDataSourceStat(
CompactSegments.TOTAL_INTERVAL_OF_SEGMENTS_AWAITING,
(final String dataSource, final long count) -> {
emitter.emit(
new ServiceMetricEvent.Builder()
.setDimension(DruidMetrics.DUTY_GROUP, groupName)
.setDimension(DruidMetrics.DATASOURCE, dataSource)
.build("interval/waitCompact/count", count)
);
}
);
stats.forEachDataSourceStat(
CompactSegments.TOTAL_SIZE_OF_SEGMENTS_SKIPPED,
(final String dataSource, final long count) -> {
emitter.emit(
new ServiceMetricEvent.Builder()
.setDimension(DruidMetrics.DUTY_GROUP, groupName)
.setDimension(DruidMetrics.DATASOURCE, dataSource)
.build("segment/skipCompact/bytes", count)
);
}
);
stats.forEachDataSourceStat(
CompactSegments.TOTAL_COUNT_OF_SEGMENTS_SKIPPED,
(final String dataSource, final long count) -> {
emitter.emit(
new ServiceMetricEvent.Builder()
.setDimension(DruidMetrics.DUTY_GROUP, groupName)
.setDimension(DruidMetrics.DATASOURCE, dataSource)
.build("segment/skipCompact/count", count)
);
}
);
stats.forEachDataSourceStat(
CompactSegments.TOTAL_INTERVAL_OF_SEGMENTS_SKIPPED,
(final String dataSource, final long count) -> {
emitter.emit(
new ServiceMetricEvent.Builder()
.setDimension(DruidMetrics.DUTY_GROUP, groupName)
.setDimension(DruidMetrics.DATASOURCE, dataSource)
.build("interval/skipCompact/count", count)
);
}
);
stats.forEachDataSourceStat(
CompactSegments.TOTAL_SIZE_OF_SEGMENTS_COMPACTED,
(final String dataSource, final long count) -> {
emitter.emit(
new ServiceMetricEvent.Builder()
.setDimension(DruidMetrics.DUTY_GROUP, groupName)
.setDimension(DruidMetrics.DATASOURCE, dataSource)
.build("segment/compacted/bytes", count)
);
}
);
stats.forEachDataSourceStat(
CompactSegments.TOTAL_COUNT_OF_SEGMENTS_COMPACTED,
(final String dataSource, final long count) -> {
emitter.emit(
new ServiceMetricEvent.Builder()
.setDimension(DruidMetrics.DUTY_GROUP, groupName)
.setDimension(DruidMetrics.DATASOURCE, dataSource)
.build("segment/compacted/count", count)
);
}
);
stats.forEachDataSourceStat(
CompactSegments.TOTAL_INTERVAL_OF_SEGMENTS_COMPACTED,
(final String dataSource, final long count) -> {
emitter.emit(
new ServiceMetricEvent.Builder()
.setDimension(DruidMetrics.DUTY_GROUP, groupName)
.setDimension(DruidMetrics.DATASOURCE, dataSource)
.build("interval/compacted/count", count)
);
}
);
}
}

View File

@ -1,53 +0,0 @@
/*
* 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 org.apache.druid.client.DataSourcesSnapshot;
import org.apache.druid.java.util.emitter.EmittingLogger;
import org.apache.druid.server.coordinator.DruidCoordinatorRuntimeParams;
import org.apache.druid.timeline.DataSegment;
public class LogUsedSegments implements CoordinatorDuty
{
private static final EmittingLogger log = new EmittingLogger(LogUsedSegments.class);
public LogUsedSegments()
{
}
@Override
public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params)
{
log.debug("Starting coordination. Getting used segments.");
// Log info about all used segments only if debug logging is enabled
if (log.isDebugEnabled()) {
DataSourcesSnapshot dataSourcesSnapshot = params.getDataSourcesSnapshot();
log.debug("Used Segments");
for (DataSegment dataSegment : dataSourcesSnapshot.iterateAllUsedSegmentsInSnapshot()) {
log.debug(" %s", dataSegment);
}
}
log.info("Found [%,d] used segments.", params.getUsedSegments().size());
return params;
}
}

View File

@ -21,29 +21,32 @@ package org.apache.druid.server.coordinator.duty;
import org.apache.druid.client.ImmutableDruidDataSource;
import org.apache.druid.client.ImmutableDruidServer;
import org.apache.druid.java.util.common.DateTimes;
import org.apache.druid.java.util.common.logger.Logger;
import org.apache.druid.server.coordinator.CoordinatorStats;
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.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 org.apache.druid.timeline.SegmentTimeline;
import org.joda.time.DateTime;
import java.time.Duration;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Map;
import java.util.Set;
import java.util.SortedSet;
public class MarkAsUnusedOvershadowedSegments implements CoordinatorDuty
public class MarkOvershadowedSegmentsAsUnused implements CoordinatorDuty
{
private static final Logger log = new Logger(MarkAsUnusedOvershadowedSegments.class);
private static final Logger log = new Logger(MarkOvershadowedSegmentsAsUnused.class);
private final DruidCoordinator coordinator;
public MarkAsUnusedOvershadowedSegments(DruidCoordinator coordinator)
public MarkOvershadowedSegmentsAsUnused(DruidCoordinator coordinator)
{
this.coordinator = coordinator;
}
@ -51,9 +54,15 @@ public class MarkAsUnusedOvershadowedSegments implements CoordinatorDuty
@Override
public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params)
{
// Mark as unused overshadowed segments only if we've had enough time to make sure we aren't flapping with old data.
if (!params.coordinatorIsLeadingEnoughTimeToMarkAsUnusedOvershadowedSegements()) {
log.info("Skipping MarkAsUnused as coordinator is not leading enough time.");
// Mark overshadowed segments as unused only if the coordinator has been running
// long enough to have refreshed its metadata view
final DateTime coordinatorStartTime = params.getCoordinatorStartTime();
final long delayMillis = params.getCoordinatorDynamicConfig().getMarkSegmentAsUnusedDelayMillis();
if (DateTimes.nowUtc().isBefore(coordinatorStartTime.plus(delayMillis))) {
log.info(
"Skipping MarkAsUnused until [%s] have elapsed after coordinator start [%s].",
Duration.ofMillis(delayMillis), coordinatorStartTime
);
return params;
}
@ -63,37 +72,40 @@ public class MarkAsUnusedOvershadowedSegments implements CoordinatorDuty
return params;
}
CoordinatorStats stats = new CoordinatorStats();
DruidCluster cluster = params.getDruidCluster();
final Map<String, SegmentTimeline> timelines = new HashMap<>();
for (SortedSet<ServerHolder> serverHolders : cluster.getSortedHistoricalsByTier()) {
for (ServerHolder serverHolder : serverHolders) {
addSegmentsFromServer(serverHolder, timelines);
}
}
for (ServerHolder serverHolder : cluster.getBrokers()) {
addSegmentsFromServer(serverHolder, timelines);
}
cluster.getHistoricals().values().forEach(
historicals -> historicals.forEach(
historical -> addSegmentsFromServer(historical, timelines)
)
);
cluster.getBrokers().forEach(
broker -> addSegmentsFromServer(broker, timelines)
);
// Note that we do not include segments from ingestion services such as tasks or indexers,
// to prevent unpublished segments from prematurely overshadowing segments.
// Mark all segments as unused in db that are overshadowed by served segments
// Mark all segments overshadowed by served segments as unused
final Map<String, Set<SegmentId>> datasourceToUnusedSegments = new HashMap<>();
for (DataSegment dataSegment : allOvershadowedSegments) {
SegmentTimeline timeline = timelines.get(dataSegment.getDataSource());
if (timeline != null && timeline.isOvershadowed(dataSegment)) {
datasourceToUnusedSegments.computeIfAbsent(dataSegment.getDataSource(), ds -> new HashSet<>())
.add(dataSegment.getId());
stats.addToGlobalStat("overShadowedCount", 1);
}
}
datasourceToUnusedSegments.forEach(coordinator::markSegmentsAsUnused);
return params.buildFromExisting().withCoordinatorStats(stats).build();
final CoordinatorRunStats stats = params.getCoordinatorStats();
datasourceToUnusedSegments.forEach(
(datasource, unusedSegments) -> {
stats.addToDatasourceStat(Stats.Segments.OVERSHADOWED, datasource, unusedSegments.size());
coordinator.markSegmentsAsUnused(datasource, unusedSegments);
}
);
return params;
}
private void addSegmentsFromServer(
@ -104,9 +116,9 @@ public class MarkAsUnusedOvershadowedSegments implements CoordinatorDuty
ImmutableDruidServer server = serverHolder.getServer();
for (ImmutableDruidDataSource dataSource : server.getDataSources()) {
SegmentTimeline timeline = timelines
.computeIfAbsent(dataSource.getName(), dsName -> new SegmentTimeline());
timeline.addSegments(dataSource.getSegments().iterator());
timelines
.computeIfAbsent(dataSource.getName(), dsName -> new SegmentTimeline())
.addSegments(dataSource.getSegments().iterator());
}
}
}

View File

@ -24,64 +24,39 @@ import org.apache.druid.client.ImmutableDruidDataSource;
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.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.loading.StrategicSegmentAssigner;
import org.apache.druid.server.coordinator.rules.BroadcastDistributionRule;
import org.apache.druid.server.coordinator.rules.Rule;
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>
* </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 DruidCoordinator coordinator;
public RunRules(DruidCoordinator coordinator)
{
this(
new ReplicationThrottler(
coordinator.getDynamicConfigs().getReplicationThrottleLimit(),
coordinator.getDynamicConfigs().getReplicantLifetime(),
false
),
coordinator
);
}
public RunRules(ReplicationThrottler replicatorThrottler, DruidCoordinator coordinator)
{
this.replicatorThrottler = replicatorThrottler;
this.coordinator = coordinator;
}
@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;
}
@ -89,60 +64,31 @@ public class RunRules implements CoordinatorDuty
// 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();
final Set<DataSegment> overshadowed = params.getDataSourcesSnapshot().getOvershadowedSegments();
final Set<DataSegment> usedSegments = params.getUsedSegments();
log.info(
"Applying retention rules on [%d] used segments, skipping [%d] overshadowed segments.",
usedSegments.size(), overshadowed.size()
);
for (String tier : cluster.getTierNames()) {
replicatorThrottler.updateReplicationState(tier);
}
final StrategicSegmentAssigner segmentAssigner = params.getSegmentAssigner();
final MetadataRuleManager databaseRuleManager = params.getDatabaseRuleManager();
DruidCoordinatorRuntimeParams paramsWithReplicationManager = params
.buildFromExistingWithoutSegmentsMetadata()
.withReplicationManager(replicatorThrottler)
.build();
int missingRules = 0;
final DateTime now = DateTimes.nowUtc();
final List<SegmentId> segmentsWithMissingRules = Lists.newArrayListWithCapacity(MAX_MISSING_RULES);
// Run through all matched rules for used segments
DateTime now = DateTimes.nowUtc();
MetadataRuleManager databaseRuleManager = paramsWithReplicationManager.getDatabaseRuleManager();
final List<SegmentId> segmentsWithMissingRules = Lists.newArrayListWithCapacity(MAX_MISSING_RULES);
int missingRules = 0;
final Set<String> broadcastDatasources = new HashSet<>();
for (ImmutableDruidDataSource dataSource : params.getDataSourcesSnapshot().getDataSourcesMap().values()) {
List<Rule> rules = databaseRuleManager.getRulesWithDefault(dataSource.getName());
for (Rule rule : rules) {
// A datasource is considered a broadcast datasource if it has any broadcast rules.
// The set of broadcast datasources is used by BalanceSegments, so it's important that RunRules
// executes before BalanceSegments.
if (rule instanceof BroadcastDistributionRule) {
broadcastDatasources.add(dataSource.getName());
break;
}
}
}
for (DataSegment segment : params.getUsedSegments()) {
for (DataSegment segment : usedSegments) {
if (overshadowed.contains(segment)) {
// Skipping overshadowed segments
// Skip overshadowed segments
continue;
}
List<Rule> rules = databaseRuleManager.getRulesWithDefault(segment.getDataSource());
boolean foundMatchingRule = false;
for (Rule rule : rules) {
if (rule.appliesTo(segment, now)) {
if (
stats.getGlobalStat(
"totalNonPrimaryReplicantsLoaded") >= paramsWithReplicationManager.getCoordinatorDynamicConfig()
.getMaxNonPrimaryReplicantsToLoad()
&& !paramsWithReplicationManager.getReplicationManager().isLoadPrimaryReplicantsOnly()
) {
log.info(
"Maximum number of non-primary replicants [%d] have been loaded for the current RunRules execution. Only loading primary replicants from here on for this coordinator run cycle.",
paramsWithReplicationManager.getCoordinatorDynamicConfig().getMaxNonPrimaryReplicantsToLoad()
);
paramsWithReplicationManager.getReplicationManager().setLoadPrimaryReplicantsOnly(true);
}
stats.accumulate(rule.run(coordinator, paramsWithReplicationManager, segment));
rule.run(segment, segmentAssigner);
foundMatchingRule = true;
break;
}
@ -164,8 +110,36 @@ public class RunRules implements CoordinatorDuty
}
return params.buildFromExisting()
.withCoordinatorStats(stats)
.withBroadcastDatasources(broadcastDatasources)
.withBroadcastDatasources(getBroadcastDatasources(params))
.build();
}
private Set<String> getBroadcastDatasources(DruidCoordinatorRuntimeParams params)
{
final Set<String> broadcastDatasources =
params.getDataSourcesSnapshot().getDataSourcesMap().values().stream()
.map(ImmutableDruidDataSource::getName)
.filter(datasource -> isBroadcastDatasource(datasource, params))
.collect(Collectors.toSet());
if (!broadcastDatasources.isEmpty()) {
log.info("Found broadcast datasources [%s] which will not participate in balancing.", broadcastDatasources);
}
return broadcastDatasources;
}
/**
* A datasource is considered a broadcast datasource if it has even one
* Broadcast Rule. Segments of broadcast datasources:
* <ul>
* <li>Do not participate in balancing</li>
* <li>Are unloaded if unused, even from realtime servers</li>
* </ul>
*/
private boolean isBroadcastDatasource(String datasource, DruidCoordinatorRuntimeParams params)
{
return params.getDatabaseRuleManager().getRulesWithDefault(datasource).stream()
.anyMatch(rule -> rule instanceof BroadcastDistributionRule);
}
}

View File

@ -1,49 +0,0 @@
/*
* 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.base.Preconditions;
import org.joda.time.Interval;
/**
* Util class used by {@link CompactSegments} and {@link CompactionSegmentSearchPolicy}.
*/
class SegmentCompactionUtil
{
/**
* Removes {@code smallInterval} from {@code largeInterval}. The end of both intervals should be same.
*
* @return an interval of {@code largeInterval} - {@code smallInterval}.
*/
static Interval removeIntervalFromEnd(Interval largeInterval, Interval smallInterval)
{
Preconditions.checkArgument(
largeInterval.getEnd().equals(smallInterval.getEnd()),
"end should be same. largeInterval[%s] smallInterval[%s]",
largeInterval,
smallInterval
);
return new Interval(largeInterval.getStart(), smallInterval.getStart());
}
private SegmentCompactionUtil()
{
}
}

View File

@ -22,20 +22,17 @@ package org.apache.druid.server.coordinator.duty;
import org.apache.druid.client.ImmutableDruidDataSource;
import org.apache.druid.client.ImmutableDruidServer;
import org.apache.druid.java.util.common.logger.Logger;
import org.apache.druid.server.coordinator.CoordinatorStats;
import org.apache.druid.server.coordinator.DruidCluster;
import org.apache.druid.server.coordinator.DruidCoordinatorRuntimeParams;
import org.apache.druid.server.coordinator.LoadQueuePeon;
import org.apache.druid.server.coordinator.ServerHolder;
import org.apache.druid.server.coordinator.loading.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.server.coordinator.stats.Stats;
import org.apache.druid.timeline.DataSegment;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.SortedSet;
/**
* Unloads segments that are no longer marked as used from servers.
@ -44,62 +41,38 @@ public class UnloadUnusedSegments implements CoordinatorDuty
{
private static final Logger log = new Logger(UnloadUnusedSegments.class);
private final SegmentLoadQueueManager loadQueueManager;
public UnloadUnusedSegments(SegmentLoadQueueManager loadQueueManager)
{
this.loadQueueManager = loadQueueManager;
}
@Override
public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params)
{
CoordinatorStats stats = new CoordinatorStats();
Set<DataSegment> usedSegments = params.getUsedSegments();
DruidCluster cluster = params.getDruidCluster();
Map<String, Boolean> broadcastStatusByDatasource = new HashMap<>();
final Map<String, Boolean> broadcastStatusByDatasource = new HashMap<>();
for (String broadcastDatasource : params.getBroadcastDatasources()) {
broadcastStatusByDatasource.put(broadcastDatasource, true);
}
for (SortedSet<ServerHolder> serverHolders : cluster.getSortedHistoricalsByTier()) {
for (ServerHolder serverHolder : serverHolders) {
handleUnusedSegmentsForServer(
serverHolder,
usedSegments,
final CoordinatorRunStats stats = params.getCoordinatorStats();
params.getDruidCluster().getAllServers().forEach(
server -> handleUnusedSegmentsForServer(
server,
params,
stats,
false,
broadcastStatusByDatasource
);
}
}
)
);
for (ServerHolder serverHolder : cluster.getBrokers()) {
handleUnusedSegmentsForServer(
serverHolder,
usedSegments,
params,
stats,
false,
broadcastStatusByDatasource
);
}
for (ServerHolder serverHolder : cluster.getRealtimes()) {
handleUnusedSegmentsForServer(
serverHolder,
usedSegments,
params,
stats,
true,
broadcastStatusByDatasource
);
}
return params.buildFromExisting().withCoordinatorStats(stats).build();
return params;
}
private void handleUnusedSegmentsForServer(
ServerHolder serverHolder,
Set<DataSegment> usedSegments,
DruidCoordinatorRuntimeParams params,
CoordinatorStats stats,
boolean dropBroadcastOnly,
CoordinatorRunStats stats,
Map<String, Boolean> broadcastStatusByDatasource
)
{
@ -107,16 +80,7 @@ public class UnloadUnusedSegments implements CoordinatorDuty
for (ImmutableDruidDataSource dataSource : server.getDataSources()) {
boolean isBroadcastDatasource = broadcastStatusByDatasource.computeIfAbsent(
dataSource.getName(),
(dataSourceName) -> {
List<Rule> rules = params.getDatabaseRuleManager().getRulesWithDefault(dataSource.getName());
for (Rule rule : rules) {
// A datasource is considered a broadcast datasource if it has any broadcast rules.
if (rule instanceof BroadcastDistributionRule) {
return true;
}
}
return false;
}
dataSourceName -> isBroadcastDatasource(dataSourceName, params)
);
// The coordinator tracks used segments by examining the metadata store.
@ -127,26 +91,35 @@ public class UnloadUnusedSegments implements CoordinatorDuty
// datasource, this will result in the those segments not being dropped from tasks.
// A more robust solution which requires a larger rework could be to expose
// the set of segments that were created by a task/indexer here, and exclude them.
if (dropBroadcastOnly && !isBroadcastDatasource) {
if (serverHolder.isRealtimeServer() && !isBroadcastDatasource) {
continue;
}
int totalUnneededCount = 0;
final Set<DataSegment> usedSegments = params.getUsedSegments();
for (DataSegment segment : dataSource.getSegments()) {
if (!usedSegments.contains(segment)) {
LoadQueuePeon queuePeon = params.getLoadManagementPeons().get(server.getName());
if (!queuePeon.getSegmentsToDrop().contains(segment)) {
queuePeon.dropSegment(segment, success -> {});
stats.addToTieredStat("unneededCount", server.getTier(), 1);
log.info(
"Dropping uneeded segment [%s] from server [%s] in tier [%s]",
segment.getId(),
server.getName(),
server.getTier()
);
}
if (!usedSegments.contains(segment)
&& loadQueueManager.dropSegment(segment, serverHolder)) {
totalUnneededCount++;
log.info(
"Dropping uneeded segment [%s] from server [%s] in tier [%s]",
segment.getId(), server.getName(), server.getTier()
);
}
}
if (totalUnneededCount > 0) {
stats.addToSegmentStat(Stats.Segments.UNNEEDED, server.getTier(), dataSource.getName(), totalUnneededCount);
}
}
}
/**
* A datasource is considered a broadcast datasource if it has even one broadcast rule.
*/
private boolean isBroadcastDatasource(String datasource, DruidCoordinatorRuntimeParams params)
{
return params.getDatabaseRuleManager().getRulesWithDefault(datasource).stream()
.anyMatch(rule -> rule instanceof BroadcastDistributionRule);
}
}

View File

@ -17,20 +17,20 @@
* under the License.
*/
package org.apache.druid.server.coordinator;
package org.apache.druid.server.coordinator.loading;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.collect.ImmutableList;
import org.apache.curator.framework.CuratorFramework;
import org.apache.curator.framework.api.CuratorWatcher;
import org.apache.curator.utils.ZKPaths;
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.emitter.EmittingLogger;
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.server.coordination.SegmentChangeRequestNoop;
import org.apache.druid.server.coordinator.DruidCoordinator;
import org.apache.druid.server.coordinator.DruidCoordinatorConfig;
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.zookeeper.CreateMode;
import org.apache.zookeeper.KeeperException;
@ -38,11 +38,7 @@ import org.apache.zookeeper.data.Stat;
import javax.annotation.Nonnull;
import javax.annotation.Nullable;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.Iterator;
import java.util.List;
import java.util.HashSet;
import java.util.Set;
import java.util.concurrent.ConcurrentSkipListMap;
import java.util.concurrent.ConcurrentSkipListSet;
@ -50,7 +46,6 @@ import java.util.concurrent.ExecutorService;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.ScheduledFuture;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong;
/**
@ -63,7 +58,7 @@ import java.util.concurrent.atomic.AtomicLong;
* of the same or different methods.
*/
@Deprecated
public class CuratorLoadQueuePeon extends LoadQueuePeon
public class CuratorLoadQueuePeon implements LoadQueuePeon
{
private static final EmittingLogger log = new EmittingLogger(CuratorLoadQueuePeon.class);
@ -80,45 +75,42 @@ public class CuratorLoadQueuePeon extends LoadQueuePeon
private final DruidCoordinatorConfig config;
private final AtomicLong queuedSize = new AtomicLong(0);
private final AtomicInteger failedAssignCount = new AtomicInteger(0);
private final CoordinatorRunStats stats = new CoordinatorRunStats();
/**
* Needs to be thread safe since it can be concurrently accessed via
* {@link #loadSegment(DataSegment, LoadPeonCallback)}, {@link #actionCompleted(SegmentHolder)},
* {@link #getSegmentsToLoad()} and {@link #stop()}
* {@link #loadSegment(DataSegment, SegmentAction, LoadPeonCallback)},
* {@link #actionCompleted(SegmentHolder)}, {@link #getSegmentsToLoad()} and
* {@link #stop()}.
*/
private final ConcurrentSkipListMap<DataSegment, SegmentHolder> segmentsToLoad = new ConcurrentSkipListMap<>(
DruidCoordinator.SEGMENT_COMPARATOR_RECENT_FIRST
);
private final ConcurrentSkipListMap<DataSegment, SegmentHolder> segmentsToLoad
= new ConcurrentSkipListMap<>(DruidCoordinator.SEGMENT_COMPARATOR_RECENT_FIRST);
/**
* Needs to be thread safe since it can be concurrently accessed via
* {@link #dropSegment(DataSegment, LoadPeonCallback)}, {@link #actionCompleted(SegmentHolder)},
* {@link #getSegmentsToDrop()} and {@link #stop()}
*/
private final ConcurrentSkipListMap<DataSegment, SegmentHolder> segmentsToDrop = new ConcurrentSkipListMap<>(
DruidCoordinator.SEGMENT_COMPARATOR_RECENT_FIRST
);
private final ConcurrentSkipListMap<DataSegment, SegmentHolder> segmentsToDrop
= new ConcurrentSkipListMap<>(DruidCoordinator.SEGMENT_COMPARATOR_RECENT_FIRST);
/**
* Needs to be thread safe since it can be concurrently accessed via
* {@link #markSegmentToDrop(DataSegment)}}, {@link #unmarkSegmentToDrop(DataSegment)}}
* and {@link #getSegmentsToDrop()}
*/
private final ConcurrentSkipListSet<DataSegment> segmentsMarkedToDrop = new ConcurrentSkipListSet<>(
DruidCoordinator.SEGMENT_COMPARATOR_RECENT_FIRST
);
private final ConcurrentSkipListSet<DataSegment> segmentsMarkedToDrop
= new ConcurrentSkipListSet<>(DruidCoordinator.SEGMENT_COMPARATOR_RECENT_FIRST);
/**
* Needs to be thread safe since it can be concurrently accessed via
* {@link #failAssign(SegmentHolder, boolean, Exception)}, {@link #actionCompleted(SegmentHolder)},
* {@link #getTimedOutSegments()} and {@link #stop()}
*/
private final ConcurrentSkipListSet<DataSegment> timedOutSegments = new ConcurrentSkipListSet<>(
DruidCoordinator.SEGMENT_COMPARATOR_RECENT_FIRST
);
private final ConcurrentSkipListSet<DataSegment> timedOutSegments =
new ConcurrentSkipListSet<>(DruidCoordinator.SEGMENT_COMPARATOR_RECENT_FIRST);
CuratorLoadQueuePeon(
public CuratorLoadQueuePeon(
CuratorFramework curator,
String basePath,
ObjectMapper jsonMapper,
@ -142,6 +134,15 @@ public class CuratorLoadQueuePeon extends LoadQueuePeon
return segmentsToLoad.keySet();
}
@Override
public Set<SegmentHolder> getSegmentsInQueue()
{
final Set<SegmentHolder> segmentsInQueue = new HashSet<>();
segmentsInQueue.addAll(segmentsToLoad.values());
segmentsInQueue.addAll(segmentsToDrop.values());
return segmentsInQueue;
}
@JsonProperty
@Override
public Set<DataSegment> getSegmentsToDrop()
@ -163,27 +164,21 @@ public class CuratorLoadQueuePeon extends LoadQueuePeon
}
@Override
public long getLoadQueueSize()
public long getSizeOfSegmentsToLoad()
{
return queuedSize.get();
}
@Override
public int getAndResetFailedAssignCount()
public CoordinatorRunStats getAndResetStats()
{
return failedAssignCount.getAndSet(0);
return stats.getSnapshotAndReset();
}
@Override
public int getNumberOfSegmentsInQueue()
public void loadSegment(final DataSegment segment, SegmentAction action, @Nullable final LoadPeonCallback callback)
{
return segmentsToLoad.size();
}
@Override
public void loadSegment(final DataSegment segment, @Nullable final LoadPeonCallback callback)
{
SegmentHolder segmentHolder = new SegmentHolder(segment, Action.LOAD, Collections.singletonList(callback));
SegmentHolder segmentHolder = new SegmentHolder(segment, action, callback);
final SegmentHolder existingHolder = segmentsToLoad.putIfAbsent(segment, segmentHolder);
if (existingHolder != null) {
existingHolder.addCallback(callback);
@ -197,7 +192,7 @@ public class CuratorLoadQueuePeon extends LoadQueuePeon
@Override
public void dropSegment(final DataSegment segment, @Nullable final LoadPeonCallback callback)
{
SegmentHolder segmentHolder = new SegmentHolder(segment, Action.DROP, Collections.singletonList(callback));
SegmentHolder segmentHolder = new SegmentHolder(segment, SegmentAction.DROP, callback);
final SegmentHolder existingHolder = segmentsToDrop.putIfAbsent(segment, segmentHolder);
if (existingHolder != null) {
existingHolder.addCallback(callback);
@ -324,11 +319,13 @@ public class CuratorLoadQueuePeon extends LoadQueuePeon
{
switch (segmentHolder.getAction()) {
case LOAD:
case REPLICATE:
case MOVE_TO:
// When load failed a segment will be removed from the segmentsToLoad twice and
// null value will be returned at the second time in which case queueSize may be negative.
// See https://github.com/apache/druid/pull/10362 for more details.
if (null != segmentsToLoad.remove(segmentHolder.getSegment())) {
queuedSize.addAndGet(-segmentHolder.getSegmentSize());
queuedSize.addAndGet(-segmentHolder.getSegment().getSize());
timedOutSegments.remove(segmentHolder.getSegment());
}
break;
@ -363,7 +360,7 @@ public class CuratorLoadQueuePeon extends LoadQueuePeon
timedOutSegments.clear();
queuedSize.set(0L);
failedAssignCount.set(0);
stats.clear();
}
private void onZkNodeDeleted(SegmentHolder segmentHolder, String path)
@ -391,7 +388,7 @@ public class CuratorLoadQueuePeon extends LoadQueuePeon
if (e != null) {
log.error(e, "Server[%s], throwable caught when submitting [%s].", basePath, segmentHolder);
}
failedAssignCount.getAndIncrement();
stats.add(Stats.SegmentQueue.FAILED_ACTIONS, 1);
if (handleTimeout) {
// Avoid removing the segment entry from the load/drop list in case config.getLoadTimeoutDelay() expires.
@ -409,91 +406,15 @@ public class CuratorLoadQueuePeon extends LoadQueuePeon
}
}
private enum Action
@Override
public boolean cancelOperation(DataSegment segment)
{
LOAD, DROP
}
private static class SegmentHolder
{
private final DataSegment segment;
private final DataSegmentChangeRequest changeRequest;
private final Action type;
// Guaranteed to store only non-null elements
private final List<LoadPeonCallback> callbacks = new ArrayList<>();
private SegmentHolder(
DataSegment segment,
Action type,
Collection<LoadPeonCallback> callbacksParam
)
{
this.segment = segment;
this.type = type;
this.changeRequest = (type == Action.LOAD)
? new SegmentChangeRequestLoad(segment)
: new SegmentChangeRequestDrop(segment);
Iterator<LoadPeonCallback> itr = callbacksParam.iterator();
while (itr.hasNext()) {
LoadPeonCallback c = itr.next();
if (c != null) {
callbacks.add(c);
}
}
}
public DataSegment getSegment()
{
return segment;
}
public Action getAction()
{
return type;
}
public String getSegmentIdentifier()
{
return segment.getId().toString();
}
public long getSegmentSize()
{
return segment.getSize();
}
public void addCallback(@Nullable LoadPeonCallback newCallback)
{
if (newCallback != null) {
synchronized (callbacks) {
callbacks.add(newCallback);
}
}
}
List<LoadPeonCallback> snapshotCallbacks()
{
synchronized (callbacks) {
// Return an immutable copy so that callers don't have to worry about concurrent modification
return ImmutableList.copyOf(callbacks);
}
}
public DataSegmentChangeRequest getChangeRequest()
{
return changeRequest;
}
@Override
public String toString()
{
return changeRequest.toString();
}
return false;
}
private void executeCallbacks(SegmentHolder holder, boolean success)
{
for (LoadPeonCallback callback : holder.snapshotCallbacks()) {
for (LoadPeonCallback callback : holder.getCallbacks()) {
callBackExecutor.submit(() -> callback.execute(success));
}
}

View File

@ -17,12 +17,11 @@
* under the License.
*/
package org.apache.druid.server.coordinator;
package org.apache.druid.server.coordinator.loading;
import com.fasterxml.jackson.core.type.TypeReference;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.fasterxml.jackson.databind.ObjectWriter;
import com.google.common.collect.Iterators;
import com.google.common.util.concurrent.FutureCallback;
import com.google.common.util.concurrent.Futures;
import com.google.common.util.concurrent.ListenableFuture;
@ -36,9 +35,12 @@ import org.apache.druid.java.util.http.client.Request;
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.SegmentChangeRequestDrop;
import org.apache.druid.server.coordination.SegmentChangeRequestLoad;
import org.apache.druid.server.coordination.SegmentLoadDropHandler;
import org.apache.druid.server.coordinator.BytesAccumulatingResponseHandler;
import org.apache.druid.server.coordinator.DruidCoordinatorConfig;
import org.apache.druid.server.coordinator.stats.CoordinatorRunStats;
import org.apache.druid.server.coordinator.stats.CoordinatorStat;
import org.apache.druid.server.coordinator.stats.Stats;
import org.apache.druid.timeline.DataSegment;
import org.jboss.netty.handler.codec.http.HttpHeaders;
import org.jboss.netty.handler.codec.http.HttpMethod;
@ -51,21 +53,22 @@ import java.net.MalformedURLException;
import java.net.URL;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashSet;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.ConcurrentSkipListMap;
import java.util.concurrent.ConcurrentSkipListSet;
import java.util.TreeSet;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong;
/**
*
*/
public class HttpLoadQueuePeon extends LoadQueuePeon
public class HttpLoadQueuePeon implements LoadQueuePeon
{
public static final TypeReference<List<DataSegmentChangeRequest>> REQUEST_ENTITY_TYPE_REF =
new TypeReference<List<DataSegmentChangeRequest>>()
@ -80,17 +83,24 @@ public class HttpLoadQueuePeon extends LoadQueuePeon
private static final EmittingLogger log = new EmittingLogger(HttpLoadQueuePeon.class);
private final AtomicLong queuedSize = new AtomicLong(0);
private final AtomicInteger failedAssignCount = new AtomicInteger(0);
private final CoordinatorRunStats stats = new CoordinatorRunStats();
private final ConcurrentSkipListMap<DataSegment, SegmentHolder> segmentsToLoad = new ConcurrentSkipListMap<>(
DruidCoordinator.SEGMENT_COMPARATOR_RECENT_FIRST
);
private final ConcurrentSkipListMap<DataSegment, SegmentHolder> segmentsToDrop = new ConcurrentSkipListMap<>(
DruidCoordinator.SEGMENT_COMPARATOR_RECENT_FIRST
);
private final ConcurrentSkipListSet<DataSegment> segmentsMarkedToDrop = new ConcurrentSkipListSet<>(
DruidCoordinator.SEGMENT_COMPARATOR_RECENT_FIRST
);
private final ConcurrentMap<DataSegment, SegmentHolder> segmentsToLoad = new ConcurrentHashMap<>();
private final ConcurrentMap<DataSegment, SegmentHolder> segmentsToDrop = new ConcurrentHashMap<>();
private final Set<DataSegment> segmentsMarkedToDrop = ConcurrentHashMap.newKeySet();
/**
* Segments currently in queue ordered by priority and interval. This includes
* drop requests as well. This need not be thread-safe as all operations on it
* are synchronized with the {@link #lock}.
*/
private final Set<SegmentHolder> queuedSegments = new TreeSet<>();
/**
* Set of segments for which requests have been sent to the server and can
* not be cancelled anymore. This need not be thread-safe.
*/
private final Set<DataSegment> activeRequestSegments = new HashSet<>();
private final ScheduledExecutorService processingExecutor;
@ -153,18 +163,25 @@ public class HttpLoadQueuePeon extends LoadQueuePeon
final List<DataSegmentChangeRequest> newRequests = new ArrayList<>(batchSize);
synchronized (lock) {
Iterator<Map.Entry<DataSegment, SegmentHolder>> iter = Iterators.concat(
segmentsToDrop.entrySet().iterator(),
segmentsToLoad.entrySet().iterator()
);
final Iterator<SegmentHolder> queuedSegmentIterator = queuedSegments.iterator();
while (newRequests.size() < batchSize && iter.hasNext()) {
Map.Entry<DataSegment, SegmentHolder> entry = iter.next();
if (entry.getValue().hasTimedOut()) {
entry.getValue().requestFailed("timed out");
iter.remove();
final long currentTimeMillis = System.currentTimeMillis();
while (newRequests.size() < batchSize && queuedSegmentIterator.hasNext()) {
final SegmentHolder holder = queuedSegmentIterator.next();
final DataSegment segment = holder.getSegment();
if (hasRequestTimedOut(holder, currentTimeMillis)) {
onRequestFailed(holder, "timed out");
queuedSegmentIterator.remove();
if (holder.isLoad()) {
segmentsToLoad.remove(segment);
} else {
segmentsToDrop.remove(segment);
}
activeRequestSegments.remove(segment);
} else {
newRequests.add(entry.getValue().getChangeRequest());
newRequests.add(holder.getChangeRequest());
holder.markRequestSentToServer();
activeRequestSegments.add(segment);
}
}
}
@ -304,10 +321,12 @@ public class HttpLoadQueuePeon extends LoadQueuePeon
return;
}
queuedSegments.remove(holder);
activeRequestSegments.remove(holder.getSegment());
if (status.getState() == SegmentLoadDropHandler.Status.STATE.FAILED) {
holder.requestFailed(status.getFailureCause());
onRequestFailed(holder, status.getFailureCause());
} else {
holder.requestSucceeded();
onRequestCompleted(holder, RequestStatus.SUCCESS);
}
}
}, null
@ -347,44 +366,51 @@ public class HttpLoadQueuePeon extends LoadQueuePeon
if (stopped) {
return;
}
log.info("Stopping load queue peon for server [%s].", serverId);
stopped = true;
for (SegmentHolder holder : segmentsToDrop.values()) {
holder.requestFailed("Stopping load queue peon.");
}
for (SegmentHolder holder : segmentsToLoad.values()) {
holder.requestFailed("Stopping load queue peon.");
}
// Cancel all queued requests
queuedSegments.forEach(holder -> onRequestCompleted(holder, RequestStatus.CANCELLED));
log.info("Cancelled [%d] requests queued on server [%s].", queuedSegments.size(), serverId);
segmentsToDrop.clear();
segmentsToLoad.clear();
queuedSegments.clear();
activeRequestSegments.clear();
queuedSize.set(0L);
failedAssignCount.set(0);
stats.clear();
}
}
@Override
public void loadSegment(DataSegment segment, LoadPeonCallback callback)
public void loadSegment(DataSegment segment, SegmentAction action, LoadPeonCallback callback)
{
if (!action.isLoad()) {
log.warn("Invalid load action [%s] for segment [%s] on server [%s].", action, segment.getId(), serverId);
return;
}
synchronized (lock) {
if (stopped) {
log.warn(
"Server[%s] cannot load segment[%s] because load queue peon is stopped.",
serverId,
segment.getId()
serverId, segment.getId()
);
callback.execute(false);
if (callback != null) {
callback.execute(false);
}
return;
}
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);
queuedSegments.add(holder);
processingExecutor.execute(this::doSegmentManagement);
incrementStat(holder, RequestStatus.ASSIGNED);
} else {
holder.addCallback(callback);
}
@ -398,18 +424,22 @@ public class HttpLoadQueuePeon extends LoadQueuePeon
if (stopped) {
log.warn(
"Server[%s] cannot drop segment[%s] because load queue peon is stopped.",
serverId,
segment.getId()
serverId, segment.getId()
);
callback.execute(false);
if (callback != null) {
callback.execute(false);
}
return;
}
SegmentHolder holder = segmentsToDrop.get(segment);
if (holder == null) {
log.trace("Server[%s] to drop segment[%s] queued.", serverId, segment.getId());
segmentsToDrop.put(segment, new DropSegmentHolder(segment, callback));
holder = new SegmentHolder(segment, SegmentAction.DROP, callback);
segmentsToDrop.put(segment, holder);
queuedSegments.add(holder);
processingExecutor.execute(this::doSegmentManagement);
incrementStat(holder, RequestStatus.ASSIGNED);
} else {
holder.addCallback(callback);
}
@ -435,15 +465,25 @@ public class HttpLoadQueuePeon extends LoadQueuePeon
}
@Override
public long getLoadQueueSize()
public Set<SegmentHolder> getSegmentsInQueue()
{
final Set<SegmentHolder> segmentsInQueue;
synchronized (lock) {
segmentsInQueue = new HashSet<>(queuedSegments);
}
return segmentsInQueue;
}
@Override
public long getSizeOfSegmentsToLoad()
{
return queuedSize.get();
}
@Override
public int getAndResetFailedAssignCount()
public CoordinatorRunStats getAndResetStats()
{
return failedAssignCount.getAndSet(0);
return stats.getSnapshotAndReset();
}
@Override
@ -458,146 +498,101 @@ public class HttpLoadQueuePeon extends LoadQueuePeon
segmentsMarkedToDrop.remove(dataSegment);
}
@Override
public int getNumberOfSegmentsInQueue()
{
return segmentsToLoad.size();
}
@Override
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(SegmentHolder holder, long currentTimeMillis)
{
private final DataSegment segment;
private final DataSegmentChangeRequest changeRequest;
private final List<LoadPeonCallback> callbacks = new ArrayList<>();
return holder.isRequestSentToServer()
&& currentTimeMillis - holder.getFirstRequestMillis()
> config.getLoadTimeoutDelay().getMillis();
}
// Time when this request was sent to target server the first time.
private volatile long scheduleTime = -1;
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, RequestStatus.FAILED);
}
private SegmentHolder(
DataSegment segment,
DataSegmentChangeRequest changeRequest,
LoadPeonCallback callback
)
{
this.segment = segment;
this.changeRequest = changeRequest;
private void onRequestCompleted(SegmentHolder holder, RequestStatus 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
);
if (callback != null) {
this.callbacks.add(callback);
if (holder.isLoad()) {
queuedSize.addAndGet(-holder.getSegment().getSize());
}
incrementStat(holder, status);
executeCallbacks(holder, status == RequestStatus.SUCCESS);
}
private void incrementStat(SegmentHolder holder, RequestStatus status)
{
stats.addToDatasourceStat(status.datasourceStat, holder.getSegment().getDataSource(), 1);
}
private void executeCallbacks(SegmentHolder holder, boolean success)
{
callBackExecutor.execute(() -> {
for (LoadPeonCallback callback : holder.getCallbacks()) {
callback.execute(success);
}
}
});
}
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 {
/**
* 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);
}
}
});
}
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();
callBackExecutor.execute(() -> {
for (LoadPeonCallback callback : callbacks) {
if (callback != null) {
callback.execute(false);
}
}
});
}
@Override
public String toString()
{
return changeRequest.toString();
queuedSegments.remove(holder);
onRequestCompleted(holder, RequestStatus.CANCELLED);
return true;
}
}
private class LoadSegmentHolder extends SegmentHolder
private enum RequestStatus
{
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 datasourceStat;
@Override
public void requestFailed(String failureCause)
RequestStatus(CoordinatorStat datasourceStat)
{
queuedSize.addAndGet(-getSegment().getSize());
super.requestFailed(failureCause);
}
}
private class DropSegmentHolder extends SegmentHolder
{
public DropSegmentHolder(DataSegment segment, LoadPeonCallback callback)
{
super(segment, new SegmentChangeRequestDrop(segment), callback);
this.datasourceStat = datasourceStat;
}
}

View File

@ -17,7 +17,7 @@
* under the License.
*/
package org.apache.druid.server.coordinator;
package org.apache.druid.server.coordinator.loading;
/**
*/

View File

@ -17,8 +17,9 @@
* under the License.
*/
package org.apache.druid.server.coordinator;
package org.apache.druid.server.coordinator.loading;
import org.apache.druid.server.coordinator.stats.CoordinatorRunStats;
import org.apache.druid.timeline.DataSegment;
import java.util.Set;
@ -28,30 +29,41 @@ import java.util.Set;
* has been verified enough in production, CuratorLoadQueuePeon and this interface would be removed.
*/
@Deprecated
public abstract class LoadQueuePeon
public interface LoadQueuePeon
{
public abstract void start();
public abstract void stop();
void start();
public abstract Set<DataSegment> getSegmentsToLoad();
void stop();
public abstract Set<DataSegment> getSegmentsToDrop();
Set<DataSegment> getSegmentsToLoad();
public abstract Set<DataSegment> getTimedOutSegments();
Set<SegmentHolder> getSegmentsInQueue();
public abstract void unmarkSegmentToDrop(DataSegment segmentToLoad);
Set<DataSegment> getSegmentsToDrop();
Set<DataSegment> getTimedOutSegments();
public abstract void markSegmentToDrop(DataSegment segmentToLoad);
void markSegmentToDrop(DataSegment segmentToLoad);
public abstract void loadSegment(DataSegment segment, LoadPeonCallback callback);
public abstract void dropSegment(DataSegment segment, LoadPeonCallback callback);
void unmarkSegmentToDrop(DataSegment segmentToLoad);
public abstract long getLoadQueueSize();
Set<DataSegment> getSegmentsMarkedToDrop();
public abstract int getAndResetFailedAssignCount();
void loadSegment(DataSegment segment, SegmentAction action, LoadPeonCallback callback);
public abstract int getNumberOfSegmentsInQueue();
public abstract Set<DataSegment> getSegmentsMarkedToDrop();
void dropSegment(DataSegment segment, LoadPeonCallback callback);
long getSizeOfSegmentsToLoad();
CoordinatorRunStats getAndResetStats();
/**
* Tries to cancel the current operation queued for the given segment on this
* server, if any. A request that has already been sent to the server cannot
* be cancelled.
*
* @return true if the operation was successfully cancelled
*/
boolean cancelOperation(DataSegment segment);
}

View File

@ -17,7 +17,7 @@
* under the License.
*/
package org.apache.druid.server.coordinator;
package org.apache.druid.server.coordinator.loading;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.inject.Provider;
@ -25,6 +25,7 @@ import org.apache.curator.framework.CuratorFramework;
import org.apache.curator.utils.ZKPaths;
import org.apache.druid.client.ImmutableDruidServer;
import org.apache.druid.java.util.http.client.HttpClient;
import org.apache.druid.server.coordinator.DruidCoordinatorConfig;
import org.apache.druid.server.initialization.ZkPathsConfig;
import java.util.concurrent.ExecutorService;

View File

@ -0,0 +1,94 @@
/*
* 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.server.coordinator.CoordinatorDynamicConfig;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Map;
import java.util.Set;
/**
* The ReplicationThrottler is used to throttle the number of segment replicas
* that are assigned to a load queue in a single run. This is achieved by setting
* the values of the following configs:
* <ul>
* <li>{@link CoordinatorDynamicConfig#getReplicationThrottleLimit()} - Maximum
* number of replicas that can be assigned to a tier in a single run.</li>
* <li>{@link CoordinatorDynamicConfig#getMaxNonPrimaryReplicantsToLoad()} -
* Maximum number of total replicas that can be assigned across all tiers in a
* single run.</li>
* </ul>
*/
public class ReplicationThrottler
{
private final int replicationThrottleLimit;
private final int maxReplicaAssignmentsInRun;
private final Map<String, Integer> tierToNumAssigned = new HashMap<>();
private final Set<String> tiersLoadingReplicas = new HashSet<>();
private int totalReplicasAssignedInRun;
/**
* Creates a new ReplicationThrottler for use during a single coordinator run.
*
* @param tiersLoadingReplicas Set of tier names which are already loading
* replicas and will not be eligible for loading
* more replicas in this run.
* @param replicationThrottleLimit Maximum number of replicas that can be
* assigned to a single tier in the current run.
* @param maxReplicaAssignmentsInRun Max number of total replicas that can be
* assigned across all tiers in the current run.
*/
public ReplicationThrottler(
Set<String> tiersLoadingReplicas,
int replicationThrottleLimit,
int maxReplicaAssignmentsInRun
)
{
this.replicationThrottleLimit = replicationThrottleLimit;
this.maxReplicaAssignmentsInRun = maxReplicaAssignmentsInRun;
this.totalReplicasAssignedInRun = 0;
if (tiersLoadingReplicas != null) {
this.tiersLoadingReplicas.addAll(tiersLoadingReplicas);
}
}
public boolean isTierLoadingReplicas(String tier)
{
return tiersLoadingReplicas.contains(tier);
}
public boolean canAssignReplica(String tier)
{
return !tiersLoadingReplicas.contains(tier)
&& totalReplicasAssignedInRun < maxReplicaAssignmentsInRun
&& tierToNumAssigned.computeIfAbsent(tier, t -> 0) < replicationThrottleLimit;
}
public void incrementAssignedReplicas(String tier)
{
++totalReplicasAssignedInRun;
tierToNumAssigned.compute(tier, (t, count) -> (count == null) ? 1 : count + 1);
}
}

View File

@ -17,8 +17,10 @@
* under the License.
*/
package org.apache.druid.server.coordinator;
package org.apache.druid.server.coordinator.loading;
import org.apache.druid.server.coordinator.DruidCluster;
import org.apache.druid.server.coordinator.ServerHolder;
import org.apache.druid.timeline.DataSegment;
import javax.annotation.concurrent.NotThreadSafe;

View File

@ -0,0 +1,81 @@
/*
* 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;
/**
* 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 a segment from a server.
*/
DROP(false),
/**
* Load a segment on a server. This should be used when trying to load a segment
* on a tier where it is currently unavailable (i.e. no replicas loaded).
* This action cannot be throttled by the {@code replicationThrottleLimit}.
*/
LOAD(true),
/**
* Load a replica of a segment on a server. This should be used when trying to
* load more replicas of a segment on a tier where it is already available
* (i.e. atleast one loaded replica).
* <p>
* This is different from LOAD in two ways:
* <ul>
* <li>this action can be throttled by the {@code replicationThrottleLimit}</li>
* <li>it is given lower priority than LOAD on the load queue peon</li>
* </ul>
* For all other purposes, REPLICATE is treated the same as LOAD.
*/
REPLICATE(true),
/**
* Move a segment to this server.
*/
MOVE_TO(true),
/**
* Move a segment from this server to another. This is essentially a pending
* DROP operation, which starts only when the corresponding MOVE_TO has succeded.
*/
MOVE_FROM(false);
private final boolean isLoad;
SegmentAction(boolean isLoad)
{
this.isLoad = isLoad;
}
/**
* True only if this action loads a segment on a server, i.e. LOAD, REPLICATE
* or MOVE_TO.
*/
public boolean isLoad()
{
return isLoad;
}
}

View File

@ -0,0 +1,178 @@
/*
* 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.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.server.coordinator.DruidCoordinator;
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>
*/
private 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);
private int runsInQueue = 0;
public 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 getFirstRequestMillis()
{
return firstRequestMillis.get();
}
public int incrementAndGetRunsInQueue()
{
return ++runsInQueue;
}
@Override
public boolean equals(Object o)
{
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
SegmentHolder that = (SegmentHolder) o;
return getSegment().equals(that.getSegment()) && getAction() == that.getAction();
}
@Override
public int hashCode()
{
return Objects.hash(getSegment(), getAction());
}
@Override
public int compareTo(SegmentHolder that)
{
return Objects.compare(this, that, COMPARE_ACTION_THEN_INTERVAL);
}
@Override
public String toString()
{
return action + "{" +
"segment=" + segment.getId() +
", runsInQueue=" + runsInQueue +
'}';
}
}

View File

@ -0,0 +1,159 @@
/*
* 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.inject.Inject;
import org.apache.druid.client.ServerInventoryView;
import org.apache.druid.java.util.common.logger.Logger;
import org.apache.druid.metadata.SegmentsMetadataManager;
import org.apache.druid.server.coordinator.ServerHolder;
import org.apache.druid.timeline.DataSegment;
/**
* Manager for addition/removal of segments to server load queues and the
* corresponding success/failure callbacks.
*/
public class SegmentLoadQueueManager
{
private static final Logger log = new Logger(SegmentLoadQueueManager.class);
private final LoadQueueTaskMaster taskMaster;
private final ServerInventoryView serverInventoryView;
private final SegmentsMetadataManager segmentsMetadataManager;
@Inject
public SegmentLoadQueueManager(
ServerInventoryView serverInventoryView,
SegmentsMetadataManager segmentsMetadataManager,
LoadQueueTaskMaster taskMaster
)
{
this.serverInventoryView = serverInventoryView;
this.segmentsMetadataManager = segmentsMetadataManager;
this.taskMaster = taskMaster;
}
/**
* Queues load of the segment on the given server.
*/
public boolean loadSegment(DataSegment segment, ServerHolder server, SegmentAction action)
{
try {
if (!server.startOperation(action, segment)) {
return false;
}
server.getPeon().loadSegment(segment, action, null);
return true;
}
catch (Exception e) {
server.cancelOperation(action, segment);
final String serverName = server.getServer().getName();
log.error(e, "Error while loading segment[%s] on server[%s]", segment.getId(), serverName);
return false;
}
}
public boolean dropSegment(DataSegment segment, ServerHolder server)
{
try {
if (server.startOperation(SegmentAction.DROP, segment)) {
server.getPeon().dropSegment(segment, null);
return true;
} else {
return false;
}
}
catch (Exception e) {
server.cancelOperation(SegmentAction.DROP, segment);
final String serverName = server.getServer().getName();
log.error(e, "Error while dropping segment[%s] from server[%s]", segment.getId(), serverName);
return false;
}
}
public boolean moveSegment(
DataSegment segment,
ServerHolder serverA,
ServerHolder serverB
)
{
final LoadQueuePeon peonA = serverA.getPeon();
final LoadPeonCallback moveFinishCallback = success -> peonA.unmarkSegmentToDrop(segment);
if (!serverA.startOperation(SegmentAction.MOVE_FROM, segment)) {
return false;
}
if (!serverB.startOperation(SegmentAction.MOVE_TO, segment)) {
serverA.cancelOperation(SegmentAction.MOVE_FROM, segment);
return false;
}
// mark segment to drop before it is actually loaded on server
// to be able to account for this information in BalancerStrategy immediately
peonA.markSegmentToDrop(segment);
final LoadQueuePeon peonB = serverB.getPeon();
final String serverNameB = serverB.getServer().getName();
try {
peonB.loadSegment(
segment,
SegmentAction.MOVE_TO,
success -> {
// Drop segment only if:
// (1) segment load was successful on serverB
// AND (2) segment is not already queued for drop on serverA
// AND (3a) loading is http-based
// OR (3b) inventory shows segment loaded on serverB
// Do not check the inventory with http loading as the HTTP
// response is enough to determine load success or failure
if (success
&& !peonA.getSegmentsToDrop().contains(segment)
&& (taskMaster.isHttpLoading()
|| serverInventoryView.isSegmentLoadedByServer(serverNameB, segment))) {
peonA.unmarkSegmentToDrop(segment);
peonA.dropSegment(segment, moveFinishCallback);
} else {
moveFinishCallback.execute(success);
}
}
);
}
catch (Exception e) {
serverA.cancelOperation(SegmentAction.MOVE_FROM, segment);
serverB.cancelOperation(SegmentAction.MOVE_TO, segment);
moveFinishCallback.execute(false);
log.error(e, "Error while moving segment[%s] to server[%s]", segment.getId(), serverNameB);
return false;
}
return true;
}
/**
* Marks the given segment as unused.
*/
public boolean deleteSegment(DataSegment segment)
{
return segmentsMetadataManager.markSegmentAsUnused(segment.getId());
}
}

View File

@ -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(),
dynamicConfig.isUseRoundRobinSegmentAssignment(),
dynamicConfig.emitBalancingStats()
);
}
}
private SegmentLoadingConfig(
int maxSegmentsInLoadQueue,
int replicationThrottleLimit,
int maxReplicaAssignmentsInRun,
int maxLifetimeInLoadQueue,
int maxSegmentsToMove,
int percentDecommSegmentsToMove,
boolean useRoundRobinSegmentAssignment,
boolean emitBalancingStats
)
{
this.maxSegmentsInLoadQueue = maxSegmentsInLoadQueue;
this.replicationThrottleLimit = replicationThrottleLimit;
this.maxReplicaAssignmentsInRun = maxReplicaAssignmentsInRun;
this.maxLifetimeInLoadQueue = maxLifetimeInLoadQueue;
this.maxSegmentsToMove = maxSegmentsToMove;
this.percentDecommSegmentsToMove = percentDecommSegmentsToMove;
this.useRoundRobinSegmentAssignment = useRoundRobinSegmentAssignment;
this.emitBalancingStats = emitBalancingStats;
}
public int getMaxSegmentsInLoadQueue()
{
return maxSegmentsInLoadQueue;
}
public int getMaxSegmentsToMove()
{
return maxSegmentsToMove;
}
public int getReplicationThrottleLimit()
{
return replicationThrottleLimit;
}
public boolean isUseRoundRobinSegmentAssignment()
{
return useRoundRobinSegmentAssignment;
}
public boolean isEmitBalancingStats()
{
return emitBalancingStats;
}
public int getMaxLifetimeInLoadQueue()
{
return maxLifetimeInLoadQueue;
}
public int getMaxReplicaAssignmentsInRun()
{
return maxReplicaAssignmentsInRun;
}
public int getPercentDecommSegmentsToMove()
{
return percentDecommSegmentsToMove;
}
}

View File

@ -0,0 +1,167 @@
/*
* 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;
/**
* Counts the number of replicas of a segment in different states (loading, loaded, etc)
* in a tier or the whole cluster.
*/
public class SegmentReplicaCount
{
private int requiredAndLoadable;
private int required;
private int loaded;
private int loadedNonHistorical;
private int loading;
private int dropping;
private int moving;
/**
* Increments number of replicas loaded on historical servers.
*/
void incrementLoaded()
{
++loaded;
}
/**
* Increments number of replicas loaded on non-historical servers. This value
* is used only for computing level of under-replication of broadcast segments.
*/
void incrementLoadedOnNonHistoricalServer()
{
++loadedNonHistorical;
}
/**
* Increments number of replicas queued for the given action.
*/
void incrementQueued(SegmentAction action)
{
switch (action) {
case REPLICATE:
case LOAD:
++loading;
break;
case MOVE_TO:
++moving;
break;
case DROP:
++dropping;
break;
default:
break;
}
}
/**
* Sets the number of required replicas of this segment.
*
* @param required Number of replicas as required by load or broadcast rules.
* @param numLoadingServers Number of servers that can load replicas of this segment.
*/
void setRequired(int required, int numLoadingServers)
{
this.required = required;
this.requiredAndLoadable = Math.min(required, numLoadingServers);
}
/**
* Required number of replicas of the segment as dictated by load rules.
* This includes replicas that may be in excess of the cluster capacity.
*/
public int required()
{
return required;
}
/**
* Required number of replicas of the segment as dictated by load rules.
* This does not include replicas that are in excess of the cluster capacity.
*/
public int requiredAndLoadable()
{
return requiredAndLoadable;
}
int loading()
{
return loading;
}
int moving()
{
return moving;
}
/**
* Number of replicas loaded on all servers. This includes replicas that are
* currently being dropped.
*/
public int totalLoaded()
{
return loaded + loadedNonHistorical;
}
/**
* Number of replicas which are safely loaded on historical servers and are
* not being dropped.
*/
int loadedNotDropping()
{
return loaded - dropping;
}
/**
* Number of replicas that are required to be loaded but are missing.
* This includes replicas that may be in excess of the cluster capacity.
*/
int missing()
{
return Math.max(required() - totalLoaded(), 0);
}
/**
* Number of replicas that are required to be loaded but are missing.
* This does not include replicas that are in excess of the cluster capacity.
*/
int missingAndLoadable()
{
return Math.max(requiredAndLoadable() - totalLoaded(), 0);
}
/**
* Accumulates counts from the given {@code SegmentReplicaCount} into this instance.
*/
void accumulate(SegmentReplicaCount other)
{
this.required += other.required;
this.requiredAndLoadable += other.requiredAndLoadable;
this.loaded += other.loaded;
this.loadedNonHistorical += other.loadedNonHistorical;
this.loading += other.loading;
this.dropping += other.dropping;
this.moving += other.moving;
}
}

View File

@ -0,0 +1,108 @@
/*
* 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.client.ImmutableDruidServer;
import org.apache.druid.server.coordinator.DruidCluster;
import org.apache.druid.timeline.DataSegment;
import org.apache.druid.timeline.SegmentId;
import java.util.Collections;
import java.util.HashMap;
import java.util.Map;
/**
* Contains a mapping from tier to {@link SegmentReplicaCount}s.
* <p>
* Used by the {@link StrategicSegmentAssigner} to make assignment decisions.
*/
public class SegmentReplicaCountMap
{
private final Map<SegmentId, Map<String, SegmentReplicaCount>> replicaCounts = new HashMap<>();
static SegmentReplicaCountMap create(DruidCluster cluster)
{
final SegmentReplicaCountMap replicaCountMap = new SegmentReplicaCountMap();
replicaCountMap.initReplicaCounts(cluster);
return replicaCountMap;
}
private void initReplicaCounts(DruidCluster cluster)
{
cluster.getHistoricals().forEach(
(tier, historicals) -> historicals.forEach(
serverHolder -> {
// Add segments already loaded on this server
for (DataSegment segment : serverHolder.getServedSegments()) {
computeIfAbsent(segment.getId(), tier).incrementLoaded();
}
// Add segments queued for load, drop or move on this server
serverHolder.getQueuedSegments().forEach(
(segment, state) -> computeIfAbsent(segment.getId(), tier)
.incrementQueued(state)
);
}
)
);
cluster.getBrokers().forEach(broker -> {
final ImmutableDruidServer server = broker.getServer();
for (DataSegment segment : server.iterateAllSegments()) {
computeIfAbsent(segment.getId(), server.getTier())
.incrementLoadedOnNonHistoricalServer();
}
});
cluster.getRealtimes().forEach(realtime -> {
final ImmutableDruidServer server = realtime.getServer();
for (DataSegment segment : server.iterateAllSegments()) {
computeIfAbsent(segment.getId(), server.getTier())
.incrementLoadedOnNonHistoricalServer();
}
});
}
SegmentReplicaCount get(SegmentId segmentId, String tier)
{
SegmentReplicaCount count = replicaCounts.getOrDefault(segmentId, Collections.emptyMap())
.get(tier);
return count == null ? new SegmentReplicaCount() : count;
}
SegmentReplicaCount getTotal(SegmentId segmentId)
{
final SegmentReplicaCount total = new SegmentReplicaCount();
replicaCounts.getOrDefault(segmentId, Collections.emptyMap())
.values().forEach(total::accumulate);
return total;
}
public SegmentReplicaCount computeIfAbsent(SegmentId segmentId, String tier)
{
return replicaCounts.computeIfAbsent(segmentId, s -> new HashMap<>())
.computeIfAbsent(tier, t -> new SegmentReplicaCount());
}
public SegmentReplicationStatus toReplicationStatus()
{
return new SegmentReplicationStatus(replicaCounts);
}
}

View File

@ -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<>();
replicaCountsInTier.forEach((segmentId, tierToReplicaCount) -> {
final SegmentReplicaCount total = new SegmentReplicaCount();
tierToReplicaCount.values().forEach(total::accumulate);
totalReplicaCounts.put(segmentId, total);
});
this.totalReplicaCounts = ImmutableMap.copyOf(totalReplicaCounts);
}
public SegmentReplicaCount getReplicaCountsInCluster(SegmentId segmentId)
{
return totalReplicaCounts.get(segmentId);
}
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, SegmentReplicaCount> tierToReplicaCount = replicaCountsInTier.get(segment.getId());
if (tierToReplicaCount == null) {
continue;
}
tierToReplicaCount.forEach((tier, counts) -> {
final int underReplicated = ignoreMissingServers ? counts.missing() : counts.missingAndLoadable();
if (underReplicated >= 0) {
Object2LongOpenHashMap<String> datasourceToUnderReplicated = (Object2LongOpenHashMap<String>)
tierToUnderReplicated.computeIfAbsent(tier, ds -> new Object2LongOpenHashMap<>());
datasourceToUnderReplicated.addTo(segment.getDataSource(), underReplicated);
}
});
}
return tierToUnderReplicated;
}
}

View File

@ -0,0 +1,78 @@
/*
* 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.server.coordinator.ServerHolder;
import org.apache.druid.timeline.DataSegment;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.NavigableSet;
/**
* Maintains a map containing the state of a segment on all servers of a tier.
*/
public class SegmentStatusInTier
{
private final DataSegment segment;
private final List<ServerHolder> eligibleLoadServers = new ArrayList<>();
private final List<ServerHolder> eligibleDropServers = new ArrayList<>();
private final Map<SegmentAction, List<ServerHolder>> serversWithQueuedActions = new HashMap<>();
public SegmentStatusInTier(DataSegment segment, NavigableSet<ServerHolder> historicals)
{
this.segment = segment;
historicals.forEach(this::handleServer);
}
public List<ServerHolder> getServersEligibleToLoad()
{
return eligibleLoadServers;
}
public List<ServerHolder> getServersEligibleToDrop()
{
return eligibleDropServers;
}
public List<ServerHolder> getServersPerforming(SegmentAction action)
{
return serversWithQueuedActions.getOrDefault(action, Collections.emptyList());
}
private void handleServer(ServerHolder server)
{
final SegmentAction action = server.getActionOnSegment(segment);
if (server.isServingSegment(segment)) {
eligibleDropServers.add(server);
} else if (server.canLoadSegment(segment)) {
eligibleLoadServers.add(server);
} else if (action != null) {
serversWithQueuedActions.computeIfAbsent(action, a -> new ArrayList<>())
.add(server);
}
}
}

View File

@ -0,0 +1,606 @@
/*
* 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.Sets;
import it.unimi.dsi.fastutil.objects.Object2IntOpenHashMap;
import org.apache.druid.java.util.emitter.EmittingLogger;
import org.apache.druid.server.coordinator.DruidCluster;
import org.apache.druid.server.coordinator.ServerHolder;
import org.apache.druid.server.coordinator.balancer.BalancerStrategy;
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.HashMap;
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.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 SegmentReplicaCountMap replicaCountMap;
private final ReplicationThrottler replicationThrottler;
private final RoundRobinServerSelector serverSelector;
private final BalancerStrategy strategy;
private final boolean useRoundRobinAssignment;
private final Set<String> tiersWithNoServer = new HashSet<>();
private final Map<String, Integer> tierToHistoricalCount = new HashMap<>();
public StrategicSegmentAssigner(
SegmentLoadQueueManager loadQueueManager,
DruidCluster cluster,
BalancerStrategy strategy,
SegmentLoadingConfig loadingConfig,
CoordinatorRunStats stats
)
{
this.stats = stats;
this.cluster = cluster;
this.strategy = strategy;
this.loadQueueManager = loadQueueManager;
this.replicaCountMap = SegmentReplicaCountMap.create(cluster);
this.replicationThrottler = createReplicationThrottler(cluster, loadingConfig);
this.useRoundRobinAssignment = loadingConfig.isUseRoundRobinSegmentAssignment();
this.serverSelector = useRoundRobinAssignment ? new RoundRobinServerSelector(cluster) : null;
cluster.getHistoricals().forEach(
(tier, historicals) -> tierToHistoricalCount.put(tier, historicals.size())
);
}
public CoordinatorRunStats getStats()
{
return stats;
}
public SegmentReplicationStatus getReplicationStatus()
{
return replicaCountMap.toReplicationStatus();
}
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</li>
* <li>or segment is already optimally placed</li>
* <li>or 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 = replicaCountMap.get(segment.getId(), tier)
.loadedNotDropping();
if (loadedCountOnTier >= 1) {
return replicateSegment(segment, serverB);
} else {
return loadSegment(segment, serverB);
}
}
// 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 replicateSegment(DataSegment segment, Map<String, Integer> tierToReplicaCount)
{
// Identify empty tiers and determine total required replicas
final Set<String> allTiersInCluster = Sets.newHashSet(cluster.getTierNames());
tierToReplicaCount.forEach((tier, requiredReplicas) -> {
reportTierCapacityStats(segment, requiredReplicas, tier);
SegmentReplicaCount replicaCount = replicaCountMap.computeIfAbsent(segment.getId(), tier);
replicaCount.setRequired(requiredReplicas, tierToHistoricalCount.getOrDefault(tier, 0));
if (!allTiersInCluster.contains(tier)) {
tiersWithNoServer.add(tier);
}
});
SegmentReplicaCount replicaCountInCluster = replicaCountMap.getTotal(segment.getId());
final int replicaSurplus = replicaCountInCluster.loadedNotDropping()
- replicaCountInCluster.requiredAndLoadable();
// Update replicas in every tier
int dropsQueued = 0;
for (String tier : allTiersInCluster) {
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 SegmentReplicaCount replicaCountOnTier
= replicaCountMap.get(segment.getId(), tier);
final int projectedReplicas = replicaCountOnTier.loadedNotDropping()
+ replicaCountOnTier.loading();
final int movingReplicas = replicaCountOnTier.moving();
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) {
int numLoadedReplicas = replicaCountOnTier.loadedNotDropping() + cancelledDrops;
int numLoadsQueued = loadReplicas(numReplicasToLoad, numLoadedReplicas, segment, tier, segmentStatus);
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 -> replicaCountMap.computeIfAbsent(segment.getId(), entry.getKey())
.setRequired(entry.getIntValue(), 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)) {
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,
int numLoadedReplicas,
DataSegment segment,
String tier,
SegmentStatusInTier segmentStatus
)
{
final boolean isAlreadyLoadedOnTier = numLoadedReplicas >= 1;
// 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()) {
ServerHolder server = serverIterator.next();
boolean queuedSuccessfully = isAlreadyLoadedOnTier ? replicateSegment(segment, server)
: loadSegment(segment, server);
numLoadsQueued += queuedSuccessfully ? 1 : 0;
}
return numLoadsQueued;
}
private boolean loadSegment(DataSegment segment, ServerHolder server)
{
final String tier = server.getServer().getTier();
final boolean assigned = loadQueueManager.loadSegment(segment, server, SegmentAction.LOAD);
if (!assigned) {
incrementSkipStat(Stats.Segments.ASSIGN_SKIPPED, "Encountered error", segment, tier);
}
return assigned;
}
private boolean replicateSegment(DataSegment segment, ServerHolder server)
{
final String tier = server.getServer().getTier();
if (!replicationThrottler.canAssignReplica(tier)) {
incrementSkipStat(Stats.Segments.ASSIGN_SKIPPED, "Throttled replication", segment, tier);
return false;
}
final boolean assigned = loadQueueManager.loadSegment(segment, server, SegmentAction.REPLICATE);
if (!assigned) {
incrementSkipStat(Stats.Segments.ASSIGN_SKIPPED, "Encountered error", segment, tier);
} else {
replicationThrottler.incrementAssignedReplicas(tier);
}
return assigned;
}
private static ReplicationThrottler createReplicationThrottler(
DruidCluster cluster,
SegmentLoadingConfig loadingConfig
)
{
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,
loadingConfig.getReplicationThrottleLimit(),
loadingConfig.getMaxReplicaAssignmentsInRun()
);
}
private int cancelOperations(
SegmentAction action,
int maxNumToCancel,
DataSegment segment,
SegmentStatusInTier segmentStatus
)
{
final List<ServerHolder> servers = segmentStatus.getServersPerforming(action);
if (servers.isEmpty() || maxNumToCancel <= 0) {
return 0;
}
int numCancelled = 0;
for (int i = 0; i < servers.size() && numCancelled < maxNumToCancel; ++i) {
numCancelled += servers.get(i).cancelOperation(action, segment) ? 1 : 0;
}
return numCancelled;
}
private void incrementSkipStat(CoordinatorStat stat, String reason, DataSegment segment, String tier)
{
final RowKey.Builder keyBuilder
= RowKey.builder()
.add(Dimension.TIER, tier)
.add(Dimension.DATASOURCE, segment.getDataSource());
if (reason != null) {
keyBuilder.add(Dimension.DESCRIPTION, reason);
}
stats.add(stat, keyBuilder.build(), 1);
}
private void incrementStat(CoordinatorStat stat, DataSegment segment, String tier, long value)
{
stats.addToSegmentStat(stat, tier, segment.getDataSource(), value);
}
}

View File

@ -19,152 +19,15 @@
package org.apache.druid.server.coordinator.rules;
import it.unimi.dsi.fastutil.objects.Object2LongMap;
import it.unimi.dsi.fastutil.objects.Object2LongOpenHashMap;
import org.apache.druid.java.util.emitter.EmittingLogger;
import org.apache.druid.server.coordination.ServerType;
import org.apache.druid.server.coordinator.CoordinatorStats;
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.SegmentReplicantLookup;
import org.apache.druid.server.coordinator.ServerHolder;
import org.apache.druid.timeline.DataSegment;
import java.util.HashSet;
import java.util.Map;
import java.util.Set;
import java.util.stream.Collectors;
public abstract class BroadcastDistributionRule implements Rule
{
private static final EmittingLogger log = new EmittingLogger(BroadcastDistributionRule.class);
@Override
public CoordinatorStats run(DruidCoordinator coordinator, DruidCoordinatorRuntimeParams params, DataSegment segment)
public void run(DataSegment segment, SegmentActionHandler handler)
{
final Set<ServerHolder> dropServerHolders = new HashSet<>();
// Find servers where we need to load the broadcast segments
final Set<ServerHolder> loadServerHolders =
params.getDruidCluster().getAllServers()
.stream()
.filter(
(serverHolder) -> {
ServerType serverType = serverHolder.getServer().getType();
if (!serverType.isSegmentBroadcastTarget()) {
return false;
}
final boolean isServingSegment =
serverHolder.isServingSegment(segment);
if (serverHolder.isDecommissioning()) {
if (isServingSegment && !serverHolder.isDroppingSegment(segment)) {
dropServerHolders.add(serverHolder);
}
return false;
}
return !isServingSegment && !serverHolder.isLoadingSegment(segment);
}
)
.collect(Collectors.toSet());
final CoordinatorStats stats = new CoordinatorStats();
return stats.accumulate(assign(loadServerHolders, segment))
.accumulate(drop(dropServerHolders, segment));
handler.broadcastSegment(segment);
}
@Override
public boolean canLoadSegments()
{
return true;
}
@Override
public void updateUnderReplicated(
Map<String, Object2LongMap<String>> underReplicatedPerTier,
SegmentReplicantLookup segmentReplicantLookup,
DataSegment segment
)
{
Object2LongMap<String> underReplicatedBroadcastTiers = segmentReplicantLookup.getBroadcastUnderReplication(segment.getId());
for (final Object2LongMap.Entry<String> entry : underReplicatedBroadcastTiers.object2LongEntrySet()) {
final String tier = entry.getKey();
final long underReplicatedCount = entry.getLongValue();
underReplicatedPerTier.compute(tier, (_tier, existing) -> {
Object2LongMap<String> underReplicationPerDataSource = existing;
if (existing == null) {
underReplicationPerDataSource = new Object2LongOpenHashMap<>();
}
underReplicationPerDataSource.compute(
segment.getDataSource(),
(_datasource, count) -> count != null ? count + underReplicatedCount : underReplicatedCount
);
return underReplicationPerDataSource;
});
}
}
@Override
public void updateUnderReplicatedWithClusterView(
Map<String, Object2LongMap<String>> underReplicatedPerTier,
SegmentReplicantLookup segmentReplicantLookup,
DruidCluster cluster,
DataSegment segment
)
{
updateUnderReplicated(
underReplicatedPerTier,
segmentReplicantLookup,
segment
);
}
private CoordinatorStats assign(
final Set<ServerHolder> serverHolders,
final DataSegment segment
)
{
final CoordinatorStats stats = new CoordinatorStats();
stats.addToGlobalStat(LoadRule.ASSIGNED_COUNT, 0);
for (ServerHolder holder : serverHolders) {
if (segment.getSize() > holder.getAvailableSize()) {
log.makeAlert("Failed to broadcast segment for [%s]", segment.getDataSource())
.addData("segmentId", segment.getId())
.addData("segmentSize", segment.getSize())
.addData("hostName", holder.getServer().getHost())
.addData("availableSize", holder.getAvailableSize())
.emit();
} else {
if (!holder.isLoadingSegment(segment)) {
holder.getPeon().loadSegment(
segment,
null
);
stats.addToGlobalStat(LoadRule.ASSIGNED_COUNT, 1);
}
}
}
return stats;
}
private CoordinatorStats drop(
final Set<ServerHolder> serverHolders,
final DataSegment segment
)
{
CoordinatorStats stats = new CoordinatorStats();
for (ServerHolder holder : serverHolders) {
holder.getPeon().dropSegment(segment, null);
stats.addToGlobalStat(LoadRule.DROPPED_COUNT, 1);
}
return stats;
}
}

View File

@ -19,9 +19,6 @@
package org.apache.druid.server.coordinator.rules;
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.timeline.DataSegment;
/**
@ -30,17 +27,9 @@ import org.apache.druid.timeline.DataSegment;
public abstract class DropRule implements Rule
{
@Override
public CoordinatorStats run(DruidCoordinator coordinator, DruidCoordinatorRuntimeParams params, DataSegment segment)
public void run(DataSegment segment, SegmentActionHandler handler)
{
CoordinatorStats stats = new CoordinatorStats();
coordinator.markSegmentAsUnused(segment);
stats.addToGlobalStat("deletedCount", 1);
return stats;
handler.deleteSegment(segment);
}
@Override
public boolean canLoadSegments()
{
return false;
}
}

View File

@ -41,7 +41,9 @@ public class ForeverLoadRule extends LoadRule
@JsonProperty("tieredReplicants") Map<String, Integer> tieredReplicants
)
{
this.tieredReplicants = tieredReplicants == null ? ImmutableMap.of(DruidServer.DEFAULT_TIER, DruidServer.DEFAULT_NUM_REPLICANTS) : tieredReplicants;
this.tieredReplicants = tieredReplicants == null
? ImmutableMap.of(DruidServer.DEFAULT_TIER, DruidServer.DEFAULT_NUM_REPLICANTS)
: tieredReplicants;
validateTieredReplicants(this.tieredReplicants);
}

View File

@ -19,535 +19,20 @@
package org.apache.druid.server.coordinator.rules;
import it.unimi.dsi.fastutil.objects.Object2IntMap;
import it.unimi.dsi.fastutil.objects.Object2IntOpenHashMap;
import it.unimi.dsi.fastutil.objects.Object2LongMap;
import it.unimi.dsi.fastutil.objects.Object2LongOpenHashMap;
import org.apache.druid.java.util.common.IAE;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.java.util.emitter.EmittingLogger;
import org.apache.druid.server.coordinator.BalancerStrategy;
import org.apache.druid.server.coordinator.CoordinatorStats;
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.SegmentReplicantLookup;
import org.apache.druid.server.coordinator.ServerHolder;
import org.apache.druid.timeline.DataSegment;
import org.apache.druid.timeline.SegmentId;
import javax.annotation.Nullable;
import java.util.Collections;
import java.util.HashMap;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.NavigableSet;
import java.util.TreeSet;
import java.util.function.Predicate;
import java.util.stream.Collectors;
/**
* LoadRules indicate the number of replicants a segment should have in a given tier.
*/
public abstract class LoadRule implements Rule
{
private static final EmittingLogger log = new EmittingLogger(LoadRule.class);
static final String ASSIGNED_COUNT = "assignedCount";
static final String DROPPED_COUNT = "droppedCount";
public final String NON_PRIMARY_ASSIGNED_COUNT = "totalNonPrimaryReplicantsLoaded";
public static final String REQUIRED_CAPACITY = "requiredCapacity";
private final Object2IntMap<String> targetReplicants = new Object2IntOpenHashMap<>();
private final Object2IntMap<String> currentReplicants = new Object2IntOpenHashMap<>();
// Cache to hold unused results from strategy call in assignPrimary
private final Map<String, ServerHolder> strategyCache = new HashMap<>();
@Override
public CoordinatorStats run(
final DruidCoordinator coordinator,
final DruidCoordinatorRuntimeParams params,
final DataSegment segment
)
public void run(DataSegment segment, SegmentActionHandler handler)
{
try {
// get the "snapshots" of targetReplicants and currentReplicants for assignments.
targetReplicants.putAll(getTieredReplicants());
currentReplicants.putAll(params.getSegmentReplicantLookup().getClusterTiers(segment.getId()));
params.getSegmentReplicantLookup().setReplicationFactor(segment.getId(), getReplicationFactor());
final CoordinatorStats stats = new CoordinatorStats();
assign(params, segment, stats);
drop(params, segment, stats);
for (String tier : targetReplicants.keySet()) {
stats.addToTieredStat(REQUIRED_CAPACITY, tier, segment.getSize() * targetReplicants.getInt(tier));
}
return stats;
}
finally {
targetReplicants.clear();
currentReplicants.clear();
strategyCache.clear();
}
}
private int getReplicationFactor()
{
return getTieredReplicants().values().stream().reduce(0, Integer::sum);
}
@Override
public boolean canLoadSegments()
{
return true;
}
@Override
public void updateUnderReplicated(
Map<String, Object2LongMap<String>> underReplicatedPerTier,
SegmentReplicantLookup segmentReplicantLookup,
DataSegment segment
)
{
getTieredReplicants().forEach((final String tier, final Integer ruleReplicants) -> {
int currentReplicants = segmentReplicantLookup.getLoadedReplicants(segment.getId(), tier);
Object2LongMap<String> underReplicationPerDataSource = underReplicatedPerTier.computeIfAbsent(
tier,
ignored -> new Object2LongOpenHashMap<>()
);
((Object2LongOpenHashMap<String>) underReplicationPerDataSource).addTo(
segment.getDataSource(),
Math.max(ruleReplicants - currentReplicants, 0)
);
});
}
@Override
public void updateUnderReplicatedWithClusterView(
Map<String, Object2LongMap<String>> underReplicatedPerTier,
SegmentReplicantLookup segmentReplicantLookup,
DruidCluster cluster,
DataSegment segment
)
{
getTieredReplicants().forEach((final String tier, final Integer ruleReplicants) -> {
int currentReplicants = segmentReplicantLookup.getLoadedReplicants(segment.getId(), tier);
Object2LongMap<String> underReplicationPerDataSource = underReplicatedPerTier.computeIfAbsent(
tier,
ignored -> new Object2LongOpenHashMap<>()
);
int possibleReplicants = Math.min(ruleReplicants, cluster.getHistoricals().get(tier).size());
log.debug(
"ruleReplicants: [%d], possibleReplicants: [%d], currentReplicants: [%d]",
ruleReplicants,
possibleReplicants,
currentReplicants
);
((Object2LongOpenHashMap<String>) underReplicationPerDataSource).addTo(
segment.getDataSource(),
Math.max(possibleReplicants - currentReplicants, 0)
);
});
}
/**
* @param stats {@link CoordinatorStats} to accumulate assignment statistics.
*/
private void assign(
final DruidCoordinatorRuntimeParams params,
final DataSegment segment,
final CoordinatorStats stats
)
{
// if primary replica already exists or is loading
final int loading = params.getSegmentReplicantLookup().getTotalReplicants(segment.getId());
if (!currentReplicants.isEmpty() || loading > 0) {
assignReplicas(params, segment, stats, null);
} else {
final ServerHolder primaryHolderToLoad = assignPrimary(params, segment);
if (primaryHolderToLoad == null) {
// cluster does not have any replicants and cannot identify primary holder
// this implies that no assignment could be done
return;
}
int numAssigned = 1; // 1 replica (i.e., primary replica) already assigned
final String tier = primaryHolderToLoad.getServer().getTier();
// assign replicas for the rest of the tier
numAssigned += assignReplicasForTier(
tier,
targetReplicants.getOrDefault(tier, 0),
numAssigned, // note that the currentReplicantsInTier is the just-assigned primary replica.
params,
createLoadQueueSizeLimitingPredicate(segment).and(holder -> !holder.equals(primaryHolderToLoad)),
segment
);
// numAssigned - 1 because we don't want to count the primary assignment
stats.addToGlobalStat(NON_PRIMARY_ASSIGNED_COUNT, numAssigned - 1);
stats.addToTieredStat(ASSIGNED_COUNT, tier, numAssigned);
// do assign replicas for the other tiers.
assignReplicas(params, segment, stats, tier /* to skip */);
}
}
private static Predicate<ServerHolder> createLoadQueueSizeLimitingPredicate(
final DataSegment segment
)
{
return server -> server != null && server.canLoadSegment(segment);
}
private static List<ServerHolder> getFilteredHolders(
final String tier,
final DruidCluster druidCluster,
final Predicate<ServerHolder> predicate
)
{
final NavigableSet<ServerHolder> queue = druidCluster.getHistoricalsByTier(tier);
if (queue == null) {
log.makeAlert("Tier[%s] has no servers! Check your cluster configuration!", tier).emit();
return Collections.emptyList();
}
Predicate<ServerHolder> isActive = s -> !s.isDecommissioning();
return queue.stream().filter(isActive.and(predicate)).collect(Collectors.toList());
}
private Iterator<ServerHolder> getRoundRobinIterator(
DruidCoordinatorRuntimeParams params,
String tier,
DataSegment segment
)
{
if (params.getRoundRobinServerSelector() == null
|| !params.getCoordinatorDynamicConfig().isUseRoundRobinSegmentAssignment()) {
return null;
}
return params.getRoundRobinServerSelector()
.getServersInTierToLoadSegment(tier, segment);
}
/**
* Iterates through each tier and find the respective segment homes; with the found segment homes, selects the one
* with the highest priority to be the holder for the primary replica.
*/
@Nullable
private ServerHolder assignPrimary(
final DruidCoordinatorRuntimeParams params,
final DataSegment segment
)
{
ServerHolder topCandidate = null;
final boolean useRoundRobinAssignment = params.getCoordinatorDynamicConfig()
.isUseRoundRobinSegmentAssignment();
for (final Object2IntMap.Entry<String> entry : targetReplicants.object2IntEntrySet()) {
final int targetReplicantsInTier = entry.getIntValue();
// sanity check: target number of replicants should be more than zero.
if (targetReplicantsInTier <= 0) {
continue;
}
final String tier = entry.getKey();
String noAvailability = StringUtils.format(
"No available [%s] servers or node capacity to assign primary segment [%s]! %s",
tier,
segment.getId(),
getReplicationLogString()
);
final List<ServerHolder> holders = getFilteredHolders(
tier,
params.getDruidCluster(),
createLoadQueueSizeLimitingPredicate(segment)
);
// no holders satisfy the predicate
if (holders.isEmpty()) {
log.warn(noAvailability);
continue;
}
final ServerHolder candidate;
if (useRoundRobinAssignment) {
Iterator<ServerHolder> roundRobinIterator = getRoundRobinIterator(params, tier, segment);
candidate = roundRobinIterator.hasNext() ? roundRobinIterator.next() : null;
} else {
candidate = params.getBalancerStrategy().findNewSegmentHomeReplicator(segment, holders);
if (candidate != null) {
strategyCache.put(tier, candidate);
}
}
if (candidate == null) {
log.warn(noAvailability);
} else {
if (topCandidate == null ||
candidate.getServer().getPriority() > topCandidate.getServer().getPriority()) {
topCandidate = candidate;
}
}
}
if (topCandidate != null) {
// remove tier for primary replica
strategyCache.remove(topCandidate.getServer().getTier());
log.info(
"Assigning 'primary' for segment [%s] to server [%s] in tier [%s]",
segment.getId(),
topCandidate.getServer().getName(),
topCandidate.getServer().getTier()
);
topCandidate.getPeon().loadSegment(segment, null);
}
return topCandidate;
}
/**
* @param stats {@link CoordinatorStats} to accumulate assignment statistics.
* @param tierToSkip if not null, this tier will be skipped from doing assignment, use when primary replica was
* assigned.
*/
private void assignReplicas(
final DruidCoordinatorRuntimeParams params,
final DataSegment segment,
final CoordinatorStats stats,
@Nullable final String tierToSkip
)
{
for (final Object2IntMap.Entry<String> entry : targetReplicants.object2IntEntrySet()) {
final String tier = entry.getKey();
if (tier.equals(tierToSkip)) {
log.info("Skipping replica assignment for segment [%s] to tier [%s]", segment.getId(), tier);
continue;
}
final int numAssigned = assignReplicasForTier(
tier,
entry.getIntValue(),
params.getSegmentReplicantLookup().getTotalReplicants(segment.getId(), tier),
params,
createLoadQueueSizeLimitingPredicate(segment),
segment
);
stats.addToGlobalStat(NON_PRIMARY_ASSIGNED_COUNT, numAssigned);
stats.addToTieredStat(ASSIGNED_COUNT, tier, numAssigned);
}
}
/**
* @param predicate {@link Predicate} used to pre-filter {@link ServerHolder}s retrieved from {@link DruidCluster}.
*/
private int assignReplicasForTier(
final String tier,
final int targetReplicantsInTier,
final int currentReplicantsInTier,
final DruidCoordinatorRuntimeParams params,
final Predicate<ServerHolder> predicate,
final DataSegment segment
)
{
final int numToAssign = targetReplicantsInTier - currentReplicantsInTier;
// if nothing to assign
if (numToAssign <= 0) {
return 0;
}
String noAvailability = StringUtils.format(
"No available [%s] servers or node capacity to assign segment [%s]! %s",
tier,
segment.getId(),
getReplicationLogString()
);
final List<ServerHolder> holders = getFilteredHolders(tier, params.getDruidCluster(), predicate);
// if no holders available for assignment
if (holders.isEmpty()) {
log.warn(noAvailability);
return 0;
}
final Iterator<ServerHolder> roundRobinServerIterator = getRoundRobinIterator(params, tier, segment);
final ReplicationThrottler throttler = params.getReplicationManager();
for (int numAssigned = 0; numAssigned < numToAssign; numAssigned++) {
if (!throttler.canCreateReplicant(tier)) {
log.info("Throttling replication for segment [%s] in tier [%s]. %s", segment.getId(), tier, getReplicationLogString());
return numAssigned;
}
// Retrieves from cache if available
final ServerHolder holder;
if (strategyCache.containsKey(tier)) {
// found in cache
holder = strategyCache.remove(tier);
} else if (roundRobinServerIterator == null) {
// Call balancer strategy
holder = params.getBalancerStrategy().findNewSegmentHomeReplicator(segment, holders);
} else {
holder = roundRobinServerIterator.hasNext() ? roundRobinServerIterator.next() : null;
}
if (holder == null) {
log.warn(noAvailability);
return numAssigned;
}
holders.remove(holder);
final SegmentId segmentId = segment.getId();
final String holderHost = holder.getServer().getHost();
throttler.registerReplicantCreation(tier, segmentId, holderHost);
log.info(
"Assigning 'replica' for segment [%s] to server [%s] in tier [%s]. %s",
segment.getId(),
holder.getServer().getName(),
holder.getServer().getTier(),
getReplicationLogString()
);
holder.getPeon().loadSegment(
segment,
loadSuccess -> throttler.unregisterReplicantCreation(tier, segmentId)
);
}
return numToAssign;
}
/**
* @param stats {@link CoordinatorStats} to accumulate assignment statistics.
*/
private void drop(
final DruidCoordinatorRuntimeParams params,
final DataSegment segment,
final CoordinatorStats stats
)
{
final DruidCluster druidCluster = params.getDruidCluster();
final boolean isLoading = loadingInProgress(druidCluster);
for (final Object2IntMap.Entry<String> entry : currentReplicants.object2IntEntrySet()) {
final String tier = entry.getKey();
final NavigableSet<ServerHolder> holders = druidCluster.getHistoricalsByTier(tier);
final int numDropped;
if (holders == null) {
log.makeAlert("No holders found for tier[%s]", tier).emit();
numDropped = 0;
} else {
final int currentReplicantsInTier = entry.getIntValue();
final int numToDrop = currentReplicantsInTier - targetReplicants.getOrDefault(tier, 0);
if (numToDrop > 0) {
// This enforces that loading is completed before we attempt to drop stuffs as a safety measure.
if (isLoading) {
log.info(
"Loading in progress for segment [%s], skipping drop from tier [%s] until loading is complete! %s",
segment.getId(),
tier,
getReplicationLogString()
);
break;
}
numDropped = dropForTier(
numToDrop,
holders,
segment,
params.getBalancerStrategy(),
getReplicationLogString()
);
} else {
numDropped = 0;
}
}
stats.addToTieredStat(DROPPED_COUNT, tier, numDropped);
}
}
/**
* Returns true if at least one tier in target replica assignment exists in cluster but does not have enough replicas.
*/
private boolean loadingInProgress(final DruidCluster druidCluster)
{
for (final Object2IntMap.Entry<String> entry : targetReplicants.object2IntEntrySet()) {
final String tier = entry.getKey();
// if there are replicants loading in cluster
if (druidCluster.hasTier(tier) && entry.getIntValue() > currentReplicants.getOrDefault(tier, 0)) {
return true;
}
}
return false;
}
private static int dropForTier(
final int numToDrop,
final NavigableSet<ServerHolder> holdersInTier,
final DataSegment segment,
final BalancerStrategy balancerStrategy,
final String replicationLog
)
{
Map<Boolean, TreeSet<ServerHolder>> holders = holdersInTier.stream()
.filter(s -> s.isServingSegment(segment))
.collect(Collectors.partitioningBy(
ServerHolder::isDecommissioning,
Collectors.toCollection(TreeSet::new)
));
TreeSet<ServerHolder> decommissioningServers = holders.get(true);
TreeSet<ServerHolder> activeServers = holders.get(false);
int left = dropSegmentFromServers(balancerStrategy, segment, decommissioningServers, numToDrop, replicationLog);
if (left > 0) {
left = dropSegmentFromServers(balancerStrategy, segment, activeServers, left, replicationLog);
}
if (left != 0) {
log.warn("I have no servers serving [%s]?", segment.getId());
}
return numToDrop - left;
}
private static int dropSegmentFromServers(
final BalancerStrategy balancerStrategy,
final DataSegment segment,
final NavigableSet<ServerHolder> holders,
int numToDrop,
final String replicationLog
)
{
final Iterator<ServerHolder> iterator = balancerStrategy.pickServersToDrop(segment, holders);
while (numToDrop > 0) {
if (!iterator.hasNext()) {
break;
}
final ServerHolder holder = iterator.next();
if (holder.isServingSegment(segment)) {
log.info(
"Dropping segment [%s] on server [%s] in tier [%s]. %s",
segment.getId(),
holder.getServer().getName(),
holder.getServer().getTier(),
replicationLog
);
holder.getPeon().dropSegment(segment, null);
numToDrop--;
} else {
log.warn(
"Server [%s] is no longer serving segment [%s], skipping drop.",
holder.getServer().getName(),
segment.getId()
);
}
}
return numToDrop;
handler.replicateSegment(segment, getTieredReplicants());
}
protected static void validateTieredReplicants(final Map<String, Integer> tieredReplicants)
@ -569,20 +54,4 @@ public abstract class LoadRule implements Rule
public abstract int getNumReplicants(String tier);
protected String getReplicationLogString()
{
StringBuilder builder = new StringBuilder("Current replication: [");
for (final Object2IntMap.Entry<String> entry : currentReplicants.object2IntEntrySet()) {
final String tier = entry.getKey();
// [hot:1/2][cold:2/2]
builder.append("[")
.append(tier)
.append(":")
.append(entry.getIntValue())
.append("/")
.append(targetReplicants.getInt(tier))
.append("]");
}
return builder.append("]").toString();
}
}

View File

@ -21,20 +21,12 @@ package org.apache.druid.server.coordinator.rules;
import com.fasterxml.jackson.annotation.JsonSubTypes;
import com.fasterxml.jackson.annotation.JsonTypeInfo;
import com.google.common.base.Preconditions;
import it.unimi.dsi.fastutil.objects.Object2LongMap;
import org.apache.druid.server.coordinator.CoordinatorStats;
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.SegmentReplicantLookup;
import org.apache.druid.timeline.DataSegment;
import org.joda.time.DateTime;
import org.joda.time.Interval;
import java.util.Map;
/**
* Retention rule that governs retention and distribution of segments in a cluster.
*/
@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type")
@JsonSubTypes(value = {
@ -57,54 +49,5 @@ public interface Rule
boolean appliesTo(Interval interval, DateTime referenceTimestamp);
/**
* Return true if this Rule can load segment onto one or more type of Druid node, otherwise return false.
* Any Rule that returns true for this method should implement logic for calculating segment under replicated
* in {@link Rule#updateUnderReplicated}
*/
boolean canLoadSegments();
/**
* This method should update the {@param underReplicatedPerTier} with the replication count of the
* {@param segment}. Rule that returns true for {@link Rule#canLoadSegments()} must override this method.
* Note that {@param underReplicatedPerTier} is a map of tier -> { dataSource -> underReplicationCount }
*/
default void updateUnderReplicated(
Map<String, Object2LongMap<String>> underReplicatedPerTier,
SegmentReplicantLookup segmentReplicantLookup,
DataSegment segment
)
{
Preconditions.checkArgument(!canLoadSegments());
}
/**
* This method should update the {@param underReplicatedPerTier} with the replication count of the
* {@param segment} taking into consideration the number of servers available in cluster that the segment can be
* replicated on. Rule that returns true for {@link Rule#canLoadSegments()} must override this method.
* Note that {@param underReplicatedPerTier} is a map of tier -> { dataSource -> underReplicationCount }
*/
default void updateUnderReplicatedWithClusterView(
Map<String, Object2LongMap<String>> underReplicatedPerTier,
SegmentReplicantLookup segmentReplicantLookup,
DruidCluster cluster,
DataSegment segment
)
{
Preconditions.checkArgument(!canLoadSegments());
}
/**
* {@link DruidCoordinatorRuntimeParams#getUsedSegments()} must not be called in Rule's code, because the used
* segments are not specified for the {@link DruidCoordinatorRuntimeParams} passed into Rule's code. This is because
* {@link DruidCoordinatorRuntimeParams} entangles two slightly different (nonexistent yet) abstractions:
* "CoordinatorDutyParams" and "RuleParams" which contain params that only {@link
* org.apache.druid.server.coordinator.duty.CoordinatorDuty} objects and Rules need, respectively. For example,
* {@link org.apache.druid.server.coordinator.ReplicationThrottler} needs to belong only to "RuleParams", but not to
* "CoordinatorDutyParams". The opposite for the collection of used segments and {@link
* org.apache.druid.client.DataSourcesSnapshot}.
*
* See https://github.com/apache/druid/issues/7228
*/
CoordinatorStats run(DruidCoordinator coordinator, DruidCoordinatorRuntimeParams params, DataSegment segment);
void run(DataSegment segment, SegmentActionHandler segmentHandler);
}

View File

@ -19,32 +19,32 @@
package org.apache.druid.server.coordinator.rules;
import java.util.ArrayList;
import java.util.List;
import org.apache.druid.timeline.DataSegment;
import java.util.Map;
/**
* Performs various actions on a given segment. Used by {@link Rule}s to load,
* drop, broadcast or delete segments.
*/
public class RuleMap
public interface SegmentActionHandler
{
private final Map<String, List<Rule>> rules;
private final List<Rule> defaultRules;
public RuleMap(Map<String, List<Rule>> rules, List<Rule> defaultRules)
{
this.rules = rules;
this.defaultRules = defaultRules;
}
/**
* Queues load or drop of replicas of the given segment to achieve the
* target replication level on all historical tiers.
*/
void replicateSegment(DataSegment segment, Map<String, Integer> tierToReplicaCount);
/**
* Marks the given segment as unused. Unused segments are eventually unloaded
* from all servers and deleted from metadata as well as deep storage.
*/
void deleteSegment(DataSegment segment);
/**
* Broadcasts the given segment to all servers that are broadcast targets.
*/
void broadcastSegment(DataSegment segment);
public List<Rule> getRules(String dataSource)
{
List<Rule> retVal = new ArrayList<>();
if (dataSource != null) {
retVal.addAll(rules.get(dataSource));
}
if (defaultRules != null) {
retVal.addAll(defaultRules);
}
return retVal;
}
}

View File

@ -0,0 +1,273 @@
/*
* 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.stats;
import it.unimi.dsi.fastutil.objects.Object2LongOpenHashMap;
import org.apache.druid.java.util.common.StringUtils;
import javax.annotation.concurrent.ThreadSafe;
import java.util.Collections;
import java.util.EnumMap;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Map;
import java.util.Objects;
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.atomic.AtomicInteger;
/**
* Contains statistics typically tracked during a single coordinator run or the
* runtime of a single coordinator duty.
*/
@ThreadSafe
public class CoordinatorRunStats
{
private final ConcurrentHashMap<RowKey, Object2LongOpenHashMap<CoordinatorStat>>
allStats = new ConcurrentHashMap<>();
private final Map<Dimension, String> debugDimensions = new HashMap<>();
public CoordinatorRunStats()
{
this(null);
}
/**
* Creates a new {@code CoordinatorRunStats}.
*
* @param debugDimensions Dimension values for which all metrics should be
* collected and logged.
*/
public CoordinatorRunStats(Map<Dimension, String> debugDimensions)
{
if (debugDimensions != null) {
this.debugDimensions.putAll(debugDimensions);
}
}
public long getSegmentStat(CoordinatorStat stat, String tier, String datasource)
{
return get(stat, RowKey.builder().add(Dimension.DATASOURCE, datasource).add(Dimension.TIER, tier).build());
}
public long getDataSourceStat(CoordinatorStat stat, String dataSource)
{
return get(stat, RowKey.forDatasource(dataSource));
}
public long get(CoordinatorStat stat)
{
return get(stat, RowKey.EMPTY);
}
public long get(CoordinatorStat stat, RowKey rowKey)
{
Object2LongOpenHashMap<CoordinatorStat> statValues = allStats.get(rowKey);
return statValues == null ? 0 : statValues.getLong(stat);
}
public void forEachStat(StatHandler handler)
{
allStats.forEach(
(rowKey, stats) -> stats.object2LongEntrySet().fastForEach(
stat -> handler.handle(rowKey.getValues(), stat.getKey(), stat.getLongValue())
)
);
}
/**
* Builds a printable table of all the collected error, info and debug level
* stats (if applicable) with non-zero values.
*/
public String buildStatsTable()
{
final StringBuilder statsTable = new StringBuilder();
final AtomicInteger hiddenStats = new AtomicInteger(0);
final AtomicInteger totalStats = new AtomicInteger();
allStats.forEach(
(rowKey, statMap) -> {
// Categorize the stats by level
final Map<CoordinatorStat.Level, Map<CoordinatorStat, Long>> levelToStats
= new EnumMap<>(CoordinatorStat.Level.class);
statMap.object2LongEntrySet().fastForEach(
stat -> levelToStats.computeIfAbsent(stat.getKey().getLevel(), l -> new HashMap<>())
.put(stat.getKey(), stat.getLongValue())
);
// Add all the errors
final Map<CoordinatorStat, Long> errorStats = levelToStats
.getOrDefault(CoordinatorStat.Level.ERROR, Collections.emptyMap());
totalStats.addAndGet(errorStats.size());
if (!errorStats.isEmpty()) {
statsTable.append(
StringUtils.format("\nError: %s ==> %s", rowKey, errorStats)
);
}
// Add all the info level stats
final Map<CoordinatorStat, Long> infoStats = levelToStats
.getOrDefault(CoordinatorStat.Level.INFO, Collections.emptyMap());
totalStats.addAndGet(infoStats.size());
if (!infoStats.isEmpty()) {
statsTable.append(
StringUtils.format("\nInfo : %s ==> %s", rowKey, infoStats)
);
}
// Add all the debug level stats if the row key has a debug dimension
final Map<CoordinatorStat, Long> debugStats = levelToStats
.getOrDefault(CoordinatorStat.Level.DEBUG, Collections.emptyMap());
totalStats.addAndGet(debugStats.size());
if (!debugStats.isEmpty() && hasDebugDimension(rowKey)) {
statsTable.append(
StringUtils.format("\nDebug: %s ==> %s", rowKey, debugStats)
);
} else {
hiddenStats.addAndGet(debugStats.size());
}
}
);
if (hiddenStats.get() > 0) {
statsTable.append(
StringUtils.format("\nDebug: %d hidden stats. Set 'debugDimensions' to see these.", hiddenStats.get())
);
}
if (totalStats.get() > 0) {
statsTable.append(
StringUtils.format("\nTOTAL: %d stats for %d dimension keys", totalStats.get(), rowCount())
);
}
return statsTable.toString();
}
public boolean hasStat(CoordinatorStat stat)
{
for (Object2LongOpenHashMap<CoordinatorStat> statValues : allStats.values()) {
if (statValues.containsKey(stat)) {
return true;
}
}
return false;
}
public int rowCount()
{
return allStats.size();
}
public void clear()
{
allStats.clear();
}
public void add(CoordinatorStat stat, long value)
{
add(stat, RowKey.EMPTY, value);
}
public void add(CoordinatorStat stat, RowKey rowKey, long value)
{
// Do not add a stat which will neither be emitted nor logged
if (!stat.shouldEmit()
&& stat.getLevel() == CoordinatorStat.Level.DEBUG
&& !hasDebugDimension(rowKey)) {
return;
}
allStats.computeIfAbsent(rowKey, d -> new Object2LongOpenHashMap<>())
.addTo(stat, value);
}
public void addToDatasourceStat(CoordinatorStat stat, String dataSource, long value)
{
add(stat, RowKey.forDatasource(dataSource), value);
}
public void addToSegmentStat(CoordinatorStat stat, String tier, String datasource, long value)
{
RowKey rowKey = RowKey.builder()
.add(Dimension.TIER, tier)
.add(Dimension.DATASOURCE, datasource).build();
add(stat, rowKey, value);
}
/**
* Updates the maximum value of the stat for the given RowKey if applicable.
*/
public void updateMax(CoordinatorStat stat, RowKey rowKey, long value)
{
allStats.computeIfAbsent(rowKey, d -> new Object2LongOpenHashMap<>())
.mergeLong(stat, value, Math::max);
}
/**
* Creates a new {@code CoordinatorRunStats} which represents the snapshot of
* the stats collected so far in this instance.
* <p>
* While this method is in progress, any updates made to the stats of this
* instance by another thread are not guaranteed to be present in the snapshot.
* But the snapshots are consistent, i.e. stats present in the snapshot created
* in one invocation of this method are permanently removed from this instance
* and will not be present in subsequent snapshots.
*
* @return Snapshot of the current state of this {@code CoordinatorRunStats}.
*/
public CoordinatorRunStats getSnapshotAndReset()
{
final CoordinatorRunStats snapshot = new CoordinatorRunStats(debugDimensions);
// Get a snapshot of all the keys, remove and copy each of them atomically
final Set<RowKey> keys = new HashSet<>(allStats.keySet());
for (RowKey key : keys) {
snapshot.allStats.put(key, allStats.remove(key));
}
return snapshot;
}
/**
* Checks if the given rowKey has any of the debug dimensions.
*/
private boolean hasDebugDimension(RowKey rowKey)
{
if (debugDimensions.isEmpty()) {
return false;
}
for (Map.Entry<Dimension, String> entry : rowKey.getValues().entrySet()) {
String expectedValue = debugDimensions.get(entry.getKey());
if (Objects.equals(expectedValue, entry.getValue())) {
return true;
}
}
return false;
}
public interface StatHandler
{
void handle(Map<Dimension, String> dimensionValues, CoordinatorStat stat, long statValue);
}
}

View File

@ -0,0 +1,99 @@
/*
* 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.stats;
/**
* A coordinator statistic, which may or may not be emitted as a metric.
*/
public class CoordinatorStat
{
private final String metricName;
private final String shortName;
private final Level level;
/**
* Creates a new non-emitting, DEBUG level stat.
*/
public CoordinatorStat(String shortStatName)
{
this(shortStatName, null, Level.DEBUG);
}
public CoordinatorStat(String shortName, Level level)
{
this(shortName, null, level);
}
/**
* Creates a new emitting, DEBUG level stat.
*/
public CoordinatorStat(String shortStatName, String metricName)
{
this(shortStatName, metricName, Level.DEBUG);
}
public CoordinatorStat(String shortStatName, String metricName, Level level)
{
this.metricName = metricName;
this.shortName = shortStatName;
this.level = level == null ? Level.DEBUG : level;
}
/**
* Name of the metric emitted for this stat, if any.
*/
public String getMetricName()
{
return metricName;
}
public String getShortName()
{
return shortName;
}
public Level getLevel()
{
return level;
}
/**
* Whether this statistic should be emitted as a metric.
*/
public boolean shouldEmit()
{
return metricName != null;
}
@Override
public String toString()
{
return shortName;
}
/**
* Level of coordinator stat, typically used for logging.
*/
public enum Level
{
DEBUG, INFO, ERROR
}
}

View File

@ -17,18 +17,38 @@
* under the License.
*/
package org.apache.druid.server.coordinator;
package org.apache.druid.server.coordinator.stats;
public class CoordinatorRuntimeParamsTestHelpers
/**
* Dimensions used while collecting or reporting coordinator run stats.
*/
public enum Dimension
{
public static DruidCoordinatorRuntimeParams.Builder newBuilder()
TIER("tier"),
DATASOURCE("dataSource"),
DUTY("duty"),
DUTY_GROUP("dutyGroup"),
DESCRIPTION("description"),
SERVER("server");
private final String reportedName;
Dimension(String name)
{
return DruidCoordinatorRuntimeParams
.newBuilder()
.withStartTimeNanos(System.nanoTime());
this.reportedName = name;
}
private CoordinatorRuntimeParamsTestHelpers()
/**
* The name of this dimension used while emitting metrics.
*/
public String reportedName()
{
return reportedName;
}
@Override
public String toString()
{
return reportedName;
}
}

View File

@ -0,0 +1,103 @@
/*
* 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.stats;
import java.util.Collections;
import java.util.EnumMap;
import java.util.Map;
import java.util.Objects;
/**
* Represents a row key against which stats are reported.
*/
public class RowKey
{
public static final RowKey EMPTY = new RowKey(Collections.emptyMap());
private final Map<Dimension, String> values;
private final int hashCode;
private RowKey(Map<Dimension, String> values)
{
this.values = values;
this.hashCode = Objects.hash(values);
}
public static Builder builder()
{
return new RowKey.Builder();
}
public static RowKey forTier(String tier)
{
return RowKey.builder().add(Dimension.TIER, tier).build();
}
public static RowKey forDatasource(String datasource)
{
return RowKey.builder().add(Dimension.DATASOURCE, datasource).build();
}
public Map<Dimension, String> getValues()
{
return values;
}
@Override
public boolean equals(Object o)
{
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
RowKey that = (RowKey) o;
return Objects.equals(values, that.values);
}
@Override
public int hashCode()
{
return hashCode;
}
public static class Builder
{
private final Map<Dimension, String> values = new EnumMap<>(Dimension.class);
public Builder add(Dimension dimension, String value)
{
values.put(dimension, value);
return this;
}
public RowKey build()
{
return new RowKey(values);
}
}
@Override
public String toString()
{
return values == null ? "{}" : values.toString();
}
}

View File

@ -0,0 +1,141 @@
/*
* 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.stats;
/**
* List of Coordinator stats.
*/
public class Stats
{
public static class Segments
{
// Decisions taken in a run
public static final CoordinatorStat ASSIGNED
= new CoordinatorStat("assigned", "segment/assigned/count", CoordinatorStat.Level.INFO);
public static final CoordinatorStat DROPPED
= new CoordinatorStat("dropped", "segment/dropped/count", CoordinatorStat.Level.INFO);
public static final CoordinatorStat DELETED
= new CoordinatorStat("deleted", "segment/deleted/count", CoordinatorStat.Level.INFO);
public static final CoordinatorStat MOVED
= new CoordinatorStat("moved", "segment/moved/count");
// Skipped decisions in a run
public static final CoordinatorStat ASSIGN_SKIPPED
= new CoordinatorStat("assignSkip", "segment/assignSkipped/count");
public static final CoordinatorStat DROP_SKIPPED
= new CoordinatorStat("dropSkip", "segment/dropSkipped/count");
public static final CoordinatorStat MOVE_SKIPPED
= new CoordinatorStat("moveSkip", "segment/moveSkipped/count");
// Current state of segments of a datasource
public static final CoordinatorStat USED
= new CoordinatorStat("usedSegments", "segment/count");
public static final CoordinatorStat USED_BYTES
= new CoordinatorStat("usedSegmentBytes", "segment/size");
public static final CoordinatorStat UNDER_REPLICATED
= new CoordinatorStat("underreplicated", "segment/underReplicated/count");
public static final CoordinatorStat UNAVAILABLE
= new CoordinatorStat("unavailable", "segment/unavailable/count");
public static final CoordinatorStat UNNEEDED
= new CoordinatorStat("unneeded", "segment/unneeded/count");
public static final CoordinatorStat OVERSHADOWED
= new CoordinatorStat("overshadowed", "segment/overshadowed/count");
}
public static class SegmentQueue
{
public static final CoordinatorStat NUM_TO_LOAD
= new CoordinatorStat("numToLoad", "segment/loadQueue/count");
public static final CoordinatorStat BYTES_TO_LOAD
= new CoordinatorStat("bytesToLoad", "segment/loadQueue/size");
public static final CoordinatorStat NUM_TO_DROP
= new CoordinatorStat("numToDrop", "segment/dropQueue/count");
public static final CoordinatorStat ASSIGNED_ACTIONS
= new CoordinatorStat("assignedActions", "segment/loadQueue/assigned");
public static final CoordinatorStat COMPLETED_ACTIONS
= new CoordinatorStat("successActions", "segment/loadQueue/success");
public static final CoordinatorStat FAILED_ACTIONS
= new CoordinatorStat("failedActions", "segment/loadQueue/failed", CoordinatorStat.Level.ERROR);
public static final CoordinatorStat CANCELLED_ACTIONS
= new CoordinatorStat("cancelledActions", "segment/loadQueue/cancelled");
}
public static class Tier
{
public static final CoordinatorStat REQUIRED_CAPACITY
= new CoordinatorStat("reqdCap", "tier/required/capacity");
public static final CoordinatorStat TOTAL_CAPACITY
= new CoordinatorStat("totalCap", "tier/total/capacity");
public static final CoordinatorStat REPLICATION_FACTOR
= new CoordinatorStat("maxRepFactor", "tier/replication/factor");
public static final CoordinatorStat HISTORICAL_COUNT
= new CoordinatorStat("numHistorical", "tier/historical/count");
}
public static class Compaction
{
public static final CoordinatorStat SUBMITTED_TASKS
= new CoordinatorStat("compactTasks", "compact/task/count");
public static final CoordinatorStat MAX_SLOTS
= new CoordinatorStat("compactMaxSlots", "compactTask/maxSlot/count");
public static final CoordinatorStat AVAILABLE_SLOTS
= new CoordinatorStat("compactAvlSlots", "compactTask/availableSlot/count");
public static final CoordinatorStat PENDING_BYTES
= new CoordinatorStat("compactPendingBytes", "segment/waitCompact/bytes");
public static final CoordinatorStat COMPACTED_BYTES
= new CoordinatorStat("compactedBytes", "segment/compacted/bytes");
public static final CoordinatorStat SKIPPED_BYTES
= new CoordinatorStat("compactSkipBytes", "segment/skipCompact/bytes");
public static final CoordinatorStat PENDING_SEGMENTS
= new CoordinatorStat("compactPendingSeg", "segment/waitCompact/count");
public static final CoordinatorStat COMPACTED_SEGMENTS
= new CoordinatorStat("compactedSeg", "segment/compacted/count");
public static final CoordinatorStat SKIPPED_SEGMENTS
= new CoordinatorStat("compactSkipSeg", "segment/skipCompact/count");
public static final CoordinatorStat PENDING_INTERVALS
= new CoordinatorStat("compactPendingIntv", "interval/waitCompact/count");
public static final CoordinatorStat COMPACTED_INTERVALS
= new CoordinatorStat("compactedIntv", "interval/compacted/count");
public static final CoordinatorStat SKIPPED_INTERVALS
= new CoordinatorStat("compactSkipIntv", "interval/skipCompact/count");
}
public static class CoordinatorRun
{
public static final CoordinatorStat DUTY_RUN_TIME
= new CoordinatorStat("dutyRunTime", "coordinator/time");
public static final CoordinatorStat GROUP_RUN_TIME
= new CoordinatorStat("groupRunTime", "coordinator/global/time");
}
public static class Balancer
{
public static final CoordinatorStat RAW_COST
= new CoordinatorStat("initialCost", "segment/cost/raw");
public static final CoordinatorStat NORMALIZATION_COST
= new CoordinatorStat("normaliznCost", "segment/cost/normalization");
public static final CoordinatorStat NORMALIZED_COST_X_1000
= new CoordinatorStat("normalizedCost", "segment/cost/normalized");
}
}

View File

@ -121,8 +121,7 @@ public class CoordinatorDynamicConfigsResource
CoordinatorDynamicConfig.CONFIG_KEY,
count
)
)
.build();
).build();
}
catch (IllegalArgumentException e) {
return Response.status(Response.Status.BAD_REQUEST)
@ -136,8 +135,7 @@ public class CoordinatorDynamicConfigsResource
CoordinatorDynamicConfig.CONFIG_KEY,
theInterval
)
)
.build();
).build();
}
}

View File

@ -19,14 +19,12 @@
package org.apache.druid.server.http;
import com.google.common.base.Function;
import com.google.common.collect.Collections2;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Maps;
import com.google.inject.Inject;
import com.sun.jersey.spi.container.ResourceFilters;
import org.apache.druid.server.coordinator.DruidCoordinator;
import org.apache.druid.server.coordinator.LoadQueuePeon;
import org.apache.druid.server.http.security.StateResourceFilter;
import org.apache.druid.timeline.DataSegment;
@ -47,9 +45,7 @@ public class CoordinatorResource
private final DruidCoordinator coordinator;
@Inject
public CoordinatorResource(
DruidCoordinator coordinator
)
public CoordinatorResource(DruidCoordinator coordinator)
{
this.coordinator = coordinator;
}
@ -91,15 +87,15 @@ public class CoordinatorResource
)
{
if (simple != null) {
return Response.ok(coordinator.computeNumsUnavailableUsedSegmentsPerDataSource()).build();
return Response.ok(coordinator.getDatasourceToUnavailableSegmentCount()).build();
}
if (full != null) {
return computeUsingClusterView != null
? Response.ok(coordinator.computeUnderReplicationCountsPerDataSourcePerTierUsingClusterView()).build() :
Response.ok(coordinator.computeUnderReplicationCountsPerDataSourcePerTier()).build();
return Response.ok(
coordinator.getTierToDatasourceToUnderReplicatedCount(computeUsingClusterView != null)
).build();
}
return Response.ok(coordinator.getLoadStatus()).build();
return Response.ok(coordinator.getDatasourceToLoadStatus()).build();
}
@GET
@ -115,28 +111,15 @@ public class CoordinatorResource
return Response.ok(
Maps.transformValues(
coordinator.getLoadManagementPeons(),
new Function<LoadQueuePeon, Object>()
{
@Override
public Object apply(LoadQueuePeon input)
{
long loadSize = 0;
for (DataSegment dataSegment : input.getSegmentsToLoad()) {
loadSize += dataSegment.getSize();
}
long dropSize = 0;
for (DataSegment dataSegment : input.getSegmentsToDrop()) {
dropSize += dataSegment.getSize();
}
return new ImmutableMap.Builder<>()
.put("segmentsToLoad", input.getSegmentsToLoad().size())
.put("segmentsToDrop", input.getSegmentsToDrop().size())
.put("segmentsToLoadSize", loadSize)
.put("segmentsToDropSize", dropSize)
.build();
}
input -> {
long loadSize = input.getSizeOfSegmentsToLoad();
long dropSize = input.getSegmentsToDrop().stream().mapToLong(DataSegment::getSize).sum();
return new ImmutableMap.Builder<>()
.put("segmentsToLoad", input.getSegmentsToLoad().size())
.put("segmentsToDrop", input.getSegmentsToDrop().size())
.put("segmentsToLoadSize", loadSize)
.put("segmentsToDropSize", dropSize)
.build();
}
)
).build();
@ -149,18 +132,11 @@ public class CoordinatorResource
return Response.ok(
Maps.transformValues(
coordinator.getLoadManagementPeons(),
new Function<LoadQueuePeon, Object>()
{
@Override
public Object apply(LoadQueuePeon peon)
{
return ImmutableMap
.builder()
.put("segmentsToLoad", Collections2.transform(peon.getSegmentsToLoad(), DataSegment::getId))
.put("segmentsToDrop", Collections2.transform(peon.getSegmentsToDrop(), DataSegment::getId))
.build();
}
}
peon -> ImmutableMap
.builder()
.put("segmentsToLoad", Collections2.transform(peon.getSegmentsToLoad(), DataSegment::getId))
.put("segmentsToDrop", Collections2.transform(peon.getSegmentsToDrop(), DataSegment::getId))
.build()
)
).build();
}

View File

@ -465,9 +465,7 @@ public class DataSourcesResource
} else if (full != null) {
// Calculate response for full mode
Map<String, Object2LongMap<String>> segmentLoadMap =
(computeUsingClusterView != null) ?
coordinator.computeUnderReplicationCountsPerDataSourcePerTierForSegmentsUsingClusterView(segments.get()) :
coordinator.computeUnderReplicationCountsPerDataSourcePerTierForSegments(segments.get());
coordinator.getTierToDatasourceToUnderReplicatedCount(segments.get(), computeUsingClusterView != null);
if (segmentLoadMap.isEmpty()) {
return Response.serverError()
.entity("Coordinator segment replicant lookup is not initialized yet. Try again later.")

View File

@ -188,23 +188,14 @@ public class MetadataResource
.flatMap(t -> t.getSegments().stream());
final Set<DataSegment> overshadowedSegments = dataSourcesSnapshot.getOvershadowedSegments();
final Stream<SegmentStatusInCluster> segmentStatus = usedSegments
.map(segment -> {
boolean isOvershadowed = overshadowedSegments.contains(segment);
Integer replicationFactor;
if (isOvershadowed) {
// If the segment is overshadowed, the replication factor won't be present in the coordinator, but we know
// that it should be 0 as we will unload it soon.
replicationFactor = 0;
} else {
replicationFactor = coordinator.getReplicationFactorForSegment(segment.getId());
}
return new SegmentStatusInCluster(
segment,
isOvershadowed,
replicationFactor
);
});
final Stream<SegmentStatusInCluster> segmentStatus = usedSegments.map(segment -> {
// The replication factor for unloaded segments is 0 as they will be unloaded soon
boolean isOvershadowed = overshadowedSegments.contains(segment);
Integer replicationFactor = isOvershadowed ? (Integer) 0
: coordinator.getReplicationFactor(segment.getId());
return new SegmentStatusInCluster(segment, isOvershadowed, replicationFactor);
});
final Function<SegmentStatusInCluster, Iterable<ResourceAction>> raGenerator = segment -> Collections
.singletonList(AuthorizationUtils.DATASOURCE_READ_RA_GENERATOR.apply(segment.getDataSegment().getDataSource()));

View File

@ -35,7 +35,7 @@ import org.apache.druid.server.coordination.ChangeRequestHistory;
import org.apache.druid.server.coordination.ChangeRequestsSnapshot;
import org.apache.druid.server.coordination.DataSegmentChangeRequest;
import org.apache.druid.server.coordination.SegmentLoadDropHandler;
import org.apache.druid.server.coordinator.HttpLoadQueuePeon;
import org.apache.druid.server.coordinator.loading.HttpLoadQueuePeon;
import org.apache.druid.server.http.security.StateResourceFilter;
import javax.annotation.Nullable;

View File

@ -40,9 +40,7 @@ public class TestUsedSegmentChecker implements UsedSegmentChecker
@Override
public Set<DataSegment> findUsedSegments(Set<SegmentIdWithShardSpec> identifiers)
{
final SegmentTimeline timeline = new SegmentTimeline();
timeline.addSegments(pushedSegments.iterator());
final SegmentTimeline timeline = SegmentTimeline.forSegments(pushedSegments);
final Set<DataSegment> retVal = new HashSet<>();
for (SegmentIdWithShardSpec identifier : identifiers) {
for (TimelineObjectHolder<String, DataSegment> holder : timeline.lookup(identifier.getInterval())) {

View File

@ -28,7 +28,11 @@ import org.apache.druid.java.util.common.DateTimes;
import org.apache.druid.java.util.emitter.EmittingLogger;
import org.apache.druid.java.util.emitter.service.ServiceEmitter;
import org.apache.druid.metadata.MetadataRuleManager;
import org.apache.druid.server.coordinator.duty.BalanceSegments;
import org.apache.druid.server.coordinator.duty.RunRules;
import org.apache.druid.server.coordinator.loading.LoadQueuePeon;
import org.apache.druid.server.coordinator.loading.LoadQueuePeonTester;
import org.apache.druid.server.coordinator.loading.SegmentLoadQueueManager;
import org.apache.druid.server.coordinator.rules.PeriodLoadRule;
import org.apache.druid.server.coordinator.rules.Rule;
import org.apache.druid.timeline.DataSegment;
@ -42,7 +46,6 @@ import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
/**
* TODO convert benchmarks to JMH
@ -50,7 +53,7 @@ import java.util.Map;
public class BalanceSegmentsProfiler
{
private static final int MAX_SEGMENTS_TO_MOVE = 5;
private DruidCoordinator coordinator;
private SegmentLoadQueueManager loadQueueManager;
private ImmutableDruidServer druidServer1;
private ImmutableDruidServer druidServer2;
List<DataSegment> segments = new ArrayList<>();
@ -62,7 +65,7 @@ public class BalanceSegmentsProfiler
@Before
public void setUp()
{
coordinator = EasyMock.createMock(DruidCoordinator.class);
loadQueueManager = new SegmentLoadQueueManager(null, null, null);
druidServer1 = EasyMock.createMock(ImmutableDruidServer.class);
druidServer2 = EasyMock.createMock(ImmutableDruidServer.class);
emitter = EasyMock.createMock(ServiceEmitter.class);
@ -80,17 +83,6 @@ public class BalanceSegmentsProfiler
EasyMock.expect(manager.getRulesWithDefault(EasyMock.anyObject())).andReturn(rules).anyTimes();
EasyMock.replay(manager);
coordinator.moveSegment(
EasyMock.anyObject(),
EasyMock.anyObject(),
EasyMock.anyObject(),
EasyMock.anyObject(),
EasyMock.anyObject()
);
EasyMock.expectLastCall().anyTimes();
EasyMock.replay(coordinator);
Map<String, LoadQueuePeon> peonMap = new HashMap<>();
List<ServerHolder> serverHolderList = new ArrayList<>();
List<DataSegment> segments = new ArrayList<>();
for (int i = 0; i < numSegments; i++) {
@ -126,19 +118,16 @@ public class BalanceSegmentsProfiler
EasyMock.replay(server);
LoadQueuePeon peon = new LoadQueuePeonTester();
peonMap.put(Integer.toString(i), peon);
serverHolderList.add(new ServerHolder(server, peon));
}
DruidCluster druidCluster = DruidClusterBuilder
.newBuilder()
DruidCluster druidCluster = DruidCluster
.builder()
.addTier("normal", serverHolderList.toArray(new ServerHolder[0]))
.build();
DruidCoordinatorRuntimeParams params = CoordinatorRuntimeParamsTestHelpers
.newBuilder()
DruidCoordinatorRuntimeParams params = DruidCoordinatorRuntimeParams
.newBuilder(DateTimes.nowUtc())
.withDruidCluster(druidCluster)
.withSegmentReplicantLookup(SegmentReplicantLookup.make(druidCluster, false))
.withLoadManagementPeons(peonMap)
.withUsedSegmentsInTest(segments)
.withDynamicConfigs(
CoordinatorDynamicConfig
@ -148,13 +137,13 @@ public class BalanceSegmentsProfiler
.withReplicationThrottleLimit(5)
.build()
)
.withSegmentAssignerUsing(loadQueueManager)
.withEmitter(emitter)
.withDatabaseRuleManager(manager)
.withReplicationManager(new ReplicationThrottler(2, 500, false))
.build();
BalanceSegmentsTester tester = new BalanceSegmentsTester(coordinator);
RunRules runner = new RunRules(coordinator);
BalanceSegments tester = new BalanceSegments();
RunRules runner = new RunRules();
watch.start();
DruidCoordinatorRuntimeParams balanceParams = tester.run(params);
DruidCoordinatorRuntimeParams assignParams = runner.run(params);
@ -183,21 +172,11 @@ public class BalanceSegmentsProfiler
EasyMock.expect(druidServer2.getSegment(EasyMock.anyObject())).andReturn(null).anyTimes();
EasyMock.replay(druidServer2);
coordinator.moveSegment(
EasyMock.anyObject(),
EasyMock.anyObject(),
EasyMock.anyObject(),
EasyMock.anyObject(),
EasyMock.anyObject()
);
EasyMock.expectLastCall().anyTimes();
EasyMock.replay(coordinator);
DruidCoordinatorRuntimeParams params = CoordinatorRuntimeParamsTestHelpers
.newBuilder()
DruidCoordinatorRuntimeParams params = DruidCoordinatorRuntimeParams
.newBuilder(DateTimes.nowUtc())
.withDruidCluster(
DruidClusterBuilder
.newBuilder()
DruidCluster
.builder()
.addTier(
"normal",
new ServerHolder(druidServer1, fromPeon),
@ -205,11 +184,11 @@ public class BalanceSegmentsProfiler
)
.build()
)
.withLoadManagementPeons(ImmutableMap.of("from", fromPeon, "to", toPeon))
.withUsedSegmentsInTest(segments)
.withDynamicConfigs(CoordinatorDynamicConfig.builder().withMaxSegmentsToMove(MAX_SEGMENTS_TO_MOVE).build())
.withSegmentAssignerUsing(loadQueueManager)
.build();
BalanceSegmentsTester tester = new BalanceSegmentsTester(coordinator);
BalanceSegments tester = new BalanceSegments();
watch.start();
DruidCoordinatorRuntimeParams balanceParams = tester.run(params);
System.out.println(watch.stop());

View File

@ -1,814 +0,0 @@
/*
* 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.ImmutableSet;
import com.google.common.util.concurrent.ListeningExecutorService;
import com.google.common.util.concurrent.MoreExecutors;
import org.apache.druid.client.ImmutableDruidServer;
import org.apache.druid.client.ImmutableDruidServerTests;
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.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.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.Iterator;
import java.util.List;
import java.util.Set;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.stream.Collectors;
import java.util.stream.IntStream;
public class BalanceSegmentsTest
{
private static final int MAX_SEGMENTS_TO_MOVE = 5;
private DruidCoordinator coordinator;
private ImmutableDruidServer druidServer1;
private ImmutableDruidServer druidServer2;
private ImmutableDruidServer druidServer3;
private ImmutableDruidServer druidServer4;
private List<ImmutableDruidServer> druidServers;
private LoadQueuePeonTester peon1;
private LoadQueuePeonTester peon2;
private LoadQueuePeonTester peon3;
private LoadQueuePeonTester peon4;
private List<LoadQueuePeon> peons;
private DataSegment segment1;
private DataSegment segment2;
private DataSegment segment3;
private DataSegment segment4;
private DataSegment segment5;
private List<DataSegment> segments;
private ListeningExecutorService balancerStrategyExecutor;
private BalancerStrategy balancerStrategy;
private Set<String> broadcastDatasources;
@Before
public void setUp()
{
coordinator = EasyMock.createMock(DruidCoordinator.class);
druidServer1 = EasyMock.createMock(ImmutableDruidServer.class);
druidServer2 = EasyMock.createMock(ImmutableDruidServer.class);
druidServer3 = EasyMock.createMock(ImmutableDruidServer.class);
druidServer4 = EasyMock.createMock(ImmutableDruidServer.class);
segment1 = EasyMock.createMock(DataSegment.class);
segment2 = EasyMock.createMock(DataSegment.class);
segment3 = EasyMock.createMock(DataSegment.class);
segment4 = EasyMock.createMock(DataSegment.class);
segment5 = EasyMock.createMock(DataSegment.class);
DateTime start1 = DateTimes.of("2012-01-01");
DateTime start2 = DateTimes.of("2012-02-01");
DateTime version = DateTimes.of("2012-03-01");
segment1 = new DataSegment(
"datasource1",
new Interval(start1, start1.plusHours(1)),
version.toString(),
new HashMap<>(),
new ArrayList<>(),
new ArrayList<>(),
NoneShardSpec.instance(),
0,
11L
);
segment2 = new DataSegment(
"datasource1",
new Interval(start2, start2.plusHours(1)),
version.toString(),
new HashMap<>(),
new ArrayList<>(),
new ArrayList<>(),
NoneShardSpec.instance(),
0,
7L
);
segment3 = new DataSegment(
"datasource2",
new Interval(start1, start1.plusHours(1)),
version.toString(),
new HashMap<>(),
new ArrayList<>(),
new ArrayList<>(),
NoneShardSpec.instance(),
0,
4L
);
segment4 = new DataSegment(
"datasource2",
new Interval(start2, start2.plusHours(1)),
version.toString(),
new HashMap<>(),
new ArrayList<>(),
new ArrayList<>(),
NoneShardSpec.instance(),
0,
8L
);
segment5 = new DataSegment(
"datasourceBroadcast",
new Interval(start2, start2.plusHours(1)),
version.toString(),
new HashMap<>(),
new ArrayList<>(),
new ArrayList<>(),
NoneShardSpec.instance(),
0,
8L
);
segments = new ArrayList<>();
segments.add(segment1);
segments.add(segment2);
segments.add(segment3);
segments.add(segment4);
segments.add(segment5);
peon1 = new LoadQueuePeonTester();
peon2 = new LoadQueuePeonTester();
peon3 = new LoadQueuePeonTester();
peon4 = new LoadQueuePeonTester();
druidServers = ImmutableList.of(druidServer1, druidServer2, druidServer3, druidServer4);
peons = ImmutableList.of(peon1, peon2, peon3, peon4);
balancerStrategyExecutor = MoreExecutors.listeningDecorator(Execs.multiThreaded(1, "BalanceSegmentsTest-%d"));
balancerStrategy = new CostBalancerStrategyFactory().createBalancerStrategy(balancerStrategyExecutor);
broadcastDatasources = Collections.singleton("datasourceBroadcast");
}
@After
public void tearDown()
{
EasyMock.verify(coordinator);
EasyMock.verify(druidServer1);
EasyMock.verify(druidServer2);
EasyMock.verify(druidServer3);
EasyMock.verify(druidServer4);
balancerStrategyExecutor.shutdownNow();
}
@Test
public void testMoveToEmptyServerBalancer()
{
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)
)
);
DruidCoordinatorRuntimeParams params = defaultRuntimeParamsBuilder(
ImmutableList.of(druidServer1, druidServer2),
ImmutableList.of(peon1, peon2)
)
.withBalancerStrategy(predefinedPickOrderStrategy)
.withBroadcastDatasources(broadcastDatasources)
.build();
params = new BalanceSegmentsTester(coordinator).run(params);
Assert.assertEquals(3, params.getCoordinatorStats().getTieredStat("movedCount", "normal"));
}
/**
* 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()
{
mockDruidServer(druidServer1, "1", "normal", 30L, 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);
EasyMock.expect(
strategy.pickSegmentsToMove(
ImmutableList.of(
new ServerHolder(druidServer2, peon2, true)
),
broadcastDatasources,
2
)
).andReturn(
ImmutableList.of(
new BalancerSegmentHolder(druidServer2, segment3),
new BalancerSegmentHolder(druidServer2, segment4)
).iterator()
);
EasyMock.expect(strategy.pickSegmentsToMove(EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyInt()))
.andReturn(
ImmutableList.of(
new BalancerSegmentHolder(druidServer1, segment1),
new BalancerSegmentHolder(druidServer1, segment2)).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, true, false)
)
.withDynamicConfigs(
CoordinatorDynamicConfig.builder()
.withMaxSegmentsToMove(3)
.withDecommissioningMaxPercentOfMaxSegmentsToMove(60)
.build() // ceil(3 * 0.6) = 2 segments from decommissioning servers
)
.withBalancerStrategy(strategy)
.withBroadcastDatasources(broadcastDatasources)
.build();
params = new BalanceSegmentsTester(coordinator).run(params);
EasyMock.verify(strategy);
Assert.assertEquals(3L, params.getCoordinatorStats().getTieredStat("movedCount", "normal"));
Assert.assertEquals(
ImmutableSet.of(segment1, segment3, segment4),
peon3.getSegmentsToLoad()
);
}
@Test
public void testZeroDecommissioningMaxPercentOfMaxSegmentsToMove()
{
DruidCoordinatorRuntimeParams params = setupParamsForDecommissioningMaxPercentOfMaxSegmentsToMove(0);
params = new BalanceSegmentsTester(coordinator).run(params);
Assert.assertEquals(1L, params.getCoordinatorStats().getTieredStat("movedCount", "normal"));
Assert.assertEquals(ImmutableSet.of(segment1), peon3.getSegmentsToLoad());
}
@Test
public void testMaxDecommissioningMaxPercentOfMaxSegmentsToMove()
{
DruidCoordinatorRuntimeParams params = setupParamsForDecommissioningMaxPercentOfMaxSegmentsToMove(10);
params = new BalanceSegmentsTester(coordinator).run(params);
Assert.assertEquals(1L, params.getCoordinatorStats().getTieredStat("movedCount", "normal"));
Assert.assertEquals(ImmutableSet.of(segment1), peon3.getSegmentsToLoad());
}
/**
* Should balance segments as usual (ignoring percent) with empty decommissioningNodes.
*/
@Test
public void testMoveDecommissioningMaxPercentOfMaxSegmentsToMoveWithNoDecommissioning()
{
mockDruidServer(druidServer1, "1", "normal", 30L, 100L, Arrays.asList(segment1, segment2));
mockDruidServer(druidServer2, "2", "normal", 0L, 100L, Arrays.asList(segment3, segment4));
mockDruidServer(druidServer3, "3", "normal", 0L, 100L, Collections.emptyList());
EasyMock.replay(druidServer4);
mockCoordinator(coordinator);
BalancerStrategy strategy = EasyMock.createMock(BalancerStrategy.class);
EasyMock.expect(
strategy.pickSegmentsToMove(EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyInt()))
.andReturn(
ImmutableList.of(
new BalancerSegmentHolder(druidServer1, segment2),
new BalancerSegmentHolder(druidServer2, segment3),
new BalancerSegmentHolder(druidServer2, segment4)).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)
)
.withDynamicConfigs(
CoordinatorDynamicConfig.builder()
.withMaxSegmentsToMove(3)
.withDecommissioningMaxPercentOfMaxSegmentsToMove(9)
.build()
)
.withBalancerStrategy(strategy)
.withBroadcastDatasources(broadcastDatasources)
.build();
params = new BalanceSegmentsTester(coordinator).run(params);
EasyMock.verify(strategy);
Assert.assertEquals(3L, params.getCoordinatorStats().getTieredStat("movedCount", "normal"));
Assert.assertEquals(
ImmutableSet.of(segment2, segment3, segment4),
peon3.getSegmentsToLoad()
);
}
/**
* Shouldn't move segments to a decommissioning server.
*/
@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);
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();
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();
params = new BalanceSegmentsTester(coordinator).run(params);
EasyMock.verify(strategy);
Assert.assertEquals(0, params.getCoordinatorStats().getTieredStat("movedCount", "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);
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();
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)
)
.withDynamicConfigs(CoordinatorDynamicConfig.builder().withMaxSegmentsToMove(1).build())
.withBalancerStrategy(strategy)
.withBroadcastDatasources(broadcastDatasources)
.build();
params = new BalanceSegmentsTester(coordinator).run(params);
EasyMock.verify(strategy);
Assert.assertEquals(1, params.getCoordinatorStats().getTieredStat("movedCount", "normal"));
Assert.assertEquals(0, peon1.getNumberOfSegmentsInQueue());
Assert.assertEquals(1, peon2.getNumberOfSegmentsInQueue());
}
@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)
)
);
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()
)
.build();
params = new BalanceSegmentsTester(coordinator).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"));
}
@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)
).build();
params = new BalanceSegmentsTester(coordinator).run(params);
Assert.assertTrue(params.getCoordinatorStats().getTieredStat("movedCount", "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();
params = new BalanceSegmentsTester(coordinator).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()
{
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
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());
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)
)
.withDynamicConfigs(
CoordinatorDynamicConfig.builder()
.withMaxSegmentsToMove(1)
.withUseBatchedSegmentSampler(false)
.withPercentOfSegmentsToConsiderPerMove(40)
.build()
)
.withBalancerStrategy(strategy)
.withBroadcastDatasources(broadcastDatasources)
.build();
params = new BalanceSegmentsTester(coordinator).run(params);
EasyMock.verify(strategy);
Assert.assertEquals(1L, params.getCoordinatorStats().getTieredStat("movedCount", "normal"));
Assert.assertEquals(ImmutableSet.of(segment3), peon3.getSegmentsToLoad());
}
@Test
public void testUseBatchedSegmentSampler()
{
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());
mockCoordinator(coordinator);
DruidCoordinatorRuntimeParams params = defaultRuntimeParamsBuilder(druidServers, peons)
.withDynamicConfigs(
CoordinatorDynamicConfig.builder()
.withMaxSegmentsToMove(2)
.withUseBatchedSegmentSampler(true)
.build()
)
.withBroadcastDatasources(broadcastDatasources)
.build();
params = new BalanceSegmentsTester(coordinator).run(params);
Assert.assertEquals(2L, params.getCoordinatorStats().getTieredStat("movedCount", "normal"));
}
private DruidCoordinatorRuntimeParams.Builder defaultRuntimeParamsBuilder(
List<ImmutableDruidServer> druidServers,
List<LoadQueuePeon> peons
)
{
return defaultRuntimeParamsBuilder(
druidServers,
peons,
druidServers.stream().map(s -> false).collect(Collectors.toList())
);
}
private DruidCoordinatorRuntimeParams.Builder defaultRuntimeParamsBuilder(
List<ImmutableDruidServer> druidServers,
List<LoadQueuePeon> peons,
List<Boolean> decommissioning
)
{
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())
.withBroadcastDatasources(broadcastDatasources)
.withBalancerStrategy(balancerStrategy);
}
private static void mockDruidServer(
ImmutableDruidServer druidServer,
String name,
String tier,
long currentSize,
long maxSize,
List<DataSegment> segments
)
{
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);
}
private static void mockCoordinator(DruidCoordinator coordinator)
{
coordinator.moveSegment(
EasyMock.anyObject(),
EasyMock.anyObject(),
EasyMock.anyObject(),
EasyMock.anyObject(),
EasyMock.anyObject()
);
EasyMock.expectLastCall().anyTimes();
EasyMock.replay(coordinator);
}
private static class PredefinedPickOrderBalancerStrategy implements BalancerStrategy
{
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();
}
@Override
public void emitStats(String tier, CoordinatorStats stats, List<ServerHolder> serverHolderList)
{
delegate.emitStats(tier, stats, serverHolderList);
}
}
private DruidCoordinatorRuntimeParams setupParamsForDecommissioningMaxPercentOfMaxSegmentsToMove(int percent)
{
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);
// either decommissioning servers list or acitve ones (ie servers list is [2] or [1, 3])
BalancerStrategy strategy = EasyMock.createMock(BalancerStrategy.class);
EasyMock.expect(
strategy.pickSegmentsToMove(
ImmutableList.of(
new ServerHolder(druidServer2, peon2, true)
),
broadcastDatasources,
100.0
)
).andReturn(
ImmutableList.of(new BalancerSegmentHolder(druidServer2, segment2)).iterator()
);
EasyMock.expect(strategy.pickSegmentsToMove(EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyInt()))
.andReturn(ImmutableList.of(new BalancerSegmentHolder(druidServer1, segment1)).iterator());
EasyMock.expect(strategy.findNewSegmentHomeBalancer(EasyMock.anyObject(), EasyMock.anyObject()))
.andReturn(new ServerHolder(druidServer3, peon3))
.anyTimes();
EasyMock.replay(strategy);
return defaultRuntimeParamsBuilder(
ImmutableList.of(druidServer1, druidServer2, druidServer3),
ImmutableList.of(peon1, peon2, peon3),
ImmutableList.of(false, true, false)
)
.withDynamicConfigs(
CoordinatorDynamicConfig.builder()
.withMaxSegmentsToMove(1)
.withDecommissioningMaxPercentOfMaxSegmentsToMove(percent)
.build()
)
.withBalancerStrategy(strategy)
.withBroadcastDatasources(broadcastDatasources)
.build();
}
}

View File

@ -1,76 +0,0 @@
/*
* 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.ImmutableDruidServer;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.server.coordinator.duty.BalanceSegments;
import org.apache.druid.timeline.DataSegment;
import org.apache.druid.timeline.SegmentId;
public class BalanceSegmentsTester extends BalanceSegments
{
public BalanceSegmentsTester(DruidCoordinator coordinator)
{
super(coordinator);
}
@Override
protected boolean moveSegment(
final BalancerSegmentHolder segment,
final ImmutableDruidServer toServer,
final DruidCoordinatorRuntimeParams params
)
{
final String toServerName = toServer.getName();
final LoadQueuePeon toPeon = params.getLoadManagementPeons().get(toServerName);
final String fromServerName = segment.getFromServer().getName();
final DataSegment segmentToMove = segment.getSegment();
final SegmentId segmentId = segmentToMove.getId();
if (!toPeon.getSegmentsToLoad().contains(segmentToMove) &&
(toServer.getSegment(segmentId) == null) &&
new ServerHolder(toServer, toPeon).getAvailableSize() > segmentToMove.getSize()) {
log.info(
"Moving [%s] from [%s] to [%s]",
segmentId,
fromServerName,
toServerName
);
try {
final LoadQueuePeon loadPeon = params.getLoadManagementPeons().get(toServerName);
loadPeon.loadSegment(segment.getSegment(), success -> {});
final LoadQueuePeon dropPeon = params.getLoadManagementPeons().get(fromServerName);
dropPeon.markSegmentToDrop(segment.getSegment());
currentlyMovingSegments.get("normal").put(segmentId, segment);
return true;
}
catch (Exception e) {
log.info(e, StringUtils.format("[%s] : Moving exception", segmentId));
}
}
return false;
}
}

View File

@ -0,0 +1,211 @@
/*
* 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.ImmutableMap;
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.junit.After;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
import java.util.HashMap;
import java.util.Map;
public class CoordinatorRunStatsTest
{
private static final CoordinatorStat STAT_1 = new CoordinatorStat("stat1", "s1");
private static final CoordinatorStat STAT_2 = new CoordinatorStat("stat2", "s2");
private static final CoordinatorStat STAT_3 = new CoordinatorStat("stat3", "s3");
private static final CoordinatorStat DEBUG_STAT_1
= new CoordinatorStat("debugStat1", CoordinatorStat.Level.DEBUG);
private static final CoordinatorStat DEBUG_STAT_2
= new CoordinatorStat("debugStat2", CoordinatorStat.Level.DEBUG);
private CoordinatorRunStats stats;
@Before
public void setUp()
{
stats = new CoordinatorRunStats();
}
@After
public void tearDown()
{
stats = null;
}
@Test
public void testAdd()
{
Assert.assertEquals(0, stats.get(STAT_1));
stats.add(STAT_1, 1);
Assert.assertEquals(1, stats.get(STAT_1));
stats.add(STAT_1, -11);
Assert.assertEquals(-10, stats.get(STAT_1));
}
@Test
public void testAddForRowKey()
{
stats.add(STAT_1, Key.TIER_1, 1);
stats.add(STAT_1, Key.TIER_2, 1);
stats.add(STAT_1, Key.TIER_1, -5);
stats.add(STAT_2, Key.TIER_1, 1);
stats.add(STAT_1, Key.TIER_2, 1);
Assert.assertFalse(stats.hasStat(STAT_3));
Assert.assertEquals(-4, stats.get(STAT_1, Key.TIER_1));
Assert.assertEquals(2, stats.get(STAT_1, Key.TIER_2));
Assert.assertEquals(1, stats.get(STAT_2, Key.TIER_1));
}
@Test
public void testGetSnapshotAndReset()
{
stats.add(STAT_1, 1);
stats.add(STAT_2, 3);
stats.add(STAT_1, Key.TIER_1, 5);
stats.add(STAT_1, Key.DUTY_1, 7);
final CoordinatorRunStats firstFlush = stats.getSnapshotAndReset();
Assert.assertEquals(1, firstFlush.get(STAT_1));
Assert.assertEquals(3, firstFlush.get(STAT_2));
Assert.assertEquals(5, firstFlush.get(STAT_1, Key.TIER_1));
Assert.assertEquals(7, firstFlush.get(STAT_1, Key.DUTY_1));
Assert.assertEquals(0, stats.rowCount());
stats.add(STAT_1, 7);
stats.add(STAT_1, Key.TIER_1, 5);
stats.add(STAT_2, Key.DUTY_1, 3);
stats.add(STAT_3, Key.TIER_1, 1);
final CoordinatorRunStats secondFlush = stats.getSnapshotAndReset();
Assert.assertEquals(7, secondFlush.get(STAT_1));
Assert.assertEquals(5, secondFlush.get(STAT_1, Key.TIER_1));
Assert.assertEquals(3, secondFlush.get(STAT_2, Key.DUTY_1));
Assert.assertEquals(1, secondFlush.get(STAT_3, Key.TIER_1));
Assert.assertEquals(0, stats.rowCount());
}
@Test
public void testUpdateMax()
{
stats.updateMax(STAT_1, Key.TIER_1, 2);
stats.updateMax(STAT_1, Key.TIER_1, 6);
stats.updateMax(STAT_1, Key.TIER_1, 5);
stats.updateMax(STAT_2, Key.TIER_1, 5);
stats.updateMax(STAT_2, Key.TIER_1, 4);
stats.updateMax(STAT_2, Key.TIER_1, 5);
stats.updateMax(STAT_1, Key.TIER_2, 7);
stats.updateMax(STAT_1, Key.TIER_2, 9);
stats.updateMax(STAT_1, Key.TIER_2, 10);
Assert.assertFalse(stats.hasStat(STAT_3));
Assert.assertEquals(6, stats.get(STAT_1, Key.TIER_1));
Assert.assertEquals(5, stats.get(STAT_2, Key.TIER_1));
Assert.assertEquals(10, stats.get(STAT_1, Key.TIER_2));
}
@Test
public void testAddToDutyStat()
{
stats.add(STAT_1, Key.DUTY_1, 1);
stats.add(STAT_1, Key.DUTY_2, 1);
stats.add(STAT_1, Key.DUTY_1, -5);
stats.add(STAT_2, Key.DUTY_1, 1);
stats.add(STAT_1, Key.DUTY_2, 1);
Assert.assertFalse(stats.hasStat(STAT_3));
Assert.assertEquals(-4, stats.get(STAT_1, Key.DUTY_1));
Assert.assertEquals(2, stats.get(STAT_1, Key.DUTY_2));
Assert.assertEquals(1, stats.get(STAT_2, Key.DUTY_1));
}
@Test
public void testForEachStat()
{
final Map<String, Long> expected = ImmutableMap.of(
"duty1", 1L,
"duty2", 2L,
"duty3", 3L
);
expected.forEach(
(duty, count) ->
stats.add(STAT_1, RowKey.builder().add(Dimension.DUTY, duty).build(), count)
);
final Map<String, Long> actual = new HashMap<>();
stats.forEachStat(
(dimensionValues, stat, value) -> {
if (stat.equals(STAT_1)) {
actual.put(dimensionValues.get(Dimension.DUTY), value);
}
}
);
Assert.assertEquals(expected, actual);
}
@Test
public void testAddWithDebugDimensions()
{
stats.add(DEBUG_STAT_1, 1);
Assert.assertFalse(stats.hasStat(DEBUG_STAT_1));
stats.add(DEBUG_STAT_1, Key.TIER_1, 1);
Assert.assertFalse(stats.hasStat(DEBUG_STAT_1));
final CoordinatorRunStats debugStats
= new CoordinatorRunStats(Key.TIER_1.getValues());
debugStats.add(DEBUG_STAT_1, 1);
Assert.assertFalse(stats.hasStat(DEBUG_STAT_1));
debugStats.add(DEBUG_STAT_1, Key.TIER_1, 1);
Assert.assertTrue(debugStats.hasStat(DEBUG_STAT_1));
debugStats.addToDatasourceStat(DEBUG_STAT_2, "wiki", 1);
Assert.assertFalse(debugStats.hasStat(DEBUG_STAT_2));
}
/**
* Dimension keys for reporting stats.
*/
private static class Key
{
static final RowKey TIER_1 = RowKey.forTier("tier1");
static final RowKey TIER_2 = RowKey.forTier("tier2");
static final RowKey DUTY_1 = RowKey.builder().add(Dimension.DUTY, "duty1").build();
static final RowKey DUTY_2 = RowKey.builder().add(Dimension.DUTY, "duty2").build();
}
}

View File

@ -1,233 +0,0 @@
/*
* 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.ImmutableMap;
import com.google.common.collect.Sets;
import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
import java.util.HashMap;
import java.util.Map;
public class CoordinatorStatsTest
{
private CoordinatorStats stats;
@Before
public void setUp()
{
stats = new CoordinatorStats();
}
@After
public void tearDown()
{
stats = null;
}
@Test
public void addToGlobalStat()
{
Assert.assertEquals(0, stats.getGlobalStat("stats"));
stats.addToGlobalStat("stats", 1);
Assert.assertEquals(1, stats.getGlobalStat("stats"));
stats.addToGlobalStat("stats", -11);
Assert.assertEquals(-10, stats.getGlobalStat("stats"));
}
@Test(expected = NullPointerException.class)
public void testAddToTieredStatNonexistentStat()
{
stats.getTieredStat("stat", "tier");
}
@Test
public void testAddToTieredStat()
{
Assert.assertFalse(stats.hasPerTierStats());
stats.addToTieredStat("stat1", "tier1", 1);
stats.addToTieredStat("stat1", "tier2", 1);
stats.addToTieredStat("stat1", "tier1", -5);
stats.addToTieredStat("stat2", "tier1", 1);
stats.addToTieredStat("stat1", "tier2", 1);
Assert.assertTrue(stats.hasPerTierStats());
Assert.assertEquals(
Sets.newHashSet("tier1", "tier2"),
stats.getTiers("stat1")
);
Assert.assertEquals(
Sets.newHashSet("tier1"),
stats.getTiers("stat2")
);
Assert.assertTrue(stats.getTiers("stat3").isEmpty());
Assert.assertEquals(-4, stats.getTieredStat("stat1", "tier1"));
Assert.assertEquals(2, stats.getTieredStat("stat1", "tier2"));
Assert.assertEquals(1, stats.getTieredStat("stat2", "tier1"));
}
@Test
public void testForEachTieredStat()
{
final Map<String, Long> expected = ImmutableMap.of(
"tier1", 1L,
"tier2", 2L,
"tier3", 3L
);
final Map<String, Long> actual = new HashMap<>();
expected.forEach(
(tier, count) -> stats.addToTieredStat("stat", tier, count)
);
stats.forEachTieredStat("stat0", (tier, count) -> Assert.fail());
stats.forEachTieredStat("stat", actual::put);
Assert.assertEquals(expected, actual);
}
@Test
public void testAccumulate()
{
stats.addToGlobalStat("stat1", 1);
stats.addToGlobalStat("stat2", 1);
stats.addToTieredStat("stat1", "tier1", 1);
stats.addToTieredStat("stat1", "tier2", 1);
stats.addToTieredStat("stat2", "tier1", 1);
stats.addToDutyStat("stat1", "duty1", 1);
stats.addToDutyStat("stat1", "duty2", 1);
stats.addToDutyStat("stat2", "duty1", 1);
final CoordinatorStats stats2 = new CoordinatorStats();
stats2.addToGlobalStat("stat1", 1);
stats2.addToTieredStat("stat1", "tier2", 1);
stats2.addToTieredStat("stat2", "tier2", 1);
stats2.addToTieredStat("stat3", "tier1", 1);
stats2.addToDutyStat("stat1", "duty2", 1);
stats2.addToDutyStat("stat2", "duty2", 1);
stats2.addToDutyStat("stat3", "duty1", 1);
stats.accumulate(stats2);
Assert.assertEquals(2, stats.getGlobalStat("stat1"));
Assert.assertEquals(1, stats.getGlobalStat("stat2"));
Assert.assertEquals(1, stats.getTieredStat("stat1", "tier1"));
Assert.assertEquals(2, stats.getTieredStat("stat1", "tier2"));
Assert.assertEquals(1, stats.getTieredStat("stat2", "tier1"));
Assert.assertEquals(1, stats.getTieredStat("stat2", "tier2"));
Assert.assertEquals(1, stats.getTieredStat("stat3", "tier1"));
Assert.assertEquals(1, stats.getDutyStat("stat1", "duty1"));
Assert.assertEquals(2, stats.getDutyStat("stat1", "duty2"));
Assert.assertEquals(1, stats.getDutyStat("stat2", "duty1"));
Assert.assertEquals(1, stats.getDutyStat("stat2", "duty2"));
Assert.assertEquals(1, stats.getDutyStat("stat3", "duty1"));
}
@Test
public void testAccumulateMaxToTieredStat()
{
Assert.assertFalse(stats.hasPerTierStats());
stats.accumulateMaxTieredStat("stat1", "tier1", 2);
stats.accumulateMaxTieredStat("stat1", "tier1", 6);
stats.accumulateMaxTieredStat("stat1", "tier1", 5);
stats.accumulateMaxTieredStat("stat2", "tier1", 5);
stats.accumulateMaxTieredStat("stat2", "tier1", 4);
stats.accumulateMaxTieredStat("stat2", "tier1", 5);
stats.accumulateMaxTieredStat("stat1", "tier2", 7);
stats.accumulateMaxTieredStat("stat1", "tier2", 9);
stats.accumulateMaxTieredStat("stat1", "tier2", 10);
Assert.assertTrue(stats.hasPerTierStats());
Assert.assertEquals(
Sets.newHashSet("tier1", "tier2"),
stats.getTiers("stat1")
);
Assert.assertEquals(
Sets.newHashSet("tier1"),
stats.getTiers("stat2")
);
Assert.assertTrue(stats.getTiers("stat3").isEmpty());
Assert.assertEquals(6, stats.getTieredStat("stat1", "tier1"));
Assert.assertEquals(5, stats.getTieredStat("stat2", "tier1"));
Assert.assertEquals(10, stats.getTieredStat("stat1", "tier2"));
}
@Test(expected = NullPointerException.class)
public void testGetNonexistentDutyStat()
{
stats.getDutyStat("stat", "duty");
}
@Test
public void testAddToDutyStat()
{
Assert.assertFalse(stats.hasPerDutyStats());
stats.addToDutyStat("stat1", "duty1", 1);
stats.addToDutyStat("stat1", "duty2", 1);
stats.addToDutyStat("stat1", "duty1", -5);
stats.addToDutyStat("stat2", "duty1", 1);
stats.addToDutyStat("stat1", "duty2", 1);
Assert.assertTrue(stats.hasPerDutyStats());
Assert.assertEquals(
Sets.newHashSet("duty1", "duty2"),
stats.getDuties("stat1")
);
Assert.assertEquals(
Sets.newHashSet("duty1"),
stats.getDuties("stat2")
);
Assert.assertTrue(stats.getDuties("stat3").isEmpty());
Assert.assertEquals(-4, stats.getDutyStat("stat1", "duty1"));
Assert.assertEquals(2, stats.getDutyStat("stat1", "duty2"));
Assert.assertEquals(1, stats.getDutyStat("stat2", "duty1"));
}
@Test
public void testForEachDutyStat()
{
final Map<String, Long> expected = ImmutableMap.of(
"duty1", 1L,
"duty2", 2L,
"duty3", 3L
);
final Map<String, Long> actual = new HashMap<>();
expected.forEach(
(duty, count) -> stats.addToDutyStat("stat", duty, count)
);
stats.forEachDutyStat("stat0", (duty, count) -> Assert.fail());
stats.forEachDutyStat("stat", actual::put);
Assert.assertEquals(expected, actual);
}
}

View File

@ -27,6 +27,7 @@ import org.apache.druid.java.util.common.Intervals;
import org.apache.druid.java.util.common.concurrent.Execs;
import org.apache.druid.java.util.common.granularity.Granularities;
import org.apache.druid.server.coordination.ServerType;
import org.apache.druid.server.coordinator.balancer.CostBalancerStrategy;
import org.apache.druid.timeline.DataSegment;
import org.joda.time.Interval;
import org.junit.Ignore;
@ -36,6 +37,7 @@ import org.junit.runners.Parameterized;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Iterator;
import java.util.List;
import java.util.Random;
import java.util.stream.Collectors;
@ -105,7 +107,8 @@ public class CostBalancerStrategyBenchmark extends AbstractBenchmark
{
DataSegment segment = DataSegment.builder().dataSource("testds").version("1000")
.interval(interval1).size(100L).build();
selected = strategy.findNewSegmentHomeReplicator(segment, serverHolderList);
Iterator<ServerHolder> candidates = strategy.findServersToLoadSegment(segment, serverHolderList);
selected = candidates.hasNext() ? candidates.next() : null;
}
// Benchmark Joda Interval Gap impl vs CostBalancer.gapMillis

View File

@ -28,6 +28,8 @@ import org.apache.druid.timeline.DataSegment;
import org.apache.druid.timeline.partition.NumberedShardSpec;
import org.joda.time.DateTime;
import org.joda.time.Interval;
import org.joda.time.format.DateTimeFormat;
import org.joda.time.format.DateTimeFormatter;
import java.util.ArrayList;
import java.util.Collections;
@ -39,10 +41,12 @@ import java.util.Objects;
*/
public class CreateDataSegments
{
private static final DateTime DEFAULT_START = DateTimes.of("2012-10-24");
private final String datasource;
private DateTime startTime;
private Granularity granularity;
private DateTime startTime = DEFAULT_START;
private Granularity granularity = Granularities.DAY;
private int numPartitions = 1;
private int numIntervals = 1;
@ -103,7 +107,7 @@ public class CreateDataSegments
nextInterval,
new NumberedShardSpec(numPartition, numPartitions),
++uniqueIdInInterval,
sizeMb
sizeMb << 20
)
);
}
@ -118,6 +122,7 @@ public class CreateDataSegments
*/
private static class NumberedDataSegment extends DataSegment
{
private static final DateTimeFormatter FORMATTER = DateTimeFormat.forPattern("yyyyMMdd");
private final int uniqueId;
private NumberedDataSegment(
@ -145,7 +150,9 @@ public class CreateDataSegments
@Override
public String toString()
{
return "{" + getDataSource() + "::" + uniqueId + "}";
return "{" + getDataSource()
+ "::" + getInterval().getStart().toString(FORMATTER)
+ "::" + uniqueId + "}";
}
}
}

View File

@ -24,7 +24,6 @@ import com.google.common.base.Predicates;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Lists;
import org.apache.curator.framework.CuratorFramework;
import org.apache.curator.framework.recipes.cache.PathChildrenCache;
import org.apache.curator.framework.recipes.cache.PathChildrenCacheEvent;
@ -35,24 +34,24 @@ import org.apache.druid.client.CoordinatorServerView;
import org.apache.druid.client.DataSourcesSnapshot;
import org.apache.druid.client.DruidServer;
import org.apache.druid.client.ImmutableDruidDataSource;
import org.apache.druid.common.config.JacksonConfigManager;
import org.apache.druid.curator.CuratorTestBase;
import org.apache.druid.curator.CuratorUtils;
import org.apache.druid.curator.discovery.LatchableServiceAnnouncer;
import org.apache.druid.discovery.DruidLeaderSelector;
import org.apache.druid.jackson.DefaultObjectMapper;
import org.apache.druid.java.util.common.Intervals;
import org.apache.druid.java.util.common.Pair;
import org.apache.druid.java.util.common.concurrent.Execs;
import org.apache.druid.metadata.MetadataRuleManager;
import org.apache.druid.metadata.SegmentsMetadataManager;
import org.apache.druid.segment.TestHelper;
import org.apache.druid.server.DruidNode;
import org.apache.druid.server.coordination.DruidServerMetadata;
import org.apache.druid.server.coordination.ServerType;
import org.apache.druid.server.coordinator.duty.CoordinatorCustomDutyGroups;
import org.apache.druid.server.coordinator.balancer.BalancerStrategy;
import org.apache.druid.server.coordinator.loading.CuratorLoadQueuePeon;
import org.apache.druid.server.coordinator.loading.LoadQueuePeon;
import org.apache.druid.server.coordinator.loading.LoadQueueTaskMaster;
import org.apache.druid.server.coordinator.loading.SegmentLoadQueueManager;
import org.apache.druid.server.coordinator.loading.SegmentLoadingConfig;
import org.apache.druid.server.coordinator.loading.StrategicSegmentAssigner;
import org.apache.druid.server.coordinator.stats.CoordinatorRunStats;
import org.apache.druid.server.initialization.ZkPathsConfig;
import org.apache.druid.server.lookup.cache.LookupCoordinatorManager;
import org.apache.druid.server.metrics.NoopServiceEmitter;
import org.apache.druid.testing.DeadlockDetectingTimeout;
import org.apache.druid.timeline.DataSegment;
@ -68,36 +67,28 @@ import org.junit.Test;
import org.junit.rules.TestRule;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.Executor;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicReference;
/**
* This tests zookeeper specific coordinator/load queue/historical interactions, such as moving segments by the balancer
*/
public class CuratorDruidCoordinatorTest extends CuratorTestBase
{
private DruidCoordinator coordinator;
private SegmentsMetadataManager segmentsMetadataManager;
private DataSourcesSnapshot dataSourcesSnapshot;
private DruidCoordinatorRuntimeParams coordinatorRuntimeParams;
private ConcurrentMap<String, LoadQueuePeon> loadManagementPeons;
private LoadQueuePeon sourceLoadQueuePeon;
private LoadQueuePeon destinationLoadQueuePeon;
private CountDownLatch leaderAnnouncerLatch;
private CountDownLatch leaderUnannouncerLatch;
private PathChildrenCache sourceLoadQueueChildrenCache;
private PathChildrenCache destinationLoadQueueChildrenCache;
private DruidCoordinatorConfig druidCoordinatorConfig;
private JacksonConfigManager configManager;
private static final String SEGPATH = "/druid/segments";
private static final String SOURCE_LOAD_PATH = "/druid/loadQueue/localhost:1";
@ -134,23 +125,6 @@ public class CuratorDruidCoordinatorTest extends CuratorTestBase
dataSourcesSnapshot = EasyMock.createNiceMock(DataSourcesSnapshot.class);
coordinatorRuntimeParams = EasyMock.createNiceMock(DruidCoordinatorRuntimeParams.class);
configManager = EasyMock.createNiceMock(JacksonConfigManager.class);
EasyMock.expect(
configManager.watch(
EasyMock.eq(CoordinatorDynamicConfig.CONFIG_KEY),
EasyMock.anyObject(Class.class),
EasyMock.anyObject()
)
).andReturn(new AtomicReference<>(CoordinatorDynamicConfig.builder().build())).anyTimes();
EasyMock.expect(
configManager.watch(
EasyMock.eq(CoordinatorCompactionConfig.CONFIG_KEY),
EasyMock.anyObject(Class.class),
EasyMock.anyObject()
)
).andReturn(new AtomicReference<>(CoordinatorCompactionConfig.empty())).anyTimes();
EasyMock.replay(configManager);
setupServerAndCurator();
curator.start();
curator.blockUntilConnected();
@ -159,7 +133,7 @@ public class CuratorDruidCoordinatorTest extends CuratorTestBase
curator.create().creatingParentsIfNeeded().forPath(DESTINATION_LOAD_PATH);
final ObjectMapper objectMapper = new DefaultObjectMapper();
druidCoordinatorConfig = new TestDruidCoordinatorConfig.Builder()
DruidCoordinatorConfig druidCoordinatorConfig = new TestDruidCoordinatorConfig.Builder()
.withCoordinatorStartDelay(new Duration(COORDINATOR_START_DELAY))
.withCoordinatorPeriod(new Duration(COORDINATOR_PERIOD))
.withCoordinatorKillPeriod(new Duration(COORDINATOR_PERIOD))
@ -196,9 +170,6 @@ public class CuratorDruidCoordinatorTest extends CuratorTestBase
callbackExec,
druidCoordinatorConfig
);
loadManagementPeons = new ConcurrentHashMap<>();
leaderAnnouncerLatch = new CountDownLatch(1);
leaderUnannouncerLatch = new CountDownLatch(1);
}
@After
@ -256,20 +227,14 @@ public class CuratorDruidCoordinatorTest extends CuratorTestBase
setupZNodeForServer(source, zkPathsConfig, jsonMapper);
setupZNodeForServer(dest, zkPathsConfig, jsonMapper);
final List<DataSegment> sourceSegments = Lists.transform(
ImmutableList.of(
Pair.of("2011-04-01/2011-04-03", "v1"),
Pair.of("2011-04-03/2011-04-06", "v1"),
Pair.of("2011-04-06/2011-04-09", "v1")
),
input -> dataSegmentWithIntervalAndVersion(input.lhs, input.rhs)
final List<DataSegment> sourceSegments = Arrays.asList(
createSegment("2011-04-01/2011-04-03", "v1"),
createSegment("2011-04-03/2011-04-06", "v1"),
createSegment("2011-04-06/2011-04-09", "v1")
);
final List<DataSegment> destinationSegments = Lists.transform(
ImmutableList.of(
Pair.of("2011-03-31/2011-04-01", "v1")
),
input -> dataSegmentWithIntervalAndVersion(input.lhs, input.rhs)
final List<DataSegment> destinationSegments = Collections.singletonList(
createSegment("2011-03-31/2011-04-01", "v1")
);
DataSegment segmentToMove = sourceSegments.get(2);
@ -320,10 +285,8 @@ public class CuratorDruidCoordinatorTest extends CuratorTestBase
Assert.assertTrue(timing.forWaiting().awaitLatch(srcCountdown));
Assert.assertTrue(timing.forWaiting().awaitLatch(destCountdown));
loadManagementPeons.put("localhost:1", sourceLoadQueuePeon);
loadManagementPeons.put("localhost:2", destinationLoadQueuePeon);
sourceSegments.forEach(source::addDataSegment);
destinationSegments.forEach(dest::addDataSegment);
segmentRemovedLatch = new CountDownLatch(1);
segmentAddedLatch = new CountDownLatch(1);
@ -333,18 +296,50 @@ public class CuratorDruidCoordinatorTest extends CuratorTestBase
EasyMock.replay(druidDataSource);
EasyMock.expect(segmentsMetadataManager.getImmutableDataSourceWithUsedSegments(EasyMock.anyString()))
.andReturn(druidDataSource);
EasyMock.expect(coordinatorRuntimeParams.getDataSourcesSnapshot()).andReturn(dataSourcesSnapshot).anyTimes();
EasyMock.replay(segmentsMetadataManager, coordinatorRuntimeParams);
EasyMock.expect(coordinatorRuntimeParams.getDataSourcesSnapshot())
.andReturn(dataSourcesSnapshot).anyTimes();
final CoordinatorDynamicConfig dynamicConfig =
CoordinatorDynamicConfig.builder().withUseRoundRobinSegmentAssignment(false).build();
EasyMock.expect(coordinatorRuntimeParams.getCoordinatorDynamicConfig())
.andReturn(dynamicConfig)
.anyTimes();
EasyMock.expect(coordinatorRuntimeParams.getSegmentLoadingConfig())
.andReturn(SegmentLoadingConfig.create(dynamicConfig, 100))
.anyTimes();
EasyMock.expect(dataSourcesSnapshot.getDataSource(EasyMock.anyString())).andReturn(druidDataSource).anyTimes();
final ServerHolder sourceServer = new ServerHolder(source.toImmutableDruidServer(), sourceLoadQueuePeon);
final ServerHolder destinationServer = new ServerHolder(dest.toImmutableDruidServer(), destinationLoadQueuePeon);
final DruidCluster cluster = DruidCluster.builder().add(sourceServer).add(destinationServer).build();
final BalancerStrategy balancerStrategy = EasyMock.mock(BalancerStrategy.class);
EasyMock.expect(
balancerStrategy.findDestinationServerToMoveSegment(
EasyMock.anyObject(),
EasyMock.anyObject(),
EasyMock.anyObject()
)
).andReturn(destinationServer).atLeastOnce();
EasyMock.expect(coordinatorRuntimeParams.getBalancerStrategy())
.andReturn(balancerStrategy).anyTimes();
EasyMock.expect(coordinatorRuntimeParams.getDruidCluster()).andReturn(cluster).anyTimes();
EasyMock.replay(segmentsMetadataManager, coordinatorRuntimeParams, balancerStrategy);
EasyMock.expect(dataSourcesSnapshot.getDataSource(EasyMock.anyString()))
.andReturn(druidDataSource).anyTimes();
EasyMock.replay(dataSourcesSnapshot);
coordinator.moveSegment(
coordinatorRuntimeParams,
source.toImmutableDruidServer(),
dest.toImmutableDruidServer(),
sourceSegments.get(2),
null
LoadQueueTaskMaster taskMaster = EasyMock.createMock(LoadQueueTaskMaster.class);
EasyMock.expect(taskMaster.isHttpLoading()).andReturn(false).anyTimes();
EasyMock.replay(taskMaster);
// Move the segment from source to dest
SegmentLoadQueueManager loadQueueManager =
new SegmentLoadQueueManager(baseView, segmentsMetadataManager, taskMaster);
StrategicSegmentAssigner segmentAssigner = createSegmentAssigner(loadQueueManager, coordinatorRuntimeParams);
segmentAssigner.moveSegment(
segmentToMove,
sourceServer,
Collections.singletonList(destinationServer)
);
// wait for destination server to load segment
@ -365,45 +360,6 @@ public class CuratorDruidCoordinatorTest extends CuratorTestBase
Assert.assertEquals(2, servers.get(1).getTotalSegments());
}
private static class TestDruidLeaderSelector implements DruidLeaderSelector
{
private volatile Listener listener;
private volatile String leader;
@Override
public String getCurrentLeader()
{
return leader;
}
@Override
public boolean isLeader()
{
return leader != null;
}
@Override
public int localTerm()
{
return 0;
}
@Override
public void registerListener(Listener listener)
{
this.listener = listener;
leader = "what:1234";
listener.becomeLeader();
}
@Override
public void unregisterListener()
{
leader = null;
listener.stopBeingLeader();
}
}
private void setupView() throws Exception
{
baseView = new BatchServerInventoryView(
@ -455,35 +411,9 @@ public class CuratorDruidCoordinatorTest extends CuratorTestBase
sourceLoadQueuePeon.start();
destinationLoadQueuePeon.start();
final LoadQueueTaskMaster loadQueueTaskMaster = EasyMock.createMock(LoadQueueTaskMaster.class);
EasyMock.expect(loadQueueTaskMaster.isHttpLoading()).andReturn(false).anyTimes();
EasyMock.replay(loadQueueTaskMaster);
coordinator = new DruidCoordinator(
druidCoordinatorConfig,
configManager,
segmentsMetadataManager,
baseView,
EasyMock.createNiceMock(MetadataRuleManager.class),
new NoopServiceEmitter(),
(corePoolSize, nameFormat) -> Executors.newSingleThreadScheduledExecutor(),
null,
loadQueueTaskMaster,
new LatchableServiceAnnouncer(leaderAnnouncerLatch, leaderUnannouncerLatch),
new DruidNode("hey", "what", false, 1234, null, true, false),
loadManagementPeons,
null,
null,
new CoordinatorCustomDutyGroups(ImmutableSet.of()),
new CostBalancerStrategyFactory(),
EasyMock.createNiceMock(LookupCoordinatorManager.class),
new TestDruidLeaderSelector(),
null
);
}
private DataSegment dataSegmentWithIntervalAndVersion(String intervalStr, String version)
private DataSegment createSegment(String intervalStr, String version)
{
return DataSegment.builder()
.dataSource("test_curator_druid_coordinator")
@ -497,4 +427,18 @@ public class CuratorDruidCoordinatorTest extends CuratorTestBase
.size(0)
.build();
}
private StrategicSegmentAssigner createSegmentAssigner(
SegmentLoadQueueManager loadQueueManager,
DruidCoordinatorRuntimeParams params
)
{
return new StrategicSegmentAssigner(
loadQueueManager,
params.getDruidCluster(),
params.getBalancerStrategy(),
params.getSegmentLoadingConfig(),
new CoordinatorRunStats()
);
}
}

View File

@ -1,68 +0,0 @@
/*
* 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 javax.annotation.Nullable;
import java.util.Arrays;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Map;
import java.util.Set;
public final class DruidClusterBuilder
{
public static DruidClusterBuilder newBuilder()
{
return new DruidClusterBuilder();
}
private @Nullable Set<ServerHolder> realtimes = null;
private final Map<String, Iterable<ServerHolder>> historicals = new HashMap<>();
private @Nullable Set<ServerHolder> brokers = null;
private DruidClusterBuilder()
{
}
public DruidClusterBuilder withRealtimes(ServerHolder... realtimes)
{
this.realtimes = new HashSet<>(Arrays.asList(realtimes));
return this;
}
public DruidClusterBuilder withBrokers(ServerHolder... brokers)
{
this.brokers = new HashSet<>(Arrays.asList(brokers));
return this;
}
public DruidClusterBuilder addTier(String tierName, ServerHolder... historicals)
{
if (this.historicals.putIfAbsent(tierName, Arrays.asList(historicals)) != null) {
throw new IllegalArgumentException("Duplicate tier: " + tierName);
}
return this;
}
public DruidCluster build()
{
return DruidCluster.createDruidClusterFromBuilderInTest(realtimes, historicals, brokers);
}
}

View File

@ -19,21 +19,16 @@
package org.apache.druid.server.coordinator;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import org.apache.druid.client.ImmutableDruidDataSource;
import org.apache.druid.client.ImmutableDruidServer;
import org.apache.druid.java.util.common.Intervals;
import org.apache.druid.server.coordination.DruidServerMetadata;
import org.apache.druid.client.DruidServer;
import org.apache.druid.java.util.common.granularity.Granularities;
import org.apache.druid.server.coordination.ServerType;
import org.apache.druid.server.coordinator.loading.LoadQueuePeonTester;
import org.apache.druid.timeline.DataSegment;
import org.apache.druid.timeline.partition.NoneShardSpec;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
import java.util.Collection;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import java.util.NavigableSet;
@ -42,100 +37,62 @@ import java.util.stream.Collectors;
public class DruidClusterTest
{
private static final List<DataSegment> SEGMENTS = ImmutableList.of(
new DataSegment(
"test",
Intervals.of("2015-04-12/2015-04-13"),
"1",
ImmutableMap.of("containerName", "container1", "blobPath", "blobPath1"),
null,
null,
NoneShardSpec.instance(),
0,
1
),
new DataSegment(
"test",
Intervals.of("2015-04-12/2015-04-13"),
"1",
ImmutableMap.of("containerName", "container2", "blobPath", "blobPath2"),
null,
null,
NoneShardSpec.instance(),
0,
1
)
);
private static final Map<String, ImmutableDruidDataSource> DATA_SOURCES = ImmutableMap.of(
"src1", new ImmutableDruidDataSource("src1", Collections.emptyMap(), Collections.singletonList(SEGMENTS.get(0))),
"src2", new ImmutableDruidDataSource("src2", Collections.emptyMap(), Collections.singletonList(SEGMENTS.get(0)))
);
private static final List<DataSegment> SEGMENTS = CreateDataSegments
.ofDatasource("test")
.forIntervals(2, Granularities.DAY)
.startingAt("2015-04-12")
.withNumPartitions(1)
.eachOfSizeInMb(100);
private static final ServerHolder NEW_REALTIME = new ServerHolder(
new ImmutableDruidServer(
new DruidServerMetadata("name1", "host2", null, 100L, ServerType.REALTIME, "tier1", 0),
0L,
ImmutableMap.of("src1", DATA_SOURCES.get("src1")),
1
),
new DruidServer("name1", "host2", null, 100L, ServerType.REALTIME, "tier1", 0)
.addDataSegment(SEGMENTS.get(0)).toImmutableDruidServer(),
new LoadQueuePeonTester()
);
private static final ServerHolder NEW_HISTORICAL = new ServerHolder(
new ImmutableDruidServer(
new DruidServerMetadata("name1", "host2", null, 100L, ServerType.HISTORICAL, "tier1", 0),
0L,
ImmutableMap.of("src1", DATA_SOURCES.get("src1")),
1
),
new DruidServer("name1", "host2", null, 100L, ServerType.HISTORICAL, "tier1", 0)
.addDataSegment(SEGMENTS.get(0)).toImmutableDruidServer(),
new LoadQueuePeonTester()
);
private DruidCluster cluster;
private DruidCluster.Builder clusterBuilder;
@Before
public void setup()
{
cluster = DruidClusterBuilder
.newBuilder()
.withRealtimes(
clusterBuilder = DruidCluster
.builder()
.add(
new ServerHolder(
new ImmutableDruidServer(
new DruidServerMetadata("name1", "host1", null, 100L, ServerType.REALTIME, "tier1", 0),
0L,
ImmutableMap.of("src1", DATA_SOURCES.get("src1")),
1
),
new DruidServer("name1", "host1", null, 100L, ServerType.REALTIME, "tier1", 0)
.addDataSegment(SEGMENTS.get(0)).toImmutableDruidServer(),
new LoadQueuePeonTester()
)
)
.addTier(
"tier1",
.add(
new ServerHolder(
new ImmutableDruidServer(
new DruidServerMetadata("name1", "host1", null, 100L, ServerType.HISTORICAL, "tier1", 0),
0L,
ImmutableMap.of("src1", DATA_SOURCES.get("src1")),
1
),
new DruidServer("name1", "host1", null, 100L, ServerType.HISTORICAL, "tier1", 0)
.addDataSegment(SEGMENTS.get(0)).toImmutableDruidServer(),
new LoadQueuePeonTester()
)
)
.build();
);
}
@Test
public void testAdd()
{
DruidCluster cluster = clusterBuilder.build();
Assert.assertEquals(1, cluster.getHistoricals().values().stream().mapToInt(Collection::size).sum());
Assert.assertEquals(1, cluster.getRealtimes().size());
cluster.add(NEW_REALTIME);
clusterBuilder.add(NEW_REALTIME);
cluster = clusterBuilder.build();
Assert.assertEquals(1, cluster.getHistoricals().values().stream().mapToInt(Collection::size).sum());
Assert.assertEquals(2, cluster.getRealtimes().size());
cluster.add(NEW_HISTORICAL);
clusterBuilder.add(NEW_HISTORICAL);
cluster = clusterBuilder.build();
Assert.assertEquals(2, cluster.getHistoricals().values().stream().mapToInt(Collection::size).sum());
Assert.assertEquals(2, cluster.getRealtimes().size());
}
@ -143,8 +100,10 @@ public class DruidClusterTest
@Test
public void testGetAllServers()
{
cluster.add(NEW_REALTIME);
cluster.add(NEW_HISTORICAL);
clusterBuilder.add(NEW_REALTIME);
clusterBuilder.add(NEW_HISTORICAL);
DruidCluster cluster = clusterBuilder.build();
final Set<ServerHolder> expectedRealtimes = cluster.getRealtimes();
final Map<String, NavigableSet<ServerHolder>> expectedHistoricals = cluster.getHistoricals();
@ -153,7 +112,9 @@ public class DruidClusterTest
Assert.assertTrue(allServers.containsAll(cluster.getRealtimes()));
Assert.assertTrue(
allServers.containsAll(
cluster.getHistoricals().values().stream().flatMap(Collection::stream).collect(Collectors.toList())
cluster.getHistoricals().values().stream()
.flatMap(Collection::stream)
.collect(Collectors.toList())
)
);
@ -164,8 +125,8 @@ public class DruidClusterTest
@Test
public void testIsEmpty()
{
final DruidCluster emptyCluster = new DruidCluster();
Assert.assertFalse(cluster.isEmpty());
final DruidCluster emptyCluster = DruidCluster.EMPTY;
Assert.assertFalse(clusterBuilder.build().isEmpty());
Assert.assertTrue(emptyCluster.isEmpty());
}
}

View File

@ -49,25 +49,25 @@ import org.apache.druid.java.util.emitter.service.ServiceEmitter;
import org.apache.druid.metadata.MetadataRuleManager;
import org.apache.druid.metadata.SegmentsMetadataManager;
import org.apache.druid.server.DruidNode;
import org.apache.druid.server.coordination.DruidServerMetadata;
import org.apache.druid.server.coordination.ServerType;
import org.apache.druid.server.coordinator.balancer.CostBalancerStrategyFactory;
import org.apache.druid.server.coordinator.duty.CompactSegments;
import org.apache.druid.server.coordinator.duty.CoordinatorCustomDuty;
import org.apache.druid.server.coordinator.duty.CoordinatorCustomDutyGroup;
import org.apache.druid.server.coordinator.duty.CoordinatorCustomDutyGroups;
import org.apache.druid.server.coordinator.duty.CoordinatorDuty;
import org.apache.druid.server.coordinator.duty.EmitClusterStatsAndMetrics;
import org.apache.druid.server.coordinator.duty.KillSupervisorsCustomDuty;
import org.apache.druid.server.coordinator.duty.LogUsedSegments;
import org.apache.druid.server.coordinator.duty.NewestSegmentFirstPolicy;
import org.apache.druid.server.coordinator.loading.CuratorLoadQueuePeon;
import org.apache.druid.server.coordinator.loading.LoadQueuePeon;
import org.apache.druid.server.coordinator.loading.LoadQueueTaskMaster;
import org.apache.druid.server.coordinator.loading.SegmentLoadQueueManager;
import org.apache.druid.server.coordinator.rules.ForeverBroadcastDistributionRule;
import org.apache.druid.server.coordinator.rules.ForeverLoadRule;
import org.apache.druid.server.coordinator.rules.IntervalLoadRule;
import org.apache.druid.server.coordinator.rules.Rule;
import org.apache.druid.server.lookup.cache.LookupCoordinatorManager;
import org.apache.druid.timeline.DataSegment;
import org.apache.druid.timeline.SegmentId;
import org.easymock.Capture;
import org.easymock.EasyMock;
import org.joda.time.Duration;
import org.junit.After;
@ -80,8 +80,6 @@ import java.util.Collections;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.Executors;
import java.util.concurrent.atomic.AtomicReference;
@ -97,12 +95,10 @@ public class DruidCoordinatorTest extends CuratorTestBase
private DruidCoordinator coordinator;
private SegmentsMetadataManager segmentsMetadataManager;
private DataSourcesSnapshot dataSourcesSnapshot;
private DruidCoordinatorRuntimeParams coordinatorRuntimeParams;
private BatchServerInventoryView serverInventoryView;
private ScheduledExecutorFactory scheduledExecutorFactory;
private DruidServer druidServer;
private ConcurrentMap<String, LoadQueuePeon> loadManagementPeons;
private LoadQueuePeon loadQueuePeon;
private LoadQueueTaskMaster loadQueueTaskMaster;
private MetadataRuleManager metadataRuleManager;
@ -118,13 +114,13 @@ public class DruidCoordinatorTest extends CuratorTestBase
@Before
public void setUp() throws Exception
{
druidServer = EasyMock.createMock(DruidServer.class);
druidServer = new DruidServer("from", "from", null, 5L, ServerType.HISTORICAL, "tier1", 0);
serverInventoryView = EasyMock.createMock(BatchServerInventoryView.class);
segmentsMetadataManager = EasyMock.createNiceMock(SegmentsMetadataManager.class);
dataSourcesSnapshot = EasyMock.createNiceMock(DataSourcesSnapshot.class);
coordinatorRuntimeParams = EasyMock.createNiceMock(DruidCoordinatorRuntimeParams.class);
metadataRuleManager = EasyMock.createNiceMock(MetadataRuleManager.class);
loadQueueTaskMaster = EasyMock.createMock(LoadQueueTaskMaster.class);
JacksonConfigManager configManager = EasyMock.createNiceMock(JacksonConfigManager.class);
EasyMock.expect(
configManager.watch(
@ -170,7 +166,6 @@ public class DruidCoordinatorTest extends CuratorTestBase
);
loadQueuePeon.start();
druidNode = new DruidNode("hey", "what", false, 1234, null, true, false);
loadManagementPeons = new ConcurrentHashMap<>();
scheduledExecutorFactory = (corePoolSize, nameFormat) -> Executors.newSingleThreadScheduledExecutor();
leaderAnnouncerLatch = new CountDownLatch(1);
leaderUnannouncerLatch = new CountDownLatch(1);
@ -184,11 +179,11 @@ public class DruidCoordinatorTest extends CuratorTestBase
scheduledExecutorFactory,
null,
loadQueueTaskMaster,
new SegmentLoadQueueManager(serverInventoryView, segmentsMetadataManager, loadQueueTaskMaster),
new LatchableServiceAnnouncer(leaderAnnouncerLatch, leaderUnannouncerLatch),
druidNode,
loadManagementPeons,
null,
new HashSet<>(),
null,
new CoordinatorCustomDutyGroups(ImmutableSet.of()),
new CostBalancerStrategyFactory(),
EasyMock.createNiceMock(LookupCoordinatorManager.class),
@ -205,101 +200,6 @@ public class DruidCoordinatorTest extends CuratorTestBase
tearDownServerAndCurator();
}
@Test
public void testMoveSegment()
{
final DataSegment segment = EasyMock.createNiceMock(DataSegment.class);
EasyMock.expect(segment.getId()).andReturn(SegmentId.dummy("dummySegment"));
EasyMock.expect(segment.getDataSource()).andReturn("dummyDataSource");
EasyMock.replay(segment);
loadQueuePeon = EasyMock.createNiceMock(LoadQueuePeon.class);
EasyMock.expect(loadQueuePeon.getLoadQueueSize()).andReturn(new Long(1));
loadQueuePeon.markSegmentToDrop(segment);
EasyMock.expectLastCall().once();
Capture<LoadPeonCallback> loadCallbackCapture = Capture.newInstance();
Capture<LoadPeonCallback> dropCallbackCapture = Capture.newInstance();
loadQueuePeon.loadSegment(EasyMock.anyObject(DataSegment.class), EasyMock.capture(loadCallbackCapture));
EasyMock.expectLastCall().once();
loadQueuePeon.dropSegment(EasyMock.anyObject(DataSegment.class), EasyMock.capture(dropCallbackCapture));
EasyMock.expectLastCall().once();
loadQueuePeon.unmarkSegmentToDrop(segment);
EasyMock.expectLastCall().once();
EasyMock.expect(loadQueuePeon.getSegmentsToDrop()).andReturn(new HashSet<>()).once();
EasyMock.replay(loadQueuePeon);
ImmutableDruidDataSource druidDataSource = EasyMock.createNiceMock(ImmutableDruidDataSource.class);
EasyMock.expect(druidDataSource.getSegment(EasyMock.anyObject(SegmentId.class))).andReturn(segment);
EasyMock.replay(druidDataSource);
EasyMock
.expect(segmentsMetadataManager.getImmutableDataSourceWithUsedSegments(EasyMock.anyString()))
.andReturn(druidDataSource);
EasyMock.replay(segmentsMetadataManager);
EasyMock.expect(dataSourcesSnapshot.getDataSource(EasyMock.anyString())).andReturn(druidDataSource).anyTimes();
EasyMock.replay(dataSourcesSnapshot);
scheduledExecutorFactory = EasyMock.createNiceMock(ScheduledExecutorFactory.class);
EasyMock.replay(scheduledExecutorFactory);
EasyMock.replay(metadataRuleManager);
ImmutableDruidDataSource dataSource = EasyMock.createMock(ImmutableDruidDataSource.class);
EasyMock.expect(dataSource.getSegments()).andReturn(Collections.singletonList(segment)).anyTimes();
EasyMock.replay(dataSource);
EasyMock.expect(druidServer.toImmutableDruidServer()).andReturn(
new ImmutableDruidServer(
new DruidServerMetadata("from", null, null, 5L, ServerType.HISTORICAL, null, 0),
1L,
ImmutableMap.of("dummyDataSource", dataSource),
1
)
).atLeastOnce();
EasyMock.replay(druidServer);
DruidServer druidServer2 = EasyMock.createMock(DruidServer.class);
EasyMock.expect(druidServer2.toImmutableDruidServer()).andReturn(
new ImmutableDruidServer(
new DruidServerMetadata("to", null, null, 5L, ServerType.HISTORICAL, null, 0),
1L,
ImmutableMap.of("dummyDataSource", dataSource),
1
)
).atLeastOnce();
EasyMock.replay(druidServer2);
loadManagementPeons.put("from", loadQueuePeon);
loadManagementPeons.put("to", loadQueuePeon);
EasyMock.expect(serverInventoryView.isSegmentLoadedByServer("to", segment)).andReturn(true).once();
EasyMock.replay(serverInventoryView);
EasyMock.expect(loadQueueTaskMaster.isHttpLoading()).andReturn(false).anyTimes();
EasyMock.replay(loadQueueTaskMaster);
mockCoordinatorRuntimeParams();
coordinator.moveSegment(
coordinatorRuntimeParams,
druidServer.toImmutableDruidServer(),
druidServer2.toImmutableDruidServer(),
segment,
null
);
LoadPeonCallback loadCallback = loadCallbackCapture.getValue();
loadCallback.execute(true);
LoadPeonCallback dropCallback = dropCallbackCapture.getValue();
dropCallback.execute(true);
EasyMock.verify(druidServer, druidServer2, loadQueuePeon, serverInventoryView, metadataRuleManager);
EasyMock.verify(coordinatorRuntimeParams);
}
private void mockCoordinatorRuntimeParams()
{
EasyMock.expect(coordinatorRuntimeParams.getDataSourcesSnapshot()).andReturn(this.dataSourcesSnapshot).anyTimes();
EasyMock.replay(coordinatorRuntimeParams);
}
@Test(timeout = 60_000L)
public void testCoordinatorRun() throws Exception
{
@ -310,8 +210,6 @@ public class DruidCoordinatorTest extends CuratorTestBase
Rule foreverLoadRule = new ForeverLoadRule(ImmutableMap.of(tier, 2));
EasyMock.expect(metadataRuleManager.getRulesWithDefault(EasyMock.anyString()))
.andReturn(ImmutableList.of(foreverLoadRule)).atLeastOnce();
EasyMock.expect(metadataRuleManager.getAllRules())
.andReturn(ImmutableMap.of(dataSource, ImmutableList.of(foreverLoadRule))).atLeastOnce();
metadataRuleManager.stop();
EasyMock.expectLastCall().once();
@ -343,12 +241,12 @@ public class DruidCoordinatorTest extends CuratorTestBase
// Setup ServerInventoryView
druidServer = new DruidServer("server1", "localhost", null, 5L, ServerType.HISTORICAL, tier, 0);
loadManagementPeons.put("server1", loadQueuePeon);
setupPeons(Collections.singletonMap("server1", loadQueuePeon));
EasyMock.expect(serverInventoryView.getInventory()).andReturn(
ImmutableList.of(druidServer)
).atLeastOnce();
EasyMock.expect(serverInventoryView.isStarted()).andReturn(true).anyTimes();
EasyMock.replay(serverInventoryView);
EasyMock.replay(serverInventoryView, loadQueueTaskMaster);
coordinator.start();
// Wait for this coordinator to become leader
@ -371,15 +269,15 @@ public class DruidCoordinatorTest extends CuratorTestBase
serviceEmitter.latch = coordinatorRunLatch;
coordinatorRunLatch.await();
Assert.assertEquals(ImmutableMap.of(dataSource, 100.0), coordinator.getLoadStatus());
Assert.assertEquals(ImmutableMap.of(dataSource, 100.0), coordinator.getDatasourceToLoadStatus());
Object2IntMap<String> numsUnavailableUsedSegmentsPerDataSource =
coordinator.computeNumsUnavailableUsedSegmentsPerDataSource();
coordinator.getDatasourceToUnavailableSegmentCount();
Assert.assertEquals(1, numsUnavailableUsedSegmentsPerDataSource.size());
Assert.assertEquals(0, numsUnavailableUsedSegmentsPerDataSource.getInt(dataSource));
Map<String, Object2LongMap<String>> underReplicationCountsPerDataSourcePerTier =
coordinator.computeUnderReplicationCountsPerDataSourcePerTier();
coordinator.getTierToDatasourceToUnderReplicatedCount(false);
Assert.assertNotNull(underReplicationCountsPerDataSourcePerTier);
Assert.assertEquals(1, underReplicationCountsPerDataSourcePerTier.size());
@ -392,9 +290,8 @@ public class DruidCoordinatorTest extends CuratorTestBase
// The load rules asks for 2 replicas, therefore 1 replica should still be pending
Assert.assertEquals(1L, underRepliicationCountsPerDataSource.getLong(dataSource));
Map<String, Object2LongMap<String>> underReplicationCountsPerDataSourcePerTierUsingClusterView =
coordinator.computeUnderReplicationCountsPerDataSourcePerTierUsingClusterView();
coordinator.getTierToDatasourceToUnderReplicatedCount(true);
Assert.assertNotNull(underReplicationCountsPerDataSourcePerTier);
Assert.assertEquals(1, underReplicationCountsPerDataSourcePerTier.size());
@ -454,7 +351,7 @@ public class DruidCoordinatorTest extends CuratorTestBase
true,
Execs.singleThreaded("coordinator_test_path_children_cache_cold-%d")
);
loadManagementPeons.putAll(ImmutableMap.of("hot", loadQueuePeon, "cold", loadQueuePeonCold));
setupPeons(ImmutableMap.of("hot", loadQueuePeon, "cold", loadQueuePeonCold));
loadQueuePeonCold.start();
pathChildrenCache.start();
@ -467,15 +364,13 @@ public class DruidCoordinatorTest extends CuratorTestBase
EasyMock.expect(metadataRuleManager.getRulesWithDefault(EasyMock.anyString()))
.andReturn(ImmutableList.of(hotTier, coldTier)).atLeastOnce();
EasyMock.expect(metadataRuleManager.getAllRules())
.andReturn(ImmutableMap.of(dataSource, ImmutableList.of(hotTier, coldTier))).atLeastOnce();
EasyMock.expect(serverInventoryView.getInventory())
.andReturn(ImmutableList.of(hotServer, coldServer))
.atLeastOnce();
EasyMock.expect(serverInventoryView.isStarted()).andReturn(true).anyTimes();
EasyMock.replay(metadataRuleManager, serverInventoryView);
EasyMock.replay(metadataRuleManager, serverInventoryView, loadQueueTaskMaster);
coordinator.start();
leaderAnnouncerLatch.await(); // Wait for this coordinator to become leader
@ -489,16 +384,16 @@ public class DruidCoordinatorTest extends CuratorTestBase
serviceEmitter.latch = coordinatorRunLatch;
coordinatorRunLatch.await();
Assert.assertEquals(ImmutableMap.of(dataSource, 100.0), coordinator.getLoadStatus());
Assert.assertEquals(ImmutableMap.of(dataSource, 100.0), coordinator.getDatasourceToLoadStatus());
Map<String, Object2LongMap<String>> underReplicationCountsPerDataSourcePerTier =
coordinator.computeUnderReplicationCountsPerDataSourcePerTier();
coordinator.getTierToDatasourceToUnderReplicatedCount(false);
Assert.assertEquals(2, underReplicationCountsPerDataSourcePerTier.size());
Assert.assertEquals(0L, underReplicationCountsPerDataSourcePerTier.get(hotTierName).getLong(dataSource));
Assert.assertEquals(0L, underReplicationCountsPerDataSourcePerTier.get(coldTierName).getLong(dataSource));
Map<String, Object2LongMap<String>> underReplicationCountsPerDataSourcePerTierUsingClusterView =
coordinator.computeUnderReplicationCountsPerDataSourcePerTierUsingClusterView();
coordinator.getTierToDatasourceToUnderReplicatedCount(true);
Assert.assertEquals(2, underReplicationCountsPerDataSourcePerTierUsingClusterView.size());
Assert.assertEquals(0L, underReplicationCountsPerDataSourcePerTierUsingClusterView.get(hotTierName).getLong(dataSource));
Assert.assertEquals(0L, underReplicationCountsPerDataSourcePerTierUsingClusterView.get(coldTierName).getLong(dataSource));
@ -519,7 +414,6 @@ public class DruidCoordinatorTest extends CuratorTestBase
final String coldTierName = "cold";
final String tierName1 = "tier1";
final String tierName2 = "tier2";
final Rule broadcastDistributionRule = new ForeverBroadcastDistributionRule();
final String loadPathCold = "/druid/loadqueue/cold:1234";
final String loadPathBroker1 = "/druid/loadqueue/broker1:1234";
final String loadPathBroker2 = "/druid/loadqueue/broker2:1234";
@ -603,11 +497,13 @@ public class DruidCoordinatorTest extends CuratorTestBase
Execs.singleThreaded("coordinator_test_path_children_cache_peon-%d")
);
loadManagementPeons.putAll(ImmutableMap.of("hot", loadQueuePeon,
"cold", loadQueuePeonCold,
"broker1", loadQueuePeonBroker1,
"broker2", loadQueuePeonBroker2,
"peon", loadQueuePeonPoenServer));
setupPeons(ImmutableMap.of(
"hot", loadQueuePeon,
"cold", loadQueuePeonCold,
"broker1", loadQueuePeonBroker1,
"broker2", loadQueuePeonBroker2,
"peon", loadQueuePeonPoenServer
));
loadQueuePeonCold.start();
loadQueuePeonBroker1.start();
@ -619,22 +515,21 @@ public class DruidCoordinatorTest extends CuratorTestBase
pathChildrenCacheBroker2.start();
pathChildrenCachePeon.start();
DruidDataSource[] druidDataSources = {new DruidDataSource(dataSource, Collections.emptyMap())};
dataSegments.values().forEach(druidDataSources[0]::addSegment);
DruidDataSource druidDataSource = new DruidDataSource(dataSource, Collections.emptyMap());
dataSegments.values().forEach(druidDataSource::addSegment);
setupSegmentsMetadataMock(druidDataSources[0]);
setupSegmentsMetadataMock(druidDataSource);
final Rule broadcastDistributionRule = new ForeverBroadcastDistributionRule();
EasyMock.expect(metadataRuleManager.getRulesWithDefault(EasyMock.anyString()))
.andReturn(ImmutableList.of(broadcastDistributionRule)).atLeastOnce();
EasyMock.expect(metadataRuleManager.getAllRules())
.andReturn(ImmutableMap.of(dataSource, ImmutableList.of(broadcastDistributionRule))).atLeastOnce();
EasyMock.expect(serverInventoryView.getInventory())
.andReturn(ImmutableList.of(hotServer, coldServer, brokerServer1, brokerServer2, peonServer))
.atLeastOnce();
EasyMock.expect(serverInventoryView.isStarted()).andReturn(true).anyTimes();
EasyMock.replay(metadataRuleManager, serverInventoryView);
EasyMock.replay(metadataRuleManager, serverInventoryView, loadQueueTaskMaster);
coordinator.start();
leaderAnnouncerLatch.await(); // Wait for this coordinator to become leader
@ -654,10 +549,11 @@ public class DruidCoordinatorTest extends CuratorTestBase
serviceEmitter.latch = coordinatorRunLatch;
coordinatorRunLatch.await();
Assert.assertEquals(ImmutableMap.of(dataSource, 100.0), coordinator.getLoadStatus());
Assert.assertEquals(ImmutableMap.of(dataSource, 100.0), coordinator.getDatasourceToLoadStatus());
// Under-replicated counts are updated only after the next coordinator run
Map<String, Object2LongMap<String>> underReplicationCountsPerDataSourcePerTier =
coordinator.computeUnderReplicationCountsPerDataSourcePerTier();
coordinator.getTierToDatasourceToUnderReplicatedCount(false);
Assert.assertEquals(4, underReplicationCountsPerDataSourcePerTier.size());
Assert.assertEquals(0L, underReplicationCountsPerDataSourcePerTier.get(hotTierName).getLong(dataSource));
Assert.assertEquals(0L, underReplicationCountsPerDataSourcePerTier.get(coldTierName).getLong(dataSource));
@ -665,7 +561,7 @@ public class DruidCoordinatorTest extends CuratorTestBase
Assert.assertEquals(0L, underReplicationCountsPerDataSourcePerTier.get(tierName2).getLong(dataSource));
Map<String, Object2LongMap<String>> underReplicationCountsPerDataSourcePerTierUsingClusterView =
coordinator.computeUnderReplicationCountsPerDataSourcePerTierUsingClusterView();
coordinator.getTierToDatasourceToUnderReplicatedCount(true);
Assert.assertEquals(4, underReplicationCountsPerDataSourcePerTierUsingClusterView.size());
Assert.assertEquals(0L, underReplicationCountsPerDataSourcePerTierUsingClusterView.get(hotTierName).getLong(dataSource));
Assert.assertEquals(0L, underReplicationCountsPerDataSourcePerTierUsingClusterView.get(coldTierName).getLong(dataSource));
@ -694,7 +590,7 @@ public class DruidCoordinatorTest extends CuratorTestBase
EasyMock.anyObject(Class.class),
EasyMock.anyObject()
)
).andReturn(new AtomicReference(dynamicConfig)).anyTimes();
).andReturn(new AtomicReference<>(dynamicConfig)).anyTimes();
ScheduledExecutorFactory scheduledExecutorFactory = EasyMock.createNiceMock(ScheduledExecutorFactory.class);
EasyMock.replay(configManager, dynamicConfig, scheduledExecutorFactory);
@ -708,6 +604,7 @@ public class DruidCoordinatorTest extends CuratorTestBase
null,
scheduledExecutorFactory,
null,
loadQueueTaskMaster,
null,
null,
null,
@ -720,32 +617,30 @@ public class DruidCoordinatorTest extends CuratorTestBase
null
);
DruidCoordinator.DutiesRunnable duty = c.new DutiesRunnable(Collections.emptyList(), 0, "TEST");
// before initialization
Assert.assertEquals(0, c.getCachedBalancerThreadNumber());
Assert.assertNull(c.getBalancerExec());
// first initialization
duty.initBalancerExecutor();
System.out.println("c.getCachedBalancerThreadNumber(): " + c.getCachedBalancerThreadNumber());
c.initBalancerExecutor();
Assert.assertEquals(5, c.getCachedBalancerThreadNumber());
ListeningExecutorService firstExec = c.getBalancerExec();
Assert.assertNotNull(firstExec);
// second initialization, expect no changes as cachedBalancerThreadNumber is not changed
duty.initBalancerExecutor();
c.initBalancerExecutor();
Assert.assertEquals(5, c.getCachedBalancerThreadNumber());
ListeningExecutorService secondExec = c.getBalancerExec();
Assert.assertNotNull(secondExec);
Assert.assertTrue(firstExec == secondExec);
Assert.assertSame(firstExec, secondExec);
// third initialization, expect executor recreated as cachedBalancerThreadNumber is changed to 10
duty.initBalancerExecutor();
c.initBalancerExecutor();
Assert.assertEquals(10, c.getCachedBalancerThreadNumber());
ListeningExecutorService thirdExec = c.getBalancerExec();
Assert.assertNotNull(thirdExec);
Assert.assertFalse(secondExec == thirdExec);
Assert.assertFalse(firstExec == thirdExec);
Assert.assertNotSame(secondExec, thirdExec);
Assert.assertNotSame(firstExec, thirdExec);
}
@Test
@ -761,12 +656,12 @@ public class DruidCoordinatorTest extends CuratorTestBase
serviceEmitter,
scheduledExecutorFactory,
null,
loadQueueTaskMaster,
null,
new LatchableServiceAnnouncer(leaderAnnouncerLatch, leaderUnannouncerLatch),
druidNode,
loadManagementPeons,
ImmutableSet.of(),
new HashSet<>(),
ImmutableSet.of(),
emptyCustomDutyGroups,
new CostBalancerStrategyFactory(),
EasyMock.createNiceMock(LookupCoordinatorManager.class),
@ -784,7 +679,6 @@ public class DruidCoordinatorTest extends CuratorTestBase
// CompactSegments returned by this method should be created using the DruidCoordinatorConfig in the DruidCoordinator
CompactSegments duty = coordinator.initializeCompactSegmentsDuty(newestSegmentFirstPolicy);
Assert.assertNotNull(duty);
Assert.assertEquals(druidCoordinatorConfig.getCompactionSkipLockedIntervals(), duty.isSkipLockedIntervals());
}
@Test
@ -801,12 +695,12 @@ public class DruidCoordinatorTest extends CuratorTestBase
serviceEmitter,
scheduledExecutorFactory,
null,
loadQueueTaskMaster,
null,
new LatchableServiceAnnouncer(leaderAnnouncerLatch, leaderUnannouncerLatch),
druidNode,
loadManagementPeons,
ImmutableSet.of(),
new HashSet<>(),
ImmutableSet.of(),
customDutyGroups,
new CostBalancerStrategyFactory(),
EasyMock.createNiceMock(LookupCoordinatorManager.class),
@ -824,7 +718,6 @@ public class DruidCoordinatorTest extends CuratorTestBase
// CompactSegments returned by this method should be created using the DruidCoordinatorConfig in the DruidCoordinator
CompactSegments duty = coordinator.initializeCompactSegmentsDuty(newestSegmentFirstPolicy);
Assert.assertNotNull(duty);
Assert.assertEquals(druidCoordinatorConfig.getCompactionSkipLockedIntervals(), duty.isSkipLockedIntervals());
}
@Test
@ -849,12 +742,12 @@ public class DruidCoordinatorTest extends CuratorTestBase
serviceEmitter,
scheduledExecutorFactory,
null,
loadQueueTaskMaster,
null,
new LatchableServiceAnnouncer(leaderAnnouncerLatch, leaderUnannouncerLatch),
druidNode,
loadManagementPeons,
ImmutableSet.of(),
new HashSet<>(),
ImmutableSet.of(),
customDutyGroups,
new CostBalancerStrategyFactory(),
EasyMock.createNiceMock(LookupCoordinatorManager.class),
@ -870,7 +763,6 @@ public class DruidCoordinatorTest extends CuratorTestBase
Assert.assertFalse(compactSegmentsDutyFromCustomGroups.isEmpty());
Assert.assertEquals(1, compactSegmentsDutyFromCustomGroups.size());
Assert.assertNotNull(compactSegmentsDutyFromCustomGroups.get(0));
Assert.assertTrue(compactSegmentsDutyFromCustomGroups.get(0) instanceof CompactSegments);
// CompactSegments returned by this method should be from the Custom Duty Group
CompactSegments duty = coordinator.initializeCompactSegmentsDuty(newestSegmentFirstPolicy);
@ -891,16 +783,15 @@ public class DruidCoordinatorTest extends CuratorTestBase
EasyMock.anyObject(Class.class),
EasyMock.anyObject()
)
).andReturn(new AtomicReference(CoordinatorDynamicConfig.builder().build())).anyTimes();
).andReturn(new AtomicReference<>(CoordinatorDynamicConfig.builder().build())).anyTimes();
EasyMock.expect(
configManager.watch(
EasyMock.eq(CoordinatorCompactionConfig.CONFIG_KEY),
EasyMock.anyObject(Class.class),
EasyMock.anyObject()
)
).andReturn(new AtomicReference(CoordinatorCompactionConfig.empty())).anyTimes();
).andReturn(new AtomicReference<>(CoordinatorCompactionConfig.empty())).anyTimes();
EasyMock.replay(configManager);
EasyMock.expect(segmentsMetadataManager.isPollingDatabasePeriodically()).andReturn(true).anyTimes();
DruidDataSource dataSource = new DruidDataSource("dataSource1", Collections.emptyMap());
DataSegment dataSegment = new DataSegment(
"dataSource1",
@ -920,33 +811,37 @@ public class DruidCoordinatorTest extends CuratorTestBase
.expect(segmentsMetadataManager.getSnapshotOfDataSourcesWithAllUsedSegments())
.andReturn(dataSourcesSnapshot)
.anyTimes();
EasyMock.expect(segmentsMetadataManager.isPollingDatabasePeriodically()).andReturn(true).anyTimes();
EasyMock.expect(segmentsMetadataManager.iterateAllUsedSegments())
.andReturn(Collections.singletonList(dataSegment)).anyTimes();
EasyMock.replay(segmentsMetadataManager);
EasyMock.expect(serverInventoryView.isStarted()).andReturn(true).anyTimes();
EasyMock.expect(serverInventoryView.getInventory()).andReturn(Collections.emptyList()).anyTimes();
EasyMock.replay(serverInventoryView);
// Create CoordinatorCustomDutyGroups
// We will have two groups and each group has one duty
CountDownLatch latch1 = new CountDownLatch(1);
CoordinatorCustomDuty duty1 = new CoordinatorCustomDuty() {
@Override
public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params)
{
latch1.countDown();
return params;
}
CoordinatorCustomDuty duty1 = params -> {
latch1.countDown();
return params;
};
CoordinatorCustomDutyGroup group1 = new CoordinatorCustomDutyGroup("group1", Duration.standardSeconds(1), ImmutableList.of(duty1));
CoordinatorCustomDutyGroup group1 = new CoordinatorCustomDutyGroup(
"group1",
Duration.standardSeconds(1),
ImmutableList.of(duty1)
);
CountDownLatch latch2 = new CountDownLatch(1);
CoordinatorCustomDuty duty2 = new CoordinatorCustomDuty() {
@Override
public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params)
{
latch2.countDown();
return params;
}
CoordinatorCustomDuty duty2 = params -> {
latch2.countDown();
return params;
};
CoordinatorCustomDutyGroup group2 = new CoordinatorCustomDutyGroup("group2", Duration.standardSeconds(1), ImmutableList.of(duty2));
CoordinatorCustomDutyGroup group2 = new CoordinatorCustomDutyGroup(
"group2",
Duration.standardSeconds(1),
ImmutableList.of(duty2)
);
CoordinatorCustomDutyGroups groups = new CoordinatorCustomDutyGroups(ImmutableSet.of(group1, group2));
coordinator = new DruidCoordinator(
@ -958,12 +853,12 @@ public class DruidCoordinatorTest extends CuratorTestBase
serviceEmitter,
scheduledExecutorFactory,
null,
null,
loadQueueTaskMaster,
new SegmentLoadQueueManager(serverInventoryView, segmentsMetadataManager, loadQueueTaskMaster),
new LatchableServiceAnnouncer(leaderAnnouncerLatch, leaderUnannouncerLatch),
druidNode,
loadManagementPeons,
null,
new HashSet<>(),
null,
groups,
new CostBalancerStrategyFactory(),
EasyMock.createNiceMock(LookupCoordinatorManager.class),
@ -978,38 +873,12 @@ public class DruidCoordinatorTest extends CuratorTestBase
latch2.await();
}
@Test
public void testEmitClusterStatsAndMetricsAddedWhenNotInDutyList()
{
DruidCoordinator.DutiesRunnable dutyRunnable = coordinator.new DutiesRunnable(ImmutableList.of(new LogUsedSegments()), 0, "TEST");
List<? extends CoordinatorDuty> duties = dutyRunnable.getDuties();
int emitDutyFound = 0;
for (CoordinatorDuty duty : duties) {
if (duty instanceof EmitClusterStatsAndMetrics) {
emitDutyFound++;
}
}
Assert.assertEquals(1, emitDutyFound);
}
@Test
public void testEmitClusterStatsAndMetricsNotAddedAgainWhenInDutyList()
{
DruidCoordinator.DutiesRunnable dutyRunnable = coordinator.new DutiesRunnable(ImmutableList.of(new LogUsedSegments(), new EmitClusterStatsAndMetrics(coordinator, "TEST", false)), 0, "TEST");
List<? extends CoordinatorDuty> duties = dutyRunnable.getDuties();
int emitDutyFound = 0;
for (CoordinatorDuty duty : duties) {
if (duty instanceof EmitClusterStatsAndMetrics) {
emitDutyFound++;
}
}
Assert.assertEquals(1, emitDutyFound);
}
private CountDownLatch createCountDownLatchAndSetPathChildrenCacheListenerWithLatch(int latchCount,
PathChildrenCache pathChildrenCache,
Map<String, DataSegment> segments,
DruidServer server)
private CountDownLatch createCountDownLatchAndSetPathChildrenCacheListenerWithLatch(
int latchCount,
PathChildrenCache pathChildrenCache,
Map<String, DataSegment> segments,
DruidServer server
)
{
final CountDownLatch countDownLatch = new CountDownLatch(latchCount);
pathChildrenCache.getListenable().addListener(
@ -1080,6 +949,13 @@ public class DruidCoordinatorTest extends CuratorTestBase
.orElse(null);
}
private void setupPeons(Map<String, LoadQueuePeon> peonMap)
{
EasyMock.expect(loadQueueTaskMaster.giveMePeon(EasyMock.anyObject())).andAnswer(
() -> peonMap.get(((ImmutableDruidServer) EasyMock.getCurrentArgument(0)).getName())
).anyTimes();
}
private static class TestDruidLeaderSelector implements DruidLeaderSelector
{
private volatile Listener listener;

View File

@ -1,203 +0,0 @@
/*
* 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.fasterxml.jackson.core.type.TypeReference;
import com.google.common.collect.ImmutableMap;
import com.google.common.util.concurrent.Futures;
import com.google.common.util.concurrent.ListenableFuture;
import org.apache.druid.java.util.common.Intervals;
import org.apache.druid.java.util.common.RE;
import org.apache.druid.java.util.common.concurrent.Execs;
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.DataSegmentChangeRequest;
import org.apache.druid.server.coordination.SegmentLoadDropHandler;
import org.apache.druid.timeline.DataSegment;
import org.apache.druid.timeline.SegmentId;
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.Test;
import java.io.ByteArrayInputStream;
import java.util.ArrayList;
import java.util.List;
import java.util.Map;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.Executors;
/**
*/
public class HttpLoadQueuePeonTest
{
final DataSegment segment1 = new DataSegment(
"test1", Intervals.of("2014/2015"), "v1",
null, null, null, null, 0, 0
);
final DataSegment segment2 = new DataSegment(
"test2", Intervals.of("2014/2015"), "v1",
null, null, null, null, 0, 0
);
final DataSegment segment3 = new DataSegment(
"test3", Intervals.of("2014/2015"), "v1",
null, null, null, null, 0, 0
);
final DataSegment segment4 = new DataSegment(
"test4", Intervals.of("2014/2015"), "v1",
null, null, null, null, 0, 0
);
final TestDruidCoordinatorConfig config = new TestDruidCoordinatorConfig.Builder()
.withCoordinatorKillMaxSegments(10)
.withCoordinatorKillIgnoreDurationToRetain(false)
.withHttpLoadQueuePeonBatchSize(2)
.build();
@Test(timeout = 60_000L)
public void testSimple() throws Exception
{
HttpLoadQueuePeon httpLoadQueuePeon = new HttpLoadQueuePeon(
"http://dummy:4000",
ServerTestHelper.MAPPER,
new TestHttpClient(),
config,
Executors.newScheduledThreadPool(
2,
Execs.makeThreadFactory("HttpLoadQueuePeonTest-%s")
),
Execs.singleThreaded("HttpLoadQueuePeonTest")
);
httpLoadQueuePeon.start();
Map<SegmentId, CountDownLatch> latches = ImmutableMap.of(
segment1.getId(), new CountDownLatch(1),
segment2.getId(), new CountDownLatch(1),
segment3.getId(), new CountDownLatch(1),
segment4.getId(), new CountDownLatch(1)
);
httpLoadQueuePeon.dropSegment(segment1, success -> latches.get(segment1.getId()).countDown());
httpLoadQueuePeon.loadSegment(segment2, success -> latches.get(segment2.getId()).countDown());
httpLoadQueuePeon.dropSegment(segment3, success -> latches.get(segment3.getId()).countDown());
httpLoadQueuePeon.loadSegment(segment4, success -> latches.get(segment4.getId()).countDown());
latches.get(segment1.getId()).await();
latches.get(segment2.getId()).await();
latches.get(segment3.getId()).await();
latches.get(segment4.getId()).await();
httpLoadQueuePeon.stop();
}
@Test(timeout = 60_000L)
public void testLoadDropAfterStop() throws Exception
{
HttpLoadQueuePeon httpLoadQueuePeon = new HttpLoadQueuePeon(
"http://dummy:4000",
ServerTestHelper.MAPPER,
new TestHttpClient(),
config,
Executors.newScheduledThreadPool(
2,
Execs.makeThreadFactory("HttpLoadQueuePeonTest-%s")
),
Execs.singleThreaded("HttpLoadQueuePeonTest")
);
httpLoadQueuePeon.start();
Map<SegmentId, CountDownLatch> latches = ImmutableMap.of(
segment1.getId(), new CountDownLatch(1),
segment2.getId(), new CountDownLatch(1),
segment3.getId(), new CountDownLatch(1),
segment4.getId(), new CountDownLatch(1)
);
httpLoadQueuePeon.dropSegment(segment1, success -> latches.get(segment1.getId()).countDown());
httpLoadQueuePeon.loadSegment(segment2, success -> latches.get(segment2.getId()).countDown());
latches.get(segment1.getId()).await();
latches.get(segment2.getId()).await();
httpLoadQueuePeon.stop();
httpLoadQueuePeon.dropSegment(segment3, success -> latches.get(segment3.getId()).countDown());
httpLoadQueuePeon.loadSegment(segment4, success -> latches.get(segment4.getId()).countDown());
latches.get(segment3.getId()).await();
latches.get(segment4.getId()).await();
}
private static class TestHttpClient implements HttpClient
{
@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) {
statuses.add(new SegmentLoadDropHandler.DataSegmentChangeRequestAndStatus(
cr,
SegmentLoadDropHandler.Status.SUCCESS
));
}
return (ListenableFuture) Futures.immediateFuture(
new ByteArrayInputStream(
ServerTestHelper.MAPPER
.writerWithType(HttpLoadQueuePeon.RESPONSE_ENTITY_TYPE_REF)
.writeValueAsBytes(statuses)
)
);
}
catch (Exception ex) {
throw new RE(ex, "Unexpected exception.");
}
}
}
}

View File

@ -23,6 +23,8 @@ import org.apache.druid.client.DruidServer;
import org.apache.druid.java.util.common.Intervals;
import org.apache.druid.segment.IndexIO;
import org.apache.druid.server.coordination.ServerType;
import org.apache.druid.server.coordinator.loading.LoadQueuePeonTester;
import org.apache.druid.server.coordinator.loading.RoundRobinServerSelector;
import org.apache.druid.timeline.DataSegment;
import org.apache.druid.timeline.partition.NumberedShardSpec;
import org.junit.Assert;
@ -51,14 +53,14 @@ public class RoundRobinServerSelectorTest
public void testSingleIterator()
{
final ServerHolder serverXL = createHistorical("serverXL", 1000);
final ServerHolder serverL = createHistorical("serverXL", 900);
final ServerHolder serverM = createHistorical("serverXL", 800);
final ServerHolder serverL = createHistorical("serverL", 900);
final ServerHolder serverM = createHistorical("serverM", 800);
// This server is too small to house the segment
final ServerHolder serverXS = createHistorical("serverXL", 10);
final ServerHolder serverXS = createHistorical("serverXS", 10);
DruidCluster cluster = DruidClusterBuilder
.newBuilder()
DruidCluster cluster = DruidCluster
.builder()
.addTier(TIER, serverXL, serverM, serverXS, serverL)
.build();
final RoundRobinServerSelector selector = new RoundRobinServerSelector(cluster);
@ -78,14 +80,14 @@ public class RoundRobinServerSelectorTest
public void testNextIteratorContinuesFromSamePosition()
{
final ServerHolder serverXL = createHistorical("serverXL", 1000);
final ServerHolder serverL = createHistorical("serverXL", 900);
final ServerHolder serverM = createHistorical("serverXL", 800);
final ServerHolder serverL = createHistorical("serverL", 900);
final ServerHolder serverM = createHistorical("serverM", 800);
// This server is too small to house the segment
final ServerHolder serverXS = createHistorical("serverXL", 10);
final ServerHolder serverXS = createHistorical("serverXS", 10);
DruidCluster cluster = DruidClusterBuilder
.newBuilder()
DruidCluster cluster = DruidCluster
.builder()
.addTier(TIER, serverXL, serverM, serverXS, serverL)
.build();
final RoundRobinServerSelector selector = new RoundRobinServerSelector(cluster);
@ -109,10 +111,7 @@ public class RoundRobinServerSelectorTest
@Test
public void testNoServersInTier()
{
DruidCluster cluster = DruidClusterBuilder
.newBuilder()
.addTier(TIER)
.build();
DruidCluster cluster = DruidCluster.builder().addTier(TIER).build();
final RoundRobinServerSelector selector = new RoundRobinServerSelector(cluster);
Iterator<ServerHolder> eligibleServers = selector.getServersInTierToLoadSegment(TIER, segment);
@ -122,16 +121,13 @@ public class RoundRobinServerSelectorTest
@Test
public void testNoEligibleServerInTier()
{
DruidCluster cluster = DruidClusterBuilder
.newBuilder()
.addTier(
TIER,
createHistorical("server1", 40),
createHistorical("server2", 30),
createHistorical("server3", 10),
createHistorical("server4", 20)
)
.build();
DruidCluster cluster = DruidCluster.builder().addTier(
TIER,
createHistorical("server1", 40),
createHistorical("server2", 30),
createHistorical("server3", 10),
createHistorical("server4", 20)
).build();
final RoundRobinServerSelector selector = new RoundRobinServerSelector(cluster);
// Verify that only eligible servers are returned in order of available size
@ -142,7 +138,8 @@ public class RoundRobinServerSelectorTest
private ServerHolder createHistorical(String name, long size)
{
return new ServerHolder(
new DruidServer(name, name, null, size, ServerType.HISTORICAL, TIER, 1).toImmutableDruidServer(),
new DruidServer(name, name, null, size, ServerType.HISTORICAL, TIER, 1)
.toImmutableDruidServer(),
new LoadQueuePeonTester()
);
}

View File

@ -26,6 +26,7 @@ import org.apache.druid.client.ImmutableDruidServer;
import org.apache.druid.java.util.common.Intervals;
import org.apache.druid.server.coordination.DruidServerMetadata;
import org.apache.druid.server.coordination.ServerType;
import org.apache.druid.server.coordinator.loading.LoadQueuePeonTester;
import org.apache.druid.timeline.DataSegment;
import org.apache.druid.timeline.partition.NoneShardSpec;
import org.junit.Assert;
@ -115,8 +116,8 @@ public class ServerHolderTest
);
Assert.assertEquals(0, h1.compareTo(h2));
Assert.assertEquals(-1, h3.compareTo(h1));
Assert.assertEquals(-1, h3.compareTo(h4));
Assert.assertEquals(1, h3.compareTo(h1));
Assert.assertEquals(1, h3.compareTo(h4));
}
@Test
@ -192,7 +193,7 @@ public class ServerHolderTest
);
Assert.assertTrue(h1.isServingSegment(SEGMENTS.get(0)));
Assert.assertFalse(h1.isServingSegment(SEGMENTS.get(1)));
Assert.assertTrue(h1.isServingSegment(SEGMENTS.get(0).getId()));
Assert.assertFalse(h1.isServingSegment(SEGMENTS.get(1).getId()));
Assert.assertTrue(h1.hasSegmentLoaded(SEGMENTS.get(0).getId()));
Assert.assertFalse(h1.hasSegmentLoaded(SEGMENTS.get(1).getId()));
}
}

View File

@ -17,12 +17,14 @@
* under the License.
*/
package org.apache.druid.server.coordinator;
package org.apache.druid.server.coordinator.balancer;
import org.apache.druid.client.DruidServer;
import org.apache.druid.java.util.common.Intervals;
import org.apache.druid.java.util.common.concurrent.Execs;
import org.apache.druid.server.coordination.ServerType;
import org.apache.druid.server.coordinator.ServerHolder;
import org.apache.druid.server.coordinator.loading.LoadQueuePeonTester;
import org.apache.druid.timeline.DataSegment;
import org.apache.druid.timeline.partition.NoneShardSpec;
import org.junit.Assert;
@ -33,6 +35,7 @@ import org.junit.runners.Parameterized;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
@ -82,31 +85,33 @@ public class BalancerStrategyTest
final ServerHolder serverHolder = new ServerHolder(
new DruidServer("server1", "host1", null, 10L, ServerType.HISTORICAL, DruidServer.DEFAULT_TIER, 0).addDataSegment(proposedDataSegment).toImmutableDruidServer(),
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.findServersToLoadSegment(
proposedDataSegment,
Collections.singletonList(serverHolder)
).hasNext()
);
}
@Test(timeout = 5000L)
public void findNewSegmentHomeReplicatorNotEnoughNodesForReplication()
{
final ServerHolder serverHolder1 = new ServerHolder(
new DruidServer("server1", "host1", null, 1000L, ServerType.HISTORICAL, DruidServer.DEFAULT_TIER, 0).addDataSegment(proposedDataSegment).toImmutableDruidServer(),
new DruidServer("server1", "host1", null, 1000L, ServerType.HISTORICAL, DruidServer.DEFAULT_TIER, 0)
.addDataSegment(proposedDataSegment).toImmutableDruidServer(),
new LoadQueuePeonTester());
final ServerHolder serverHolder2 = new ServerHolder(
new DruidServer("server2", "host2", null, 1000L, ServerType.HISTORICAL, DruidServer.DEFAULT_TIER, 0).addDataSegment(proposedDataSegment).toImmutableDruidServer(),
new DruidServer("server2", "host2", null, 1000L, ServerType.HISTORICAL, DruidServer.DEFAULT_TIER, 0)
.addDataSegment(proposedDataSegment).toImmutableDruidServer(),
new LoadQueuePeonTester());
serverHolders = new ArrayList<>();
serverHolders.add(serverHolder1);
serverHolders.add(serverHolder2);
final ServerHolder foundServerHolder = balancerStrategy.findNewSegmentHomeReplicator(proposedDataSegment, serverHolders);
// since there is not enough nodes to load 3 replicas of segment
Assert.assertNull(foundServerHolder);
Assert.assertFalse(balancerStrategy.findServersToLoadSegment(proposedDataSegment, serverHolders).hasNext());
}
@Test
@ -117,7 +122,8 @@ public class BalancerStrategyTest
new LoadQueuePeonTester());
serverHolders = new ArrayList<>();
serverHolders.add(serverHolder);
final ServerHolder foundServerHolder = balancerStrategy.findNewSegmentHomeReplicator(proposedDataSegment, serverHolders);
final ServerHolder foundServerHolder = balancerStrategy
.findServersToLoadSegment(proposedDataSegment, serverHolders).next();
// since there is enough space on server it should be selected
Assert.assertEquals(serverHolder, foundServerHolder);
}

View File

@ -17,7 +17,7 @@
* under the License.
*/
package org.apache.druid.server.coordinator.cost;
package org.apache.druid.server.coordinator.balancer;
import com.google.common.util.concurrent.ListeningExecutorService;
import com.google.common.util.concurrent.MoreExecutors;
@ -25,10 +25,8 @@ 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.CachingCostBalancerStrategy;
import org.apache.druid.server.coordinator.CostBalancerStrategy;
import org.apache.druid.server.coordinator.LoadQueuePeonTester;
import org.apache.druid.server.coordinator.ServerHolder;
import org.apache.druid.server.coordinator.loading.LoadQueuePeonTester;
import org.apache.druid.timeline.DataSegment;
import org.joda.time.DateTime;
import org.joda.time.Interval;
@ -95,12 +93,15 @@ public class CachingCostBalancerStrategyTest
executorService
);
CostBalancerStrategy costBalancerStrategy = createCostBalancerStrategy(executorService);
final ServerHolder firstServer = serverHolderList.get(0);
int notEqual = segmentQueries
.stream()
.mapToInt(
s -> {
ServerHolder s1 = cachingCostBalancerStrategy.findNewSegmentHomeBalancer(s, serverHolderList);
ServerHolder s2 = costBalancerStrategy.findNewSegmentHomeBalancer(s, serverHolderList);
ServerHolder s1 = cachingCostBalancerStrategy
.findDestinationServerToMoveSegment(s, firstServer, serverHolderList);
ServerHolder s2 = costBalancerStrategy
.findDestinationServerToMoveSegment(s, firstServer, serverHolderList);
return (s1.getServer().getName().equals(s2.getServer().getName())) ? 0 : 1;
}
)

View File

@ -17,7 +17,7 @@
* under the License.
*/
package org.apache.druid.server.coordinator;
package org.apache.druid.server.coordinator.balancer;
import com.google.common.util.concurrent.MoreExecutors;
import org.apache.druid.client.DruidServer;
@ -28,6 +28,9 @@ import org.apache.druid.java.util.emitter.core.Event;
import org.apache.druid.java.util.emitter.service.AlertEvent;
import org.apache.druid.java.util.metrics.StubServiceEmitter;
import org.apache.druid.server.coordination.ServerType;
import org.apache.druid.server.coordinator.CreateDataSegments;
import org.apache.druid.server.coordinator.ServerHolder;
import org.apache.druid.server.coordinator.loading.LoadQueuePeonTester;
import org.apache.druid.server.coordinator.simulate.BlockingExecutorService;
import org.apache.druid.timeline.DataSegment;
import org.junit.After;
@ -320,13 +323,14 @@ public class CostBalancerStrategyTest
.startingAt("2012-10-24")
.eachOfSizeInMb(100).get(0);
ServerHolder serverA = new ServerHolder(createHistorical().toImmutableDruidServer(), null);
ServerHolder serverB = new ServerHolder(createHistorical().toImmutableDruidServer(), null);
final LoadQueuePeonTester peon = new LoadQueuePeonTester();
ServerHolder serverA = new ServerHolder(createHistorical().toImmutableDruidServer(), peon);
ServerHolder serverB = new ServerHolder(createHistorical().toImmutableDruidServer(), peon);
balancerExecutor.shutdownNow();
Assert.assertThrows(
RejectedExecutionException.class,
() -> strategy.findNewSegmentHomeBalancer(segment, Arrays.asList(serverA, serverB))
() -> strategy.findServersToLoadSegment(segment, Arrays.asList(serverA, serverB))
);
}
@ -338,14 +342,22 @@ public class CostBalancerStrategyTest
.startingAt("2012-10-24")
.eachOfSizeInMb(100).get(0);
ServerHolder serverA = new ServerHolder(createHistorical().toImmutableDruidServer(), null);
ServerHolder serverB = new ServerHolder(createHistorical().toImmutableDruidServer(), null);
final LoadQueuePeonTester peon = new LoadQueuePeonTester();
ServerHolder serverA = new ServerHolder(createHistorical().toImmutableDruidServer(), peon);
ServerHolder serverB = new ServerHolder(createHistorical().toImmutableDruidServer(), peon);
strategy.findNewSegmentHomeBalancer(segment, Arrays.asList(serverA, serverB));
strategy.findServersToLoadSegment(segment, Arrays.asList(serverA, serverB));
List<Event> events = serviceEmitter.getEvents();
Assert.assertEquals(1, events.size());
Assert.assertTrue(events.get(0) instanceof AlertEvent);
AlertEvent alertEvent = (AlertEvent) events.get(0);
Assert.assertEquals(
"Cost balancer strategy timed out in action [findServersToLoadSegment]."
+ " Try setting a higher value of 'balancerComputeThreads'.",
alertEvent.getDescription()
);
}
private void verifyServerCosts(

View File

@ -17,7 +17,7 @@
* under the License.
*/
package org.apache.druid.server.coordinator;
package org.apache.druid.server.coordinator.balancer;
import com.google.common.collect.ImmutableMap;
import com.google.common.util.concurrent.MoreExecutors;
@ -28,6 +28,8 @@ import org.apache.druid.java.util.common.Intervals;
import org.apache.druid.java.util.common.concurrent.Execs;
import org.apache.druid.server.coordination.DruidServerMetadata;
import org.apache.druid.server.coordination.ServerType;
import org.apache.druid.server.coordinator.ServerHolder;
import org.apache.druid.server.coordinator.loading.LoadQueuePeonTester;
import org.apache.druid.timeline.DataSegment;
import org.easymock.EasyMock;
import org.joda.time.Interval;
@ -134,7 +136,7 @@ public class DiskNormalizedCostBalancerStrategyTest
BalancerStrategy strategy = new DiskNormalizedCostBalancerStrategy(
MoreExecutors.listeningDecorator(Execs.multiThreaded(4, "DiskNormalizedCostBalancerStrategyTest-%d"))
);
ServerHolder holder = strategy.findNewSegmentHomeReplicator(segment, serverHolderList);
ServerHolder holder = strategy.findServersToLoadSegment(segment, serverHolderList).next();
Assert.assertNotNull("Should be able to find a place for new segment!!", holder);
Assert.assertEquals("Best Server should be BEST_SERVER", "BEST_SERVER", holder.getServer().getName());
}
@ -148,7 +150,7 @@ public class DiskNormalizedCostBalancerStrategyTest
BalancerStrategy strategy = new DiskNormalizedCostBalancerStrategy(
MoreExecutors.listeningDecorator(Execs.multiThreaded(1, "DiskNormalizedCostBalancerStrategyTest-%d"))
);
ServerHolder holder = strategy.findNewSegmentHomeReplicator(segment, serverHolderList);
ServerHolder holder = strategy.findServersToLoadSegment(segment, serverHolderList).next();
Assert.assertNotNull("Should be able to find a place for new segment!!", holder);
Assert.assertEquals("Best Server should be BEST_SERVER", "BEST_SERVER", holder.getServer().getName());
}

View File

@ -17,23 +17,30 @@
* under the License.
*/
package org.apache.druid.server.coordinator;
package org.apache.druid.server.coordinator.balancer;
import org.apache.druid.client.DruidServer;
import org.apache.druid.java.util.common.granularity.Granularities;
import org.apache.druid.server.coordination.ServerType;
import org.apache.druid.server.coordinator.CreateDataSegments;
import org.apache.druid.server.coordinator.ServerHolder;
import org.apache.druid.server.coordinator.loading.LoadQueuePeonTester;
import org.apache.druid.server.coordinator.loading.SegmentAction;
import org.apache.druid.timeline.DataSegment;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.function.Function;
import java.util.stream.Collectors;
public class ReservoirSegmentSamplerTest
{
@ -48,6 +55,9 @@ public class ReservoirSegmentSamplerTest
.withNumPartitions(10)
.eachOfSizeInMb(100);
private final Function<ServerHolder, Collection<DataSegment>> GET_SERVED_SEGMENTS
= serverHolder -> serverHolder.getServer().iterateAllSegments();
@Before
public void setUp()
{
@ -70,7 +80,7 @@ public class ReservoirSegmentSamplerTest
// due to the pseudo-randomness of this method, we may not select a segment every single time no matter what.
segmentCountMap.compute(
ReservoirSegmentSampler
.getRandomBalancerSegmentHolders(servers, Collections.emptySet(), 1)
.pickMovableSegmentsFrom(servers, 1, GET_SERVED_SEGMENTS, Collections.emptySet())
.get(0).getSegment(),
(segment, count) -> count == null ? 1 : count + 1
);
@ -114,6 +124,42 @@ public class ReservoirSegmentSamplerTest
Assert.assertEquals(3, segmentCountMap.size());
}
@Test
public void testPickLoadingOrLoadedSegments()
{
final List<DataSegment> loadedSegments = Arrays.asList(segments.get(0), segments.get(1));
final List<DataSegment> loadingSegments = Arrays.asList(segments.get(2), segments.get(3));
final ServerHolder server1 = createHistorical("server1", loadedSegments.get(0));
server1.startOperation(SegmentAction.LOAD, loadingSegments.get(0));
final ServerHolder server2 = createHistorical("server2", loadedSegments.get(1));
server2.startOperation(SegmentAction.LOAD, loadingSegments.get(1));
// Pick only loading segments
Set<DataSegment> pickedSegments = ReservoirSegmentSampler
.pickMovableSegmentsFrom(
Arrays.asList(server1, server2),
10,
ServerHolder::getLoadingSegments,
Collections.emptySet()
)
.stream().map(BalancerSegmentHolder::getSegment).collect(Collectors.toSet());
// Verify that only loading segments are picked
Assert.assertEquals(loadingSegments.size(), pickedSegments.size());
Assert.assertTrue(pickedSegments.containsAll(loadingSegments));
// Pick only loaded segments
pickedSegments = ReservoirSegmentSampler
.pickMovableSegmentsFrom(Arrays.asList(server1, server2), 10, GET_SERVED_SEGMENTS, Collections.emptySet())
.stream().map(BalancerSegmentHolder::getSegment).collect(Collectors.toSet());
// Verify that only loaded segments are picked
Assert.assertEquals(loadedSegments.size(), pickedSegments.size());
Assert.assertTrue(pickedSegments.containsAll(loadedSegments));
}
@Test
public void testSegmentsOnBrokersAreIgnored()
{
@ -128,16 +174,17 @@ public class ReservoirSegmentSamplerTest
);
// Try to pick all the segments on the servers
List<BalancerSegmentHolder> pickedSegments = ReservoirSegmentSampler.getRandomBalancerSegmentHolders(
List<BalancerSegmentHolder> pickedSegments = ReservoirSegmentSampler.pickMovableSegmentsFrom(
Arrays.asList(historical, broker),
Collections.emptySet(),
10
10,
GET_SERVED_SEGMENTS,
Collections.emptySet()
);
// Verify that only the segments on the historical are picked
Assert.assertEquals(2, pickedSegments.size());
for (BalancerSegmentHolder holder : pickedSegments) {
Assert.assertEquals(historical.getServer(), holder.getFromServer());
Assert.assertEquals(historical, holder.getServer());
}
}
@ -160,7 +207,7 @@ public class ReservoirSegmentSamplerTest
// Try to pick all the segments on the servers
List<BalancerSegmentHolder> pickedSegments = ReservoirSegmentSampler
.getRandomBalancerSegmentHolders(servers, Collections.singleton(broadcastDatasource), 10);
.pickMovableSegmentsFrom(servers, 10, GET_SERVED_SEGMENTS, Collections.singleton(broadcastDatasource));
// Verify that none of the broadcast segments are picked
Assert.assertEquals(2, pickedSegments.size());
@ -212,7 +259,7 @@ public class ReservoirSegmentSamplerTest
int numIterations = 1;
for (; numIterations < 10000; ++numIterations) {
ReservoirSegmentSampler
.getRandomBalancerSegmentHolders(servers, Collections.emptySet(), sampleSize)
.pickMovableSegmentsFrom(servers, sampleSize, GET_SERVED_SEGMENTS, Collections.emptySet())
.forEach(holder -> pickedSegments.add(holder.getSegment()));
if (pickedSegments.size() >= numSegments) {

View File

@ -17,7 +17,7 @@
* under the License.
*/
package org.apache.druid.server.coordinator.cost;
package org.apache.druid.server.coordinator.balancer;
import org.apache.druid.java.util.common.DateTimes;
import org.apache.druid.timeline.DataSegment;

View File

@ -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)
.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().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 DruidCoordinatorRuntimeParams
.newBuilder(DateTimes.nowUtc())
.withDruidCluster(DruidCluster.builder().addTier("normal", servers).build())
.withUsedSegmentsInTest(allSegments)
.withBroadcastDatasources(broadcastDatasources)
.withBalancerStrategy(balancerStrategy)
.withSegmentAssignerUsing(loadQueueManager);
}
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,
10
);
}
private void expectFindDestinationAndReturn(BalancerStrategy strategy, ServerHolder chosenServer)
{
EasyMock.expect(
strategy.findDestinationServerToMoveSegment(
EasyMock.anyObject(),
EasyMock.anyObject(),
EasyMock.anyObject()
)
).andReturn(chosenServer).anyTimes();
}
private DataSegment createHourlySegment(String datasource, DateTime start, String version)
{
return new DataSegment(
datasource,
new Interval(start, start.plusHours(1)),
version,
Collections.emptyMap(),
Collections.emptyList(),
Collections.emptyList(),
NoneShardSpec.instance(),
0,
8L
);
}
}

View File

@ -0,0 +1,71 @@
/*
* 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 it.unimi.dsi.fastutil.objects.Object2IntMaps;
import it.unimi.dsi.fastutil.objects.Object2LongMaps;
import org.apache.druid.java.util.common.DateTimes;
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.balancer.RandomBalancerStrategy;
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.junit.Assert;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.mockito.Mock;
import org.mockito.Mockito;
import org.mockito.junit.MockitoJUnitRunner;
import java.util.Collections;
@RunWith(MockitoJUnitRunner.class)
public class CollectSegmentAndServerStatsTest
{
@Mock
private DruidCoordinator mockDruidCoordinator;
@Test
public void testCollectedSegmentStats()
{
DruidCoordinatorRuntimeParams runtimeParams =
DruidCoordinatorRuntimeParams.newBuilder(DateTimes.nowUtc())
.withDruidCluster(DruidCluster.EMPTY)
.withUsedSegmentsInTest()
.withBalancerStrategy(new RandomBalancerStrategy())
.withSegmentAssignerUsing(new SegmentLoadQueueManager(null, null, null))
.build();
Mockito.when(mockDruidCoordinator.getDatasourceToUnavailableSegmentCount())
.thenReturn(Object2IntMaps.singleton("ds", 10));
Mockito.when(mockDruidCoordinator.getTierToDatasourceToUnderReplicatedCount(false))
.thenReturn(Collections.singletonMap("ds", Object2LongMaps.singleton("tier1", 100)));
CoordinatorDuty duty = new CollectSegmentAndServerStats(mockDruidCoordinator);
DruidCoordinatorRuntimeParams params = duty.run(runtimeParams);
CoordinatorRunStats stats = params.getCoordinatorStats();
Assert.assertTrue(stats.hasStat(Stats.Segments.UNAVAILABLE));
Assert.assertTrue(stats.hasStat(Stats.Segments.UNDER_REPLICATED));
}
}

View File

@ -74,8 +74,6 @@ import org.apache.druid.segment.transform.TransformSpec;
import org.apache.druid.server.DruidNode;
import org.apache.druid.server.coordinator.AutoCompactionSnapshot;
import org.apache.druid.server.coordinator.CoordinatorCompactionConfig;
import org.apache.druid.server.coordinator.CoordinatorRuntimeParamsTestHelpers;
import org.apache.druid.server.coordinator.CoordinatorStats;
import org.apache.druid.server.coordinator.DataSourceCompactionConfig;
import org.apache.druid.server.coordinator.DruidCoordinatorConfig;
import org.apache.druid.server.coordinator.DruidCoordinatorRuntimeParams;
@ -84,6 +82,8 @@ import org.apache.druid.server.coordinator.UserCompactionTaskGranularityConfig;
import org.apache.druid.server.coordinator.UserCompactionTaskIOConfig;
import org.apache.druid.server.coordinator.UserCompactionTaskQueryTuningConfig;
import org.apache.druid.server.coordinator.UserCompactionTaskTransformConfig;
import org.apache.druid.server.coordinator.stats.CoordinatorRunStats;
import org.apache.druid.server.coordinator.stats.Stats;
import org.apache.druid.timeline.CompactionState;
import org.apache.druid.timeline.DataSegment;
import org.apache.druid.timeline.SegmentTimeline;
@ -122,6 +122,7 @@ import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.function.BiFunction;
import java.util.function.BooleanSupplier;
import java.util.function.Supplier;
@ -178,7 +179,7 @@ public class CompactSegmentsTest
private final PartitionsSpec partitionsSpec;
private final BiFunction<Integer, Integer, ShardSpec> shardSpecFactory;
private Map<String, SegmentTimeline> dataSources;
private DataSourcesSnapshot dataSources;
Map<String, List<DataSegment>> datasourceToSegments = new HashMap<>();
public CompactSegmentsTest(PartitionsSpec partitionsSpec, BiFunction<Integer, Integer, ShardSpec> shardSpecFactory)
@ -205,9 +206,7 @@ public class CompactSegmentsTest
}
}
}
dataSources = DataSourcesSnapshot
.fromUsedSegments(allSegments, ImmutableMap.of())
.getUsedSegmentsTimelinesPerDataSource();
dataSources = DataSourcesSnapshot.fromUsedSegments(allSegments, ImmutableMap.of());
Mockito.when(COORDINATOR_CONFIG.getCompactionSkipLockedIntervals()).thenReturn(true);
}
@ -234,8 +233,8 @@ public class CompactSegmentsTest
interval,
"version",
null,
ImmutableList.of(),
ImmutableList.of(),
Collections.emptyList(),
Collections.emptyList(),
shardSpec,
0,
10L
@ -260,8 +259,6 @@ public class CompactSegmentsTest
CompactSegments serdeCompactSegments = JSON_MAPPER.readValue(compactSegmentString, CompactSegments.class);
Assert.assertNotNull(serdeCompactSegments);
Assert.assertEquals(COORDINATOR_CONFIG.getCompactionSkipLockedIntervals(), serdeCompactSegments.isSkipLockedIntervals());
Assert.assertEquals(indexingServiceClient, serdeCompactSegments.getIndexingServiceClient());
}
@Test
@ -358,10 +355,10 @@ public class CompactSegmentsTest
doCompactionAndAssertCompactSegmentStatistics(compactSegments, compactionRunCount);
}
// Test that stats does not change (and is still correct) when auto compaction runs with everything is fully compacted
final CoordinatorStats stats = doCompactSegments(compactSegments);
final CoordinatorRunStats stats = doCompactSegments(compactSegments);
Assert.assertEquals(
0,
stats.getGlobalStat(CompactSegments.COMPACTION_TASK_COUNT)
stats.get(Stats.Compaction.SUBMITTED_TASKS)
);
for (int i = 0; i < 3; i++) {
verifySnapshot(
@ -407,7 +404,7 @@ public class CompactSegmentsTest
doCompactSegments(compactSegments, new ArrayList<>());
Assert.assertEquals(
0,
stats.getGlobalStat(CompactSegments.COMPACTION_TASK_COUNT)
stats.get(Stats.Compaction.SUBMITTED_TASKS)
);
Assert.assertTrue(compactSegments.getAutoCompactionSnapshot().isEmpty());
@ -439,10 +436,7 @@ public class CompactSegmentsTest
}
}
dataSources = DataSourcesSnapshot
.fromUsedSegments(segments, ImmutableMap.of())
.getUsedSegmentsTimelinesPerDataSource();
dataSources = DataSourcesSnapshot.fromUsedSegments(segments, ImmutableMap.of());
final TestDruidLeaderClient leaderClient = new TestDruidLeaderClient(JSON_MAPPER);
leaderClient.start();
@ -456,10 +450,10 @@ public class CompactSegmentsTest
// 3 intervals, 120 byte, 12 segments already compacted before the run
for (int compactionRunCount = 0; compactionRunCount < 8; compactionRunCount++) {
// Do a cycle of auto compaction which creates one compaction task
final CoordinatorStats stats = doCompactSegments(compactSegments);
final CoordinatorRunStats stats = doCompactSegments(compactSegments);
Assert.assertEquals(
1,
stats.getGlobalStat(CompactSegments.COMPACTION_TASK_COUNT)
stats.get(Stats.Compaction.SUBMITTED_TASKS)
);
verifySnapshot(
@ -482,10 +476,10 @@ public class CompactSegmentsTest
}
// Test that stats does not change (and is still correct) when auto compaction runs with everything is fully compacted
final CoordinatorStats stats = doCompactSegments(compactSegments);
final CoordinatorRunStats stats = doCompactSegments(compactSegments);
Assert.assertEquals(
0,
stats.getGlobalStat(CompactSegments.COMPACTION_TASK_COUNT)
stats.get(Stats.Compaction.SUBMITTED_TASKS)
);
verifySnapshot(
compactSegments,
@ -521,10 +515,10 @@ public class CompactSegmentsTest
doCompactionAndAssertCompactSegmentStatistics(compactSegments, compactionRunCount);
}
// Test that stats does not change (and is still correct) when auto compaction runs with everything is fully compacted
final CoordinatorStats stats = doCompactSegments(compactSegments);
final CoordinatorRunStats stats = doCompactSegments(compactSegments);
Assert.assertEquals(
0,
stats.getGlobalStat(CompactSegments.COMPACTION_TASK_COUNT)
stats.get(Stats.Compaction.SUBMITTED_TASKS)
);
for (int i = 0; i < 3; i++) {
verifySnapshot(
@ -544,7 +538,8 @@ public class CompactSegmentsTest
}
// Deactivate one datasource (datasource 0 no longer exist in timeline)
dataSources.remove(DATA_SOURCE_PREFIX + 0);
dataSources.getUsedSegmentsTimelinesPerDataSource()
.remove(DATA_SOURCE_PREFIX + 0);
// Test run auto compaction with one datasource deactivated
// Snapshot should not contain deactivated datasource
@ -599,10 +594,7 @@ public class CompactSegmentsTest
}
}
dataSources = DataSourcesSnapshot
.fromUsedSegments(segments, ImmutableMap.of())
.getUsedSegmentsTimelinesPerDataSource();
dataSources = DataSourcesSnapshot.fromUsedSegments(segments, ImmutableMap.of());
final TestDruidLeaderClient leaderClient = new TestDruidLeaderClient(JSON_MAPPER);
leaderClient.start();
@ -616,10 +608,10 @@ public class CompactSegmentsTest
// 3 intervals, 1200 byte (each segment is 100 bytes), 12 segments will be skipped by auto compaction
for (int compactionRunCount = 0; compactionRunCount < 8; compactionRunCount++) {
// Do a cycle of auto compaction which creates one compaction task
final CoordinatorStats stats = doCompactSegments(compactSegments);
final CoordinatorRunStats stats = doCompactSegments(compactSegments);
Assert.assertEquals(
1,
stats.getGlobalStat(CompactSegments.COMPACTION_TASK_COUNT)
stats.get(Stats.Compaction.SUBMITTED_TASKS)
);
verifySnapshot(
@ -640,10 +632,10 @@ public class CompactSegmentsTest
}
// Test that stats does not change (and is still correct) when auto compaction runs with everything is fully compacted
final CoordinatorStats stats = doCompactSegments(compactSegments);
final CoordinatorRunStats stats = doCompactSegments(compactSegments);
Assert.assertEquals(
0,
stats.getGlobalStat(CompactSegments.COMPACTION_TASK_COUNT)
stats.get(Stats.Compaction.SUBMITTED_TASKS)
);
verifySnapshot(
compactSegments,
@ -670,10 +662,10 @@ public class CompactSegmentsTest
final HttpIndexingServiceClient indexingServiceClient = new HttpIndexingServiceClient(JSON_MAPPER, leaderClient);
final CompactSegments compactSegments = new CompactSegments(COORDINATOR_CONFIG, SEARCH_POLICY, indexingServiceClient);
final CoordinatorStats stats = doCompactSegments(compactSegments, 3);
Assert.assertEquals(3, stats.getGlobalStat(CompactSegments.AVAILABLE_COMPACTION_TASK_SLOT));
Assert.assertEquals(3, stats.getGlobalStat(CompactSegments.MAX_COMPACTION_TASK_SLOT));
Assert.assertEquals(3, stats.getGlobalStat(CompactSegments.COMPACTION_TASK_COUNT));
final CoordinatorRunStats stats = doCompactSegments(compactSegments, 3);
Assert.assertEquals(3, stats.get(Stats.Compaction.AVAILABLE_SLOTS));
Assert.assertEquals(3, stats.get(Stats.Compaction.MAX_SLOTS));
Assert.assertEquals(3, stats.get(Stats.Compaction.SUBMITTED_TASKS));
}
@Test
@ -685,10 +677,10 @@ public class CompactSegmentsTest
leaderClient.start();
final HttpIndexingServiceClient indexingServiceClient = new HttpIndexingServiceClient(JSON_MAPPER, leaderClient);
final CompactSegments compactSegments = new CompactSegments(COORDINATOR_CONFIG, SEARCH_POLICY, indexingServiceClient);
final CoordinatorStats stats = doCompactSegments(compactSegments, createCompactionConfigs(), maxCompactionSlot, true);
Assert.assertEquals(maxCompactionSlot, stats.getGlobalStat(CompactSegments.AVAILABLE_COMPACTION_TASK_SLOT));
Assert.assertEquals(maxCompactionSlot, stats.getGlobalStat(CompactSegments.MAX_COMPACTION_TASK_SLOT));
Assert.assertEquals(maxCompactionSlot, stats.getGlobalStat(CompactSegments.COMPACTION_TASK_COUNT));
final CoordinatorRunStats stats = doCompactSegments(compactSegments, createCompactionConfigs(), maxCompactionSlot, true);
Assert.assertEquals(maxCompactionSlot, stats.get(Stats.Compaction.AVAILABLE_SLOTS));
Assert.assertEquals(maxCompactionSlot, stats.get(Stats.Compaction.MAX_SLOTS));
Assert.assertEquals(maxCompactionSlot, stats.get(Stats.Compaction.SUBMITTED_TASKS));
}
@Test
@ -700,10 +692,10 @@ public class CompactSegmentsTest
leaderClient.start();
final HttpIndexingServiceClient indexingServiceClient = new HttpIndexingServiceClient(JSON_MAPPER, leaderClient);
final CompactSegments compactSegments = new CompactSegments(COORDINATOR_CONFIG, SEARCH_POLICY, indexingServiceClient);
final CoordinatorStats stats = doCompactSegments(compactSegments, createCompactionConfigs(), maxCompactionSlot, true);
Assert.assertEquals(MAXIMUM_CAPACITY_WITH_AUTO_SCALE, stats.getGlobalStat(CompactSegments.AVAILABLE_COMPACTION_TASK_SLOT));
Assert.assertEquals(MAXIMUM_CAPACITY_WITH_AUTO_SCALE, stats.getGlobalStat(CompactSegments.MAX_COMPACTION_TASK_SLOT));
Assert.assertEquals(MAXIMUM_CAPACITY_WITH_AUTO_SCALE, stats.getGlobalStat(CompactSegments.COMPACTION_TASK_COUNT));
final CoordinatorRunStats stats = doCompactSegments(compactSegments, createCompactionConfigs(), maxCompactionSlot, true);
Assert.assertEquals(MAXIMUM_CAPACITY_WITH_AUTO_SCALE, stats.get(Stats.Compaction.AVAILABLE_SLOTS));
Assert.assertEquals(MAXIMUM_CAPACITY_WITH_AUTO_SCALE, stats.get(Stats.Compaction.MAX_SLOTS));
Assert.assertEquals(MAXIMUM_CAPACITY_WITH_AUTO_SCALE, stats.get(Stats.Compaction.SUBMITTED_TASKS));
}
@Test
@ -1269,10 +1261,10 @@ public class CompactSegmentsTest
final HttpIndexingServiceClient indexingServiceClient = new HttpIndexingServiceClient(JSON_MAPPER, leaderClient);
final CompactSegments compactSegments = new CompactSegments(COORDINATOR_CONFIG, SEARCH_POLICY, indexingServiceClient);
final CoordinatorStats stats = doCompactSegments(compactSegments, createCompactionConfigs(2), 4);
Assert.assertEquals(4, stats.getGlobalStat(CompactSegments.AVAILABLE_COMPACTION_TASK_SLOT));
Assert.assertEquals(4, stats.getGlobalStat(CompactSegments.MAX_COMPACTION_TASK_SLOT));
Assert.assertEquals(2, stats.getGlobalStat(CompactSegments.COMPACTION_TASK_COUNT));
final CoordinatorRunStats stats = doCompactSegments(compactSegments, createCompactionConfigs(2), 4);
Assert.assertEquals(4, stats.get(Stats.Compaction.AVAILABLE_SLOTS));
Assert.assertEquals(4, stats.get(Stats.Compaction.MAX_SLOTS));
Assert.assertEquals(2, stats.get(Stats.Compaction.SUBMITTED_TASKS));
}
@Test
@ -1302,8 +1294,8 @@ public class CompactSegmentsTest
// Verify that locked intervals are skipped and only one compaction task
// is submitted for dataSource_0
CompactSegments compactSegments = new CompactSegments(COORDINATOR_CONFIG, SEARCH_POLICY, indexingServiceClient);
final CoordinatorStats stats = doCompactSegments(compactSegments, createCompactionConfigs(2), 4);
Assert.assertEquals(1, stats.getGlobalStat(CompactSegments.COMPACTION_TASK_COUNT));
final CoordinatorRunStats stats = doCompactSegments(compactSegments, createCompactionConfigs(2), 4);
Assert.assertEquals(1, stats.get(Stats.Compaction.SUBMITTED_TASKS));
Assert.assertEquals(1, leaderClient.submittedCompactionTasks.size());
final ClientCompactionTaskQuery compactionTask = leaderClient.submittedCompactionTasks.get(0);
@ -1535,8 +1527,8 @@ public class CompactSegmentsTest
// Verify that no locked intervals are skipped
CompactSegments compactSegments = new CompactSegments(COORDINATOR_CONFIG, SEARCH_POLICY, indexingServiceClient);
int maxTaskSlots = partitionsSpec instanceof SingleDimensionPartitionsSpec ? 5 : 3;
final CoordinatorStats stats = doCompactSegments(compactSegments, createCompactionConfigs(1), maxTaskSlots);
Assert.assertEquals(3, stats.getGlobalStat(CompactSegments.COMPACTION_TASK_COUNT));
final CoordinatorRunStats stats = doCompactSegments(compactSegments, createCompactionConfigs(1), maxTaskSlots);
Assert.assertEquals(3, stats.get(Stats.Compaction.SUBMITTED_TASKS));
Assert.assertEquals(3, leaderClient.submittedCompactionTasks.size());
leaderClient.submittedCompactionTasks.forEach(task -> {
System.out.println(task.getDataSource() + " : " + task.getIoConfig().getInputSpec().getInterval());
@ -1592,10 +1584,7 @@ public class CompactSegmentsTest
10L
)
);
dataSources = DataSourcesSnapshot
.fromUsedSegments(segments, ImmutableMap.of())
.getUsedSegmentsTimelinesPerDataSource();
dataSources = DataSourcesSnapshot.fromUsedSegments(segments, ImmutableMap.of());
final HttpIndexingServiceClient mockIndexingServiceClient = Mockito.mock(HttpIndexingServiceClient.class);
final CompactSegments compactSegments = new CompactSegments(COORDINATOR_CONFIG, SEARCH_POLICY, mockIndexingServiceClient);
@ -1690,10 +1679,7 @@ public class CompactSegmentsTest
10L
)
);
dataSources = DataSourcesSnapshot
.fromUsedSegments(segments, ImmutableMap.of())
.getUsedSegmentsTimelinesPerDataSource();
dataSources = DataSourcesSnapshot.fromUsedSegments(segments, ImmutableMap.of());
final HttpIndexingServiceClient mockIndexingServiceClient = Mockito.mock(HttpIndexingServiceClient.class);
final CompactSegments compactSegments = new CompactSegments(COORDINATOR_CONFIG, SEARCH_POLICY, mockIndexingServiceClient);
@ -1917,10 +1903,10 @@ public class CompactSegmentsTest
{
for (int dataSourceIndex = 0; dataSourceIndex < 3; dataSourceIndex++) {
// One compaction task triggered
final CoordinatorStats stats = doCompactSegments(compactSegments);
final CoordinatorRunStats stats = doCompactSegments(compactSegments);
Assert.assertEquals(
1,
stats.getGlobalStat(CompactSegments.COMPACTION_TASK_COUNT)
stats.get(Stats.Compaction.SUBMITTED_TASKS)
);
// Note: Subsequent compaction run after the dataSource was compacted will show different numbers than
// on the run it was compacted. For example, in a compaction run, if a dataSource had 4 segments compacted,
@ -1934,14 +1920,14 @@ public class CompactSegmentsTest
compactSegments,
AutoCompactionSnapshot.AutoCompactionScheduleStatus.RUNNING,
DATA_SOURCE_PREFIX + i,
TOTAL_BYTE_PER_DATASOURCE - 40 * (compactionRunCount + 1),
40 * (compactionRunCount + 1),
TOTAL_BYTE_PER_DATASOURCE - 40L * (compactionRunCount + 1),
40L * (compactionRunCount + 1),
40,
TOTAL_INTERVAL_PER_DATASOURCE - (compactionRunCount + 1),
(compactionRunCount + 1),
1,
TOTAL_SEGMENT_PER_DATASOURCE - 4 * (compactionRunCount + 1),
2 * (compactionRunCount + 1),
TOTAL_SEGMENT_PER_DATASOURCE - 4L * (compactionRunCount + 1),
2L * (compactionRunCount + 1),
4
);
} else {
@ -1949,14 +1935,14 @@ public class CompactSegmentsTest
compactSegments,
AutoCompactionSnapshot.AutoCompactionScheduleStatus.RUNNING,
DATA_SOURCE_PREFIX + i,
TOTAL_BYTE_PER_DATASOURCE - 40 * (compactionRunCount + 1),
40 * (compactionRunCount + 1),
TOTAL_BYTE_PER_DATASOURCE - 40L * (compactionRunCount + 1),
40L * (compactionRunCount + 1),
40,
TOTAL_INTERVAL_PER_DATASOURCE - (compactionRunCount + 1),
(compactionRunCount + 1),
1,
TOTAL_SEGMENT_PER_DATASOURCE - 4 * (compactionRunCount + 1),
2 * compactionRunCount + 4,
TOTAL_SEGMENT_PER_DATASOURCE - 4L * (compactionRunCount + 1),
2L * compactionRunCount + 4,
4
);
}
@ -1968,39 +1954,39 @@ public class CompactSegmentsTest
compactSegments,
AutoCompactionSnapshot.AutoCompactionScheduleStatus.RUNNING,
DATA_SOURCE_PREFIX + i,
TOTAL_BYTE_PER_DATASOURCE - 40 * compactionRunCount,
40 * compactionRunCount,
TOTAL_BYTE_PER_DATASOURCE - 40L * compactionRunCount,
40L * compactionRunCount,
40,
TOTAL_INTERVAL_PER_DATASOURCE - compactionRunCount,
compactionRunCount,
1,
TOTAL_SEGMENT_PER_DATASOURCE - 4 * compactionRunCount,
2 * compactionRunCount,
TOTAL_SEGMENT_PER_DATASOURCE - 4L * compactionRunCount,
2L * compactionRunCount,
4
);
}
}
}
private CoordinatorStats doCompactSegments(CompactSegments compactSegments)
private CoordinatorRunStats doCompactSegments(CompactSegments compactSegments)
{
return doCompactSegments(compactSegments, (Integer) null);
}
private CoordinatorStats doCompactSegments(CompactSegments compactSegments, @Nullable Integer numCompactionTaskSlots)
private CoordinatorRunStats doCompactSegments(CompactSegments compactSegments, @Nullable Integer numCompactionTaskSlots)
{
return doCompactSegments(compactSegments, createCompactionConfigs(), numCompactionTaskSlots);
}
private CoordinatorStats doCompactSegments(
private void doCompactSegments(
CompactSegments compactSegments,
List<DataSourceCompactionConfig> compactionConfigs
)
{
return doCompactSegments(compactSegments, compactionConfigs, null);
doCompactSegments(compactSegments, compactionConfigs, null);
}
private CoordinatorStats doCompactSegments(
private CoordinatorRunStats doCompactSegments(
CompactSegments compactSegments,
List<DataSourceCompactionConfig> compactionConfigs,
@Nullable Integer numCompactionTaskSlots
@ -2009,20 +1995,20 @@ public class CompactSegmentsTest
return doCompactSegments(compactSegments, compactionConfigs, numCompactionTaskSlots, false);
}
private CoordinatorStats doCompactSegments(
private CoordinatorRunStats doCompactSegments(
CompactSegments compactSegments,
List<DataSourceCompactionConfig> compactionConfigs,
@Nullable Integer numCompactionTaskSlots,
boolean useAutoScaleSlots
)
{
DruidCoordinatorRuntimeParams params = CoordinatorRuntimeParamsTestHelpers
.newBuilder()
.withUsedSegmentsTimelinesPerDataSourceInTest(dataSources)
DruidCoordinatorRuntimeParams params = DruidCoordinatorRuntimeParams
.newBuilder(DateTimes.nowUtc())
.withSnapshotOfDataSourcesWithAllUsedSegments(dataSources)
.withCompactionConfig(
new CoordinatorCompactionConfig(
compactionConfigs,
numCompactionTaskSlots == null ? null : 1., // 100% when numCompactionTaskSlots is not null
numCompactionTaskSlots == null ? null : 1.0, // 100% when numCompactionTaskSlots is not null
numCompactionTaskSlots,
useAutoScaleSlots
)
@ -2040,35 +2026,37 @@ public class CompactSegmentsTest
)
{
for (int i = 0; i < 3; i++) {
final CoordinatorStats stats = doCompactSegments(compactSegments);
final CoordinatorRunStats stats = doCompactSegments(compactSegments);
Assert.assertEquals(
expectedCompactTaskCount,
stats.getGlobalStat(CompactSegments.COMPACTION_TASK_COUNT)
stats.get(Stats.Compaction.SUBMITTED_TASKS)
);
// If expectedRemainingSegments is positive, we count the number of datasources
// which have that many segments waiting for compaction. Otherwise, we count
// all the datasources in the coordinator stats
final AtomicInteger numDatasources = new AtomicInteger();
stats.forEachStat(
(dimensionValues, stat, value) -> {
if (stat.equals(Stats.Compaction.PENDING_BYTES)
&& (expectedRemainingSegments <= 0 || value == expectedRemainingSegments)) {
numDatasources.incrementAndGet();
}
}
);
// One of dataSource is compacted
if (expectedRemainingSegments > 0) {
// If expectedRemainingSegments is positive, we check how many dataSources have the segments waiting for
// compaction.
long numDataSourceOfExpectedRemainingSegments = stats
.getDataSources(CompactSegments.TOTAL_SIZE_OF_SEGMENTS_AWAITING)
.stream()
.mapToLong(ds -> stats.getDataSourceStat(CompactSegments.TOTAL_SIZE_OF_SEGMENTS_AWAITING, ds))
.filter(stat -> stat == expectedRemainingSegments)
.count();
Assert.assertEquals(i + 1, numDataSourceOfExpectedRemainingSegments);
Assert.assertEquals(i + 1, numDatasources.get());
} else {
// Otherwise, we check how many dataSources are in the coordinator stats.
Assert.assertEquals(
2 - i,
stats.getDataSources(CompactSegments.TOTAL_SIZE_OF_SEGMENTS_AWAITING).size()
);
Assert.assertEquals(2 - i, numDatasources.get());
}
}
final Map<String, SegmentTimeline> dataSourceToTimeline
= dataSources.getUsedSegmentsTimelinesPerDataSource();
for (int i = 0; i < 3; i++) {
final String dataSource = DATA_SOURCE_PREFIX + i;
List<TimelineObjectHolder<String, DataSegment>> holders = dataSources.get(dataSource).lookup(expectedInterval);
List<TimelineObjectHolder<String, DataSegment>> holders = dataSourceToTimeline.get(dataSource).lookup(expectedInterval);
Assert.assertEquals(1, holders.size());
List<PartitionChunk<DataSegment>> chunks = Lists.newArrayList(holders.get(0).getObject());
Assert.assertEquals(2, chunks.size());
@ -2083,10 +2071,12 @@ public class CompactSegmentsTest
private void assertLastSegmentNotCompacted(CompactSegments compactSegments)
{
// Segments of the latest interval should not be compacted
final Map<String, SegmentTimeline> dataSourceToTimeline
= dataSources.getUsedSegmentsTimelinesPerDataSource();
for (int i = 0; i < 3; i++) {
final String dataSource = DATA_SOURCE_PREFIX + i;
final Interval interval = Intervals.of(StringUtils.format("2017-01-09T12:00:00/2017-01-10"));
List<TimelineObjectHolder<String, DataSegment>> holders = dataSources.get(dataSource).lookup(interval);
List<TimelineObjectHolder<String, DataSegment>> holders = dataSourceToTimeline.get(dataSource).lookup(interval);
Assert.assertEquals(1, holders.size());
for (TimelineObjectHolder<String, DataSegment> holder : holders) {
List<PartitionChunk<DataSegment>> chunks = Lists.newArrayList(holder.getObject());
@ -2103,10 +2093,10 @@ public class CompactSegmentsTest
final String dataSource = DATA_SOURCE_PREFIX + 0;
addMoreData(dataSource, 9);
CoordinatorStats stats = doCompactSegments(compactSegments);
CoordinatorRunStats stats = doCompactSegments(compactSegments);
Assert.assertEquals(
1,
stats.getGlobalStat(CompactSegments.COMPACTION_TASK_COUNT)
stats.get(Stats.Compaction.SUBMITTED_TASKS)
);
addMoreData(dataSource, 10);
@ -2114,21 +2104,23 @@ public class CompactSegmentsTest
stats = doCompactSegments(compactSegments);
Assert.assertEquals(
1,
stats.getGlobalStat(CompactSegments.COMPACTION_TASK_COUNT)
stats.get(Stats.Compaction.SUBMITTED_TASKS)
);
}
private void addMoreData(String dataSource, int day)
{
final SegmentTimeline timeline
= dataSources.getUsedSegmentsTimelinesPerDataSource().get(dataSource);
for (int i = 0; i < 2; i++) {
DataSegment newSegment = createSegment(dataSource, day, true, i);
dataSources.get(dataSource).add(
timeline.add(
newSegment.getInterval(),
newSegment.getVersion(),
newSegment.getShardSpec().createChunk(newSegment)
);
newSegment = createSegment(dataSource, day, false, i);
dataSources.get(dataSource).add(
timeline.add(
newSegment.getInterval(),
newSegment.getVersion(),
newSegment.getShardSpec().createChunk(newSegment)
@ -2277,19 +2269,15 @@ public class CompactSegmentsTest
submittedCompactionTasks.add(compactionTaskQuery);
final Interval intervalToCompact = compactionTaskQuery.getIoConfig().getInputSpec().getInterval();
final VersionedIntervalTimeline<String, DataSegment> timeline = dataSources.get(
compactionTaskQuery.getDataSource()
);
final SegmentTimeline timeline = dataSources.getUsedSegmentsTimelinesPerDataSource()
.get(compactionTaskQuery.getDataSource());
final List<DataSegment> segments = timeline.lookup(intervalToCompact)
.stream()
.flatMap(holder -> Streams.sequentialStreamFrom(holder.getObject()))
.map(PartitionChunk::getObject)
.collect(Collectors.toList());
compactSegments(
timeline,
segments,
compactionTaskQuery
);
compactSegments(timeline, segments, compactionTaskQuery);
return createStringFullResponseHolder(jsonMapper.writeValueAsString(ImmutableMap.of("task", taskQuery.getId())));
}

Some files were not shown because too many files have changed in this diff Show More