mirror of https://github.com/apache/druid.git
Enable round-robin segment assignment and batch segment allocation by default (#13942)
Changes: - Set `useRoundRobinSegmentAssignment` in coordinator dynamic config to `true` by default. - Set `batchSegmentAllocation` in `TaskLockConfig` (used in Overlord runtime properties) to `true` by default.
This commit is contained in:
parent
ede9903ff4
commit
b7752a909c
|
@ -31,7 +31,7 @@ public class TaskLockConfig
|
|||
private boolean forceTimeChunkLock = true;
|
||||
|
||||
@JsonProperty
|
||||
private boolean batchSegmentAllocation = false;
|
||||
private boolean batchSegmentAllocation = true;
|
||||
|
||||
@JsonProperty
|
||||
private long batchAllocationWaitTime = 500L;
|
||||
|
|
|
@ -530,7 +530,7 @@ public class CoordinatorDynamicConfig
|
|||
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 = false;
|
||||
private static final boolean DEFAULT_USE_ROUND_ROBIN_ASSIGNMENT = true;
|
||||
|
||||
private Long leadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments;
|
||||
private Long mergeBytesLimit;
|
||||
|
|
|
@ -81,7 +81,7 @@ public class DruidCoordinatorRuntimeParams
|
|||
@Nullable DataSourcesSnapshot dataSourcesSnapshot,
|
||||
Map<String, LoadQueuePeon> loadManagementPeons,
|
||||
ReplicationThrottler replicationManager,
|
||||
RoundRobinServerSelector roundRobinServerSelector,
|
||||
@Nullable RoundRobinServerSelector roundRobinServerSelector,
|
||||
ServiceEmitter emitter,
|
||||
CoordinatorDynamicConfig coordinatorDynamicConfig,
|
||||
CoordinatorCompactionConfig coordinatorCompactionConfig,
|
||||
|
@ -296,7 +296,7 @@ public class DruidCoordinatorRuntimeParams
|
|||
@Nullable DataSourcesSnapshot dataSourcesSnapshot,
|
||||
Map<String, LoadQueuePeon> loadManagementPeons,
|
||||
ReplicationThrottler replicationManager,
|
||||
RoundRobinServerSelector roundRobinServerSelector,
|
||||
@Nullable RoundRobinServerSelector roundRobinServerSelector,
|
||||
ServiceEmitter emitter,
|
||||
CoordinatorDynamicConfig coordinatorDynamicConfig,
|
||||
CoordinatorCompactionConfig coordinatorCompactionConfig,
|
||||
|
@ -334,7 +334,7 @@ public class DruidCoordinatorRuntimeParams
|
|||
dataSourcesSnapshot,
|
||||
loadManagementPeons,
|
||||
replicationManager,
|
||||
roundRobinServerSelector,
|
||||
getOrCreateRoundRobinServerSelector(),
|
||||
emitter,
|
||||
coordinatorDynamicConfig,
|
||||
coordinatorCompactionConfig,
|
||||
|
@ -344,6 +344,18 @@ public class DruidCoordinatorRuntimeParams
|
|||
);
|
||||
}
|
||||
|
||||
private RoundRobinServerSelector getOrCreateRoundRobinServerSelector()
|
||||
{
|
||||
if (druidCluster == null || coordinatorDynamicConfig == null
|
||||
|| !coordinatorDynamicConfig.isUseRoundRobinSegmentAssignment()) {
|
||||
return null;
|
||||
} else if (roundRobinServerSelector == null) {
|
||||
return new RoundRobinServerSelector(druidCluster);
|
||||
} else {
|
||||
return roundRobinServerSelector;
|
||||
}
|
||||
}
|
||||
|
||||
public Builder withStartTimeNanos(long startTimeNanos)
|
||||
{
|
||||
this.startTimeNanos = startTimeNanos;
|
||||
|
|
|
@ -135,7 +135,9 @@ public class BalanceSegmentsProfiler
|
|||
.addTier("normal", serverHolderList.toArray(new ServerHolder[0]))
|
||||
.build();
|
||||
DruidCoordinatorRuntimeParams params = CoordinatorRuntimeParamsTestHelpers
|
||||
.newBuilder(druidCluster)
|
||||
.newBuilder()
|
||||
.withDruidCluster(druidCluster)
|
||||
.withSegmentReplicantLookup(SegmentReplicantLookup.make(druidCluster, false))
|
||||
.withLoadManagementPeons(peonMap)
|
||||
.withUsedSegmentsInTest(segments)
|
||||
.withDynamicConfigs(
|
||||
|
|
|
@ -28,13 +28,6 @@ public class CoordinatorRuntimeParamsTestHelpers
|
|||
.withStartTimeNanos(System.nanoTime());
|
||||
}
|
||||
|
||||
public static DruidCoordinatorRuntimeParams.Builder newBuilder(DruidCluster druidCluster)
|
||||
{
|
||||
return newBuilder()
|
||||
.withDruidCluster(druidCluster)
|
||||
.withSegmentReplicantLookup(SegmentReplicantLookup.make(druidCluster, false));
|
||||
}
|
||||
|
||||
private CoordinatorRuntimeParamsTestHelpers()
|
||||
{
|
||||
}
|
||||
|
|
|
@ -574,9 +574,11 @@ public class LoadRuleTest
|
|||
mockPeon.loadSegment(EasyMock.anyObject(), EasyMock.anyObject());
|
||||
EasyMock.expectLastCall().atLeastOnce();
|
||||
|
||||
EasyMock.expect(mockBalancerStrategy.findNewSegmentHomeReplicator(EasyMock.anyObject(), EasyMock.anyObject()))
|
||||
.andDelegateTo(balancerStrategy)
|
||||
.times(1);
|
||||
if (!useRoundRobinAssignment) {
|
||||
EasyMock.expect(mockBalancerStrategy.findNewSegmentHomeReplicator(EasyMock.anyObject(), EasyMock.anyObject()))
|
||||
.andDelegateTo(balancerStrategy)
|
||||
.times(1);
|
||||
}
|
||||
|
||||
EasyMock.replay(throttler, mockPeon, mockBalancerStrategy);
|
||||
|
||||
|
@ -596,6 +598,10 @@ public class LoadRuleTest
|
|||
|
||||
final DataSegment segment = createDataSegment("foo");
|
||||
|
||||
final CoordinatorDynamicConfig dynamicConfig =
|
||||
CoordinatorDynamicConfig.builder()
|
||||
.withUseRoundRobinSegmentAssignment(useRoundRobinAssignment)
|
||||
.build();
|
||||
CoordinatorStats stats = rule.run(
|
||||
null,
|
||||
CoordinatorRuntimeParamsTestHelpers
|
||||
|
@ -604,6 +610,7 @@ public class LoadRuleTest
|
|||
.withSegmentReplicantLookup(SegmentReplicantLookup.make(new DruidCluster(), false))
|
||||
.withReplicationManager(throttler)
|
||||
.withBalancerStrategy(mockBalancerStrategy)
|
||||
.withDynamicConfigs(dynamicConfig)
|
||||
.withUsedSegmentsInTest(segment)
|
||||
.build(),
|
||||
segment
|
||||
|
@ -654,9 +661,11 @@ public class LoadRuleTest
|
|||
public void testMaxLoadingQueueSize()
|
||||
{
|
||||
final int maxSegmentsInLoadQueue = 2;
|
||||
EasyMock.expect(mockBalancerStrategy.findNewSegmentHomeReplicator(EasyMock.anyObject(), EasyMock.anyObject()))
|
||||
.andDelegateTo(balancerStrategy)
|
||||
.times(2);
|
||||
if (!useRoundRobinAssignment) {
|
||||
EasyMock.expect(mockBalancerStrategy.findNewSegmentHomeReplicator(EasyMock.anyObject(), EasyMock.anyObject()))
|
||||
.andDelegateTo(balancerStrategy)
|
||||
.times(2);
|
||||
}
|
||||
|
||||
EasyMock.replay(throttler, mockBalancerStrategy);
|
||||
|
||||
|
@ -692,6 +701,7 @@ public class LoadRuleTest
|
|||
.withDynamicConfigs(
|
||||
CoordinatorDynamicConfig.builder()
|
||||
.withMaxSegmentsInNodeLoadingQueue(maxSegmentsInLoadQueue)
|
||||
.withUseRoundRobinSegmentAssignment(useRoundRobinAssignment)
|
||||
.build()
|
||||
).build();
|
||||
|
||||
|
|
Loading…
Reference in New Issue