mirror of https://github.com/apache/druid.git
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:
parent
cfd07a95b7
commit
50461c3bd5
|
@ -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());
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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());
|
||||
}
|
||||
|
||||
|
|
|
@ -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());
|
||||
}
|
||||
|
||||
|
|
|
@ -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";
|
||||
|
|
|
@ -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) -> {
|
||||
|
|
|
@ -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);
|
||||
}
|
|
@ -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())
|
||||
);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
|
File diff suppressed because it is too large
Load Diff
|
@ -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;
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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() + "}";
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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--;
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
|
@ -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
|
||||
{
|
|
@ -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);
|
|
@ -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;
|
||||
|
|
@ -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;
|
|
@ -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;
|
|
@ -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,
|
|
@ -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;
|
||||
|
|
@ -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
|
|
@ -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;
|
||||
|
|
@ -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)
|
||||
{
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
|
|
@ -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;
|
||||
}
|
|
@ -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;
|
||||
|
|
@ -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;
|
|
@ -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);
|
||||
}
|
||||
|
||||
}
|
|
@ -18,6 +18,6 @@
|
|||
*/
|
||||
|
||||
@ParametersAreNonnullByDefault
|
||||
package org.apache.druid.server.coordinator.cost;
|
||||
package org.apache.druid.server.coordinator.balancer;
|
||||
|
||||
import javax.annotation.ParametersAreNonnullByDefault;
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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
|
||||
);
|
||||
}
|
||||
|
||||
}
|
|
@ -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
|
||||
|
|
|
@ -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)
|
||||
);
|
||||
}
|
||||
);
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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());
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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()
|
||||
{
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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));
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
|
@ -17,7 +17,7 @@
|
|||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.server.coordinator;
|
||||
package org.apache.druid.server.coordinator.loading;
|
||||
|
||||
/**
|
||||
*/
|
|
@ -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);
|
||||
|
||||
}
|
|
@ -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;
|
|
@ -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);
|
||||
}
|
||||
|
||||
}
|
|
@ -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;
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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 +
|
||||
'}';
|
||||
}
|
||||
}
|
|
@ -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());
|
||||
}
|
||||
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
|
||||
}
|
|
@ -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
|
||||
}
|
||||
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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();
|
||||
}
|
||||
}
|
|
@ -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");
|
||||
}
|
||||
}
|
|
@ -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();
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
|
|
|
@ -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.")
|
||||
|
|
|
@ -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()));
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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())) {
|
||||
|
|
|
@ -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());
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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();
|
||||
}
|
||||
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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
|
||||
|
|
|
@ -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 + "}";
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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()
|
||||
);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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());
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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.");
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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()
|
||||
);
|
||||
}
|
||||
|
|
File diff suppressed because it is too large
Load Diff
|
@ -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()));
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
)
|
|
@ -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(
|
|
@ -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());
|
||||
}
|
|
@ -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) {
|
|
@ -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;
|
|
@ -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
|
||||
);
|
||||
}
|
||||
}
|
|
@ -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));
|
||||
}
|
||||
|
||||
}
|
|
@ -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
Loading…
Reference in New Issue