Skip to content

Commit

Permalink
Optimize CostBalancerStrategy (#2910)
Browse files Browse the repository at this point in the history
* Optimize CostBalancerStrategy

Ignore benchmark test in normal run

fix test

review comments

fix compilation

fix test

* review comments

* review comment
  • Loading branch information
nishantmonu51 authored and fjy committed May 5, 2016
1 parent b489f63 commit a2dd57c
Show file tree
Hide file tree
Showing 11 changed files with 272 additions and 91 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,9 @@

import org.joda.time.DateTime;

public interface BalancerStrategyFactory
import java.io.Closeable;

public interface BalancerStrategyFactory extends Closeable
{
public BalancerStrategy createBalancerStrategy(DateTime referenceTimestamp);
}
Original file line number Diff line number Diff line change
Expand Up @@ -32,7 +32,7 @@

import java.util.List;
import java.util.concurrent.Callable;
import java.util.concurrent.Executors;
import java.util.concurrent.ExecutorService;

public class CostBalancerStrategy implements BalancerStrategy
{
Expand All @@ -41,12 +41,23 @@ public class CostBalancerStrategy implements BalancerStrategy
private static final long SEVEN_DAYS_IN_MILLIS = 7 * DAY_IN_MILLIS;
private static final long THIRTY_DAYS_IN_MILLIS = 30 * DAY_IN_MILLIS;
private final long referenceTimestamp;
private final int threadCount;
private final ListeningExecutorService exec;

public CostBalancerStrategy(DateTime referenceTimestamp, int threadCount)
public static long gapMillis(Interval interval1, Interval interval2)
{
if (interval1.getStartMillis() > interval2.getEndMillis()) {
return interval1.getStartMillis() - interval2.getEndMillis();
} else if (interval2.getStartMillis() > interval1.getEndMillis()) {
return interval2.getStartMillis() - interval1.getEndMillis();
} else {
return 0;
}
}

public CostBalancerStrategy(DateTime referenceTimestamp, ListeningExecutorService exec)
{
this.referenceTimestamp = referenceTimestamp.getMillis();
this.threadCount = threadCount;
this.exec = exec;
}

@Override
Expand Down Expand Up @@ -85,7 +96,7 @@ public ServerHolder findNewSegmentHomeBalancer(
*/
public double computeJointSegmentCosts(final DataSegment segment1, final DataSegment segment2)
{
final Interval gap = segment1.getInterval().gap(segment2.getInterval());
final long gapMillis = gapMillis(segment1.getInterval(), segment2.getInterval());

final double baseCost = Math.min(segment1.getSize(), segment2.getSize());
double recencyPenalty = 1;
Expand All @@ -103,10 +114,9 @@ public double computeJointSegmentCosts(final DataSegment segment1, final DataSeg
}

/** gap is null if the two segment intervals overlap or if they're adjacent */
if (gap == null) {
if (gapMillis == 0) {
gapPenalty = 2;
} else {
long gapMillis = gap.toDurationMillis();
if (gapMillis < THIRTY_DAYS_IN_MILLIS) {
gapPenalty = 2 - gapMillis / THIRTY_DAYS_IN_MILLIS;
}
Expand Down Expand Up @@ -151,8 +161,8 @@ public double calculateInitialTotalCost(final List<ServerHolder> serverHolders)
* @param serverHolders A list of ServerHolders for a particular tier.
*
* @return The normalization value (the sum of the diagonal entries in the
* pairwise cost matrix). This is the cost of a cluster if each
* segment were to get its own historical node.
* pairwise cost matrix). This is the cost of a cluster if each
* segment were to get its own historical node.
*/
public double calculateNormalization(final List<ServerHolder> serverHolders)
{
Expand Down Expand Up @@ -234,12 +244,11 @@ protected Pair<Double, ServerHolder> chooseBestServer(
{
Pair<Double, ServerHolder> bestServer = Pair.of(Double.POSITIVE_INFINITY, null);

ListeningExecutorService service = MoreExecutors.listeningDecorator(Executors.newFixedThreadPool(threadCount));
List<ListenableFuture<Pair<Double, ServerHolder>>> futures = Lists.newArrayList();

for (final ServerHolder server : serverHolders) {
futures.add(
service.submit(
exec.submit(
new Callable<Pair<Double, ServerHolder>>()
{
@Override
Expand All @@ -264,9 +273,7 @@ public Pair<Double, ServerHolder> call() throws Exception
catch (Exception e) {
log.makeAlert(e, "Cost Balancer Multithread strategy wasn't able to complete cost computation.").emit();
}
service.shutdown();
return bestServer;
}

}

Original file line number Diff line number Diff line change
Expand Up @@ -18,20 +18,31 @@
*/
package io.druid.server.coordinator;

import com.google.common.util.concurrent.ListeningExecutorService;
import com.google.common.util.concurrent.MoreExecutors;
import org.joda.time.DateTime;

import java.io.IOException;
import java.util.concurrent.Executors;

public class CostBalancerStrategyFactory implements BalancerStrategyFactory
{
private final int threadCount;
private final ListeningExecutorService exec;

public CostBalancerStrategyFactory(int costBalancerStrategyThreadCount)
{
this.threadCount = costBalancerStrategyThreadCount;
this.exec = MoreExecutors.listeningDecorator(Executors.newFixedThreadPool(costBalancerStrategyThreadCount));
}

@Override
public BalancerStrategy createBalancerStrategy(DateTime referenceTimestamp)
{
return new CostBalancerStrategy(referenceTimestamp, threadCount);
return new CostBalancerStrategy(referenceTimestamp, exec);
}

@Override
public void close() throws IOException
{
exec.shutdownNow();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -686,23 +686,22 @@ public void run()
}
}

BalancerStrategyFactory factory =
new CostBalancerStrategyFactory(getDynamicConfigs().getBalancerComputeThreads());

// Do coordinator stuff.
DruidCoordinatorRuntimeParams params =
DruidCoordinatorRuntimeParams.newBuilder()
.withStartTime(startTime)
.withDatasources(metadataSegmentManager.getInventory())
.withDynamicConfigs(getDynamicConfigs())
.withEmitter(emitter)
.withBalancerStrategyFactory(factory)
.build();

for (DruidCoordinatorHelper helper : helpers) {
// Don't read state and run state in the same helper otherwise racy conditions may exist
if (leader && startingLeaderCounter == leaderCounter) {
params = helper.run(params);
try (BalancerStrategyFactory factory =
new CostBalancerStrategyFactory(getDynamicConfigs().getBalancerComputeThreads())) {
// Do coordinator stuff.
DruidCoordinatorRuntimeParams params =
DruidCoordinatorRuntimeParams.newBuilder()
.withStartTime(startTime)
.withDatasources(metadataSegmentManager.getInventory())
.withDynamicConfigs(getDynamicConfigs())
.withEmitter(emitter)
.withBalancerStrategyFactory(factory)
.build();
for (DruidCoordinatorHelper helper : helpers) {
// Don't read state and run state in the same helper otherwise racy conditions may exist
if (leader && startingLeaderCounter == leaderCounter) {
params = helper.run(params);
}
}
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,6 @@

import com.google.common.collect.Maps;
import com.google.common.collect.Sets;
import com.metamx.common.guava.Comparators;
import com.metamx.emitter.service.ServiceEmitter;
import io.druid.client.DruidDataSource;
import io.druid.metadata.MetadataRuleManager;
Expand Down Expand Up @@ -206,7 +205,6 @@ public static class Builder
this.stats = new CoordinatorStats();
this.coordinatorDynamicConfig = new CoordinatorDynamicConfig.Builder().build();
this.balancerReferenceTimestamp = DateTime.now();
this.strategyFactory = new CostBalancerStrategyFactory(1);
}

Builder(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,11 +20,19 @@

import org.joda.time.DateTime;

import java.io.IOException;

public class RandomBalancerStrategyFactory implements BalancerStrategyFactory
{
@Override
public BalancerStrategy createBalancerStrategy(DateTime referenceTimestamp)
{
return new RandomBalancerStrategy();
}

@Override
public void close() throws IOException
{
// No-op
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,114 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/

package io.druid.server.coordination;

import com.carrotsearch.junitbenchmarks.AbstractBenchmark;
import com.carrotsearch.junitbenchmarks.BenchmarkOptions;
import io.druid.server.coordinator.CostBalancerStrategy;
import io.druid.server.coordinator.CostBalancerStrategyFactory;
import io.druid.server.coordinator.ServerHolder;
import io.druid.timeline.DataSegment;
import org.joda.time.DateTime;
import org.joda.time.Interval;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Ignore;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;

import java.util.Arrays;
import java.util.List;

@Ignore
@RunWith(Parameterized.class)
public class CostBalancerStrategyBenchmark extends AbstractBenchmark
{
@Parameterized.Parameters
public static List<CostBalancerStrategyFactory[]> factoryClasses()
{
return Arrays.asList(
(CostBalancerStrategyFactory[]) Arrays.asList(
new CostBalancerStrategyFactory(1)
).toArray(),
(CostBalancerStrategyFactory[]) Arrays.asList(
new CostBalancerStrategyFactory(4)
).toArray()
);
}

private final CostBalancerStrategy strategy;

public CostBalancerStrategyBenchmark(CostBalancerStrategyFactory factory)
{
this.strategy = (CostBalancerStrategy) factory.createBalancerStrategy(DateTime.now());
}

private static List<ServerHolder> serverHolderList;
volatile ServerHolder selected;

@BeforeClass
public static void setup()
{
serverHolderList = CostBalancerStrategyTest.setupDummyCluster(5, 20000);
}

@AfterClass
public static void tearDown(){
serverHolderList = null;
}

@Test
@BenchmarkOptions(warmupRounds = 10, benchmarkRounds = 1000)
public void testBenchmark() throws InterruptedException
{
DataSegment segment = CostBalancerStrategyTest.getSegment(1000, "testds", interval1);
selected = strategy.findNewSegmentHomeReplicator(segment, serverHolderList);
}


// Benchmark Joda Interval Gap impl vs CostBalancer.gapMillis
private final Interval interval1 = new Interval("2015-01-01T01:00:00Z/2015-01-01T02:00:00Z");
private final Interval interval2 = new Interval("2015-02-01T01:00:00Z/2015-02-01T02:00:00Z");
volatile Long sum;

@BenchmarkOptions(warmupRounds = 1000, benchmarkRounds = 1000000)
@Test
public void testJodaGap()
{
long diff = 0;
for (int i = 0; i < 1000; i++) {
diff = diff + interval1.gap(interval2).toDurationMillis();
}
sum = diff;
}

@BenchmarkOptions(warmupRounds = 1000, benchmarkRounds = 1000000)
@Test
public void testBalancerGapMillis()
{
long diff = 0;
for (int i = 0; i < 1000; i++) {
diff = diff + CostBalancerStrategy.gapMillis(interval1, interval2);
}
sum = diff;
}

}
Loading

0 comments on commit a2dd57c

Please sign in to comment.