mirror of https://github.com/apache/druid.git
maintenance mode for Historical (#6349)
* maintenance mode for Historical forbidden api fix, config deserialization fix logging fix, unit tests * addressed comments * addressed comments * a style fix * addressed comments * a unit-test fix due to recent code-refactoring * docs & refactoring * addressed comments * addressed a LoadRule drop flaw * post merge cleaning up
This commit is contained in:
parent
7e48593b57
commit
97b6407983
|
@ -779,7 +779,9 @@ A sample Coordinator dynamic config JSON object is shown below:
|
|||
"replicantLifetime": 15,
|
||||
"replicationThrottleLimit": 10,
|
||||
"emitBalancingStats": false,
|
||||
"killDataSourceWhitelist": ["wikipedia", "testDatasource"]
|
||||
"killDataSourceWhitelist": ["wikipedia", "testDatasource"],
|
||||
"historicalNodesInMaintenance": ["localhost:8182", "localhost:8282"],
|
||||
"nodesInMaintenancePriority": 7
|
||||
}
|
||||
```
|
||||
|
||||
|
@ -799,6 +801,8 @@ Issuing a GET request at the same URL will return the spec that is currently in
|
|||
|`killAllDataSources`|Send kill tasks for ALL dataSources if property `druid.coordinator.kill.on` is true. If this is set to true then `killDataSourceWhitelist` must not be specified or be empty list.|false|
|
||||
|`killPendingSegmentsSkipList`|List of dataSources for which pendingSegments are _NOT_ cleaned up if property `druid.coordinator.kill.pendingSegments.on` is true. This can be a list of comma-separated dataSources or a JSON array.|none|
|
||||
|`maxSegmentsInNodeLoadingQueue`|The maximum number of segments that could be queued for loading to any given server. This parameter could be used to speed up segments loading process, especially if there are "slow" nodes in the cluster (with low loading speed) or if too much segments scheduled to be replicated to some particular node (faster loading could be preferred to better segments distribution). Desired value depends on segments loading speed, acceptable replication time and number of nodes. Value 1000 could be a start point for a rather big cluster. Default value is 0 (loading queue is unbounded) |0|
|
||||
|`historicalNodesInMaintenance`| List of Historical nodes in maintenance mode. Coordinator doesn't assign new segments on those nodes and moves segments from the nodes according to a specified priority.|none|
|
||||
|`nodesInMaintenancePriority`| Priority of segments from servers in maintenance. Coordinator takes ceil(maxSegmentsToMove * (priority / 10)) from servers in maitenance during balancing phase, i.e.:<br>0 - no segments from servers in maintenance will be processed during balancing<br>5 - 50% segments from servers in maintenance<br>10 - 100% segments from servers in maintenance<br>By leveraging the priority an operator can prevent general nodes from overload or decrease maitenance time instead.|7|
|
||||
|
||||
To view the audit history of Coordinator dynamic config issue a GET request to the URL -
|
||||
|
||||
|
|
|
@ -191,6 +191,12 @@
|
|||
<artifactId>junit</artifactId>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.hamcrest</groupId>
|
||||
<artifactId>hamcrest-all</artifactId>
|
||||
<version>1.3</version>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.carrotsearch</groupId>
|
||||
<artifactId>junit-benchmarks</artifactId>
|
||||
|
|
|
@ -55,10 +55,12 @@ public class CoordinatorDynamicConfig
|
|||
private final int balancerComputeThreads;
|
||||
private final boolean emitBalancingStats;
|
||||
private final boolean killAllDataSources;
|
||||
private final Set<String> killDataSourceWhitelist;
|
||||
private final Set<String> killableDataSources;
|
||||
private final Set<String> historicalNodesInMaintenance;
|
||||
private final int nodesInMaintenancePriority;
|
||||
|
||||
// The pending segments of the dataSources in this list are not killed.
|
||||
private final Set<String> killPendingSegmentsSkipList;
|
||||
private final Set<String> protectedPendingSegmentDatasources;
|
||||
|
||||
/**
|
||||
* The maximum number of segments that could be queued for loading to any given server.
|
||||
|
@ -82,10 +84,12 @@ public class CoordinatorDynamicConfig
|
|||
// Type is Object here so that we can support both string and list as
|
||||
// coordinator console can not send array of strings in the update request.
|
||||
// See https://github.com/apache/incubator-druid/issues/3055
|
||||
@JsonProperty("killDataSourceWhitelist") Object killDataSourceWhitelist,
|
||||
@JsonProperty("killDataSourceWhitelist") Object killableDataSources,
|
||||
@JsonProperty("killAllDataSources") boolean killAllDataSources,
|
||||
@JsonProperty("killPendingSegmentsSkipList") Object killPendingSegmentsSkipList,
|
||||
@JsonProperty("maxSegmentsInNodeLoadingQueue") int maxSegmentsInNodeLoadingQueue
|
||||
@JsonProperty("killPendingSegmentsSkipList") Object protectedPendingSegmentDatasources,
|
||||
@JsonProperty("maxSegmentsInNodeLoadingQueue") int maxSegmentsInNodeLoadingQueue,
|
||||
@JsonProperty("historicalNodesInMaintenance") Object historicalNodesInMaintenance,
|
||||
@JsonProperty("nodesInMaintenancePriority") int nodesInMaintenancePriority
|
||||
)
|
||||
{
|
||||
this.millisToWaitBeforeDeleting = millisToWaitBeforeDeleting;
|
||||
|
@ -97,11 +101,17 @@ public class CoordinatorDynamicConfig
|
|||
this.balancerComputeThreads = Math.max(balancerComputeThreads, 1);
|
||||
this.emitBalancingStats = emitBalancingStats;
|
||||
this.killAllDataSources = killAllDataSources;
|
||||
this.killDataSourceWhitelist = parseJsonStringOrArray(killDataSourceWhitelist);
|
||||
this.killPendingSegmentsSkipList = parseJsonStringOrArray(killPendingSegmentsSkipList);
|
||||
this.killableDataSources = parseJsonStringOrArray(killableDataSources);
|
||||
this.protectedPendingSegmentDatasources = parseJsonStringOrArray(protectedPendingSegmentDatasources);
|
||||
this.maxSegmentsInNodeLoadingQueue = maxSegmentsInNodeLoadingQueue;
|
||||
this.historicalNodesInMaintenance = parseJsonStringOrArray(historicalNodesInMaintenance);
|
||||
Preconditions.checkArgument(
|
||||
nodesInMaintenancePriority >= 0 && nodesInMaintenancePriority <= 10,
|
||||
"nodesInMaintenancePriority should be in range [0, 10]"
|
||||
);
|
||||
this.nodesInMaintenancePriority = nodesInMaintenancePriority;
|
||||
|
||||
if (this.killAllDataSources && !this.killDataSourceWhitelist.isEmpty()) {
|
||||
if (this.killAllDataSources && !this.killableDataSources.isEmpty()) {
|
||||
throw new IAE("can't have killAllDataSources and non-empty killDataSourceWhitelist");
|
||||
}
|
||||
}
|
||||
|
@ -188,10 +198,14 @@ public class CoordinatorDynamicConfig
|
|||
return balancerComputeThreads;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public Set<String> getKillDataSourceWhitelist()
|
||||
/**
|
||||
* List of dataSources for which kill tasks are sent in
|
||||
* {@link org.apache.druid.server.coordinator.helper.DruidCoordinatorSegmentKiller}.
|
||||
*/
|
||||
@JsonProperty("killDataSourceWhitelist")
|
||||
public Set<String> getKillableDataSources()
|
||||
{
|
||||
return killDataSourceWhitelist;
|
||||
return killableDataSources;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
|
@ -200,10 +214,14 @@ public class CoordinatorDynamicConfig
|
|||
return killAllDataSources;
|
||||
}
|
||||
|
||||
/**
|
||||
* List of dataSources for which pendingSegments are NOT cleaned up
|
||||
* in {@link DruidCoordinatorCleanupPendingSegments}.
|
||||
*/
|
||||
@JsonProperty
|
||||
public Set<String> getKillPendingSegmentsSkipList()
|
||||
public Set<String> getProtectedPendingSegmentDatasources()
|
||||
{
|
||||
return killPendingSegmentsSkipList;
|
||||
return protectedPendingSegmentDatasources;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
|
@ -212,6 +230,35 @@ public class CoordinatorDynamicConfig
|
|||
return maxSegmentsInNodeLoadingQueue;
|
||||
}
|
||||
|
||||
/**
|
||||
* Historical nodes list in maintenance mode. Coordinator doesn't assign new segments on those nodes and moves
|
||||
* segments from those nodes according to a specified priority.
|
||||
*
|
||||
* @return list of host:port entries
|
||||
*/
|
||||
@JsonProperty
|
||||
public Set<String> getHistoricalNodesInMaintenance()
|
||||
{
|
||||
return historicalNodesInMaintenance;
|
||||
}
|
||||
|
||||
/**
|
||||
* Priority of segments from servers in maintenance. Coordinator takes ceil(maxSegmentsToMove * (priority / 10))
|
||||
* from servers in maitenance during balancing phase, i.e.:
|
||||
* 0 - no segments from servers in maintenance will be processed during balancing
|
||||
* 5 - 50% segments from servers in maintenance
|
||||
* 10 - 100% segments from servers in maintenance
|
||||
* By leveraging the priority an operator can prevent general nodes from overload or decrease maitenance time
|
||||
* instead.
|
||||
*
|
||||
* @return number in range [0, 10]
|
||||
*/
|
||||
@JsonProperty
|
||||
public int getNodesInMaintenancePriority()
|
||||
{
|
||||
return nodesInMaintenancePriority;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString()
|
||||
{
|
||||
|
@ -224,10 +271,12 @@ public class CoordinatorDynamicConfig
|
|||
", replicationThrottleLimit=" + replicationThrottleLimit +
|
||||
", balancerComputeThreads=" + balancerComputeThreads +
|
||||
", emitBalancingStats=" + emitBalancingStats +
|
||||
", killDataSourceWhitelist=" + killDataSourceWhitelist +
|
||||
", killAllDataSources=" + killAllDataSources +
|
||||
", killPendingSegmentsSkipList=" + killPendingSegmentsSkipList +
|
||||
", killDataSourceWhitelist=" + killableDataSources +
|
||||
", protectedPendingSegmentDatasources=" + protectedPendingSegmentDatasources +
|
||||
", maxSegmentsInNodeLoadingQueue=" + maxSegmentsInNodeLoadingQueue +
|
||||
", historicalNodesInMaintenance=" + historicalNodesInMaintenance +
|
||||
", nodesInMaintenancePriority=" + nodesInMaintenancePriority +
|
||||
'}';
|
||||
}
|
||||
|
||||
|
@ -273,10 +322,16 @@ public class CoordinatorDynamicConfig
|
|||
if (maxSegmentsInNodeLoadingQueue != that.maxSegmentsInNodeLoadingQueue) {
|
||||
return false;
|
||||
}
|
||||
if (!Objects.equals(killDataSourceWhitelist, that.killDataSourceWhitelist)) {
|
||||
if (!Objects.equals(killableDataSources, that.killableDataSources)) {
|
||||
return false;
|
||||
}
|
||||
return Objects.equals(killPendingSegmentsSkipList, that.killPendingSegmentsSkipList);
|
||||
if (!Objects.equals(protectedPendingSegmentDatasources, that.protectedPendingSegmentDatasources)) {
|
||||
return false;
|
||||
}
|
||||
if (!Objects.equals(historicalNodesInMaintenance, that.historicalNodesInMaintenance)) {
|
||||
return false;
|
||||
}
|
||||
return nodesInMaintenancePriority == that.nodesInMaintenancePriority;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -293,8 +348,10 @@ public class CoordinatorDynamicConfig
|
|||
emitBalancingStats,
|
||||
killAllDataSources,
|
||||
maxSegmentsInNodeLoadingQueue,
|
||||
killDataSourceWhitelist,
|
||||
killPendingSegmentsSkipList
|
||||
killableDataSources,
|
||||
protectedPendingSegmentDatasources,
|
||||
historicalNodesInMaintenance,
|
||||
nodesInMaintenancePriority
|
||||
);
|
||||
}
|
||||
|
||||
|
@ -315,6 +372,7 @@ public class CoordinatorDynamicConfig
|
|||
private static final boolean DEFAULT_EMIT_BALANCING_STATS = false;
|
||||
private static final boolean DEFAULT_KILL_ALL_DATA_SOURCES = false;
|
||||
private static final int DEFAULT_MAX_SEGMENTS_IN_NODE_LOADING_QUEUE = 0;
|
||||
private static final int DEFAULT_MAINTENANCE_MODE_SEGMENTS_PRIORITY = 7;
|
||||
|
||||
private Long millisToWaitBeforeDeleting;
|
||||
private Long mergeBytesLimit;
|
||||
|
@ -324,10 +382,12 @@ public class CoordinatorDynamicConfig
|
|||
private Integer replicationThrottleLimit;
|
||||
private Boolean emitBalancingStats;
|
||||
private Integer balancerComputeThreads;
|
||||
private Object killDataSourceWhitelist;
|
||||
private Object killableDataSources;
|
||||
private Boolean killAllDataSources;
|
||||
private Object killPendingSegmentsSkipList;
|
||||
private Integer maxSegmentsInNodeLoadingQueue;
|
||||
private Object maintenanceList;
|
||||
private Integer maintenanceModeSegmentsPriority;
|
||||
|
||||
public Builder()
|
||||
{
|
||||
|
@ -343,10 +403,12 @@ public class CoordinatorDynamicConfig
|
|||
@JsonProperty("replicationThrottleLimit") @Nullable Integer replicationThrottleLimit,
|
||||
@JsonProperty("balancerComputeThreads") @Nullable Integer balancerComputeThreads,
|
||||
@JsonProperty("emitBalancingStats") @Nullable Boolean emitBalancingStats,
|
||||
@JsonProperty("killDataSourceWhitelist") @Nullable Object killDataSourceWhitelist,
|
||||
@JsonProperty("killDataSourceWhitelist") @Nullable Object killableDataSources,
|
||||
@JsonProperty("killAllDataSources") @Nullable Boolean killAllDataSources,
|
||||
@JsonProperty("killPendingSegmentsSkipList") @Nullable Object killPendingSegmentsSkipList,
|
||||
@JsonProperty("maxSegmentsInNodeLoadingQueue") @Nullable Integer maxSegmentsInNodeLoadingQueue
|
||||
@JsonProperty("maxSegmentsInNodeLoadingQueue") @Nullable Integer maxSegmentsInNodeLoadingQueue,
|
||||
@JsonProperty("historicalNodesInMaintenance") @Nullable Object maintenanceList,
|
||||
@JsonProperty("nodesInMaintenancePriority") @Nullable Integer maintenanceModeSegmentsPriority
|
||||
)
|
||||
{
|
||||
this.millisToWaitBeforeDeleting = millisToWaitBeforeDeleting;
|
||||
|
@ -358,9 +420,11 @@ public class CoordinatorDynamicConfig
|
|||
this.balancerComputeThreads = balancerComputeThreads;
|
||||
this.emitBalancingStats = emitBalancingStats;
|
||||
this.killAllDataSources = killAllDataSources;
|
||||
this.killDataSourceWhitelist = killDataSourceWhitelist;
|
||||
this.killableDataSources = killableDataSources;
|
||||
this.killPendingSegmentsSkipList = killPendingSegmentsSkipList;
|
||||
this.maxSegmentsInNodeLoadingQueue = maxSegmentsInNodeLoadingQueue;
|
||||
this.maintenanceList = maintenanceList;
|
||||
this.maintenanceModeSegmentsPriority = maintenanceModeSegmentsPriority;
|
||||
}
|
||||
|
||||
public Builder withMillisToWaitBeforeDeleting(long millisToWaitBeforeDeleting)
|
||||
|
@ -413,7 +477,7 @@ public class CoordinatorDynamicConfig
|
|||
|
||||
public Builder withKillDataSourceWhitelist(Set<String> killDataSourceWhitelist)
|
||||
{
|
||||
this.killDataSourceWhitelist = killDataSourceWhitelist;
|
||||
this.killableDataSources = killDataSourceWhitelist;
|
||||
return this;
|
||||
}
|
||||
|
||||
|
@ -429,6 +493,18 @@ public class CoordinatorDynamicConfig
|
|||
return this;
|
||||
}
|
||||
|
||||
public Builder withMaintenanceList(Set<String> list)
|
||||
{
|
||||
this.maintenanceList = list;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withMaintenanceModeSegmentsPriority(Integer priority)
|
||||
{
|
||||
this.maintenanceModeSegmentsPriority = priority;
|
||||
return this;
|
||||
}
|
||||
|
||||
public CoordinatorDynamicConfig build()
|
||||
{
|
||||
return new CoordinatorDynamicConfig(
|
||||
|
@ -440,12 +516,16 @@ public class CoordinatorDynamicConfig
|
|||
replicationThrottleLimit == null ? DEFAULT_REPLICATION_THROTTLE_LIMIT : replicationThrottleLimit,
|
||||
balancerComputeThreads == null ? DEFAULT_BALANCER_COMPUTE_THREADS : balancerComputeThreads,
|
||||
emitBalancingStats == null ? DEFAULT_EMIT_BALANCING_STATS : emitBalancingStats,
|
||||
killDataSourceWhitelist,
|
||||
killableDataSources,
|
||||
killAllDataSources == null ? DEFAULT_KILL_ALL_DATA_SOURCES : killAllDataSources,
|
||||
killPendingSegmentsSkipList,
|
||||
maxSegmentsInNodeLoadingQueue == null
|
||||
? DEFAULT_MAX_SEGMENTS_IN_NODE_LOADING_QUEUE
|
||||
: maxSegmentsInNodeLoadingQueue
|
||||
: maxSegmentsInNodeLoadingQueue,
|
||||
maintenanceList,
|
||||
maintenanceModeSegmentsPriority == null
|
||||
? DEFAULT_MAINTENANCE_MODE_SEGMENTS_PRIORITY
|
||||
: maintenanceModeSegmentsPriority
|
||||
);
|
||||
}
|
||||
|
||||
|
@ -460,12 +540,18 @@ public class CoordinatorDynamicConfig
|
|||
replicationThrottleLimit == null ? defaults.getReplicationThrottleLimit() : replicationThrottleLimit,
|
||||
balancerComputeThreads == null ? defaults.getBalancerComputeThreads() : balancerComputeThreads,
|
||||
emitBalancingStats == null ? defaults.emitBalancingStats() : emitBalancingStats,
|
||||
killDataSourceWhitelist == null ? defaults.getKillDataSourceWhitelist() : killDataSourceWhitelist,
|
||||
killableDataSources == null ? defaults.getKillableDataSources() : killableDataSources,
|
||||
killAllDataSources == null ? defaults.isKillAllDataSources() : killAllDataSources,
|
||||
killPendingSegmentsSkipList == null ? defaults.getKillPendingSegmentsSkipList() : killPendingSegmentsSkipList,
|
||||
killPendingSegmentsSkipList == null
|
||||
? defaults.getProtectedPendingSegmentDatasources()
|
||||
: killPendingSegmentsSkipList,
|
||||
maxSegmentsInNodeLoadingQueue == null
|
||||
? defaults.getMaxSegmentsInNodeLoadingQueue()
|
||||
: maxSegmentsInNodeLoadingQueue
|
||||
: maxSegmentsInNodeLoadingQueue,
|
||||
maintenanceList == null ? defaults.getHistoricalNodesInMaintenance() : maintenanceList,
|
||||
maintenanceModeSegmentsPriority == null
|
||||
? defaults.getNodesInMaintenancePriority()
|
||||
: maintenanceModeSegmentsPriority
|
||||
);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -34,6 +34,8 @@ import java.util.Map;
|
|||
import java.util.NavigableSet;
|
||||
import java.util.Set;
|
||||
import java.util.TreeSet;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.StreamSupport;
|
||||
|
||||
/**
|
||||
* Contains a representation of the current state of the cluster by tier.
|
||||
|
@ -53,11 +55,19 @@ public class DruidCluster
|
|||
@VisibleForTesting
|
||||
public DruidCluster(
|
||||
@Nullable Set<ServerHolder> realtimes,
|
||||
Map<String, NavigableSet<ServerHolder>> historicals
|
||||
Map<String, Iterable<ServerHolder>> historicals
|
||||
)
|
||||
{
|
||||
this.realtimes = realtimes == null ? new HashSet<>() : new HashSet<>(realtimes);
|
||||
this.historicals = historicals;
|
||||
this.historicals = historicals
|
||||
.entrySet()
|
||||
.stream()
|
||||
.collect(Collectors.toMap(
|
||||
Map.Entry::getKey,
|
||||
e -> StreamSupport
|
||||
.stream(e.getValue().spliterator(), false)
|
||||
.collect(Collectors.toCollection(() -> new TreeSet<>(Collections.reverseOrder())))
|
||||
));
|
||||
}
|
||||
|
||||
public void add(ServerHolder serverHolder)
|
||||
|
|
|
@ -694,6 +694,7 @@ public class DruidCoordinator
|
|||
}
|
||||
|
||||
// Find all historical servers, group them by subType and sort by ascending usage
|
||||
Set<String> nodesInMaintenance = params.getCoordinatorDynamicConfig().getHistoricalNodesInMaintenance();
|
||||
final DruidCluster cluster = new DruidCluster();
|
||||
for (ImmutableDruidServer server : servers) {
|
||||
if (!loadManagementPeons.containsKey(server.getName())) {
|
||||
|
@ -704,7 +705,13 @@ public class DruidCoordinator
|
|||
loadManagementPeons.put(server.getName(), loadQueuePeon);
|
||||
}
|
||||
|
||||
cluster.add(new ServerHolder(server, loadManagementPeons.get(server.getName())));
|
||||
cluster.add(
|
||||
new ServerHolder(
|
||||
server,
|
||||
loadManagementPeons.get(server.getName()),
|
||||
nodesInMaintenance.contains(server.getHost())
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
segmentReplicantLookup = SegmentReplicantLookup.make(cluster);
|
||||
|
|
|
@ -89,7 +89,7 @@ public class DruidCoordinatorCleanupPendingSegments implements DruidCoordinatorH
|
|||
// (DateTimes.nowUtc() - KEEP_PENDING_SEGMENTS_OFFSET).
|
||||
final DateTime pendingSegmentsCleanupEndTime = createdTimes.get(0).minus(KEEP_PENDING_SEGMENTS_OFFSET);
|
||||
for (String dataSource : params.getDataSources().keySet()) {
|
||||
if (!params.getCoordinatorDynamicConfig().getKillPendingSegmentsSkipList().contains(dataSource)) {
|
||||
if (!params.getCoordinatorDynamicConfig().getProtectedPendingSegmentDatasources().contains(dataSource)) {
|
||||
log.info(
|
||||
"Killed [%d] pendingSegments created until [%s] for dataSource[%s]",
|
||||
indexingServiceClient.killPendingSegments(dataSource, pendingSegmentsCleanupEndTime),
|
||||
|
|
|
@ -19,6 +19,7 @@
|
|||
|
||||
package org.apache.druid.server.coordinator;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import org.apache.druid.client.ImmutableDruidDataSource;
|
||||
import org.apache.druid.java.util.common.DateTimes;
|
||||
import org.apache.druid.java.util.emitter.service.ServiceEmitter;
|
||||
|
@ -27,6 +28,7 @@ import org.apache.druid.timeline.DataSegment;
|
|||
import org.apache.druid.timeline.VersionedIntervalTimeline;
|
||||
import org.joda.time.DateTime;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
|
@ -344,9 +346,16 @@ public class DruidCoordinatorRuntimeParams
|
|||
return this;
|
||||
}
|
||||
|
||||
public Builder withAvailableSegments(Collection<DataSegment> availableSegmentsCollection)
|
||||
@VisibleForTesting
|
||||
public Builder withAvailableSegments(DataSegment... availableSegments)
|
||||
{
|
||||
availableSegments.addAll(Collections.unmodifiableCollection(availableSegmentsCollection));
|
||||
this.availableSegments.addAll(Arrays.asList(availableSegments));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withAvailableSegments(Collection<DataSegment> availableSegments)
|
||||
{
|
||||
this.availableSegments.addAll(Collections.unmodifiableCollection(availableSegments));
|
||||
return this;
|
||||
}
|
||||
|
||||
|
|
|
@ -32,11 +32,18 @@ public class ServerHolder implements Comparable<ServerHolder>
|
|||
private static final Logger log = new Logger(ServerHolder.class);
|
||||
private final ImmutableDruidServer server;
|
||||
private final LoadQueuePeon peon;
|
||||
private final boolean inMaintenance;
|
||||
|
||||
public ServerHolder(ImmutableDruidServer server, LoadQueuePeon peon)
|
||||
{
|
||||
this(server, peon, false);
|
||||
}
|
||||
|
||||
public ServerHolder(ImmutableDruidServer server, LoadQueuePeon peon, boolean inMaintenance)
|
||||
{
|
||||
this.server = server;
|
||||
this.peon = peon;
|
||||
this.inMaintenance = inMaintenance;
|
||||
}
|
||||
|
||||
public ImmutableDruidServer getServer()
|
||||
|
@ -74,6 +81,17 @@ public class ServerHolder implements Comparable<ServerHolder>
|
|||
return (100.0 * getSizeUsed()) / getMaxSize();
|
||||
}
|
||||
|
||||
/**
|
||||
* Historical nodes can be placed in maintenance mode, which instructs Coordinator to move segments from them
|
||||
* according to a specified priority. The mechanism allows to drain segments from nodes which are planned for
|
||||
* replacement.
|
||||
* @return true if the node is in maitenance mode
|
||||
*/
|
||||
public boolean isInMaintenance()
|
||||
{
|
||||
return inMaintenance;
|
||||
}
|
||||
|
||||
public long getAvailableSize()
|
||||
{
|
||||
long maxSize = getMaxSize();
|
||||
|
|
|
@ -21,6 +21,7 @@ package org.apache.druid.server.coordinator.helper;
|
|||
|
||||
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.common.StringUtils;
|
||||
import org.apache.druid.java.util.emitter.EmittingLogger;
|
||||
import org.apache.druid.server.coordinator.BalancerSegmentHolder;
|
||||
|
@ -34,7 +35,6 @@ import org.apache.druid.server.coordinator.ServerHolder;
|
|||
import org.apache.druid.timeline.DataSegment;
|
||||
import org.apache.druid.timeline.SegmentId;
|
||||
|
||||
import java.util.Comparator;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
@ -48,8 +48,6 @@ import java.util.stream.Collectors;
|
|||
*/
|
||||
public class DruidCoordinatorBalancer implements DruidCoordinatorHelper
|
||||
{
|
||||
public static final Comparator<ServerHolder> percentUsedComparator =
|
||||
Comparator.comparing(ServerHolder::getPercentUsed).reversed();
|
||||
|
||||
protected static final EmittingLogger log = new EmittingLogger(DruidCoordinatorBalancer.class);
|
||||
|
||||
|
@ -108,27 +106,73 @@ public class DruidCoordinatorBalancer implements DruidCoordinatorHelper
|
|||
return;
|
||||
}
|
||||
|
||||
final List<ServerHolder> toMoveFrom = Lists.newArrayList(servers);
|
||||
final List<ServerHolder> toMoveTo = Lists.newArrayList(servers);
|
||||
/*
|
||||
Take as much segments from maintenance servers as priority allows and find the best location for them on
|
||||
available servers. After that, balance segments within available servers pool.
|
||||
*/
|
||||
Map<Boolean, List<ServerHolder>> partitions =
|
||||
servers.stream().collect(Collectors.partitioningBy(ServerHolder::isInMaintenance));
|
||||
final List<ServerHolder> maintenanceServers = partitions.get(true);
|
||||
final List<ServerHolder> availableServers = partitions.get(false);
|
||||
log.info(
|
||||
"Found %d servers in maintenance, %d available servers servers",
|
||||
maintenanceServers.size(),
|
||||
availableServers.size()
|
||||
);
|
||||
|
||||
if (toMoveTo.size() <= 1) {
|
||||
log.info("[%s]: One or fewer servers found. Cannot balance.", tier);
|
||||
return;
|
||||
if (maintenanceServers.isEmpty()) {
|
||||
if (availableServers.size() <= 1) {
|
||||
log.info("[%s]: %d available servers servers found. Cannot balance.", tier, availableServers.size());
|
||||
}
|
||||
} else if (availableServers.isEmpty()) {
|
||||
log.info("[%s]: no available servers servers found during maintenance. Cannot balance.", tier);
|
||||
}
|
||||
|
||||
int numSegments = 0;
|
||||
for (ServerHolder sourceHolder : toMoveFrom) {
|
||||
for (ServerHolder sourceHolder : servers) {
|
||||
numSegments += sourceHolder.getServer().getSegments().size();
|
||||
}
|
||||
|
||||
|
||||
if (numSegments == 0) {
|
||||
log.info("No segments found. Cannot balance.");
|
||||
return;
|
||||
}
|
||||
|
||||
final BalancerStrategy strategy = params.getBalancerStrategy();
|
||||
final int maxSegmentsToMove = Math.min(params.getCoordinatorDynamicConfig().getMaxSegmentsToMove(), numSegments);
|
||||
int priority = params.getCoordinatorDynamicConfig().getNodesInMaintenancePriority();
|
||||
int maxMaintenanceSegmentsToMove = (int) Math.ceil(maxSegmentsToMove * priority / 10.0);
|
||||
log.info("Processing %d segments from servers in maintenance mode", maxMaintenanceSegmentsToMove);
|
||||
Pair<Integer, Integer> maintenanceResult =
|
||||
balanceServers(params, maintenanceServers, availableServers, maxMaintenanceSegmentsToMove);
|
||||
int maxGeneralSegmentsToMove = maxSegmentsToMove - maintenanceResult.lhs;
|
||||
log.info("Processing %d segments from servers in general mode", maxGeneralSegmentsToMove);
|
||||
Pair<Integer, Integer> generalResult =
|
||||
balanceServers(params, availableServers, availableServers, maxGeneralSegmentsToMove);
|
||||
|
||||
int moved = generalResult.lhs + maintenanceResult.lhs;
|
||||
int unmoved = generalResult.rhs + maintenanceResult.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 BalancerStrategy strategy = params.getBalancerStrategy();
|
||||
final int maxIterations = 2 * maxSegmentsToMove;
|
||||
final int maxToLoad = params.getCoordinatorDynamicConfig().getMaxSegmentsInNodeLoadingQueue();
|
||||
int moved = 0, unmoved = 0;
|
||||
|
@ -136,7 +180,6 @@ public class DruidCoordinatorBalancer implements DruidCoordinatorHelper
|
|||
//noinspection ForLoopThatDoesntUseLoopVariable
|
||||
for (int iter = 0; (moved + unmoved) < maxSegmentsToMove; ++iter) {
|
||||
final BalancerSegmentHolder segmentToMoveHolder = strategy.pickSegmentToMove(toMoveFrom);
|
||||
|
||||
if (segmentToMoveHolder != null && params.getAvailableSegments().contains(segmentToMoveHolder.getSegment())) {
|
||||
final DataSegment segmentToMove = segmentToMoveHolder.getSegment();
|
||||
final ImmutableDruidServer fromServer = segmentToMoveHolder.getFromServer();
|
||||
|
@ -154,8 +197,11 @@ public class DruidCoordinatorBalancer implements DruidCoordinatorHelper
|
|||
strategy.findNewSegmentHomeBalancer(segmentToMove, toMoveToWithLoadQueueCapacityAndNotServingSegment);
|
||||
|
||||
if (destinationHolder != null && !destinationHolder.getServer().equals(fromServer)) {
|
||||
moveSegment(segmentToMoveHolder, destinationHolder.getServer(), params);
|
||||
moved++;
|
||||
if (moveSegment(segmentToMoveHolder, destinationHolder.getServer(), params)) {
|
||||
moved++;
|
||||
} else {
|
||||
unmoved++;
|
||||
}
|
||||
} else {
|
||||
log.debug("Segment [%s] is 'optimally' placed.", segmentToMove.getId());
|
||||
unmoved++;
|
||||
|
@ -174,25 +220,10 @@ public class DruidCoordinatorBalancer implements DruidCoordinatorHelper
|
|||
break;
|
||||
}
|
||||
}
|
||||
|
||||
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()) {
|
||||
strategy.emitStats(tier, stats, toMoveFrom);
|
||||
}
|
||||
log.info(
|
||||
"[%s]: Segments Moved: [%d] Segments Let Alone: [%d]",
|
||||
tier,
|
||||
moved,
|
||||
unmoved
|
||||
);
|
||||
return new Pair<>(moved, unmoved);
|
||||
}
|
||||
|
||||
protected void moveSegment(
|
||||
protected boolean moveSegment(
|
||||
final BalancerSegmentHolder segment,
|
||||
final ImmutableDruidServer toServer,
|
||||
final DruidCoordinatorRuntimeParams params
|
||||
|
@ -221,6 +252,7 @@ public class DruidCoordinatorBalancer implements DruidCoordinatorHelper
|
|||
segmentToMove,
|
||||
callback
|
||||
);
|
||||
return true;
|
||||
}
|
||||
catch (Exception e) {
|
||||
log.makeAlert(e, StringUtils.format("[%s] : Moving exception", segmentId)).emit();
|
||||
|
@ -229,5 +261,6 @@ public class DruidCoordinatorBalancer implements DruidCoordinatorHelper
|
|||
}
|
||||
}
|
||||
}
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -83,7 +83,7 @@ public class DruidCoordinatorSegmentKiller implements DruidCoordinatorHelper
|
|||
public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params)
|
||||
{
|
||||
boolean killAllDataSources = params.getCoordinatorDynamicConfig().isKillAllDataSources();
|
||||
Collection<String> whitelist = params.getCoordinatorDynamicConfig().getKillDataSourceWhitelist();
|
||||
Collection<String> whitelist = params.getCoordinatorDynamicConfig().getKillableDataSources();
|
||||
|
||||
if (killAllDataSources && whitelist != null && !whitelist.isEmpty()) {
|
||||
log.error("killAllDataSources can't be true when killDataSourceWhitelist is non-empty, No kill tasks are scheduled.");
|
||||
|
|
|
@ -46,8 +46,9 @@ public abstract class BroadcastDistributionRule implements Rule
|
|||
} else {
|
||||
params.getDruidCluster().getAllServers().forEach(
|
||||
eachHolder -> {
|
||||
if (colocatedDataSources.stream()
|
||||
.anyMatch(source -> eachHolder.getServer().getDataSource(source) != null)) {
|
||||
if (!eachHolder.isInMaintenance()
|
||||
&& colocatedDataSources.stream()
|
||||
.anyMatch(source -> eachHolder.getServer().getDataSource(source) != null)) {
|
||||
loadServerHolders.add(eachHolder);
|
||||
} else if (eachHolder.isServingSegment(segment)) {
|
||||
if (!eachHolder.getPeon().getSegmentsToDrop().contains(segment)) {
|
||||
|
|
|
@ -153,8 +153,8 @@ public abstract class LoadRule implements Rule
|
|||
log.makeAlert("Tier[%s] has no servers! Check your cluster configuration!", tier).emit();
|
||||
return Collections.emptyList();
|
||||
}
|
||||
|
||||
return queue.stream().filter(predicate).collect(Collectors.toList());
|
||||
Predicate<ServerHolder> isNotInMaintenance = s -> !s.isInMaintenance();
|
||||
return queue.stream().filter(isNotInMaintenance.and(predicate)).collect(Collectors.toList());
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -382,21 +382,38 @@ public abstract class LoadRule implements Rule
|
|||
final BalancerStrategy balancerStrategy
|
||||
)
|
||||
{
|
||||
int numDropped = 0;
|
||||
Map<Boolean, TreeSet<ServerHolder>> holders = holdersInTier.stream()
|
||||
.filter(s -> s.isServingSegment(segment))
|
||||
.collect(Collectors.partitioningBy(
|
||||
ServerHolder::isInMaintenance,
|
||||
Collectors.toCollection(TreeSet::new)
|
||||
));
|
||||
TreeSet<ServerHolder> maintenanceServers = holders.get(true);
|
||||
TreeSet<ServerHolder> availableServers = holders.get(false);
|
||||
int left = dropSegmentFromServers(balancerStrategy, segment, maintenanceServers, numToDrop);
|
||||
if (left > 0) {
|
||||
left = dropSegmentFromServers(balancerStrategy, segment, availableServers, left);
|
||||
}
|
||||
if (left != 0) {
|
||||
log.warn("Wtf, holder was null? I have no servers serving [%s]?", segment.getId());
|
||||
}
|
||||
return numToDrop - left;
|
||||
}
|
||||
|
||||
final NavigableSet<ServerHolder> isServingSubset =
|
||||
holdersInTier.stream().filter(s -> s.isServingSegment(segment)).collect(Collectors.toCollection(TreeSet::new));
|
||||
private static int dropSegmentFromServers(
|
||||
BalancerStrategy balancerStrategy,
|
||||
DataSegment segment,
|
||||
NavigableSet<ServerHolder> holders, int numToDrop
|
||||
)
|
||||
{
|
||||
final Iterator<ServerHolder> iterator = balancerStrategy.pickServersToDrop(segment, holders);
|
||||
|
||||
final Iterator<ServerHolder> iterator = balancerStrategy.pickServersToDrop(segment, isServingSubset);
|
||||
|
||||
while (numDropped < numToDrop) {
|
||||
while (numToDrop > 0) {
|
||||
if (!iterator.hasNext()) {
|
||||
log.warn("Wtf, holder was null? I have no servers serving [%s]?", segment.getId());
|
||||
break;
|
||||
}
|
||||
|
||||
final ServerHolder holder = iterator.next();
|
||||
|
||||
if (holder.isServingSegment(segment)) {
|
||||
log.info(
|
||||
"Dropping segment [%s] on server [%s] in tier [%s]",
|
||||
|
@ -405,7 +422,7 @@ public abstract class LoadRule implements Rule
|
|||
holder.getServer().getTier()
|
||||
);
|
||||
holder.getPeon().dropSegment(segment, null);
|
||||
++numDropped;
|
||||
numToDrop--;
|
||||
} else {
|
||||
log.warn(
|
||||
"Server [%s] is no longer serving segment [%s], skipping drop.",
|
||||
|
@ -414,8 +431,7 @@ public abstract class LoadRule implements Rule
|
|||
);
|
||||
}
|
||||
}
|
||||
|
||||
return numDropped;
|
||||
return numToDrop;
|
||||
}
|
||||
|
||||
protected static void validateTieredReplicants(final Map<String, Integer> tieredReplicants)
|
||||
|
|
|
@ -19,12 +19,12 @@
|
|||
|
||||
package org.apache.druid.server.http;
|
||||
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.sun.jersey.spi.container.ResourceFilters;
|
||||
import org.apache.druid.audit.AuditInfo;
|
||||
import org.apache.druid.audit.AuditManager;
|
||||
import org.apache.druid.common.config.ConfigManager.SetResult;
|
||||
import org.apache.druid.common.config.JacksonConfigManager;
|
||||
import org.apache.druid.common.utils.ServletResourceUtils;
|
||||
import org.apache.druid.java.util.common.Intervals;
|
||||
import org.apache.druid.server.coordinator.CoordinatorDynamicConfig;
|
||||
import org.apache.druid.server.http.security.ConfigResourceFilter;
|
||||
|
@ -80,19 +80,26 @@ public class CoordinatorDynamicConfigsResource
|
|||
@Context HttpServletRequest req
|
||||
)
|
||||
{
|
||||
CoordinatorDynamicConfig current = CoordinatorDynamicConfig.current(manager);
|
||||
try {
|
||||
CoordinatorDynamicConfig current = CoordinatorDynamicConfig.current(manager);
|
||||
|
||||
final SetResult setResult = manager.set(
|
||||
CoordinatorDynamicConfig.CONFIG_KEY,
|
||||
dynamicConfigBuilder.build(current),
|
||||
new AuditInfo(author, comment, req.getRemoteAddr())
|
||||
);
|
||||
final SetResult setResult = manager.set(
|
||||
CoordinatorDynamicConfig.CONFIG_KEY,
|
||||
dynamicConfigBuilder.build(current),
|
||||
new AuditInfo(author, comment, req.getRemoteAddr())
|
||||
);
|
||||
|
||||
if (setResult.isOk()) {
|
||||
return Response.ok().build();
|
||||
} else {
|
||||
if (setResult.isOk()) {
|
||||
return Response.ok().build();
|
||||
} else {
|
||||
return Response.status(Response.Status.BAD_REQUEST)
|
||||
.entity(ServletResourceUtils.sanitizeException(setResult.getException()))
|
||||
.build();
|
||||
}
|
||||
}
|
||||
catch (IllegalArgumentException e) {
|
||||
return Response.status(Response.Status.BAD_REQUEST)
|
||||
.entity(ImmutableMap.of("error", setResult.getException()))
|
||||
.entity(ServletResourceUtils.sanitizeException(e))
|
||||
.build();
|
||||
}
|
||||
}
|
||||
|
@ -119,7 +126,7 @@ public class CoordinatorDynamicConfigsResource
|
|||
}
|
||||
catch (IllegalArgumentException e) {
|
||||
return Response.status(Response.Status.BAD_REQUEST)
|
||||
.entity(ImmutableMap.<String, Object>of("error", e.getMessage()))
|
||||
.entity(ServletResourceUtils.sanitizeException(e))
|
||||
.build();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -27,7 +27,6 @@ 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.helper.DruidCoordinatorBalancer;
|
||||
import org.apache.druid.server.coordinator.helper.DruidCoordinatorRuleRunner;
|
||||
import org.apache.druid.server.coordinator.rules.PeriodLoadRule;
|
||||
import org.apache.druid.server.coordinator.rules.Rule;
|
||||
|
@ -139,7 +138,7 @@ public class DruidCoordinatorBalancerProfiler
|
|||
serverHolderList.stream().collect(
|
||||
Collectors.toCollection(
|
||||
() -> new TreeSet<>(
|
||||
DruidCoordinatorBalancer.percentUsedComparator
|
||||
DruidCoordinatorBalancerTester.percentUsedComparator
|
||||
)
|
||||
)
|
||||
)
|
||||
|
@ -170,7 +169,7 @@ public class DruidCoordinatorBalancerProfiler
|
|||
serverHolderList.stream().collect(
|
||||
Collectors.toCollection(
|
||||
() -> new TreeSet<>(
|
||||
DruidCoordinatorBalancer.percentUsedComparator
|
||||
DruidCoordinatorBalancerTester.percentUsedComparator
|
||||
)
|
||||
)
|
||||
)
|
||||
|
@ -232,7 +231,7 @@ public class DruidCoordinatorBalancerProfiler
|
|||
).collect(
|
||||
Collectors.toCollection(
|
||||
() -> new TreeSet<>(
|
||||
DruidCoordinatorBalancer.percentUsedComparator
|
||||
DruidCoordinatorBalancerTester.percentUsedComparator
|
||||
)
|
||||
)
|
||||
)
|
||||
|
|
|
@ -21,11 +21,12 @@ package org.apache.druid.server.coordinator;
|
|||
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
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.java.util.common.DateTimes;
|
||||
import org.apache.druid.server.coordinator.helper.DruidCoordinatorBalancer;
|
||||
import org.apache.druid.server.coordination.ServerType;
|
||||
import org.apache.druid.timeline.DataSegment;
|
||||
import org.apache.druid.timeline.partition.NoneShardSpec;
|
||||
import org.easymock.EasyMock;
|
||||
|
@ -37,15 +38,20 @@ 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.List;
|
||||
import java.util.TreeSet;
|
||||
import java.util.concurrent.Executors;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.IntStream;
|
||||
|
||||
import static org.easymock.EasyMock.anyObject;
|
||||
import static org.easymock.EasyMock.replay;
|
||||
import static org.hamcrest.Matchers.equalTo;
|
||||
import static org.hamcrest.Matchers.is;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class DruidCoordinatorBalancerTest
|
||||
|
@ -193,6 +199,187 @@ public class DruidCoordinatorBalancerTest
|
|||
Assert.assertEquals(2, params.getCoordinatorStats().getTieredStat("movedCount", "normal"));
|
||||
}
|
||||
|
||||
/**
|
||||
* Server 1 has 2 segments.
|
||||
* Server 2 (maintenance) has 2 segments.
|
||||
* Server 3 is empty.
|
||||
* Maintenance has priority 7.
|
||||
* 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 testMoveMaintenancePriority()
|
||||
{
|
||||
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.pickSegmentToMove(ImmutableList.of(new ServerHolder(druidServer2, peon2, false))))
|
||||
.andReturn(new BalancerSegmentHolder(druidServer2, segment3))
|
||||
.andReturn(new BalancerSegmentHolder(druidServer2, segment4));
|
||||
EasyMock.expect(strategy.pickSegmentToMove(anyObject()))
|
||||
.andReturn(new BalancerSegmentHolder(druidServer1, segment1))
|
||||
.andReturn(new BalancerSegmentHolder(druidServer1, segment2));
|
||||
|
||||
EasyMock.expect(strategy.findNewSegmentHomeBalancer(anyObject(), anyObject()))
|
||||
.andReturn(new ServerHolder(druidServer3, peon3))
|
||||
.anyTimes();
|
||||
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)
|
||||
.withMaintenanceModeSegmentsPriority(6)
|
||||
.build() // ceil(3 * 0.6) = 2 segments from servers in maintenance
|
||||
)
|
||||
.withBalancerStrategy(strategy)
|
||||
.build();
|
||||
|
||||
params = new DruidCoordinatorBalancerTester(coordinator).run(params);
|
||||
Assert.assertEquals(3L, params.getCoordinatorStats().getTieredStat("movedCount", "normal"));
|
||||
Assert.assertThat(peon3.getSegmentsToLoad(), is(equalTo(ImmutableSet.of(segment1, segment3, segment4))));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testZeroMaintenancePriority()
|
||||
{
|
||||
DruidCoordinatorRuntimeParams params = setupParamsForMaintenancePriority(0);
|
||||
params = new DruidCoordinatorBalancerTester(coordinator).run(params);
|
||||
Assert.assertEquals(1L, params.getCoordinatorStats().getTieredStat("movedCount", "normal"));
|
||||
Assert.assertThat(peon3.getSegmentsToLoad(), is(equalTo(ImmutableSet.of(segment1))));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMaxMaintenancePriority()
|
||||
{
|
||||
DruidCoordinatorRuntimeParams params = setupParamsForMaintenancePriority(10);
|
||||
params = new DruidCoordinatorBalancerTester(coordinator).run(params);
|
||||
Assert.assertEquals(1L, params.getCoordinatorStats().getTieredStat("movedCount", "normal"));
|
||||
Assert.assertThat(peon3.getSegmentsToLoad(), is(equalTo(ImmutableSet.of(segment2))));
|
||||
}
|
||||
|
||||
/**
|
||||
* Should balance segments as usual (ignoring priority) with empty maintenanceList.
|
||||
*/
|
||||
@Test
|
||||
public void testMoveMaintenancePriorityWithNoMaintenance()
|
||||
{
|
||||
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.pickSegmentToMove(anyObject()))
|
||||
.andReturn(new BalancerSegmentHolder(druidServer1, segment1))
|
||||
.andReturn(new BalancerSegmentHolder(druidServer1, segment2))
|
||||
.andReturn(new BalancerSegmentHolder(druidServer2, segment3))
|
||||
.andReturn(new BalancerSegmentHolder(druidServer2, segment4));
|
||||
|
||||
EasyMock.expect(strategy.findNewSegmentHomeBalancer(anyObject(), anyObject()))
|
||||
.andReturn(new ServerHolder(druidServer3, peon3))
|
||||
.anyTimes();
|
||||
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).withMaintenanceModeSegmentsPriority(9).build()
|
||||
)
|
||||
.withBalancerStrategy(strategy)
|
||||
.build();
|
||||
|
||||
params = new DruidCoordinatorBalancerTester(coordinator).run(params);
|
||||
Assert.assertEquals(3L, params.getCoordinatorStats().getTieredStat("movedCount", "normal"));
|
||||
Assert.assertThat(peon3.getSegmentsToLoad(), is(equalTo(ImmutableSet.of(segment1, segment2, segment3))));
|
||||
}
|
||||
|
||||
/**
|
||||
* Shouldn't move segments to a server in maintenance mode.
|
||||
*/
|
||||
@Test
|
||||
public void testMoveToServerInMaintenance()
|
||||
{
|
||||
mockDruidServer(druidServer1, "1", "normal", 30L, 100L, segments);
|
||||
mockDruidServer(druidServer2, "2", "normal", 0L, 100L, Collections.emptyList());
|
||||
|
||||
replay(druidServer3);
|
||||
replay(druidServer4);
|
||||
|
||||
mockCoordinator(coordinator);
|
||||
|
||||
BalancerStrategy strategy = EasyMock.createMock(BalancerStrategy.class);
|
||||
EasyMock.expect(strategy.pickSegmentToMove(anyObject()))
|
||||
.andReturn(new BalancerSegmentHolder(druidServer1, segment1))
|
||||
.anyTimes();
|
||||
EasyMock.expect(strategy.findNewSegmentHomeBalancer(anyObject(), anyObject())).andAnswer(() -> {
|
||||
List<ServerHolder> holders = (List<ServerHolder>) EasyMock.getCurrentArguments()[1];
|
||||
return holders.get(0);
|
||||
}).anyTimes();
|
||||
replay(strategy);
|
||||
|
||||
DruidCoordinatorRuntimeParams params = defaultRuntimeParamsBuilder(
|
||||
ImmutableList.of(druidServer1, druidServer2),
|
||||
ImmutableList.of(peon1, peon2),
|
||||
ImmutableList.of(false, true)
|
||||
)
|
||||
.withBalancerStrategy(strategy)
|
||||
.build();
|
||||
|
||||
params = new DruidCoordinatorBalancerTester(coordinator).run(params);
|
||||
Assert.assertEquals(0, params.getCoordinatorStats().getTieredStat("movedCount", "normal"));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMoveFromServerInMaintenance()
|
||||
{
|
||||
mockDruidServer(druidServer1, "1", "normal", 30L, 100L, segments);
|
||||
mockDruidServer(druidServer2, "2", "normal", 0L, 100L, Collections.emptyList());
|
||||
|
||||
replay(druidServer3);
|
||||
replay(druidServer4);
|
||||
|
||||
mockCoordinator(coordinator);
|
||||
|
||||
ServerHolder holder2 = new ServerHolder(druidServer2, peon2, false);
|
||||
BalancerStrategy strategy = EasyMock.createMock(BalancerStrategy.class);
|
||||
EasyMock.expect(strategy.pickSegmentToMove(anyObject()))
|
||||
.andReturn(new BalancerSegmentHolder(druidServer1, segment1))
|
||||
.once();
|
||||
EasyMock.expect(strategy.findNewSegmentHomeBalancer(anyObject(), anyObject())).andReturn(holder2).once();
|
||||
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)
|
||||
.build();
|
||||
|
||||
params = new DruidCoordinatorBalancerTester(coordinator).run(params);
|
||||
Assert.assertEquals(1, params.getCoordinatorStats().getTieredStat("movedCount", "normal"));
|
||||
Assert.assertEquals(0, peon1.getNumberOfSegmentsInQueue());
|
||||
Assert.assertEquals(1, peon2.getNumberOfSegmentsInQueue());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMoveMaxLoadQueueServerBalancer()
|
||||
{
|
||||
|
@ -314,6 +501,19 @@ public class DruidCoordinatorBalancerTest
|
|||
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> maintenance
|
||||
)
|
||||
{
|
||||
return DruidCoordinatorRuntimeParams
|
||||
.newBuilder()
|
||||
|
@ -324,12 +524,8 @@ public class DruidCoordinatorBalancerTest
|
|||
"normal",
|
||||
IntStream
|
||||
.range(0, druidServers.size())
|
||||
.mapToObj(i -> new ServerHolder(druidServers.get(i), peons.get(i)))
|
||||
.collect(
|
||||
Collectors.toCollection(
|
||||
() -> new TreeSet<>(DruidCoordinatorBalancer.percentUsedComparator)
|
||||
)
|
||||
)
|
||||
.mapToObj(i -> new ServerHolder(druidServers.get(i), peons.get(i), maintenance.get(i)))
|
||||
.collect(Collectors.toSet())
|
||||
)
|
||||
)
|
||||
)
|
||||
|
@ -349,7 +545,7 @@ public class DruidCoordinatorBalancerTest
|
|||
.withBalancerReferenceTimestamp(DateTimes.of("2013-01-01"));
|
||||
}
|
||||
|
||||
private void mockDruidServer(
|
||||
private static void mockDruidServer(
|
||||
ImmutableDruidServer druidServer,
|
||||
String name,
|
||||
String tier,
|
||||
|
@ -363,17 +559,18 @@ public class DruidCoordinatorBalancerTest
|
|||
EasyMock.expect(druidServer.getCurrSize()).andReturn(currentSize).atLeastOnce();
|
||||
EasyMock.expect(druidServer.getMaxSize()).andReturn(maxSize).atLeastOnce();
|
||||
EasyMock.expect(druidServer.getSegments()).andReturn(segments).anyTimes();
|
||||
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()
|
||||
);
|
||||
} else {
|
||||
EasyMock.expect(druidServer.getSegment(EasyMock.anyObject())).andReturn(null).anyTimes();
|
||||
}
|
||||
EasyMock.replay(druidServer);
|
||||
EasyMock.expect(druidServer.getSegment(anyObject())).andReturn(null).anyTimes();
|
||||
replay(druidServer);
|
||||
}
|
||||
|
||||
private void mockCoordinator(DruidCoordinator coordinator)
|
||||
private static void mockCoordinator(DruidCoordinator coordinator)
|
||||
{
|
||||
coordinator.moveSegment(
|
||||
EasyMock.anyObject(),
|
||||
|
@ -424,4 +621,40 @@ public class DruidCoordinatorBalancerTest
|
|||
delegate.emitStats(tier, stats, serverHolderList);
|
||||
}
|
||||
}
|
||||
|
||||
private DruidCoordinatorRuntimeParams setupParamsForMaintenancePriority(int priority)
|
||||
{
|
||||
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 maintenance servers list or general ones (ie servers list is [2] or [1, 3])
|
||||
BalancerStrategy strategy = EasyMock.createMock(BalancerStrategy.class);
|
||||
EasyMock.expect(strategy.pickSegmentToMove(ImmutableList.of(new ServerHolder(druidServer2, peon2, true))))
|
||||
.andReturn(new BalancerSegmentHolder(druidServer2, segment2));
|
||||
EasyMock.expect(strategy.pickSegmentToMove(anyObject()))
|
||||
.andReturn(new BalancerSegmentHolder(druidServer1, segment1));
|
||||
EasyMock.expect(strategy.findNewSegmentHomeBalancer(anyObject(), anyObject()))
|
||||
.andReturn(new ServerHolder(druidServer3, peon3))
|
||||
.anyTimes();
|
||||
replay(strategy);
|
||||
|
||||
return defaultRuntimeParamsBuilder(
|
||||
ImmutableList.of(druidServer1, druidServer2, druidServer3),
|
||||
ImmutableList.of(peon1, peon2, peon3),
|
||||
ImmutableList.of(false, true, false)
|
||||
)
|
||||
.withDynamicConfigs(
|
||||
CoordinatorDynamicConfig.builder()
|
||||
.withMaxSegmentsToMove(1)
|
||||
.withMaintenanceModeSegmentsPriority(priority)
|
||||
.build()
|
||||
)
|
||||
.withBalancerStrategy(strategy)
|
||||
.build();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -25,15 +25,25 @@ import org.apache.druid.server.coordinator.helper.DruidCoordinatorBalancer;
|
|||
import org.apache.druid.timeline.DataSegment;
|
||||
import org.apache.druid.timeline.SegmentId;
|
||||
|
||||
import java.util.Comparator;
|
||||
|
||||
public class DruidCoordinatorBalancerTester extends DruidCoordinatorBalancer
|
||||
{
|
||||
public static final Comparator<ServerHolder> percentUsedComparator = (ServerHolder a, ServerHolder b) -> {
|
||||
int c = Double.compare(a.getPercentUsed(), b.getPercentUsed());
|
||||
if (c == 0) {
|
||||
return a.getServer().getName().compareTo(b.getServer().getName());
|
||||
}
|
||||
return c;
|
||||
};
|
||||
|
||||
public DruidCoordinatorBalancerTester(DruidCoordinator coordinator)
|
||||
{
|
||||
super(coordinator);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void moveSegment(
|
||||
protected boolean moveSegment(
|
||||
final BalancerSegmentHolder segment,
|
||||
final ImmutableDruidServer toServer,
|
||||
final DruidCoordinatorRuntimeParams params
|
||||
|
@ -64,10 +74,12 @@ public class DruidCoordinatorBalancerTester extends DruidCoordinatorBalancer
|
|||
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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -946,9 +946,9 @@ public class DruidCoordinatorRuleRunnerTest
|
|||
ImmutableMap.of(
|
||||
"normal",
|
||||
Stream.of(
|
||||
new ServerHolder(server1.toImmutableDruidServer(), mockPeon),
|
||||
new ServerHolder(server2.toImmutableDruidServer(), anotherMockPeon),
|
||||
new ServerHolder(server3.toImmutableDruidServer(), anotherMockPeon)
|
||||
new ServerHolder(server1.toImmutableDruidServer(), mockPeon, false),
|
||||
new ServerHolder(server2.toImmutableDruidServer(), anotherMockPeon, false),
|
||||
new ServerHolder(server3.toImmutableDruidServer(), anotherMockPeon, false)
|
||||
).collect(Collectors.toCollection(() -> new TreeSet<>(Collections.reverseOrder())))
|
||||
)
|
||||
);
|
||||
|
|
|
@ -21,7 +21,6 @@ package org.apache.druid.server.coordinator.rules;
|
|||
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.Lists;
|
||||
import org.apache.druid.client.DruidServer;
|
||||
import org.apache.druid.java.util.common.DateTimes;
|
||||
import org.apache.druid.java.util.common.Intervals;
|
||||
|
@ -58,6 +57,10 @@ public class BroadcastDistributionRuleTest
|
|||
private final List<DataSegment> largeSegments = new ArrayList<>();
|
||||
private final List<DataSegment> largeSegments2 = new ArrayList<>();
|
||||
private DataSegment smallSegment;
|
||||
private DruidCluster secondCluster;
|
||||
private ServerHolder generalServer;
|
||||
private ServerHolder maintenanceServer2;
|
||||
private ServerHolder maintenanceServer1;
|
||||
|
||||
@Before
|
||||
public void setUp()
|
||||
|
@ -197,6 +200,50 @@ public class BroadcastDistributionRuleTest
|
|||
)
|
||||
);
|
||||
|
||||
generalServer = new ServerHolder(
|
||||
new DruidServer(
|
||||
"general",
|
||||
"host1",
|
||||
null,
|
||||
100,
|
||||
ServerType.HISTORICAL,
|
||||
"tier1",
|
||||
0
|
||||
).addDataSegment(largeSegments.get(0))
|
||||
.toImmutableDruidServer(),
|
||||
new LoadQueuePeonTester()
|
||||
);
|
||||
|
||||
maintenanceServer1 = new ServerHolder(
|
||||
new DruidServer(
|
||||
"maintenance1",
|
||||
"host2",
|
||||
null,
|
||||
100,
|
||||
ServerType.HISTORICAL,
|
||||
"tier1",
|
||||
0
|
||||
).addDataSegment(smallSegment)
|
||||
.toImmutableDruidServer(),
|
||||
new LoadQueuePeonTester(),
|
||||
true
|
||||
);
|
||||
|
||||
maintenanceServer2 = new ServerHolder(
|
||||
new DruidServer(
|
||||
"maintenance2",
|
||||
"host3",
|
||||
null,
|
||||
100,
|
||||
ServerType.HISTORICAL,
|
||||
"tier1",
|
||||
0
|
||||
).addDataSegment(largeSegments.get(1))
|
||||
.toImmutableDruidServer(),
|
||||
new LoadQueuePeonTester(),
|
||||
true
|
||||
);
|
||||
|
||||
druidCluster = new DruidCluster(
|
||||
null,
|
||||
ImmutableMap.of(
|
||||
|
@ -214,6 +261,18 @@ public class BroadcastDistributionRuleTest
|
|||
).collect(Collectors.toCollection(() -> new TreeSet<>(Collections.reverseOrder())))
|
||||
)
|
||||
);
|
||||
|
||||
secondCluster = new DruidCluster(
|
||||
null,
|
||||
ImmutableMap.of(
|
||||
"tier1",
|
||||
Stream.of(
|
||||
generalServer,
|
||||
maintenanceServer1,
|
||||
maintenanceServer2
|
||||
).collect(Collectors.toCollection(() -> new TreeSet<>(Collections.reverseOrder())))
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -227,14 +286,14 @@ public class BroadcastDistributionRuleTest
|
|||
.withDruidCluster(druidCluster)
|
||||
.withSegmentReplicantLookup(SegmentReplicantLookup.make(druidCluster))
|
||||
.withBalancerReferenceTimestamp(DateTimes.of("2013-01-01"))
|
||||
.withAvailableSegments(Lists.newArrayList(
|
||||
.withAvailableSegments(
|
||||
smallSegment,
|
||||
largeSegments.get(0),
|
||||
largeSegments.get(1),
|
||||
largeSegments.get(2),
|
||||
largeSegments2.get(0),
|
||||
largeSegments2.get(1)
|
||||
)).build(),
|
||||
).build(),
|
||||
smallSegment
|
||||
);
|
||||
|
||||
|
@ -254,6 +313,46 @@ public class BroadcastDistributionRuleTest
|
|||
assertFalse(holderOfSmallSegment.getPeon().getSegmentsToLoad().contains(smallSegment));
|
||||
}
|
||||
|
||||
/**
|
||||
* Servers:
|
||||
* name | segments
|
||||
* -------------+--------------
|
||||
* general | large segment
|
||||
* maintenance1 | small segment
|
||||
* maintenance2 | large segment
|
||||
*
|
||||
* After running the rule for the small segment:
|
||||
* general | large & small segments
|
||||
* maintenance1 |
|
||||
* maintenance2 | large segment
|
||||
*/
|
||||
@Test
|
||||
public void testBroadcastWithMaintenance()
|
||||
{
|
||||
final ForeverBroadcastDistributionRule rule = new ForeverBroadcastDistributionRule(ImmutableList.of("large_source"));
|
||||
|
||||
CoordinatorStats stats = rule.run(
|
||||
null,
|
||||
DruidCoordinatorRuntimeParams.newBuilder()
|
||||
.withDruidCluster(secondCluster)
|
||||
.withSegmentReplicantLookup(SegmentReplicantLookup.make(secondCluster))
|
||||
.withBalancerReferenceTimestamp(DateTimes.of("2013-01-01"))
|
||||
.withAvailableSegments(
|
||||
smallSegment,
|
||||
largeSegments.get(0),
|
||||
largeSegments.get(1)
|
||||
).build(),
|
||||
smallSegment
|
||||
);
|
||||
|
||||
assertEquals(1L, stats.getGlobalStat(LoadRule.ASSIGNED_COUNT));
|
||||
assertEquals(false, stats.hasPerTierStats());
|
||||
|
||||
assertEquals(1, generalServer.getPeon().getSegmentsToLoad().size());
|
||||
assertEquals(1, maintenanceServer1.getPeon().getSegmentsToDrop().size());
|
||||
assertEquals(0, maintenanceServer2.getPeon().getSegmentsToLoad().size());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testBroadcastToMultipleDataSources()
|
||||
{
|
||||
|
@ -267,14 +366,14 @@ public class BroadcastDistributionRuleTest
|
|||
.withDruidCluster(druidCluster)
|
||||
.withSegmentReplicantLookup(SegmentReplicantLookup.make(druidCluster))
|
||||
.withBalancerReferenceTimestamp(DateTimes.of("2013-01-01"))
|
||||
.withAvailableSegments(Lists.newArrayList(
|
||||
.withAvailableSegments(
|
||||
smallSegment,
|
||||
largeSegments.get(0),
|
||||
largeSegments.get(1),
|
||||
largeSegments.get(2),
|
||||
largeSegments2.get(0),
|
||||
largeSegments2.get(1)
|
||||
)).build(),
|
||||
).build(),
|
||||
smallSegment
|
||||
);
|
||||
|
||||
|
@ -305,14 +404,14 @@ public class BroadcastDistributionRuleTest
|
|||
.withDruidCluster(druidCluster)
|
||||
.withSegmentReplicantLookup(SegmentReplicantLookup.make(druidCluster))
|
||||
.withBalancerReferenceTimestamp(DateTimes.of("2013-01-01"))
|
||||
.withAvailableSegments(Lists.newArrayList(
|
||||
.withAvailableSegments(
|
||||
smallSegment,
|
||||
largeSegments.get(0),
|
||||
largeSegments.get(1),
|
||||
largeSegments.get(2),
|
||||
largeSegments2.get(0),
|
||||
largeSegments2.get(1)
|
||||
)).build(),
|
||||
).build(),
|
||||
smallSegment
|
||||
);
|
||||
|
||||
|
|
|
@ -65,6 +65,7 @@ import java.util.Map;
|
|||
import java.util.Set;
|
||||
import java.util.TreeSet;
|
||||
import java.util.concurrent.Executors;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
|
@ -181,7 +182,7 @@ public class LoadRuleTest
|
|||
.withReplicationManager(throttler)
|
||||
.withBalancerStrategy(mockBalancerStrategy)
|
||||
.withBalancerReferenceTimestamp(DateTimes.of("2013-01-01"))
|
||||
.withAvailableSegments(Collections.singletonList(segment)).build(),
|
||||
.withAvailableSegments(segment).build(),
|
||||
segment
|
||||
);
|
||||
|
||||
|
@ -252,7 +253,7 @@ public class LoadRuleTest
|
|||
.withReplicationManager(throttler)
|
||||
.withBalancerStrategy(mockBalancerStrategy)
|
||||
.withBalancerReferenceTimestamp(DateTimes.of("2013-01-01"))
|
||||
.withAvailableSegments(Collections.singletonList(segment)).build(),
|
||||
.withAvailableSegments(segment).build(),
|
||||
segment
|
||||
);
|
||||
|
||||
|
@ -302,7 +303,7 @@ public class LoadRuleTest
|
|||
.withReplicationManager(throttler)
|
||||
.withBalancerStrategy(mockBalancerStrategy)
|
||||
.withBalancerReferenceTimestamp(DateTimes.of("2013-01-01"))
|
||||
.withAvailableSegments(Collections.singletonList(segment)).build(),
|
||||
.withAvailableSegments(segment).build(),
|
||||
segment
|
||||
);
|
||||
|
||||
|
@ -392,7 +393,7 @@ public class LoadRuleTest
|
|||
.withReplicationManager(throttler)
|
||||
.withBalancerStrategy(mockBalancerStrategy)
|
||||
.withBalancerReferenceTimestamp(DateTimes.of("2013-01-01"))
|
||||
.withAvailableSegments(Collections.singletonList(segment)).build(),
|
||||
.withAvailableSegments(segment).build(),
|
||||
segment
|
||||
);
|
||||
|
||||
|
@ -410,7 +411,7 @@ public class LoadRuleTest
|
|||
EasyMock.expectLastCall().atLeastOnce();
|
||||
EasyMock.expect(mockBalancerStrategy.pickServersToDrop(EasyMock.anyObject(), EasyMock.anyObject()))
|
||||
.andDelegateTo(balancerStrategy)
|
||||
.times(2);
|
||||
.times(4);
|
||||
EasyMock.replay(throttler, mockPeon, mockBalancerStrategy);
|
||||
|
||||
LoadRule rule = createLoadRule(ImmutableMap.of(
|
||||
|
@ -481,7 +482,7 @@ public class LoadRuleTest
|
|||
.withReplicationManager(throttler)
|
||||
.withBalancerStrategy(mockBalancerStrategy)
|
||||
.withBalancerReferenceTimestamp(DateTimes.of("2013-01-01"))
|
||||
.withAvailableSegments(Collections.singletonList(segment)).build(),
|
||||
.withAvailableSegments(segment).build(),
|
||||
segment
|
||||
);
|
||||
|
||||
|
@ -540,7 +541,7 @@ public class LoadRuleTest
|
|||
.withReplicationManager(throttler)
|
||||
.withBalancerStrategy(mockBalancerStrategy)
|
||||
.withBalancerReferenceTimestamp(DateTimes.of("2013-01-01"))
|
||||
.withAvailableSegments(Collections.singletonList(segment)).build(),
|
||||
.withAvailableSegments(segment).build(),
|
||||
segment
|
||||
);
|
||||
|
||||
|
@ -557,7 +558,7 @@ public class LoadRuleTest
|
|||
EasyMock.expectLastCall().atLeastOnce();
|
||||
EasyMock.expect(mockBalancerStrategy.pickServersToDrop(EasyMock.anyObject(), EasyMock.anyObject()))
|
||||
.andDelegateTo(balancerStrategy)
|
||||
.times(1);
|
||||
.times(2);
|
||||
EasyMock.replay(throttler, mockPeon, mockBalancerStrategy);
|
||||
|
||||
LoadRule rule = createLoadRule(ImmutableMap.of(
|
||||
|
@ -613,7 +614,7 @@ public class LoadRuleTest
|
|||
.withReplicationManager(throttler)
|
||||
.withBalancerStrategy(mockBalancerStrategy)
|
||||
.withBalancerReferenceTimestamp(DateTimes.of("2013-01-01"))
|
||||
.withAvailableSegments(Collections.singletonList(segment)).build(),
|
||||
.withAvailableSegments(segment).build(),
|
||||
segment
|
||||
);
|
||||
|
||||
|
@ -670,7 +671,7 @@ public class LoadRuleTest
|
|||
.withReplicationManager(throttler)
|
||||
.withBalancerStrategy(mockBalancerStrategy)
|
||||
.withBalancerReferenceTimestamp(DateTimes.of("2013-01-01"))
|
||||
.withAvailableSegments(Arrays.asList(dataSegment1, dataSegment2, dataSegment3))
|
||||
.withAvailableSegments(dataSegment1, dataSegment2, dataSegment3)
|
||||
.withDynamicConfigs(CoordinatorDynamicConfig.builder().withMaxSegmentsInNodeLoadingQueue(2).build())
|
||||
.build();
|
||||
|
||||
|
@ -685,6 +686,239 @@ public class LoadRuleTest
|
|||
EasyMock.verify(throttler, mockBalancerStrategy);
|
||||
}
|
||||
|
||||
/**
|
||||
* 2 servers in different tiers, the first is in maitenance mode.
|
||||
* Should not load a segment to the server in maintenance mode.
|
||||
*/
|
||||
@Test
|
||||
public void testLoadDuringMaitenance()
|
||||
{
|
||||
final LoadQueuePeon mockPeon1 = createEmptyPeon();
|
||||
final LoadQueuePeon mockPeon2 = createOneCallPeonMock();
|
||||
|
||||
LoadRule rule = createLoadRule(ImmutableMap.of(
|
||||
"tier1", 1,
|
||||
"tier2", 1
|
||||
));
|
||||
|
||||
final DataSegment segment = createDataSegment("foo");
|
||||
|
||||
EasyMock.expect(mockBalancerStrategy.findNewSegmentHomeReplicator(EasyMock.anyObject(), EasyMock.anyObject()))
|
||||
.andDelegateTo(balancerStrategy)
|
||||
.times(1);
|
||||
|
||||
EasyMock.replay(mockPeon1, mockPeon2, mockBalancerStrategy);
|
||||
|
||||
|
||||
DruidCluster druidCluster = new DruidCluster(
|
||||
null,
|
||||
ImmutableMap.of(
|
||||
"tier1",
|
||||
Collections.singleton(createServerHolder("tier1", mockPeon1, true)),
|
||||
"tier2",
|
||||
Collections.singleton(createServerHolder("tier2", mockPeon2, false))
|
||||
)
|
||||
);
|
||||
|
||||
CoordinatorStats stats = rule.run(
|
||||
null,
|
||||
DruidCoordinatorRuntimeParams.newBuilder()
|
||||
.withDruidCluster(druidCluster)
|
||||
.withSegmentReplicantLookup(SegmentReplicantLookup.make(druidCluster))
|
||||
.withReplicationManager(throttler)
|
||||
.withBalancerStrategy(mockBalancerStrategy)
|
||||
.withBalancerReferenceTimestamp(DateTimes.of("2013-01-01"))
|
||||
.withAvailableSegments(segment).build(),
|
||||
segment
|
||||
);
|
||||
|
||||
Assert.assertEquals(1L, stats.getTieredStat(LoadRule.ASSIGNED_COUNT, "tier2"));
|
||||
EasyMock.verify(mockPeon1, mockPeon2, mockBalancerStrategy);
|
||||
}
|
||||
|
||||
/**
|
||||
* 2 tiers, 2 servers each, 1 server of the second tier is in maintenance.
|
||||
* Should not load a segment to the server in maintenance mode.
|
||||
*/
|
||||
@Test
|
||||
public void testLoadReplicaDuringMaitenance()
|
||||
{
|
||||
EasyMock.expect(throttler.canCreateReplicant(EasyMock.anyString())).andReturn(true).anyTimes();
|
||||
|
||||
final LoadQueuePeon mockPeon1 = createEmptyPeon();
|
||||
final LoadQueuePeon mockPeon2 = createOneCallPeonMock();
|
||||
final LoadQueuePeon mockPeon3 = createOneCallPeonMock();
|
||||
final LoadQueuePeon mockPeon4 = createOneCallPeonMock();
|
||||
|
||||
LoadRule rule = createLoadRule(ImmutableMap.of("tier1", 2, "tier2", 2));
|
||||
|
||||
final DataSegment segment = createDataSegment("foo");
|
||||
|
||||
throttler.registerReplicantCreation(EasyMock.eq("tier2"), EasyMock.anyObject(), EasyMock.anyObject());
|
||||
EasyMock.expectLastCall().times(2);
|
||||
|
||||
ServerHolder holder1 = createServerHolder("tier1", mockPeon1, true);
|
||||
ServerHolder holder2 = createServerHolder("tier1", mockPeon2, false);
|
||||
ServerHolder holder3 = createServerHolder("tier2", mockPeon3, false);
|
||||
ServerHolder holder4 = createServerHolder("tier2", mockPeon4, false);
|
||||
|
||||
EasyMock.expect(mockBalancerStrategy.findNewSegmentHomeReplicator(segment, ImmutableList.of(holder2)))
|
||||
.andReturn(holder2);
|
||||
EasyMock.expect(mockBalancerStrategy.findNewSegmentHomeReplicator(segment, ImmutableList.of(holder4, holder3)))
|
||||
.andReturn(holder3);
|
||||
EasyMock.expect(mockBalancerStrategy.findNewSegmentHomeReplicator(segment, ImmutableList.of(holder4)))
|
||||
.andReturn(holder4);
|
||||
|
||||
EasyMock.replay(throttler, mockPeon1, mockPeon2, mockPeon3, mockPeon4, mockBalancerStrategy);
|
||||
|
||||
|
||||
DruidCluster druidCluster = new DruidCluster(
|
||||
null,
|
||||
ImmutableMap.of("tier1", Arrays.asList(holder1, holder2), "tier2", Arrays.asList(holder3, holder4))
|
||||
);
|
||||
|
||||
CoordinatorStats stats = rule.run(
|
||||
null,
|
||||
DruidCoordinatorRuntimeParams.newBuilder()
|
||||
.withDruidCluster(druidCluster)
|
||||
.withSegmentReplicantLookup(SegmentReplicantLookup.make(druidCluster))
|
||||
.withReplicationManager(throttler)
|
||||
.withBalancerStrategy(mockBalancerStrategy)
|
||||
.withBalancerReferenceTimestamp(DateTimes.of("2013-01-01"))
|
||||
.withAvailableSegments(segment).build(),
|
||||
segment
|
||||
);
|
||||
|
||||
Assert.assertEquals(1L, stats.getTieredStat(LoadRule.ASSIGNED_COUNT, "tier1"));
|
||||
Assert.assertEquals(2L, stats.getTieredStat(LoadRule.ASSIGNED_COUNT, "tier2"));
|
||||
|
||||
EasyMock.verify(throttler, mockPeon1, mockPeon2, mockPeon3, mockPeon4, mockBalancerStrategy);
|
||||
}
|
||||
|
||||
/**
|
||||
* 2 servers with a segment, one server in maintenance mode.
|
||||
* Should drop a segment from both.
|
||||
*/
|
||||
@Test
|
||||
public void testDropDuringMaintenance()
|
||||
{
|
||||
final LoadQueuePeon mockPeon = createEmptyPeon();
|
||||
mockPeon.dropSegment(EasyMock.anyObject(), EasyMock.anyObject());
|
||||
EasyMock.expectLastCall().times(2);
|
||||
EasyMock.expect(mockBalancerStrategy.pickServersToDrop(EasyMock.anyObject(), EasyMock.anyObject()))
|
||||
.andDelegateTo(balancerStrategy)
|
||||
.times(4);
|
||||
EasyMock.replay(throttler, mockPeon, mockBalancerStrategy);
|
||||
|
||||
LoadRule rule = createLoadRule(ImmutableMap.of("tier1", 0));
|
||||
|
||||
final DataSegment segment1 = createDataSegment("foo1");
|
||||
final DataSegment segment2 = createDataSegment("foo2");
|
||||
|
||||
DruidServer server1 = createServer("tier1");
|
||||
server1.addDataSegment(segment1);
|
||||
DruidServer server2 = createServer("tier1");
|
||||
server2.addDataSegment(segment2);
|
||||
|
||||
DruidCluster druidCluster = new DruidCluster(
|
||||
null,
|
||||
ImmutableMap.of(
|
||||
"tier1",
|
||||
Arrays.asList(
|
||||
new ServerHolder(server1.toImmutableDruidServer(), mockPeon, true),
|
||||
new ServerHolder(server2.toImmutableDruidServer(), mockPeon, false)
|
||||
)
|
||||
)
|
||||
);
|
||||
|
||||
DruidCoordinatorRuntimeParams params = DruidCoordinatorRuntimeParams
|
||||
.newBuilder()
|
||||
.withDruidCluster(druidCluster)
|
||||
.withSegmentReplicantLookup(SegmentReplicantLookup.make(druidCluster))
|
||||
.withReplicationManager(throttler)
|
||||
.withBalancerStrategy(mockBalancerStrategy)
|
||||
.withBalancerReferenceTimestamp(DateTimes.of("2013-01-01"))
|
||||
.withAvailableSegments(segment1, segment2)
|
||||
.build();
|
||||
CoordinatorStats stats = rule.run(
|
||||
null,
|
||||
params,
|
||||
segment1
|
||||
);
|
||||
Assert.assertEquals(1L, stats.getTieredStat("droppedCount", "tier1"));
|
||||
stats = rule.run(
|
||||
null,
|
||||
params,
|
||||
segment2
|
||||
);
|
||||
Assert.assertEquals(1L, stats.getTieredStat("droppedCount", "tier1"));
|
||||
|
||||
|
||||
EasyMock.verify(throttler, mockPeon);
|
||||
}
|
||||
|
||||
/**
|
||||
* 3 servers hosting 3 replicas of the segment.
|
||||
* 1 servers is in maitenance.
|
||||
* 1 replica is redundant.
|
||||
* Should drop from the server in maintenance.
|
||||
*/
|
||||
@Test
|
||||
public void testRedundantReplicaDropDuringMaintenance()
|
||||
{
|
||||
final LoadQueuePeon mockPeon1 = new LoadQueuePeonTester();
|
||||
final LoadQueuePeon mockPeon2 = new LoadQueuePeonTester();
|
||||
final LoadQueuePeon mockPeon3 = new LoadQueuePeonTester();
|
||||
EasyMock.expect(mockBalancerStrategy.pickServersToDrop(EasyMock.anyObject(), EasyMock.anyObject()))
|
||||
.andDelegateTo(balancerStrategy)
|
||||
.times(4);
|
||||
EasyMock.replay(throttler, mockBalancerStrategy);
|
||||
|
||||
LoadRule rule = createLoadRule(ImmutableMap.of("tier1", 2));
|
||||
|
||||
final DataSegment segment1 = createDataSegment("foo1");
|
||||
|
||||
DruidServer server1 = createServer("tier1");
|
||||
server1.addDataSegment(segment1);
|
||||
DruidServer server2 = createServer("tier1");
|
||||
server2.addDataSegment(segment1);
|
||||
DruidServer server3 = createServer("tier1");
|
||||
server3.addDataSegment(segment1);
|
||||
|
||||
DruidCluster druidCluster = new DruidCluster(
|
||||
null,
|
||||
ImmutableMap.of(
|
||||
"tier1",
|
||||
Arrays.asList(
|
||||
new ServerHolder(server1.toImmutableDruidServer(), mockPeon1, false),
|
||||
new ServerHolder(server2.toImmutableDruidServer(), mockPeon2, true),
|
||||
new ServerHolder(server3.toImmutableDruidServer(), mockPeon3, false)
|
||||
)
|
||||
)
|
||||
);
|
||||
|
||||
DruidCoordinatorRuntimeParams params = DruidCoordinatorRuntimeParams
|
||||
.newBuilder()
|
||||
.withDruidCluster(druidCluster)
|
||||
.withSegmentReplicantLookup(SegmentReplicantLookup.make(druidCluster))
|
||||
.withReplicationManager(throttler)
|
||||
.withBalancerStrategy(mockBalancerStrategy)
|
||||
.withBalancerReferenceTimestamp(DateTimes.of("2013-01-01"))
|
||||
.withAvailableSegments(segment1)
|
||||
.build();
|
||||
CoordinatorStats stats = rule.run(
|
||||
null,
|
||||
params,
|
||||
segment1
|
||||
);
|
||||
Assert.assertEquals(1L, stats.getTieredStat("droppedCount", "tier1"));
|
||||
Assert.assertEquals(0, mockPeon1.getSegmentsToDrop().size());
|
||||
Assert.assertEquals(1, mockPeon2.getSegmentsToDrop().size());
|
||||
Assert.assertEquals(0, mockPeon3.getSegmentsToDrop().size());
|
||||
|
||||
EasyMock.verify(throttler);
|
||||
}
|
||||
|
||||
private DataSegment createDataSegment(String dataSource)
|
||||
{
|
||||
return new DataSegment(
|
||||
|
@ -760,4 +994,37 @@ public class LoadRuleTest
|
|||
|
||||
return mockPeon;
|
||||
}
|
||||
|
||||
private static final AtomicInteger serverId = new AtomicInteger();
|
||||
|
||||
private static DruidServer createServer(String tier)
|
||||
{
|
||||
int serverId = LoadRuleTest.serverId.incrementAndGet();
|
||||
return new DruidServer(
|
||||
"server" + serverId,
|
||||
"127.0.0.1:800" + serverId,
|
||||
null,
|
||||
1000,
|
||||
ServerType.HISTORICAL,
|
||||
tier,
|
||||
0
|
||||
);
|
||||
}
|
||||
|
||||
private static LoadQueuePeon createOneCallPeonMock()
|
||||
{
|
||||
final LoadQueuePeon mockPeon2 = createEmptyPeon();
|
||||
mockPeon2.loadSegment(EasyMock.anyObject(), EasyMock.anyObject());
|
||||
EasyMock.expectLastCall().once();
|
||||
return mockPeon2;
|
||||
}
|
||||
|
||||
private static ServerHolder createServerHolder(String tier, LoadQueuePeon mockPeon1, boolean maintenance)
|
||||
{
|
||||
return new ServerHolder(
|
||||
createServer(tier).toImmutableDruidServer(),
|
||||
mockPeon1,
|
||||
maintenance
|
||||
);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -38,6 +38,44 @@ public class CoordinatorDynamicConfigTest
|
|||
|
||||
@Test
|
||||
public void testSerde() throws Exception
|
||||
{
|
||||
String jsonStr = "{\n"
|
||||
+ " \"millisToWaitBeforeDeleting\": 1,\n"
|
||||
+ " \"mergeBytesLimit\": 1,\n"
|
||||
+ " \"mergeSegmentsLimit\" : 1,\n"
|
||||
+ " \"maxSegmentsToMove\": 1,\n"
|
||||
+ " \"replicantLifetime\": 1,\n"
|
||||
+ " \"replicationThrottleLimit\": 1,\n"
|
||||
+ " \"balancerComputeThreads\": 2, \n"
|
||||
+ " \"emitBalancingStats\": true,\n"
|
||||
+ " \"killDataSourceWhitelist\": [\"test1\",\"test2\"],\n"
|
||||
+ " \"maxSegmentsInNodeLoadingQueue\": 1,\n"
|
||||
+ " \"historicalNodesInMaintenance\": [\"host1\", \"host2\"],\n"
|
||||
+ " \"nodesInMaintenancePriority\": 9\n"
|
||||
+ "}\n";
|
||||
|
||||
CoordinatorDynamicConfig actual = mapper.readValue(
|
||||
mapper.writeValueAsString(
|
||||
mapper.readValue(
|
||||
jsonStr,
|
||||
CoordinatorDynamicConfig.class
|
||||
)
|
||||
),
|
||||
CoordinatorDynamicConfig.class
|
||||
);
|
||||
ImmutableSet<String> maintenance = ImmutableSet.of("host1", "host2");
|
||||
ImmutableSet<String> whitelist = ImmutableSet.of("test1", "test2");
|
||||
assertConfig(actual, 1, 1, 1, 1, 1, 1, 2, true, whitelist, false, 1, maintenance, 9);
|
||||
|
||||
actual = CoordinatorDynamicConfig.builder().withMaintenanceList(ImmutableSet.of("host1")).build(actual);
|
||||
assertConfig(actual, 1, 1, 1, 1, 1, 1, 2, true, whitelist, false, 1, ImmutableSet.of("host1"), 9);
|
||||
|
||||
actual = CoordinatorDynamicConfig.builder().withMaintenanceModeSegmentsPriority(5).build(actual);
|
||||
assertConfig(actual, 1, 1, 1, 1, 1, 1, 2, true, whitelist, false, 1, ImmutableSet.of("host1"), 5);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMaintenanceParametersBackwardCompatibility() throws Exception
|
||||
{
|
||||
String jsonStr = "{\n"
|
||||
+ " \"millisToWaitBeforeDeleting\": 1,\n"
|
||||
|
@ -61,7 +99,15 @@ public class CoordinatorDynamicConfigTest
|
|||
),
|
||||
CoordinatorDynamicConfig.class
|
||||
);
|
||||
assertConfig(actual, 1, 1, 1, 1, 1, 1, 2, true, ImmutableSet.of("test1", "test2"), false, 1);
|
||||
ImmutableSet<String> maintenance = ImmutableSet.of();
|
||||
ImmutableSet<String> whitelist = ImmutableSet.of("test1", "test2");
|
||||
assertConfig(actual, 1, 1, 1, 1, 1, 1, 2, true, whitelist, false, 1, maintenance, 0);
|
||||
|
||||
actual = CoordinatorDynamicConfig.builder().withMaintenanceList(ImmutableSet.of("host1")).build(actual);
|
||||
assertConfig(actual, 1, 1, 1, 1, 1, 1, 2, true, whitelist, false, 1, ImmutableSet.of("host1"), 0);
|
||||
|
||||
actual = CoordinatorDynamicConfig.builder().withMaintenanceModeSegmentsPriority(5).build(actual);
|
||||
assertConfig(actual, 1, 1, 1, 1, 1, 1, 2, true, whitelist, false, 1, ImmutableSet.of("host1"), 5);
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -89,7 +135,7 @@ public class CoordinatorDynamicConfigTest
|
|||
),
|
||||
CoordinatorDynamicConfig.class
|
||||
);
|
||||
assertConfig(actual, 1, 1, 1, 1, 1, 1, 2, true, ImmutableSet.of("test1", "test2"), false, 1);
|
||||
assertConfig(actual, 1, 1, 1, 1, 1, 1, 2, true, ImmutableSet.of("test1", "test2"), false, 1, ImmutableSet.of(), 0);
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -118,7 +164,7 @@ public class CoordinatorDynamicConfigTest
|
|||
CoordinatorDynamicConfig.class
|
||||
);
|
||||
|
||||
assertConfig(actual, 1, 1, 1, 1, 1, 1, 2, true, ImmutableSet.of(), true, 1);
|
||||
assertConfig(actual, 1, 1, 1, 1, 1, 1, 2, true, ImmutableSet.of(), true, 1, ImmutableSet.of(), 0);
|
||||
|
||||
//ensure whitelist is empty when killAllDataSources is true
|
||||
try {
|
||||
|
@ -163,15 +209,15 @@ public class CoordinatorDynamicConfigTest
|
|||
CoordinatorDynamicConfig.class
|
||||
);
|
||||
|
||||
assertConfig(actual, 1, 1, 1, 1, 1, 1, 2, true, ImmutableSet.of(), true, 0);
|
||||
assertConfig(actual, 1, 1, 1, 1, 1, 1, 2, true, ImmutableSet.of(), true, 0, ImmutableSet.of(), 0);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testBuilderDefaults()
|
||||
{
|
||||
|
||||
CoordinatorDynamicConfig defaultConfig = CoordinatorDynamicConfig.builder().build();
|
||||
assertConfig(defaultConfig, 900000, 524288000, 100, 5, 15, 10, 1, false, ImmutableSet.of(), false, 0);
|
||||
ImmutableSet<String> emptyList = ImmutableSet.of();
|
||||
assertConfig(defaultConfig, 900000, 524288000, 100, 5, 15, 10, 1, false, emptyList, false, 0, emptyList, 7);
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -184,7 +230,7 @@ public class CoordinatorDynamicConfigTest
|
|||
Assert.assertEquals(
|
||||
current,
|
||||
new CoordinatorDynamicConfig
|
||||
.Builder(null, null, null, null, null, null, null, null, null, null, null, null)
|
||||
.Builder(null, null, null, null, null, null, null, null, null, null, null, null, null, null)
|
||||
.build(current)
|
||||
);
|
||||
}
|
||||
|
@ -198,18 +244,22 @@ public class CoordinatorDynamicConfigTest
|
|||
Assert.assertEquals(config1.hashCode(), config2.hashCode());
|
||||
}
|
||||
|
||||
private void assertConfig(CoordinatorDynamicConfig config,
|
||||
long expectedMillisToWaitBeforeDeleting,
|
||||
long expectedMergeBytesLimit,
|
||||
int expectedMergeSegmentsLimit,
|
||||
int expectedMaxSegmentsToMove,
|
||||
int expectedReplicantLifetime,
|
||||
int expectedReplicationThrottleLimit,
|
||||
int expectedBalancerComputeThreads,
|
||||
boolean expectedEmitingBalancingStats,
|
||||
Set<String> expectedKillDataSourceWhitelist,
|
||||
boolean expectedKillAllDataSources,
|
||||
int expectedMaxSegmentsInNodeLoadingQueue)
|
||||
private void assertConfig(
|
||||
CoordinatorDynamicConfig config,
|
||||
long expectedMillisToWaitBeforeDeleting,
|
||||
long expectedMergeBytesLimit,
|
||||
int expectedMergeSegmentsLimit,
|
||||
int expectedMaxSegmentsToMove,
|
||||
int expectedReplicantLifetime,
|
||||
int expectedReplicationThrottleLimit,
|
||||
int expectedBalancerComputeThreads,
|
||||
boolean expectedEmitingBalancingStats,
|
||||
Set<String> expectedKillableDatasources,
|
||||
boolean expectedKillAllDataSources,
|
||||
int expectedMaxSegmentsInNodeLoadingQueue,
|
||||
Set<String> maintenanceList,
|
||||
int maintenancePriority
|
||||
)
|
||||
{
|
||||
Assert.assertEquals(expectedMillisToWaitBeforeDeleting, config.getMillisToWaitBeforeDeleting());
|
||||
Assert.assertEquals(expectedMergeBytesLimit, config.getMergeBytesLimit());
|
||||
|
@ -219,8 +269,10 @@ public class CoordinatorDynamicConfigTest
|
|||
Assert.assertEquals(expectedReplicationThrottleLimit, config.getReplicationThrottleLimit());
|
||||
Assert.assertEquals(expectedBalancerComputeThreads, config.getBalancerComputeThreads());
|
||||
Assert.assertEquals(expectedEmitingBalancingStats, config.emitBalancingStats());
|
||||
Assert.assertEquals(expectedKillDataSourceWhitelist, config.getKillDataSourceWhitelist());
|
||||
Assert.assertEquals(expectedKillableDatasources, config.getKillableDataSources());
|
||||
Assert.assertEquals(expectedKillAllDataSources, config.isKillAllDataSources());
|
||||
Assert.assertEquals(expectedMaxSegmentsInNodeLoadingQueue, config.getMaxSegmentsInNodeLoadingQueue());
|
||||
Assert.assertEquals(maintenanceList, config.getHistoricalNodesInMaintenance());
|
||||
Assert.assertEquals(maintenancePriority, config.getNodesInMaintenancePriority());
|
||||
}
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue