mirror of https://github.com/apache/druid.git
Do not remove segments from currentlyMovingSegments in DruidBalancer if move is impossible or not needed (#4472)
* Do not remove segment that should not be moved from currentlyMovingSegments (segments are removed by callbacks or not inserted) * Replace putIfAbsent with computeIfAbsent in DruidBalancer * Refactoring
This commit is contained in:
parent
bbe7fb8c46
commit
4dd1e2b59e
|
@ -20,12 +20,10 @@
|
||||||
package io.druid.server.coordinator.helper;
|
package io.druid.server.coordinator.helper;
|
||||||
|
|
||||||
import com.google.common.collect.Lists;
|
import com.google.common.collect.Lists;
|
||||||
import com.google.common.collect.Maps;
|
|
||||||
import com.google.common.collect.MinMaxPriorityQueue;
|
import com.google.common.collect.MinMaxPriorityQueue;
|
||||||
import com.metamx.emitter.EmittingLogger;
|
import com.metamx.emitter.EmittingLogger;
|
||||||
import io.druid.client.ImmutableDruidServer;
|
import io.druid.client.ImmutableDruidServer;
|
||||||
import io.druid.java.util.common.StringUtils;
|
import io.druid.java.util.common.StringUtils;
|
||||||
import io.druid.java.util.common.guava.Comparators;
|
|
||||||
import io.druid.server.coordinator.BalancerSegmentHolder;
|
import io.druid.server.coordinator.BalancerSegmentHolder;
|
||||||
import io.druid.server.coordinator.BalancerStrategy;
|
import io.druid.server.coordinator.BalancerStrategy;
|
||||||
import io.druid.server.coordinator.CoordinatorStats;
|
import io.druid.server.coordinator.CoordinatorStats;
|
||||||
|
@ -37,6 +35,7 @@ import io.druid.server.coordinator.ServerHolder;
|
||||||
import io.druid.timeline.DataSegment;
|
import io.druid.timeline.DataSegment;
|
||||||
|
|
||||||
import java.util.Comparator;
|
import java.util.Comparator;
|
||||||
|
import java.util.HashMap;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.concurrent.ConcurrentHashMap;
|
import java.util.concurrent.ConcurrentHashMap;
|
||||||
|
@ -45,21 +44,15 @@ import java.util.concurrent.ConcurrentHashMap;
|
||||||
*/
|
*/
|
||||||
public class DruidCoordinatorBalancer implements DruidCoordinatorHelper
|
public class DruidCoordinatorBalancer implements DruidCoordinatorHelper
|
||||||
{
|
{
|
||||||
public static final Comparator<ServerHolder> percentUsedComparator = Comparators.inverse(
|
public static final Comparator<ServerHolder> percentUsedComparator =
|
||||||
new Comparator<ServerHolder>()
|
Comparator.comparing(ServerHolder::getPercentUsed).reversed();
|
||||||
{
|
|
||||||
@Override
|
|
||||||
public int compare(ServerHolder lhs, ServerHolder rhs)
|
|
||||||
{
|
|
||||||
return lhs.getPercentUsed().compareTo(rhs.getPercentUsed());
|
|
||||||
}
|
|
||||||
}
|
|
||||||
);
|
|
||||||
protected static final EmittingLogger log = new EmittingLogger(DruidCoordinatorBalancer.class);
|
protected static final EmittingLogger log = new EmittingLogger(DruidCoordinatorBalancer.class);
|
||||||
|
|
||||||
protected final DruidCoordinator coordinator;
|
protected final DruidCoordinator coordinator;
|
||||||
|
|
||||||
protected final Map<String, ConcurrentHashMap<String, BalancerSegmentHolder>> currentlyMovingSegments = Maps.newHashMap();
|
protected final Map<String, ConcurrentHashMap<String, BalancerSegmentHolder>> currentlyMovingSegments =
|
||||||
|
new HashMap<>();
|
||||||
|
|
||||||
public DruidCoordinatorBalancer(
|
public DruidCoordinatorBalancer(
|
||||||
DruidCoordinator coordinator
|
DruidCoordinator coordinator
|
||||||
|
@ -85,28 +78,36 @@ public class DruidCoordinatorBalancer implements DruidCoordinatorHelper
|
||||||
public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params)
|
public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params)
|
||||||
{
|
{
|
||||||
final CoordinatorStats stats = new CoordinatorStats();
|
final CoordinatorStats stats = new CoordinatorStats();
|
||||||
|
params.getDruidCluster().getHistoricals().forEach((String tier, MinMaxPriorityQueue<ServerHolder> servers) -> {
|
||||||
|
balanceTier(params, tier, servers, stats);
|
||||||
|
});
|
||||||
|
return params.buildFromExisting().withCoordinatorStats(stats).build();
|
||||||
|
}
|
||||||
|
|
||||||
|
private void balanceTier(
|
||||||
|
DruidCoordinatorRuntimeParams params,
|
||||||
|
String tier,
|
||||||
|
MinMaxPriorityQueue<ServerHolder> servers,
|
||||||
|
CoordinatorStats stats
|
||||||
|
)
|
||||||
|
{
|
||||||
final BalancerStrategy strategy = params.getBalancerStrategy();
|
final BalancerStrategy strategy = params.getBalancerStrategy();
|
||||||
final int maxSegmentsToMove = params.getCoordinatorDynamicConfig().getMaxSegmentsToMove();
|
final int maxSegmentsToMove = params.getCoordinatorDynamicConfig().getMaxSegmentsToMove();
|
||||||
|
|
||||||
for (Map.Entry<String, MinMaxPriorityQueue<ServerHolder>> entry :
|
currentlyMovingSegments.computeIfAbsent(tier, t -> new ConcurrentHashMap<>());
|
||||||
params.getDruidCluster().getHistoricals().entrySet()) {
|
|
||||||
String tier = entry.getKey();
|
|
||||||
|
|
||||||
if (currentlyMovingSegments.get(tier) == null) {
|
|
||||||
currentlyMovingSegments.put(tier, new ConcurrentHashMap<String, BalancerSegmentHolder>());
|
|
||||||
}
|
|
||||||
|
|
||||||
if (!currentlyMovingSegments.get(tier).isEmpty()) {
|
if (!currentlyMovingSegments.get(tier).isEmpty()) {
|
||||||
reduceLifetimes(tier);
|
reduceLifetimes(tier);
|
||||||
log.info("[%s]: Still waiting on %,d segments to be moved", tier, currentlyMovingSegments.size());
|
log.info("[%s]: Still waiting on %,d segments to be moved", tier, currentlyMovingSegments.size());
|
||||||
continue;
|
return;
|
||||||
}
|
}
|
||||||
|
|
||||||
final List<ServerHolder> serverHolderList = Lists.newArrayList(entry.getValue());
|
|
||||||
|
final List<ServerHolder> serverHolderList = Lists.newArrayList(servers);
|
||||||
|
|
||||||
if (serverHolderList.size() <= 1) {
|
if (serverHolderList.size() <= 1) {
|
||||||
log.info("[%s]: One or fewer servers found. Cannot balance.", tier);
|
log.info("[%s]: One or fewer servers found. Cannot balance.", tier);
|
||||||
continue;
|
return;
|
||||||
}
|
}
|
||||||
|
|
||||||
int numSegments = 0;
|
int numSegments = 0;
|
||||||
|
@ -116,7 +117,7 @@ public class DruidCoordinatorBalancer implements DruidCoordinatorHelper
|
||||||
|
|
||||||
if (numSegments == 0) {
|
if (numSegments == 0) {
|
||||||
log.info("No segments found. Cannot balance.");
|
log.info("No segments found. Cannot balance.");
|
||||||
continue;
|
return;
|
||||||
}
|
}
|
||||||
long unmoved = 0L;
|
long unmoved = 0L;
|
||||||
for (int iter = 0; iter < maxSegmentsToMove; iter++) {
|
for (int iter = 0; iter < maxSegmentsToMove; iter++) {
|
||||||
|
@ -147,12 +148,6 @@ public class DruidCoordinatorBalancer implements DruidCoordinatorHelper
|
||||||
currentlyMovingSegments.get(tier).size(),
|
currentlyMovingSegments.get(tier).size(),
|
||||||
unmoved
|
unmoved
|
||||||
);
|
);
|
||||||
|
|
||||||
}
|
|
||||||
|
|
||||||
return params.buildFromExisting()
|
|
||||||
.withCoordinatorStats(stats)
|
|
||||||
.build();
|
|
||||||
}
|
}
|
||||||
|
|
||||||
protected void moveSegment(
|
protected void moveSegment(
|
||||||
|
@ -174,18 +169,9 @@ public class DruidCoordinatorBalancer implements DruidCoordinatorHelper
|
||||||
|
|
||||||
LoadPeonCallback callback = null;
|
LoadPeonCallback callback = null;
|
||||||
try {
|
try {
|
||||||
currentlyMovingSegments.get(toServer.getTier()).put(segmentName, segment);
|
|
||||||
callback = new LoadPeonCallback()
|
|
||||||
{
|
|
||||||
@Override
|
|
||||||
public void execute()
|
|
||||||
{
|
|
||||||
Map<String, BalancerSegmentHolder> movingSegments = currentlyMovingSegments.get(toServer.getTier());
|
Map<String, BalancerSegmentHolder> movingSegments = currentlyMovingSegments.get(toServer.getTier());
|
||||||
if (movingSegments != null) {
|
movingSegments.put(segmentName, segment);
|
||||||
movingSegments.remove(segmentName);
|
callback = () -> movingSegments.remove(segmentName);
|
||||||
}
|
|
||||||
}
|
|
||||||
};
|
|
||||||
coordinator.moveSegment(
|
coordinator.moveSegment(
|
||||||
fromServer,
|
fromServer,
|
||||||
toServer,
|
toServer,
|
||||||
|
@ -199,9 +185,6 @@ public class DruidCoordinatorBalancer implements DruidCoordinatorHelper
|
||||||
callback.execute();
|
callback.execute();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
} else {
|
}
|
||||||
currentlyMovingSegments.get(toServer.getTier()).remove(segmentName);
|
|
||||||
}
|
|
||||||
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -19,6 +19,7 @@
|
||||||
|
|
||||||
package io.druid.server.coordinator;
|
package io.druid.server.coordinator;
|
||||||
|
|
||||||
|
import com.google.common.collect.ImmutableList;
|
||||||
import com.google.common.collect.ImmutableMap;
|
import com.google.common.collect.ImmutableMap;
|
||||||
import com.google.common.collect.Lists;
|
import com.google.common.collect.Lists;
|
||||||
import com.google.common.collect.Maps;
|
import com.google.common.collect.Maps;
|
||||||
|
@ -37,10 +38,14 @@ import org.junit.Before;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.Arrays;
|
import java.util.Collections;
|
||||||
import java.util.HashMap;
|
import java.util.HashMap;
|
||||||
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.concurrent.Executors;
|
import java.util.concurrent.Executors;
|
||||||
|
import java.util.concurrent.atomic.AtomicInteger;
|
||||||
|
import java.util.stream.Collectors;
|
||||||
|
import java.util.stream.IntStream;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
*/
|
*/
|
||||||
|
@ -52,11 +57,19 @@ public class DruidCoordinatorBalancerTest
|
||||||
private ImmutableDruidServer druidServer2;
|
private ImmutableDruidServer druidServer2;
|
||||||
private ImmutableDruidServer druidServer3;
|
private ImmutableDruidServer druidServer3;
|
||||||
private ImmutableDruidServer druidServer4;
|
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 segment1;
|
||||||
private DataSegment segment2;
|
private DataSegment segment2;
|
||||||
private DataSegment segment3;
|
private DataSegment segment3;
|
||||||
private DataSegment segment4;
|
private DataSegment segment4;
|
||||||
Map<String, DataSegment> segments;
|
private Map<String, DataSegment> segments;
|
||||||
|
private ListeningExecutorService balancerStrategyExecutor;
|
||||||
|
private BalancerStrategy balancerStrategy;
|
||||||
|
|
||||||
@Before
|
@Before
|
||||||
public void setUp() throws Exception
|
public void setUp() throws Exception
|
||||||
|
@ -78,9 +91,9 @@ public class DruidCoordinatorBalancerTest
|
||||||
"datasource1",
|
"datasource1",
|
||||||
new Interval(start1, start1.plusHours(1)),
|
new Interval(start1, start1.plusHours(1)),
|
||||||
version.toString(),
|
version.toString(),
|
||||||
Maps.<String, Object>newHashMap(),
|
Maps.newHashMap(),
|
||||||
Lists.<String>newArrayList(),
|
Lists.newArrayList(),
|
||||||
Lists.<String>newArrayList(),
|
Lists.newArrayList(),
|
||||||
NoneShardSpec.instance(),
|
NoneShardSpec.instance(),
|
||||||
0,
|
0,
|
||||||
11L
|
11L
|
||||||
|
@ -89,9 +102,9 @@ public class DruidCoordinatorBalancerTest
|
||||||
"datasource1",
|
"datasource1",
|
||||||
new Interval(start2, start2.plusHours(1)),
|
new Interval(start2, start2.plusHours(1)),
|
||||||
version.toString(),
|
version.toString(),
|
||||||
Maps.<String, Object>newHashMap(),
|
Maps.newHashMap(),
|
||||||
Lists.<String>newArrayList(),
|
Lists.newArrayList(),
|
||||||
Lists.<String>newArrayList(),
|
Lists.newArrayList(),
|
||||||
NoneShardSpec.instance(),
|
NoneShardSpec.instance(),
|
||||||
0,
|
0,
|
||||||
7L
|
7L
|
||||||
|
@ -100,9 +113,9 @@ public class DruidCoordinatorBalancerTest
|
||||||
"datasource2",
|
"datasource2",
|
||||||
new Interval(start1, start1.plusHours(1)),
|
new Interval(start1, start1.plusHours(1)),
|
||||||
version.toString(),
|
version.toString(),
|
||||||
Maps.<String, Object>newHashMap(),
|
Maps.newHashMap(),
|
||||||
Lists.<String>newArrayList(),
|
Lists.newArrayList(),
|
||||||
Lists.<String>newArrayList(),
|
Lists.newArrayList(),
|
||||||
NoneShardSpec.instance(),
|
NoneShardSpec.instance(),
|
||||||
0,
|
0,
|
||||||
4L
|
4L
|
||||||
|
@ -111,19 +124,30 @@ public class DruidCoordinatorBalancerTest
|
||||||
"datasource2",
|
"datasource2",
|
||||||
new Interval(start2, start2.plusHours(1)),
|
new Interval(start2, start2.plusHours(1)),
|
||||||
version.toString(),
|
version.toString(),
|
||||||
Maps.<String, Object>newHashMap(),
|
Maps.newHashMap(),
|
||||||
Lists.<String>newArrayList(),
|
Lists.newArrayList(),
|
||||||
Lists.<String>newArrayList(),
|
Lists.newArrayList(),
|
||||||
NoneShardSpec.instance(),
|
NoneShardSpec.instance(),
|
||||||
0,
|
0,
|
||||||
8L
|
8L
|
||||||
);
|
);
|
||||||
|
|
||||||
segments = new HashMap<String, DataSegment>();
|
segments = new HashMap<>();
|
||||||
segments.put("datasource1_2012-01-01T00:00:00.000Z_2012-01-01T01:00:00.000Z_2012-03-01T00:00:00.000Z", segment1);
|
segments.put("datasource1_2012-01-01T00:00:00.000Z_2012-01-01T01:00:00.000Z_2012-03-01T00:00:00.000Z", segment1);
|
||||||
segments.put("datasource1_2012-02-01T00:00:00.000Z_2012-02-01T01:00:00.000Z_2012-03-01T00:00:00.000Z", segment2);
|
segments.put("datasource1_2012-02-01T00:00:00.000Z_2012-02-01T01:00:00.000Z_2012-03-01T00:00:00.000Z", segment2);
|
||||||
segments.put("datasource2_2012-01-01T00:00:00.000Z_2012-01-01T01:00:00.000Z_2012-03-01T00:00:00.000Z", segment3);
|
segments.put("datasource2_2012-01-01T00:00:00.000Z_2012-01-01T01:00:00.000Z_2012-03-01T00:00:00.000Z", segment3);
|
||||||
segments.put("datasource2_2012-02-01T00:00:00.000Z_2012-02-01T01:00:00.000Z_2012-03-01T00:00:00.000Z", segment4);
|
segments.put("datasource2_2012-02-01T00:00:00.000Z_2012-02-01T01:00:00.000Z_2012-03-01T00:00:00.000Z", segment4);
|
||||||
|
|
||||||
|
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(Executors.newFixedThreadPool(1));
|
||||||
|
balancerStrategy = new CostBalancerStrategyFactory().createBalancerStrategy(balancerStrategyExecutor);
|
||||||
}
|
}
|
||||||
|
|
||||||
@After
|
@After
|
||||||
|
@ -134,166 +158,87 @@ public class DruidCoordinatorBalancerTest
|
||||||
EasyMock.verify(druidServer2);
|
EasyMock.verify(druidServer2);
|
||||||
EasyMock.verify(druidServer3);
|
EasyMock.verify(druidServer3);
|
||||||
EasyMock.verify(druidServer4);
|
EasyMock.verify(druidServer4);
|
||||||
|
balancerStrategyExecutor.shutdownNow();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testMoveToEmptyServerBalancer() throws IOException
|
public void testMoveToEmptyServerBalancer() throws IOException
|
||||||
{
|
{
|
||||||
EasyMock.expect(druidServer1.getName()).andReturn("from").atLeastOnce();
|
mockDruidServer(druidServer1, "1", "normal", 30L, 100L, segments);
|
||||||
EasyMock.expect(druidServer1.getCurrSize()).andReturn(30L).atLeastOnce();
|
mockDruidServer(druidServer2, "2", "normal", 0L, 100L, Collections.emptyMap());
|
||||||
EasyMock.expect(druidServer1.getMaxSize()).andReturn(100L).atLeastOnce();
|
|
||||||
EasyMock.expect(druidServer1.getSegments()).andReturn(segments).anyTimes();
|
|
||||||
EasyMock.expect(druidServer1.getSegment(EasyMock.<String>anyObject())).andReturn(null).anyTimes();
|
|
||||||
EasyMock.replay(druidServer1);
|
|
||||||
|
|
||||||
EasyMock.expect(druidServer2.getName()).andReturn("to").atLeastOnce();
|
|
||||||
EasyMock.expect(druidServer2.getTier()).andReturn("normal").anyTimes();
|
|
||||||
EasyMock.expect(druidServer2.getCurrSize()).andReturn(0L).atLeastOnce();
|
|
||||||
EasyMock.expect(druidServer2.getMaxSize()).andReturn(100L).atLeastOnce();
|
|
||||||
EasyMock.expect(druidServer2.getSegments()).andReturn(new HashMap<String, DataSegment>()).anyTimes();
|
|
||||||
EasyMock.expect(druidServer2.getSegment(EasyMock.<String>anyObject())).andReturn(null).anyTimes();
|
|
||||||
EasyMock.replay(druidServer2);
|
|
||||||
|
|
||||||
EasyMock.replay(druidServer3);
|
EasyMock.replay(druidServer3);
|
||||||
EasyMock.replay(druidServer4);
|
EasyMock.replay(druidServer4);
|
||||||
|
|
||||||
// Mock stuff that the coordinator needs
|
// Mock stuff that the coordinator needs
|
||||||
coordinator.moveSegment(
|
mockCoordinator(coordinator);
|
||||||
EasyMock.<ImmutableDruidServer>anyObject(),
|
|
||||||
EasyMock.<ImmutableDruidServer>anyObject(),
|
|
||||||
EasyMock.<DataSegment>anyObject(),
|
|
||||||
EasyMock.<LoadPeonCallback>anyObject()
|
|
||||||
);
|
|
||||||
EasyMock.expectLastCall().anyTimes();
|
|
||||||
EasyMock.replay(coordinator);
|
|
||||||
|
|
||||||
LoadQueuePeonTester fromPeon = new LoadQueuePeonTester();
|
DruidCoordinatorRuntimeParams params = defaullRuntimeParamsBuilder(
|
||||||
LoadQueuePeonTester toPeon = new LoadQueuePeonTester();
|
ImmutableList.of(druidServer1, druidServer2),
|
||||||
ListeningExecutorService exec = MoreExecutors.listeningDecorator(
|
ImmutableList.of(peon1, peon2)
|
||||||
Executors.newFixedThreadPool(1));
|
).build();
|
||||||
BalancerStrategy balancerStrategy =
|
|
||||||
new CostBalancerStrategyFactory().createBalancerStrategy(exec);
|
|
||||||
|
|
||||||
DruidCoordinatorRuntimeParams params =
|
|
||||||
DruidCoordinatorRuntimeParams.newBuilder()
|
|
||||||
.withDruidCluster(
|
|
||||||
new DruidCluster(
|
|
||||||
null,
|
|
||||||
ImmutableMap.<String, MinMaxPriorityQueue<ServerHolder>>of(
|
|
||||||
"normal",
|
|
||||||
MinMaxPriorityQueue.orderedBy(DruidCoordinatorBalancerTester.percentUsedComparator)
|
|
||||||
.create(
|
|
||||||
Arrays.asList(
|
|
||||||
new ServerHolder(druidServer1, fromPeon),
|
|
||||||
new ServerHolder(druidServer2, toPeon)
|
|
||||||
)
|
|
||||||
)
|
|
||||||
)
|
|
||||||
)
|
|
||||||
)
|
|
||||||
.withLoadManagementPeons(
|
|
||||||
ImmutableMap.<String, LoadQueuePeon>of(
|
|
||||||
"from",
|
|
||||||
fromPeon,
|
|
||||||
"to",
|
|
||||||
toPeon
|
|
||||||
)
|
|
||||||
)
|
|
||||||
.withAvailableSegments(segments.values())
|
|
||||||
.withDynamicConfigs(
|
|
||||||
new CoordinatorDynamicConfig.Builder().withMaxSegmentsToMove(
|
|
||||||
MAX_SEGMENTS_TO_MOVE
|
|
||||||
).build()
|
|
||||||
)
|
|
||||||
.withBalancerStrategy(balancerStrategy)
|
|
||||||
.withBalancerReferenceTimestamp(new DateTime("2013-01-01"))
|
|
||||||
.build();
|
|
||||||
|
|
||||||
params = new DruidCoordinatorBalancerTester(coordinator).run(params);
|
params = new DruidCoordinatorBalancerTester(coordinator).run(params);
|
||||||
Assert.assertTrue(params.getCoordinatorStats().getTieredStat("movedCount", "normal") > 0);
|
Assert.assertTrue(params.getCoordinatorStats().getTieredStat("movedCount", "normal") > 0);
|
||||||
Assert.assertTrue(params.getCoordinatorStats().getTieredStat("movedCount", "normal") < segments.size());
|
Assert.assertTrue(params.getCoordinatorStats().getTieredStat("movedCount", "normal") < segments.size());
|
||||||
exec.shutdown();
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testMoveSameSegmentTwice() throws Exception
|
||||||
|
{
|
||||||
|
mockDruidServer(druidServer1, "1", "normal", 30L, 100L, segments);
|
||||||
|
mockDruidServer(druidServer2, "2", "normal", 0L, 100L, Collections.emptyMap());
|
||||||
|
|
||||||
|
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 = defaullRuntimeParamsBuilder(
|
||||||
|
ImmutableList.of(druidServer1, druidServer2),
|
||||||
|
ImmutableList.of(peon1, peon2)
|
||||||
|
)
|
||||||
|
.withBalancerStrategy(predefinedPickOrderStrategy)
|
||||||
|
.withDynamicConfigs(
|
||||||
|
new CoordinatorDynamicConfig.Builder().withMaxSegmentsToMove(
|
||||||
|
2
|
||||||
|
).build()
|
||||||
|
)
|
||||||
|
.build();
|
||||||
|
|
||||||
|
params = new DruidCoordinatorBalancerTester(coordinator).run(params);
|
||||||
|
Assert.assertEquals(1, params.getCoordinatorStats().getTieredStat("movedCount", "normal"));
|
||||||
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testRun1() throws IOException
|
public void testRun1() throws IOException
|
||||||
{
|
{
|
||||||
// Mock some servers of different usages
|
// Mock some servers of different usages
|
||||||
|
mockDruidServer(druidServer1, "1", "normal", 30L, 100L, segments);
|
||||||
EasyMock.expect(druidServer1.getName()).andReturn("from").atLeastOnce();
|
mockDruidServer(druidServer2, "2", "normal", 0L, 100L, Collections.emptyMap());
|
||||||
EasyMock.expect(druidServer1.getCurrSize()).andReturn(30L).atLeastOnce();
|
|
||||||
EasyMock.expect(druidServer1.getMaxSize()).andReturn(100L).atLeastOnce();
|
|
||||||
EasyMock.expect(druidServer1.getSegments()).andReturn(segments).anyTimes();
|
|
||||||
EasyMock.expect(druidServer1.getSegment(EasyMock.<String>anyObject())).andReturn(null).anyTimes();
|
|
||||||
EasyMock.replay(druidServer1);
|
|
||||||
|
|
||||||
EasyMock.expect(druidServer2.getName()).andReturn("to").atLeastOnce();
|
|
||||||
EasyMock.expect(druidServer2.getTier()).andReturn("normal").anyTimes();
|
|
||||||
EasyMock.expect(druidServer2.getCurrSize()).andReturn(0L).atLeastOnce();
|
|
||||||
EasyMock.expect(druidServer2.getMaxSize()).andReturn(100L).atLeastOnce();
|
|
||||||
EasyMock.expect(druidServer2.getSegments()).andReturn(new HashMap<String, DataSegment>()).anyTimes();
|
|
||||||
EasyMock.expect(druidServer2.getSegment(EasyMock.<String>anyObject())).andReturn(null).anyTimes();
|
|
||||||
EasyMock.replay(druidServer2);
|
|
||||||
|
|
||||||
EasyMock.replay(druidServer3);
|
EasyMock.replay(druidServer3);
|
||||||
EasyMock.replay(druidServer4);
|
EasyMock.replay(druidServer4);
|
||||||
|
|
||||||
// Mock stuff that the coordinator needs
|
// Mock stuff that the coordinator needs
|
||||||
coordinator.moveSegment(
|
mockCoordinator(coordinator);
|
||||||
EasyMock.<ImmutableDruidServer>anyObject(),
|
|
||||||
EasyMock.<ImmutableDruidServer>anyObject(),
|
|
||||||
EasyMock.<DataSegment>anyObject(),
|
|
||||||
EasyMock.<LoadPeonCallback>anyObject()
|
|
||||||
);
|
|
||||||
EasyMock.expectLastCall().anyTimes();
|
|
||||||
EasyMock.replay(coordinator);
|
|
||||||
|
|
||||||
ListeningExecutorService exec = MoreExecutors.listeningDecorator(
|
DruidCoordinatorRuntimeParams params = defaullRuntimeParamsBuilder(
|
||||||
Executors.newFixedThreadPool(1));
|
ImmutableList.of(druidServer1, druidServer2),
|
||||||
BalancerStrategy balancerStrategy =
|
ImmutableList.of(peon1, peon2)
|
||||||
new CostBalancerStrategyFactory().createBalancerStrategy(exec);
|
).build();
|
||||||
|
|
||||||
LoadQueuePeonTester fromPeon = new LoadQueuePeonTester();
|
|
||||||
LoadQueuePeonTester toPeon = new LoadQueuePeonTester();
|
|
||||||
DruidCoordinatorRuntimeParams params =
|
|
||||||
DruidCoordinatorRuntimeParams.newBuilder()
|
|
||||||
.withDruidCluster(
|
|
||||||
new DruidCluster(
|
|
||||||
null,
|
|
||||||
ImmutableMap.<String, MinMaxPriorityQueue<ServerHolder>>of(
|
|
||||||
"normal",
|
|
||||||
MinMaxPriorityQueue.orderedBy(DruidCoordinatorBalancerTester.percentUsedComparator)
|
|
||||||
.create(
|
|
||||||
Arrays.asList(
|
|
||||||
new ServerHolder(druidServer1, fromPeon),
|
|
||||||
new ServerHolder(druidServer2, toPeon)
|
|
||||||
)
|
|
||||||
)
|
|
||||||
)
|
|
||||||
)
|
|
||||||
)
|
|
||||||
.withLoadManagementPeons(
|
|
||||||
ImmutableMap.<String, LoadQueuePeon>of(
|
|
||||||
"from",
|
|
||||||
fromPeon,
|
|
||||||
"to",
|
|
||||||
toPeon
|
|
||||||
)
|
|
||||||
)
|
|
||||||
.withAvailableSegments(segments.values())
|
|
||||||
.withDynamicConfigs(
|
|
||||||
new CoordinatorDynamicConfig.Builder().withMaxSegmentsToMove(MAX_SEGMENTS_TO_MOVE)
|
|
||||||
.build()
|
|
||||||
)
|
|
||||||
.withBalancerStrategy(balancerStrategy)
|
|
||||||
.withBalancerReferenceTimestamp(new DateTime("2013-01-01"))
|
|
||||||
.build();
|
|
||||||
|
|
||||||
params = new DruidCoordinatorBalancerTester(coordinator).run(params);
|
params = new DruidCoordinatorBalancerTester(coordinator).run(params);
|
||||||
Assert.assertTrue(params.getCoordinatorStats().getTieredStat("movedCount", "normal") > 0);
|
Assert.assertTrue(params.getCoordinatorStats().getTieredStat("movedCount", "normal") > 0);
|
||||||
exec.shutdown();
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
@ -301,87 +246,47 @@ public class DruidCoordinatorBalancerTest
|
||||||
public void testRun2() throws IOException
|
public void testRun2() throws IOException
|
||||||
{
|
{
|
||||||
// Mock some servers of different usages
|
// Mock some servers of different usages
|
||||||
EasyMock.expect(druidServer1.getName()).andReturn("1").atLeastOnce();
|
mockDruidServer(druidServer1, "1", "normal", 30L, 100L, segments);
|
||||||
EasyMock.expect(druidServer1.getCurrSize()).andReturn(30L).atLeastOnce();
|
mockDruidServer(druidServer2, "2", "normal", 0L, 100L, Collections.emptyMap());
|
||||||
EasyMock.expect(druidServer1.getMaxSize()).andReturn(100L).atLeastOnce();
|
mockDruidServer(druidServer3, "3", "normal", 0L, 100L, Collections.emptyMap());
|
||||||
EasyMock.expect(druidServer1.getSegments()).andReturn(segments).anyTimes();
|
mockDruidServer(druidServer4, "4", "normal", 0L, 100L, Collections.emptyMap());
|
||||||
EasyMock.expect(druidServer1.getSegment(EasyMock.<String>anyObject())).andReturn(null).anyTimes();
|
|
||||||
EasyMock.replay(druidServer1);
|
|
||||||
|
|
||||||
EasyMock.expect(druidServer2.getName()).andReturn("2").atLeastOnce();
|
|
||||||
EasyMock.expect(druidServer2.getTier()).andReturn("normal").anyTimes();
|
|
||||||
EasyMock.expect(druidServer2.getCurrSize()).andReturn(0L).atLeastOnce();
|
|
||||||
EasyMock.expect(druidServer2.getMaxSize()).andReturn(100L).atLeastOnce();
|
|
||||||
EasyMock.expect(druidServer2.getSegments()).andReturn(new HashMap<String, DataSegment>()).anyTimes();
|
|
||||||
EasyMock.expect(druidServer2.getSegment(EasyMock.<String>anyObject())).andReturn(null).anyTimes();
|
|
||||||
EasyMock.replay(druidServer2);
|
|
||||||
|
|
||||||
EasyMock.expect(druidServer3.getName()).andReturn("3").atLeastOnce();
|
|
||||||
EasyMock.expect(druidServer3.getTier()).andReturn("normal").anyTimes();
|
|
||||||
EasyMock.expect(druidServer3.getCurrSize()).andReturn(0L).atLeastOnce();
|
|
||||||
EasyMock.expect(druidServer3.getMaxSize()).andReturn(100L).atLeastOnce();
|
|
||||||
EasyMock.expect(druidServer3.getSegments()).andReturn(new HashMap<String, DataSegment>()).anyTimes();
|
|
||||||
EasyMock.expect(druidServer3.getSegment(EasyMock.<String>anyObject())).andReturn(null).anyTimes();
|
|
||||||
EasyMock.replay(druidServer3);
|
|
||||||
|
|
||||||
EasyMock.expect(druidServer4.getName()).andReturn("4").atLeastOnce();
|
|
||||||
EasyMock.expect(druidServer4.getTier()).andReturn("normal").anyTimes();
|
|
||||||
EasyMock.expect(druidServer4.getCurrSize()).andReturn(0L).atLeastOnce();
|
|
||||||
EasyMock.expect(druidServer4.getMaxSize()).andReturn(100L).atLeastOnce();
|
|
||||||
EasyMock.expect(druidServer4.getSegments()).andReturn(new HashMap<String, DataSegment>()).anyTimes();
|
|
||||||
EasyMock.expect(druidServer4.getSegment(EasyMock.<String>anyObject())).andReturn(null).anyTimes();
|
|
||||||
EasyMock.replay(druidServer4);
|
|
||||||
|
|
||||||
// Mock stuff that the coordinator needs
|
// Mock stuff that the coordinator needs
|
||||||
coordinator.moveSegment(
|
mockCoordinator(coordinator);
|
||||||
EasyMock.<ImmutableDruidServer>anyObject(),
|
|
||||||
EasyMock.<ImmutableDruidServer>anyObject(),
|
|
||||||
EasyMock.<DataSegment>anyObject(),
|
|
||||||
EasyMock.<LoadPeonCallback>anyObject()
|
|
||||||
);
|
|
||||||
EasyMock.expectLastCall().anyTimes();
|
|
||||||
EasyMock.replay(coordinator);
|
|
||||||
|
|
||||||
LoadQueuePeonTester peon1 = new LoadQueuePeonTester();
|
DruidCoordinatorRuntimeParams params = defaullRuntimeParamsBuilder(druidServers, peons).build();
|
||||||
LoadQueuePeonTester peon2 = new LoadQueuePeonTester();
|
|
||||||
LoadQueuePeonTester peon3 = new LoadQueuePeonTester();
|
|
||||||
LoadQueuePeonTester peon4 = new LoadQueuePeonTester();
|
|
||||||
|
|
||||||
ListeningExecutorService exec = MoreExecutors.listeningDecorator(
|
params = new DruidCoordinatorBalancerTester(coordinator).run(params);
|
||||||
Executors.newFixedThreadPool(1));
|
Assert.assertTrue(params.getCoordinatorStats().getTieredStat("movedCount", "normal") > 0);
|
||||||
BalancerStrategy balancerStrategy =
|
}
|
||||||
new CostBalancerStrategyFactory().createBalancerStrategy(exec);
|
|
||||||
|
|
||||||
DruidCoordinatorRuntimeParams params =
|
private DruidCoordinatorRuntimeParams.Builder defaullRuntimeParamsBuilder(
|
||||||
DruidCoordinatorRuntimeParams.newBuilder()
|
List<ImmutableDruidServer> druidServers,
|
||||||
|
List<LoadQueuePeon> peons
|
||||||
|
)
|
||||||
|
{
|
||||||
|
return DruidCoordinatorRuntimeParams
|
||||||
|
.newBuilder()
|
||||||
.withDruidCluster(
|
.withDruidCluster(
|
||||||
new DruidCluster(
|
new DruidCluster(
|
||||||
null,
|
null,
|
||||||
ImmutableMap.<String, MinMaxPriorityQueue<ServerHolder>>of(
|
ImmutableMap.of(
|
||||||
"normal",
|
"normal",
|
||||||
MinMaxPriorityQueue.orderedBy(DruidCoordinatorBalancerTester.percentUsedComparator)
|
MinMaxPriorityQueue.orderedBy(DruidCoordinatorBalancerTester.percentUsedComparator)
|
||||||
.create(
|
.create(
|
||||||
Arrays.asList(
|
IntStream
|
||||||
new ServerHolder(druidServer1, peon1),
|
.range(0, druidServers.size())
|
||||||
new ServerHolder(druidServer2, peon2),
|
.mapToObj(i -> new ServerHolder(druidServers.get(i), peons.get(i)))
|
||||||
new ServerHolder(druidServer3, peon3),
|
.collect(Collectors.toList())
|
||||||
new ServerHolder(druidServer4, peon4)
|
|
||||||
)
|
|
||||||
)
|
)
|
||||||
)
|
)
|
||||||
)
|
)
|
||||||
)
|
)
|
||||||
.withLoadManagementPeons(
|
.withLoadManagementPeons(
|
||||||
ImmutableMap.<String, LoadQueuePeon>of(
|
IntStream
|
||||||
"1",
|
.range(0, peons.size())
|
||||||
peon1,
|
.boxed()
|
||||||
"2",
|
.collect(Collectors.toMap(i -> String.valueOf(i + 1), peons::get))
|
||||||
peon2,
|
|
||||||
"3",
|
|
||||||
peon3,
|
|
||||||
"4",
|
|
||||||
peon4
|
|
||||||
)
|
|
||||||
)
|
)
|
||||||
.withAvailableSegments(segments.values())
|
.withAvailableSegments(segments.values())
|
||||||
.withDynamicConfigs(
|
.withDynamicConfigs(
|
||||||
|
@ -390,12 +295,83 @@ public class DruidCoordinatorBalancerTest
|
||||||
).build()
|
).build()
|
||||||
)
|
)
|
||||||
.withBalancerStrategy(balancerStrategy)
|
.withBalancerStrategy(balancerStrategy)
|
||||||
.withBalancerReferenceTimestamp(new DateTime("2013-01-01"))
|
.withBalancerReferenceTimestamp(new DateTime("2013-01-01"));
|
||||||
.build();
|
}
|
||||||
|
|
||||||
params = new DruidCoordinatorBalancerTester(coordinator).run(params);
|
private void mockDruidServer(
|
||||||
Assert.assertTrue(params.getCoordinatorStats().getTieredStat("movedCount", "normal") > 0);
|
ImmutableDruidServer druidServer,
|
||||||
exec.shutdown();
|
String name,
|
||||||
|
String tier,
|
||||||
|
long currentSize,
|
||||||
|
long maxSize,
|
||||||
|
Map<String, DataSegment> segments
|
||||||
|
)
|
||||||
|
{
|
||||||
|
EasyMock.expect(druidServer.getName()).andReturn(name).atLeastOnce();
|
||||||
|
EasyMock.expect(druidServer.getTier()).andReturn(tier).anyTimes();
|
||||||
|
EasyMock.expect(druidServer.getCurrSize()).andReturn(currentSize).atLeastOnce();
|
||||||
|
EasyMock.expect(druidServer.getMaxSize()).andReturn(maxSize).atLeastOnce();
|
||||||
|
EasyMock.expect(druidServer.getSegments()).andReturn(segments).anyTimes();
|
||||||
|
EasyMock.expect(druidServer.getSegment(EasyMock.anyObject())).andReturn(null).anyTimes();
|
||||||
|
EasyMock.replay(druidServer);
|
||||||
|
}
|
||||||
|
|
||||||
|
private void mockCoordinator(DruidCoordinator coordinator)
|
||||||
|
{
|
||||||
|
coordinator.moveSegment(
|
||||||
|
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);
|
||||||
|
|
||||||
|
public 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 BalancerSegmentHolder pickSegmentToMove(List<ServerHolder> serverHolders)
|
||||||
|
{
|
||||||
|
return pickOrder.get(pickCounter.getAndIncrement() % pickOrder.size());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void emitStats(
|
||||||
|
String tier, CoordinatorStats stats, List<ServerHolder> serverHolderList
|
||||||
|
)
|
||||||
|
{
|
||||||
|
delegate.emitStats(tier, stats, serverHolderList);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -70,8 +70,6 @@ public class DruidCoordinatorBalancerTester extends DruidCoordinatorBalancer
|
||||||
catch (Exception e) {
|
catch (Exception e) {
|
||||||
log.info(e, StringUtils.format("[%s] : Moving exception", segmentName));
|
log.info(e, StringUtils.format("[%s] : Moving exception", segmentName));
|
||||||
}
|
}
|
||||||
} else {
|
|
||||||
currentlyMovingSegments.get("normal").remove(segmentName);
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
Loading…
Reference in New Issue