Add tests for CostBalancerStrategy (#14230)

Changes:
- `CostBalancerStrategyTest`
  - Focus on verification of cost computations rather than choosing servers in this test
  - Add new tests `testComputeCost` and `testJointSegmentsCost`
  - Add tests to demonstrate that with a long enough interval gap, all costs become negligible
  - Retain `testIntervalCost` and `testIntervalCostAdditivity`
  - Remove redundant tests such as `testStrategyMultiThreaded`, `testStrategySingleThreaded`as
verification of this behaviour is better suited to `BalancingStrategiesTest`.
- `CostBalancerStrategyBenchmark`
  - Remove usage of static method from `CostBalancerStrategyTest`
  - Explicitly setup cluster and segments to use for benchmarking
This commit is contained in:
Kashif Faraz 2023-05-30 08:52:56 +05:30 committed by GitHub
parent 8091c6a547
commit d4cacebf79
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
3 changed files with 352 additions and 196 deletions

View File

@ -22,19 +22,23 @@ package org.apache.druid.server.coordinator;
import com.carrotsearch.junitbenchmarks.AbstractBenchmark; import com.carrotsearch.junitbenchmarks.AbstractBenchmark;
import com.carrotsearch.junitbenchmarks.BenchmarkOptions; import com.carrotsearch.junitbenchmarks.BenchmarkOptions;
import com.google.common.util.concurrent.MoreExecutors; import com.google.common.util.concurrent.MoreExecutors;
import org.apache.druid.client.DruidServer;
import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.Intervals;
import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.concurrent.Execs;
import org.apache.druid.java.util.common.granularity.Granularities;
import org.apache.druid.server.coordination.ServerType;
import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.DataSegment;
import org.joda.time.Interval; import org.joda.time.Interval;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Ignore; import org.junit.Ignore;
import org.junit.Test; import org.junit.Test;
import org.junit.runner.RunWith; import org.junit.runner.RunWith;
import org.junit.runners.Parameterized; import org.junit.runners.Parameterized;
import java.util.ArrayList;
import java.util.Arrays; import java.util.Arrays;
import java.util.List; import java.util.List;
import java.util.Random;
import java.util.stream.Collectors;
@Ignore @Ignore
@RunWith(Parameterized.class) @RunWith(Parameterized.class)
@ -56,36 +60,54 @@ public class CostBalancerStrategyBenchmark extends AbstractBenchmark
} }
private final CostBalancerStrategy strategy; private final CostBalancerStrategy strategy;
private final List<ServerHolder> serverHolderList;
public CostBalancerStrategyBenchmark(CostBalancerStrategy costBalancerStrategy) public CostBalancerStrategyBenchmark(CostBalancerStrategy costBalancerStrategy)
{ {
this.strategy = costBalancerStrategy; this.strategy = costBalancerStrategy;
this.serverHolderList = initServers();
}
private List<ServerHolder> initServers()
{
final List<DruidServer> servers = new ArrayList<>();
for (int i = 0; i < 6; ++i) {
DruidServer druidServer = new DruidServer(
"server_" + i,
"localhost", null, 10_000_000L, ServerType.HISTORICAL, "hot", 1
);
servers.add(druidServer);
}
// Create and randomly distribute some segments amongst the servers
final List<DataSegment> segments =
CreateDataSegments.ofDatasource("wikipedia")
.forIntervals(200, Granularities.DAY)
.withNumPartitions(100)
.eachOfSizeInMb(200);
final Random random = new Random(100);
segments.forEach(
segment -> servers.get(random.nextInt(servers.size()))
.addDataSegment(segment)
);
return servers.stream()
.map(DruidServer::toImmutableDruidServer)
.map(server -> new ServerHolder(server, null))
.collect(Collectors.toList());
} }
private static List<ServerHolder> serverHolderList;
volatile ServerHolder selected; volatile ServerHolder selected;
@BeforeClass
public static void setup()
{
serverHolderList = CostBalancerStrategyTest.setupDummyCluster(5, 20000);
}
@AfterClass
public static void tearDown()
{
serverHolderList = null;
}
@Test @Test
@BenchmarkOptions(warmupRounds = 10, benchmarkRounds = 1000) @BenchmarkOptions(warmupRounds = 10, benchmarkRounds = 1000)
public void testBenchmark() public void testBenchmark()
{ {
DataSegment segment = CostBalancerStrategyTest.getSegment(1000, "testds", interval1); DataSegment segment = DataSegment.builder().dataSource("testds").version("1000")
.interval(interval1).size(100L).build();
selected = strategy.findNewSegmentHomeReplicator(segment, serverHolderList); selected = strategy.findNewSegmentHomeReplicator(segment, serverHolderList);
} }
// Benchmark Joda Interval Gap impl vs CostBalancer.gapMillis // Benchmark Joda Interval Gap impl vs CostBalancer.gapMillis
private final Interval interval1 = Intervals.of("2015-01-01T01:00:00Z/2015-01-01T02:00:00Z"); private final Interval interval1 = Intervals.of("2015-01-01T01:00:00Z/2015-01-01T02:00:00Z");
private final Interval interval2 = Intervals.of("2015-02-01T01:00:00Z/2015-02-01T02:00:00Z"); private final Interval interval2 = Intervals.of("2015-02-01T01:00:00Z/2015-02-01T02:00:00Z");

View File

@ -19,211 +19,329 @@
package org.apache.druid.server.coordinator; package org.apache.druid.server.coordinator;
import com.google.common.collect.ImmutableMap;
import com.google.common.util.concurrent.MoreExecutors; import com.google.common.util.concurrent.MoreExecutors;
import org.apache.druid.client.ImmutableDruidDataSource; import org.apache.druid.client.DruidServer;
import org.apache.druid.client.ImmutableDruidServer; import org.apache.druid.java.util.common.granularity.Granularities;
import org.apache.druid.client.ImmutableDruidServerTests; import org.apache.druid.java.util.common.granularity.GranularityType;
import org.apache.druid.java.util.common.DateTimes;
import org.apache.druid.java.util.common.Intervals;
import org.apache.druid.java.util.common.concurrent.Execs;
import org.apache.druid.server.coordination.DruidServerMetadata;
import org.apache.druid.server.coordination.ServerType; import org.apache.druid.server.coordination.ServerType;
import org.apache.druid.server.coordinator.simulate.BlockingExecutorService;
import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.DataSegment;
import org.apache.druid.timeline.SegmentId; import org.junit.After;
import org.easymock.EasyMock;
import org.joda.time.DateTime;
import org.joda.time.Interval;
import org.junit.Assert; import org.junit.Assert;
import org.junit.Before;
import org.junit.Test; import org.junit.Test;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Random;
import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ExecutorService;
import java.util.concurrent.TimeUnit;
import java.util.stream.Collectors; import java.util.stream.Collectors;
import java.util.stream.IntStream; import java.util.stream.IntStream;
public class CostBalancerStrategyTest public class CostBalancerStrategyTest
{ {
private static final Interval DAY = Intervals.of("2015-01-01T00/2015-01-01T01"); private static final double DELTA = 1e-6;
private static final String DS_WIKI = "wiki";
/** private ExecutorService balancerExecutor;
* Create Druid cluster with serverCount servers having maxSegments segments each, and 1 server with 98 segment private CostBalancerStrategy strategy;
* Cost Balancer Strategy should assign the next segment to the server with less segments. private int uniqueServerId;
*/
public static List<ServerHolder> setupDummyCluster(int serverCount, int maxSegments) @Before
public void setup()
{ {
List<ServerHolder> serverHolderList = new ArrayList<>(); balancerExecutor = new BlockingExecutorService("test-balance-exec-%d");
// Create 10 servers with current size being 3K & max size being 10K strategy = new CostBalancerStrategy(MoreExecutors.listeningDecorator(balancerExecutor));
// Each having having 100 segments }
for (int i = 0; i < serverCount; i++) {
LoadQueuePeonTester fromPeon = new LoadQueuePeonTester();
List<DataSegment> segments = IntStream @After
.range(0, maxSegments) public void tearDown()
.mapToObj(j -> getSegment(j)) {
.collect(Collectors.toList()); if (balancerExecutor != null) {
ImmutableDruidDataSource dataSource = new ImmutableDruidDataSource("DUMMY", Collections.emptyMap(), segments); balancerExecutor.shutdownNow();
String serverName = "DruidServer_Name_" + i;
ServerHolder serverHolder = new ServerHolder(
new ImmutableDruidServer(
new DruidServerMetadata(serverName, "localhost", null, 10000000L, ServerType.HISTORICAL, "hot", 1),
3000L,
ImmutableMap.of("DUMMY", dataSource),
segments.size()
),
fromPeon
);
serverHolderList.add(serverHolder);
} }
// The best server to be available for next segment assignment has only 98 Segments
LoadQueuePeonTester fromPeon = new LoadQueuePeonTester();
ImmutableDruidServer druidServer = EasyMock.createMock(ImmutableDruidServer.class);
EasyMock.expect(druidServer.getName()).andReturn("BEST_SERVER").anyTimes();
EasyMock.expect(druidServer.getCurrSize()).andReturn(3000L).anyTimes();
EasyMock.expect(druidServer.getMaxSize()).andReturn(10000000L).anyTimes();
EasyMock.expect(druidServer.getSegment(EasyMock.anyObject())).andReturn(null).anyTimes();
Map<SegmentId, DataSegment> segments = new HashMap<>();
for (int j = 0; j < (maxSegments - 2); j++) {
DataSegment segment = getSegment(j);
segments.put(segment.getId(), segment);
EasyMock.expect(druidServer.getSegment(segment.getId())).andReturn(segment).anyTimes();
}
ImmutableDruidServerTests.expectSegments(druidServer, segments.values());
EasyMock.replay(druidServer);
serverHolderList.add(new ServerHolder(druidServer, fromPeon));
return serverHolderList;
}
/**
* Returns segment with dummy id and size 100
*
* @param index
*
* @return segment
*/
public static DataSegment getSegment(int index)
{
return getSegment(index, "DUMMY", DAY);
}
public static DataSegment getSegment(int index, String dataSource, Interval interval)
{
// Not using EasyMock as it hampers the performance of multithreads.
DataSegment segment = new DataSegment(
dataSource,
interval,
String.valueOf(index),
new ConcurrentHashMap<>(),
new ArrayList<>(),
new ArrayList<>(),
null,
0,
index * 100L
);
return segment;
} }
@Test @Test
public void testCostBalancerMultiThreadedStrategy() public void testIntervalCostAdditivity()
{ {
List<ServerHolder> serverHolderList = setupDummyCluster(10, 20); Assert.assertEquals(
DataSegment segment = getSegment(1000); intervalCost(1, 1, 3),
intervalCost(1, 1, 2) + intervalCost(1, 2, 3),
BalancerStrategy strategy = new CostBalancerStrategy( DELTA
MoreExecutors.listeningDecorator(Execs.multiThreaded(4, "CostBalancerStrategyTest-%d"))
);
ServerHolder holder = strategy.findNewSegmentHomeReplicator(segment, serverHolderList);
Assert.assertNotNull("Should be able to find a place for new segment!!", holder);
Assert.assertEquals("Best Server should be BEST_SERVER", "BEST_SERVER", holder.getServer().getName());
}
@Test
public void testCostBalancerSingleThreadStrategy()
{
List<ServerHolder> serverHolderList = setupDummyCluster(10, 20);
DataSegment segment = getSegment(1000);
BalancerStrategy strategy = new CostBalancerStrategy(
MoreExecutors.listeningDecorator(Execs.multiThreaded(1, "CostBalancerStrategyTest-%d"))
);
ServerHolder holder = strategy.findNewSegmentHomeReplicator(segment, serverHolderList);
Assert.assertNotNull("Should be able to find a place for new segment!!", holder);
Assert.assertEquals("Best Server should be BEST_SERVER", "BEST_SERVER", holder.getServer().getName());
}
@Test
public void testComputeJointSegmentCost()
{
DateTime referenceTime = DateTimes.of("2014-01-01T00:00:00");
double segmentCost = CostBalancerStrategy.computeJointSegmentsCost(
getSegment(
100,
"DUMMY",
new Interval(
referenceTime,
referenceTime.plusHours(1)
)
),
getSegment(
101,
"DUMMY",
new Interval(
referenceTime.minusHours(2),
referenceTime.minusHours(2).plusHours(1)
)
)
); );
Assert.assertEquals( Assert.assertEquals(
CostBalancerStrategy.INV_LAMBDA_SQUARE * CostBalancerStrategy.intervalCost( intervalCost(2, 1, 3),
1 * CostBalancerStrategy.LAMBDA, intervalCost(2, 1, 2) + intervalCost(2, 2, 3),
-2 * CostBalancerStrategy.LAMBDA, DELTA
-1 * CostBalancerStrategy.LAMBDA );
) * 2,
segmentCost, 1e-6); Assert.assertEquals(
intervalCost(3, 1, 2),
intervalCost(1, 0, 1) + intervalCost(1, 1, 2) + intervalCost(1, 1, 2),
DELTA
);
}
private double intervalCost(double x1, double y0, double y1)
{
return CostBalancerStrategy.intervalCost(x1, y0, y1);
} }
@Test @Test
public void testIntervalCost() public void testIntervalCost()
{ {
// additivity // no overlap
Assert.assertEquals(CostBalancerStrategy.intervalCost(1, 1, 3), // [0, 1) [1, 2)
CostBalancerStrategy.intervalCost(1, 1, 2) + Assert.assertEquals(0.3995764, intervalCost(1, 1, 2), DELTA);
CostBalancerStrategy.intervalCost(1, 2, 3), 1e-6); // [0, 1) [-1, 0)
Assert.assertEquals(0.3995764, intervalCost(1, -1, 0), DELTA);
Assert.assertEquals(CostBalancerStrategy.intervalCost(2, 1, 3), // exact overlap
CostBalancerStrategy.intervalCost(2, 1, 2) + // [0, 1), [0, 1)
CostBalancerStrategy.intervalCost(2, 2, 3), 1e-6); Assert.assertEquals(0.7357589, intervalCost(1, 0, 1), DELTA);
// [0, 2), [0, 2)
Assert.assertEquals(2.270671, intervalCost(2, 0, 2), DELTA);
Assert.assertEquals(CostBalancerStrategy.intervalCost(3, 1, 2), // partial overlap
CostBalancerStrategy.intervalCost(1, 1, 2) + // [0, 2), [1, 3)
CostBalancerStrategy.intervalCost(1, 0, 1) + Assert.assertEquals(1.681908, intervalCost(2, 1, 3), DELTA);
CostBalancerStrategy.intervalCost(1, 1, 2), 1e-6); // [0, 2), [1, 2)
Assert.assertEquals(1.135335, intervalCost(2, 1, 2), DELTA);
// no overlap [0, 1) [1, 2) // [0, 2), [0, 1)
Assert.assertEquals(0.3995764, CostBalancerStrategy.intervalCost(1, 1, 2), 1e-6); Assert.assertEquals(1.135335, intervalCost(2, 0, 1), DELTA);
// no overlap [0, 1) [-1, 0) // [0, 3), [1, 2)
Assert.assertEquals(0.3995764, CostBalancerStrategy.intervalCost(1, -1, 0), 1e-6); Assert.assertEquals(1.534912, intervalCost(3, 1, 2), DELTA);
// exact overlap [0, 1), [0, 1)
Assert.assertEquals(0.7357589, CostBalancerStrategy.intervalCost(1, 0, 1), 1e-6);
// exact overlap [0, 2), [0, 2)
Assert.assertEquals(2.270671, CostBalancerStrategy.intervalCost(2, 0, 2), 1e-6);
// partial overlap [0, 2), [1, 3)
Assert.assertEquals(1.681908, CostBalancerStrategy.intervalCost(2, 1, 3), 1e-6);
// partial overlap [0, 2), [1, 2)
Assert.assertEquals(1.135335, CostBalancerStrategy.intervalCost(2, 1, 2), 1e-6);
// partial overlap [0, 2), [0, 1)
Assert.assertEquals(1.135335, CostBalancerStrategy.intervalCost(2, 0, 1), 1e-6);
// partial overlap [0, 3), [1, 2)
Assert.assertEquals(1.534912, CostBalancerStrategy.intervalCost(3, 1, 2), 1e-6);
} }
@Test
public void testJointSegmentsCost()
{
final long noGap = 0;
final long oneDayGap = TimeUnit.DAYS.toMillis(1);
verifyJointSegmentsCost(GranularityType.HOUR, GranularityType.HOUR, noGap, 1.980884);
verifyJointSegmentsCost(GranularityType.HOUR, GranularityType.HOUR, oneDayGap, 1.000070);
verifyJointSegmentsCost(GranularityType.HOUR, GranularityType.DAY, noGap, 35.110275);
verifyJointSegmentsCost(GranularityType.DAY, GranularityType.DAY, noGap, 926.232308);
verifyJointSegmentsCost(GranularityType.DAY, GranularityType.DAY, oneDayGap, 599.434267);
verifyJointSegmentsCost(GranularityType.DAY, GranularityType.DAY, 7 * oneDayGap, 9.366160);
verifyJointSegmentsCost(GranularityType.DAY, GranularityType.MONTH, noGap, 2125.100840);
verifyJointSegmentsCost(GranularityType.MONTH, GranularityType.MONTH, noGap, 98247.576470);
verifyJointSegmentsCost(GranularityType.MONTH, GranularityType.MONTH, 7 * oneDayGap, 79719.068161);
verifyJointSegmentsCost(GranularityType.MONTH, GranularityType.YEAR, noGap, 100645.313535);
verifyJointSegmentsCost(GranularityType.YEAR, GranularityType.YEAR, noGap, 1208453.347454);
verifyJointSegmentsCost(GranularityType.YEAR, GranularityType.YEAR, 7 * oneDayGap, 1189943.571325);
}
@Test
public void testJointSegmentsCostSymmetry()
{
final DataSegment segmentA = CreateDataSegments.ofDatasource(DS_WIKI)
.forIntervals(1, Granularities.DAY)
.startingAt("2010-01-01")
.eachOfSizeInMb(100).get(0);
final DataSegment segmentB = CreateDataSegments.ofDatasource(DS_WIKI)
.forIntervals(1, Granularities.MONTH)
.startingAt("2010-01-01")
.eachOfSizeInMb(100).get(0);
Assert.assertEquals(
CostBalancerStrategy.computeJointSegmentsCost(segmentA, segmentB),
CostBalancerStrategy.computeJointSegmentsCost(segmentB, segmentA),
DELTA
);
}
@Test
public void testJointSegmentsCostMultipleDatasources()
{
final DataSegment wikiSegment = CreateDataSegments.ofDatasource(DS_WIKI)
.forIntervals(1, Granularities.DAY)
.startingAt("2010-01-01")
.eachOfSizeInMb(100).get(0);
final DataSegment koalaSegment = CreateDataSegments.ofDatasource("koala")
.forIntervals(1, Granularities.DAY)
.startingAt("2010-01-01")
.eachOfSizeInMb(100).get(0);
// Verify that cross datasource cost is twice that of same datasource cost
final double crossDatasourceCost =
CostBalancerStrategy.computeJointSegmentsCost(koalaSegment, wikiSegment);
Assert.assertEquals(
2 * crossDatasourceCost,
CostBalancerStrategy.computeJointSegmentsCost(wikiSegment, wikiSegment),
DELTA
);
Assert.assertEquals(
2 * crossDatasourceCost,
CostBalancerStrategy.computeJointSegmentsCost(koalaSegment, koalaSegment),
DELTA
);
}
@Test
public void testJointSegmentsCostWith45DayGap()
{
// start of 2nd segment - end of 1st segment = 45 days
final long gap1Day = TimeUnit.DAYS.toMillis(1);
final long gap45Days = 45 * gap1Day;
// This test establishes that after 45 days, all costs become negligible
// (except with ALL granularity)
// Add extra gap to ensure that segments have no overlap
final long gap1Hour = TimeUnit.HOURS.toMillis(1);
verifyJointSegmentsCost(GranularityType.HOUR, GranularityType.HOUR, gap1Hour + gap45Days, 0);
verifyJointSegmentsCost(GranularityType.HOUR, GranularityType.DAY, gap1Hour + gap45Days, 0);
verifyJointSegmentsCost(GranularityType.DAY, GranularityType.DAY, gap1Day + gap45Days, 0);
verifyJointSegmentsCost(GranularityType.DAY, GranularityType.MONTH, gap1Day + gap45Days, 0);
verifyJointSegmentsCost(GranularityType.MONTH, GranularityType.MONTH, 30 * gap1Day + gap45Days, 0);
verifyJointSegmentsCost(GranularityType.YEAR, GranularityType.YEAR, 365 * gap1Day + gap45Days, 0);
}
@Test
public void testJointSegmentsCostAllGranularity()
{
// Cost of ALL with other granularities
verifyJointSegmentsCost(GranularityType.HOUR, GranularityType.ALL, 0, 138.516732);
verifyJointSegmentsCost(GranularityType.DAY, GranularityType.ALL, 0, 3323.962523);
verifyJointSegmentsCost(GranularityType.MONTH, GranularityType.ALL, 0, 103043.057744);
verifyJointSegmentsCost(GranularityType.YEAR, GranularityType.ALL, 0, 1213248.808913);
// Self cost of an ALL granularity segment
DataSegment segmentAllGranularity =
CreateDataSegments.ofDatasource("ds")
.forIntervals(1, Granularities.ALL)
.eachOfSizeInMb(100).get(0);
double cost = CostBalancerStrategy.computeJointSegmentsCost(
segmentAllGranularity,
segmentAllGranularity
);
Assert.assertTrue(cost >= 3.548e14 && cost <= 3.549e14);
}
@Test
public void testComputeCost()
{
// Create segments for different granularities
final List<DataSegment> daySegments =
CreateDataSegments.ofDatasource(DS_WIKI)
.forIntervals(10, Granularities.DAY)
.startingAt("2022-01-01")
.withNumPartitions(10)
.eachOfSizeInMb(100);
final List<DataSegment> monthSegments =
CreateDataSegments.ofDatasource(DS_WIKI)
.forIntervals(10, Granularities.MONTH)
.startingAt("2022-03-01")
.withNumPartitions(10)
.eachOfSizeInMb(100);
final List<DataSegment> yearSegments =
CreateDataSegments.ofDatasource(DS_WIKI)
.forIntervals(1, Granularities.YEAR)
.startingAt("2023-01-01")
.withNumPartitions(30)
.eachOfSizeInMb(100);
// Distribute the segments randomly amongst 2 servers
final List<DataSegment> segments = new ArrayList<>(daySegments);
segments.addAll(monthSegments);
segments.addAll(yearSegments);
List<DruidServer> historicals = IntStream.range(0, 3)
.mapToObj(i -> createHistorical())
.collect(Collectors.toList());
final Random random = new Random(100);
segments.forEach(
segment -> historicals.get(random.nextInt(historicals.size()))
.addDataSegment(segment)
);
// Create ServerHolder for each server
final List<ServerHolder> serverHolders = historicals.stream().map(
server -> new ServerHolder(server.toImmutableDruidServer(), new LoadQueuePeonTester())
).collect(Collectors.toList());
// Verify costs for DAY, MONTH and YEAR segments
verifyServerCosts(
daySegments.get(0),
serverHolders,
5191.500804, 8691.392080, 6418.467818
);
verifyServerCosts(
monthSegments.get(0),
serverHolders,
301935.940609, 301935.940606, 304333.677669
);
verifyServerCosts(
yearSegments.get(0),
serverHolders,
8468764.380437, 12098919.896931, 14501440.169452
);
// Verify costs for an ALL granularity segment
DataSegment allGranularitySegment =
CreateDataSegments.ofDatasource(DS_WIKI)
.forIntervals(1, Granularities.ALL)
.eachOfSizeInMb(100).get(0);
verifyServerCosts(
allGranularitySegment,
serverHolders,
1.1534173737329768e7,
1.6340633534241956e7,
1.9026400521582970e7
);
}
private void verifyServerCosts(
DataSegment segment,
List<ServerHolder> serverHolders,
double... expectedCosts
)
{
for (int i = 0; i < serverHolders.size(); ++i) {
double observedCost = strategy.computeCost(segment, serverHolders.get(i), true);
Assert.assertEquals(expectedCosts[i], observedCost, DELTA);
}
}
private void verifyJointSegmentsCost(
GranularityType granularityX,
GranularityType granularityY,
long startGapMillis,
double expectedCost
)
{
final DataSegment segmentX =
CreateDataSegments.ofDatasource(DS_WIKI)
.forIntervals(1, granularityX.getDefaultGranularity())
.startingAt("2012-10-24")
.eachOfSizeInMb(100).get(0);
long startTimeY = segmentX.getInterval().getStartMillis() + startGapMillis;
final DataSegment segmentY =
CreateDataSegments.ofDatasource(DS_WIKI)
.forIntervals(1, granularityY.getDefaultGranularity())
.startingAt(startTimeY)
.eachOfSizeInMb(100).get(0);
double observedCost = CostBalancerStrategy.computeJointSegmentsCost(segmentX, segmentY);
Assert.assertEquals(expectedCost, observedCost, DELTA);
}
private DruidServer createHistorical()
{
String serverName = "hist_" + uniqueServerId++;
return new DruidServer(serverName, serverName, null, 10L << 30, ServerType.HISTORICAL, "hot", 1);
}
} }

View File

@ -20,6 +20,8 @@
package org.apache.druid.server.coordinator; package org.apache.druid.server.coordinator;
import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.DateTimes;
import org.apache.druid.java.util.common.Intervals;
import org.apache.druid.java.util.common.granularity.Granularities;
import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.granularity.Granularity;
import org.apache.druid.segment.IndexIO; import org.apache.druid.segment.IndexIO;
import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.DataSegment;
@ -30,6 +32,7 @@ import org.joda.time.Interval;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Collections; import java.util.Collections;
import java.util.List; import java.util.List;
import java.util.Objects;
/** /**
* Test utility to create {@link DataSegment}s for a given datasource. * Test utility to create {@link DataSegment}s for a given datasource.
@ -40,8 +43,8 @@ public class CreateDataSegments
private DateTime startTime; private DateTime startTime;
private Granularity granularity; private Granularity granularity;
private int numPartitions; private int numPartitions = 1;
private int numIntervals; private int numIntervals = 1;
public static CreateDataSegments ofDatasource(String datasource) public static CreateDataSegments ofDatasource(String datasource)
{ {
@ -66,6 +69,12 @@ public class CreateDataSegments
return this; return this;
} }
public CreateDataSegments startingAt(long startOfFirstInterval)
{
this.startTime = DateTimes.utc(startOfFirstInterval);
return this;
}
public CreateDataSegments withNumPartitions(int numPartitions) public CreateDataSegments withNumPartitions(int numPartitions)
{ {
this.numPartitions = numPartitions; this.numPartitions = numPartitions;
@ -74,12 +83,19 @@ public class CreateDataSegments
public List<DataSegment> eachOfSizeInMb(long sizeMb) public List<DataSegment> eachOfSizeInMb(long sizeMb)
{ {
final List<DataSegment> segments = new ArrayList<>(); boolean isEternityInterval = Objects.equals(granularity, Granularities.ALL);
if (isEternityInterval) {
numIntervals = 1;
}
int uniqueIdInInterval = 0; int uniqueIdInInterval = 0;
DateTime nextStart = startTime; DateTime nextStart = startTime;
final List<DataSegment> segments = new ArrayList<>();
for (int numInterval = 0; numInterval < numIntervals; ++numInterval) { for (int numInterval = 0; numInterval < numIntervals; ++numInterval) {
Interval nextInterval = new Interval(nextStart, granularity.increment(nextStart)); Interval nextInterval = isEternityInterval
? Intervals.ETERNITY
: new Interval(nextStart, granularity.increment(nextStart));
for (int numPartition = 0; numPartition < numPartitions; ++numPartition) { for (int numPartition = 0; numPartition < numPartitions; ++numPartition) {
segments.add( segments.add(
new NumberedDataSegment( new NumberedDataSegment(