mirror of https://github.com/apache/druid.git
fixed with Erics feedback
This commit is contained in:
parent
0d5b222f8f
commit
c4f0e4d96d
|
@ -31,11 +31,9 @@ import com.metamx.druid.client.DruidDataSource;
|
|||
import com.metamx.druid.client.DruidServer;
|
||||
import com.metamx.druid.client.InventoryView;
|
||||
import com.metamx.druid.client.indexing.IndexingServiceClient;
|
||||
import com.metamx.druid.config.JacksonConfigManager;
|
||||
import com.metamx.druid.db.DatabaseRuleManager;
|
||||
import com.metamx.druid.db.DatabaseSegmentManager;
|
||||
import com.metamx.druid.master.DruidMaster;
|
||||
import com.metamx.druid.master.DynamicConfigs;
|
||||
import com.metamx.druid.master.rules.Rule;
|
||||
import org.joda.time.Interval;
|
||||
|
||||
|
@ -103,7 +101,6 @@ public class InfoResource
|
|||
private final DatabaseSegmentManager databaseSegmentManager;
|
||||
private final DatabaseRuleManager databaseRuleManager;
|
||||
private final IndexingServiceClient indexingServiceClient;
|
||||
private final JacksonConfigManager configManager;
|
||||
|
||||
|
||||
@Inject
|
||||
|
@ -112,7 +109,6 @@ public class InfoResource
|
|||
InventoryView serverInventoryView,
|
||||
DatabaseSegmentManager databaseSegmentManager,
|
||||
DatabaseRuleManager databaseRuleManager,
|
||||
JacksonConfigManager configManager,
|
||||
@Nullable
|
||||
IndexingServiceClient indexingServiceClient
|
||||
)
|
||||
|
@ -121,7 +117,6 @@ public class InfoResource
|
|||
this.serverInventoryView = serverInventoryView;
|
||||
this.databaseSegmentManager = databaseSegmentManager;
|
||||
this.databaseRuleManager = databaseRuleManager;
|
||||
this.configManager = configManager;
|
||||
this.indexingServiceClient = indexingServiceClient;
|
||||
}
|
||||
|
||||
|
@ -135,20 +130,6 @@ public class InfoResource
|
|||
.build();
|
||||
}
|
||||
|
||||
@GET
|
||||
@Path("/master/config")
|
||||
@Produces("application/json")
|
||||
public Response getDynamicConfigs()
|
||||
{
|
||||
Response.ResponseBuilder builder = Response.status(Response.Status.OK)
|
||||
.entity(
|
||||
configManager.watch(
|
||||
DynamicConfigs.CONFIG_KEY,
|
||||
DynamicConfigs.class
|
||||
).get()
|
||||
);
|
||||
return builder.build();
|
||||
}
|
||||
|
||||
@GET
|
||||
@Path("/cluster")
|
||||
|
@ -394,20 +375,6 @@ public class InfoResource
|
|||
return Response.status(Response.Status.INTERNAL_SERVER_ERROR).build();
|
||||
}
|
||||
|
||||
@POST
|
||||
@Path("master/config")
|
||||
@Consumes("application/json")
|
||||
public Response setDynamicConfigs(
|
||||
final DynamicConfigs dynamicConfigs
|
||||
)
|
||||
{
|
||||
if (!configManager.set(DynamicConfigs.CONFIG_KEY, dynamicConfigs)) {
|
||||
return Response.status(Response.Status.BAD_REQUEST).build();
|
||||
}
|
||||
return Response.status(Response.Status.OK).build();
|
||||
}
|
||||
|
||||
|
||||
@GET
|
||||
@Path("/datasources")
|
||||
@Produces("application/json")
|
||||
|
|
|
@ -332,6 +332,7 @@ public class MasterMain
|
|||
);
|
||||
root.addFilter(GuiceFilter.class, "/info/*", 0);
|
||||
root.addFilter(GuiceFilter.class, "/master/*", 0);
|
||||
root.addFilter(GuiceFilter.class, "/masterSegmentSettings/*",0);
|
||||
|
||||
server.start();
|
||||
server.join();
|
||||
|
|
|
@ -24,13 +24,10 @@ import com.metamx.druid.master.LoadPeonCallback;
|
|||
|
||||
import javax.inject.Inject;
|
||||
import javax.ws.rs.Consumes;
|
||||
import javax.ws.rs.GET;
|
||||
import javax.ws.rs.POST;
|
||||
import javax.ws.rs.Path;
|
||||
import javax.ws.rs.Produces;
|
||||
import javax.ws.rs.core.Response;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
*/
|
||||
|
@ -110,11 +107,5 @@ public class MasterResource
|
|||
return resp;
|
||||
}
|
||||
|
||||
@GET
|
||||
@Path("/loadstatus")
|
||||
@Produces("application/json")
|
||||
public Map<String, Double> getLoadStatus()
|
||||
{
|
||||
return master.getLoadStatus();
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,71 @@
|
|||
/*
|
||||
* Druid - a distributed column store.
|
||||
* Copyright (C) 2012 Metamarkets Group Inc.
|
||||
*
|
||||
* This program is free software; you can redistribute it and/or
|
||||
* modify it under the terms of the GNU General Public License
|
||||
* as published by the Free Software Foundation; either version 2
|
||||
* of the License, or (at your option) any later version.
|
||||
*
|
||||
* This program is distributed in the hope that it will be useful,
|
||||
* but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||
* GNU General Public License for more details.
|
||||
*
|
||||
* You should have received a copy of the GNU General Public License
|
||||
* along with this program; if not, write to the Free Software
|
||||
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
|
||||
*/
|
||||
package com.metamx.druid.http;
|
||||
|
||||
import com.metamx.druid.config.JacksonConfigManager;
|
||||
import com.metamx.druid.master.MasterSegmentSettings;
|
||||
|
||||
import javax.inject.Inject;
|
||||
import javax.ws.rs.Consumes;
|
||||
import javax.ws.rs.GET;
|
||||
import javax.ws.rs.POST;
|
||||
import javax.ws.rs.Path;
|
||||
import javax.ws.rs.Produces;
|
||||
import javax.ws.rs.core.Response;
|
||||
|
||||
/**
|
||||
*/
|
||||
@Path("/masterSegmentSettings")
|
||||
public class MasterSegmentSettingsResource
|
||||
{
|
||||
private final JacksonConfigManager manager;
|
||||
|
||||
@Inject
|
||||
public MasterSegmentSettingsResource(
|
||||
JacksonConfigManager manager
|
||||
)
|
||||
{
|
||||
this.manager=manager;
|
||||
}
|
||||
@GET
|
||||
@Path("/config")
|
||||
@Produces("application/json")
|
||||
public Response getDynamicConfigs()
|
||||
{
|
||||
Response.ResponseBuilder builder = Response.status(Response.Status.OK)
|
||||
.entity(
|
||||
manager.watch(MasterSegmentSettings.CONFIG_KEY,MasterSegmentSettings.class).get()
|
||||
);
|
||||
return builder.build();
|
||||
}
|
||||
|
||||
@POST
|
||||
@Path("/config")
|
||||
@Consumes("application/json")
|
||||
public Response setDynamicConfigs(
|
||||
final MasterSegmentSettings masterSegmentSettings
|
||||
)
|
||||
{
|
||||
if (!manager.set(MasterSegmentSettings.CONFIG_KEY, masterSegmentSettings)) {
|
||||
return Response.status(Response.Status.BAD_REQUEST).build();
|
||||
}
|
||||
return Response.status(Response.Status.OK).build();
|
||||
}
|
||||
|
||||
}
|
|
@ -46,6 +46,7 @@ public class MasterServletModule extends JerseyServletModule
|
|||
private final IndexingServiceClient indexingServiceClient;
|
||||
private final JacksonConfigManager configManager;
|
||||
|
||||
|
||||
public MasterServletModule(
|
||||
InventoryView serverInventoryView,
|
||||
DatabaseSegmentManager segmentInventoryManager,
|
||||
|
@ -68,6 +69,7 @@ public class MasterServletModule extends JerseyServletModule
|
|||
@Override
|
||||
protected void configureServlets()
|
||||
{
|
||||
bind(MasterSegmentSettingsResource.class);
|
||||
bind(InfoResource.class);
|
||||
bind(MasterResource.class);
|
||||
bind(InventoryView.class).toInstance(serverInventoryView);
|
||||
|
|
|
@ -95,6 +95,7 @@ public class DruidMaster
|
|||
|
||||
private final Map<String, LoadQueuePeon> loadManagementPeons;
|
||||
private final AtomicReference<LeaderLatch> leaderLatch;
|
||||
private AtomicReference<MasterSegmentSettings> segmentSettingsAtomicReference;
|
||||
|
||||
public DruidMaster(
|
||||
DruidMasterConfig config,
|
||||
|
@ -156,6 +157,7 @@ public class DruidMaster
|
|||
this.exec = scheduledExecutorFactory.create(1, "Master-Exec--%d");
|
||||
|
||||
this.leaderLatch = new AtomicReference<LeaderLatch>(null);
|
||||
this.segmentSettingsAtomicReference= new AtomicReference<MasterSegmentSettings>(null);
|
||||
this.loadManagementPeons = loadQueuePeonMap;
|
||||
}
|
||||
|
||||
|
@ -465,8 +467,7 @@ public class DruidMaster
|
|||
serverInventoryView.start();
|
||||
|
||||
final List<Pair<? extends MasterRunnable, Duration>> masterRunnables = Lists.newArrayList();
|
||||
configManager.watch(DynamicConfigs.CONFIG_KEY, DynamicConfigs.class);
|
||||
configManager.set(DynamicConfigs.CONFIG_KEY, (new DynamicConfigs.Builder()).build());
|
||||
segmentSettingsAtomicReference = configManager.watch(MasterSegmentSettings.CONFIG_KEY, MasterSegmentSettings.class,(new MasterSegmentSettings.Builder()).build());
|
||||
masterRunnables.add(Pair.of(new MasterComputeManagerRunnable(), config.getMasterPeriod()));
|
||||
if (indexingServiceClient != null) {
|
||||
|
||||
|
@ -651,13 +652,11 @@ public class DruidMaster
|
|||
}
|
||||
|
||||
// Do master stuff.
|
||||
DynamicConfigs dynamicConfigs = configManager.watch(DynamicConfigs.CONFIG_KEY,DynamicConfigs.class).get();
|
||||
|
||||
DruidMasterRuntimeParams params =
|
||||
DruidMasterRuntimeParams.newBuilder()
|
||||
.withStartTime(startTime)
|
||||
.withDatasources(databaseSegmentManager.getInventory())
|
||||
.withDynamicConfigs(dynamicConfigs)
|
||||
.withMasterSegmentSettings(segmentSettingsAtomicReference.get())
|
||||
.withEmitter(emitter)
|
||||
.build();
|
||||
|
||||
|
@ -756,7 +755,9 @@ public class DruidMaster
|
|||
.withLoadManagementPeons(loadManagementPeons)
|
||||
.withSegmentReplicantLookup(segmentReplicantLookup)
|
||||
.withBalancerReferenceTimestamp(DateTime.now())
|
||||
.withDynamicConfigs(configManager.watch(DynamicConfigs.CONFIG_KEY,DynamicConfigs.class).get())
|
||||
.withMasterSegmentSettings(
|
||||
segmentSettingsAtomicReference.get()
|
||||
)
|
||||
.build();
|
||||
}
|
||||
},
|
||||
|
|
|
@ -79,7 +79,7 @@ public class DruidMasterBalancer implements DruidMasterHelper
|
|||
final MasterStats stats = new MasterStats();
|
||||
final DateTime referenceTimestamp = params.getBalancerReferenceTimestamp();
|
||||
final BalancerCostAnalyzer analyzer = params.getBalancerCostAnalyzer(referenceTimestamp);
|
||||
final int maxSegmentsToMove = params.getMaxSegmentsToMove();
|
||||
final int maxSegmentsToMove = params.getMasterSegmentSettings().getMaxSegmentsToMove();
|
||||
|
||||
for (Map.Entry<String, MinMaxPriorityQueue<ServerHolder>> entry :
|
||||
params.getDruidCluster().getCluster().entrySet()) {
|
||||
|
|
|
@ -46,7 +46,7 @@ public class DruidMasterRuntimeParams
|
|||
private final Map<String, LoadQueuePeon> loadManagementPeons;
|
||||
private final ReplicationThrottler replicationManager;
|
||||
private final ServiceEmitter emitter;
|
||||
private final DynamicConfigs dynamicConfigs;
|
||||
private final MasterSegmentSettings masterSegmentSettings;
|
||||
private final MasterStats stats;
|
||||
private final DateTime balancerReferenceTimestamp;
|
||||
|
||||
|
@ -60,7 +60,7 @@ public class DruidMasterRuntimeParams
|
|||
Map<String, LoadQueuePeon> loadManagementPeons,
|
||||
ReplicationThrottler replicationManager,
|
||||
ServiceEmitter emitter,
|
||||
DynamicConfigs dynamicConfigs,
|
||||
MasterSegmentSettings masterSegmentSettings,
|
||||
MasterStats stats,
|
||||
DateTime balancerReferenceTimestamp
|
||||
)
|
||||
|
@ -74,7 +74,7 @@ public class DruidMasterRuntimeParams
|
|||
this.loadManagementPeons = loadManagementPeons;
|
||||
this.replicationManager = replicationManager;
|
||||
this.emitter = emitter;
|
||||
this.dynamicConfigs = dynamicConfigs;
|
||||
this.masterSegmentSettings = masterSegmentSettings;
|
||||
this.stats = stats;
|
||||
this.balancerReferenceTimestamp = balancerReferenceTimestamp;
|
||||
}
|
||||
|
@ -124,9 +124,9 @@ public class DruidMasterRuntimeParams
|
|||
return emitter;
|
||||
}
|
||||
|
||||
public long getMillisToWaitBeforeDeleting()
|
||||
public MasterSegmentSettings getMasterSegmentSettings()
|
||||
{
|
||||
return dynamicConfigs.getMillisToWaitBeforeDeleting();
|
||||
return masterSegmentSettings;
|
||||
}
|
||||
|
||||
public MasterStats getMasterStats()
|
||||
|
@ -134,21 +134,6 @@ public class DruidMasterRuntimeParams
|
|||
return stats;
|
||||
}
|
||||
|
||||
public long getMergeBytesLimit()
|
||||
{
|
||||
return dynamicConfigs.getMergeBytesLimit();
|
||||
}
|
||||
|
||||
public int getMergeSegmentsLimit()
|
||||
{
|
||||
return dynamicConfigs.getMergeSegmentsLimit();
|
||||
}
|
||||
|
||||
public int getMaxSegmentsToMove()
|
||||
{
|
||||
return dynamicConfigs.getMaxSegmentsToMove();
|
||||
}
|
||||
|
||||
public DateTime getBalancerReferenceTimestamp()
|
||||
{
|
||||
return balancerReferenceTimestamp;
|
||||
|
@ -161,7 +146,7 @@ public class DruidMasterRuntimeParams
|
|||
|
||||
public boolean hasDeletionWaitTimeElapsed()
|
||||
{
|
||||
return (System.currentTimeMillis() - getStartTime() > getMillisToWaitBeforeDeleting());
|
||||
return (System.currentTimeMillis() - getStartTime() > masterSegmentSettings.getMillisToWaitBeforeDeleting());
|
||||
}
|
||||
|
||||
public static Builder newBuilder()
|
||||
|
@ -181,7 +166,7 @@ public class DruidMasterRuntimeParams
|
|||
loadManagementPeons,
|
||||
replicationManager,
|
||||
emitter,
|
||||
dynamicConfigs,
|
||||
masterSegmentSettings,
|
||||
stats,
|
||||
balancerReferenceTimestamp
|
||||
);
|
||||
|
@ -198,7 +183,7 @@ public class DruidMasterRuntimeParams
|
|||
private final Map<String, LoadQueuePeon> loadManagementPeons;
|
||||
private ReplicationThrottler replicationManager;
|
||||
private ServiceEmitter emitter;
|
||||
private DynamicConfigs dynamicConfigs;
|
||||
private MasterSegmentSettings masterSegmentSettings;
|
||||
private MasterStats stats;
|
||||
private DateTime balancerReferenceTimestamp;
|
||||
|
||||
|
@ -214,7 +199,7 @@ public class DruidMasterRuntimeParams
|
|||
this.replicationManager = null;
|
||||
this.emitter = null;
|
||||
this.stats = new MasterStats();
|
||||
this.dynamicConfigs = new DynamicConfigs.Builder().build();
|
||||
this.masterSegmentSettings = new MasterSegmentSettings.Builder().build();
|
||||
this.balancerReferenceTimestamp = null;
|
||||
}
|
||||
|
||||
|
@ -228,7 +213,7 @@ public class DruidMasterRuntimeParams
|
|||
Map<String, LoadQueuePeon> loadManagementPeons,
|
||||
ReplicationThrottler replicationManager,
|
||||
ServiceEmitter emitter,
|
||||
DynamicConfigs dynamicConfigs,
|
||||
MasterSegmentSettings masterSegmentSettings,
|
||||
MasterStats stats,
|
||||
DateTime balancerReferenceTimestamp
|
||||
)
|
||||
|
@ -242,7 +227,7 @@ public class DruidMasterRuntimeParams
|
|||
this.loadManagementPeons = loadManagementPeons;
|
||||
this.replicationManager = replicationManager;
|
||||
this.emitter = emitter;
|
||||
this.dynamicConfigs=dynamicConfigs;
|
||||
this.masterSegmentSettings = masterSegmentSettings;
|
||||
this.stats = stats;
|
||||
this.balancerReferenceTimestamp = balancerReferenceTimestamp;
|
||||
}
|
||||
|
@ -259,7 +244,7 @@ public class DruidMasterRuntimeParams
|
|||
loadManagementPeons,
|
||||
replicationManager,
|
||||
emitter,
|
||||
dynamicConfigs,
|
||||
masterSegmentSettings,
|
||||
stats,
|
||||
balancerReferenceTimestamp
|
||||
);
|
||||
|
@ -325,9 +310,9 @@ public class DruidMasterRuntimeParams
|
|||
return this;
|
||||
}
|
||||
|
||||
public Builder withDynamicConfigs(DynamicConfigs configs)
|
||||
public Builder withMasterSegmentSettings(MasterSegmentSettings configs)
|
||||
{
|
||||
this.dynamicConfigs = configs;
|
||||
this.masterSegmentSettings = configs;
|
||||
return this;
|
||||
}
|
||||
|
||||
|
|
|
@ -107,9 +107,9 @@ public class DruidMasterSegmentMerger implements DruidMasterHelper
|
|||
throw Throwables.propagate(e);
|
||||
}
|
||||
|
||||
if (segmentsToMerge.getByteCount() > params.getMergeBytesLimit()
|
||||
|| segmentsToMerge.getSegmentCount() >= params.getMergeSegmentsLimit()) {
|
||||
i -= segmentsToMerge.backtrack(params.getMergeBytesLimit());
|
||||
if (segmentsToMerge.getByteCount() > params.getMasterSegmentSettings().getMergeBytesLimit()
|
||||
|| segmentsToMerge.getSegmentCount() >= params.getMasterSegmentSettings().getMergeSegmentsLimit()) {
|
||||
i -= segmentsToMerge.backtrack(params.getMasterSegmentSettings().getMergeBytesLimit());
|
||||
|
||||
if (segmentsToMerge.getSegmentCount() > 1) {
|
||||
stats.addToGlobalStat("mergedCount", mergeSegments(segmentsToMerge, entry.getKey()));
|
||||
|
@ -125,7 +125,7 @@ public class DruidMasterSegmentMerger implements DruidMasterHelper
|
|||
}
|
||||
|
||||
// Finish any timelineObjects to merge that may have not hit threshold
|
||||
segmentsToMerge.backtrack(params.getMergeBytesLimit());
|
||||
segmentsToMerge.backtrack(params.getMasterSegmentSettings().getMergeBytesLimit());
|
||||
if (segmentsToMerge.getSegmentCount() > 1) {
|
||||
stats.addToGlobalStat("mergedCount", mergeSegments(segmentsToMerge, entry.getKey()));
|
||||
}
|
||||
|
|
|
@ -21,7 +21,7 @@ package com.metamx.druid.master;
|
|||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
|
||||
public class DynamicConfigs
|
||||
public class MasterSegmentSettings
|
||||
{
|
||||
public static final String CONFIG_KEY = "master.dynamicConfigs";
|
||||
private long millisToWaitBeforeDeleting=15 * 60 * 1000L;
|
||||
|
@ -30,10 +30,11 @@ public class DynamicConfigs
|
|||
private int maxSegmentsToMove = 5;
|
||||
|
||||
@JsonCreator
|
||||
public DynamicConfigs(@JsonProperty("millisToWaitBeforeDeleting") Long millisToWaitBeforeDeleting,
|
||||
@JsonProperty("mergeBytesLimit") Long mergeBytesLimit,
|
||||
@JsonProperty("mergeSegmentsLimit") Integer mergeSegmentsLimit,
|
||||
@JsonProperty("maxSegmentsToMove") Integer maxSegmentsToMove
|
||||
public MasterSegmentSettings(
|
||||
@JsonProperty("millisToWaitBeforeDeleting") Long millisToWaitBeforeDeleting,
|
||||
@JsonProperty("mergeBytesLimit") Long mergeBytesLimit,
|
||||
@JsonProperty("mergeSegmentsLimit") Integer mergeSegmentsLimit,
|
||||
@JsonProperty("maxSegmentsToMove") Integer maxSegmentsToMove
|
||||
)
|
||||
{
|
||||
this.maxSegmentsToMove=maxSegmentsToMove;
|
||||
|
@ -120,9 +121,9 @@ public class DynamicConfigs
|
|||
return this;
|
||||
}
|
||||
|
||||
public DynamicConfigs build()
|
||||
public MasterSegmentSettings build()
|
||||
{
|
||||
return new DynamicConfigs(millisToWaitBeforeDeleting,mergeBytesLimit,mergeSegmentsLimit,maxSegmentsToMove);
|
||||
return new MasterSegmentSettings(millisToWaitBeforeDeleting,mergeBytesLimit,mergeSegmentsLimit,maxSegmentsToMove);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -38,7 +38,7 @@
|
|||
<a href="rules.html">Configure Compute Node Rules</a>
|
||||
</div>
|
||||
<div>
|
||||
<a href="dynamicConfigs.html">Configure Dynamic Master Parameters</a>
|
||||
<a href="masterSegmentSettings.html">Configure Dynamic Master Parameters</a>
|
||||
</div>
|
||||
<div>
|
||||
<a href="enable.html">Enable/Disable Datasources</a>
|
||||
|
|
|
@ -1,5 +1,5 @@
|
|||
$(function () {
|
||||
$.get('../info/master/config', function (data) {
|
||||
$.get('../masterSegmentSettings/config', function (data) {
|
||||
document.getElementById("millis").value=data["millisToWaitBeforeDeleting"];
|
||||
document.getElementById("mergeBytes").value = data["mergeBytesLimit"];
|
||||
document.getElementById("mergeSegments").value = data["mergeSegmentsLimit"];
|
||||
|
@ -15,7 +15,7 @@ $(function () {
|
|||
values[list[i]["name"]]=list[i]["value"];
|
||||
}
|
||||
$.ajax({
|
||||
url:'../info/master/config',
|
||||
url:'../masterSegmentSettings/config',
|
||||
type:"POST",
|
||||
data: JSON.stringify(values),
|
||||
contentType:"application/json; charset=utf-8",
|
|
@ -17,7 +17,7 @@
|
|||
</form>
|
||||
</body>
|
||||
<script type="text/javascript" src="js/jquery-1.8.3.js"></script>
|
||||
<script type="text/javascript" src="js/dynamicConfigs.js"></script>
|
||||
<script type="text/javascript" src="js/masterSegmentSettings.js"></script>
|
||||
|
||||
</script>
|
||||
</html>
|
|
@ -192,8 +192,9 @@ public class DruidMasterBalancerTest
|
|||
)
|
||||
)
|
||||
.withAvailableSegments(segments.values())
|
||||
.withDynamicConfigs(
|
||||
new DynamicConfigs.Builder().withMaxSegmentsToMove(MAX_SEGMENTS_TO_MOVE).build()
|
||||
.withMasterSegmentSettings(
|
||||
new MasterSegmentSettings.Builder().withMaxSegmentsToMove(MAX_SEGMENTS_TO_MOVE)
|
||||
.build()
|
||||
)
|
||||
.withBalancerReferenceTimestamp(new DateTime("2013-01-01"))
|
||||
.build();
|
||||
|
@ -283,7 +284,11 @@ public class DruidMasterBalancerTest
|
|||
)
|
||||
)
|
||||
.withAvailableSegments(segments.values())
|
||||
.withDynamicConfigs(new DynamicConfigs.Builder().withMaxSegmentsToMove(MAX_SEGMENTS_TO_MOVE).build())
|
||||
.withMasterSegmentSettings(
|
||||
new MasterSegmentSettings.Builder().withMaxSegmentsToMove(
|
||||
MAX_SEGMENTS_TO_MOVE
|
||||
).build()
|
||||
)
|
||||
.withBalancerReferenceTimestamp(new DateTime("2013-01-01"))
|
||||
.build();
|
||||
|
||||
|
|
|
@ -177,7 +177,7 @@ public class DruidMasterRuleRunnerTest
|
|||
.withAvailableSegments(availableSegments)
|
||||
.withDatabaseRuleManager(databaseRuleManager)
|
||||
.withSegmentReplicantLookup(SegmentReplicantLookup.make(new DruidCluster()))
|
||||
.withDynamicConfigs(new DynamicConfigs.Builder().withMaxSegmentsToMove(5).build())
|
||||
.withMasterSegmentSettings(new MasterSegmentSettings.Builder().withMaxSegmentsToMove(5).build())
|
||||
.build();
|
||||
|
||||
DruidMasterRuntimeParams afterParams = ruleRunner.run(params);
|
||||
|
@ -522,7 +522,7 @@ public class DruidMasterRuleRunnerTest
|
|||
|
||||
DruidMasterRuntimeParams params = new DruidMasterRuntimeParams.Builder()
|
||||
.withDruidCluster(druidCluster)
|
||||
.withDynamicConfigs(new DynamicConfigs.Builder().withMillisToWaitBeforeDeleting(0L).build())
|
||||
.withMasterSegmentSettings(new MasterSegmentSettings.Builder().withMillisToWaitBeforeDeleting(0L).build())
|
||||
.withAvailableSegments(availableSegments)
|
||||
.withDatabaseRuleManager(databaseRuleManager)
|
||||
.withSegmentReplicantLookup(segmentReplicantLookup)
|
||||
|
@ -596,7 +596,7 @@ public class DruidMasterRuleRunnerTest
|
|||
|
||||
DruidMasterRuntimeParams params = new DruidMasterRuntimeParams.Builder()
|
||||
.withDruidCluster(druidCluster)
|
||||
.withDynamicConfigs(new DynamicConfigs.Builder().withMillisToWaitBeforeDeleting(0L).build())
|
||||
.withMasterSegmentSettings(new MasterSegmentSettings.Builder().withMillisToWaitBeforeDeleting(0L).build())
|
||||
.withAvailableSegments(availableSegments)
|
||||
.withDatabaseRuleManager(databaseRuleManager)
|
||||
.withSegmentReplicantLookup(segmentReplicantLookup)
|
||||
|
@ -677,7 +677,7 @@ public class DruidMasterRuleRunnerTest
|
|||
|
||||
DruidMasterRuntimeParams params = new DruidMasterRuntimeParams.Builder()
|
||||
.withDruidCluster(druidCluster)
|
||||
.withDynamicConfigs(new DynamicConfigs.Builder().withMillisToWaitBeforeDeleting(0L).build())
|
||||
.withMasterSegmentSettings(new MasterSegmentSettings.Builder().withMillisToWaitBeforeDeleting(0L).build())
|
||||
.withAvailableSegments(availableSegments)
|
||||
.withDatabaseRuleManager(databaseRuleManager)
|
||||
.withSegmentReplicantLookup(segmentReplicantLookup)
|
||||
|
@ -754,7 +754,7 @@ public class DruidMasterRuleRunnerTest
|
|||
|
||||
DruidMasterRuntimeParams params = new DruidMasterRuntimeParams.Builder()
|
||||
.withDruidCluster(druidCluster)
|
||||
.withDynamicConfigs(new DynamicConfigs.Builder().withMillisToWaitBeforeDeleting(0L).build())
|
||||
.withMasterSegmentSettings(new MasterSegmentSettings.Builder().withMillisToWaitBeforeDeleting(0L).build())
|
||||
.withAvailableSegments(availableSegments)
|
||||
.withDatabaseRuleManager(databaseRuleManager)
|
||||
.withSegmentReplicantLookup(segmentReplicantLookup)
|
||||
|
@ -843,7 +843,7 @@ public class DruidMasterRuleRunnerTest
|
|||
|
||||
DruidMasterRuntimeParams params = new DruidMasterRuntimeParams.Builder()
|
||||
.withDruidCluster(druidCluster)
|
||||
.withDynamicConfigs(new DynamicConfigs.Builder().withMillisToWaitBeforeDeleting(0L).build())
|
||||
.withMasterSegmentSettings(new MasterSegmentSettings.Builder().withMillisToWaitBeforeDeleting(0L).build())
|
||||
.withAvailableSegments(availableSegments)
|
||||
.withDatabaseRuleManager(databaseRuleManager)
|
||||
.withSegmentReplicantLookup(segmentReplicantLookup)
|
||||
|
@ -1029,7 +1029,7 @@ public class DruidMasterRuleRunnerTest
|
|||
|
||||
DruidMasterRuntimeParams params = new DruidMasterRuntimeParams.Builder()
|
||||
.withDruidCluster(druidCluster)
|
||||
.withDynamicConfigs(new DynamicConfigs.Builder().withMillisToWaitBeforeDeleting(0L).build())
|
||||
.withMasterSegmentSettings(new MasterSegmentSettings.Builder().withMillisToWaitBeforeDeleting(0L).build())
|
||||
.withAvailableSegments(longerAvailableSegments)
|
||||
.withDatabaseRuleManager(databaseRuleManager)
|
||||
.withSegmentReplicantLookup(segmentReplicantLookup)
|
||||
|
|
|
@ -421,8 +421,15 @@ public class DruidMasterSegmentMergerTest
|
|||
final DruidMasterSegmentMerger merger = new DruidMasterSegmentMerger(indexingServiceClient, whitelistRef);
|
||||
final DruidMasterRuntimeParams params = DruidMasterRuntimeParams.newBuilder()
|
||||
.withAvailableSegments(ImmutableSet.copyOf(segments))
|
||||
.withDynamicConfigs(
|
||||
new DynamicConfigs.Builder().withMergeBytesLimit(mergeBytesLimit).withMergeSegmentsLimit(mergeSegmentsLimit).build())
|
||||
.withMasterSegmentSettings(
|
||||
new MasterSegmentSettings.Builder().withMergeBytesLimit(
|
||||
mergeBytesLimit
|
||||
)
|
||||
.withMergeSegmentsLimit(
|
||||
mergeSegmentsLimit
|
||||
)
|
||||
.build()
|
||||
)
|
||||
.build();
|
||||
merger.run(params);
|
||||
return retVal;
|
||||
|
|
Loading…
Reference in New Issue