mirror of https://github.com/apache/druid.git
simplified console code. Added Builder. Fixed urls
This commit is contained in:
parent
dd689ca828
commit
9b340d9f6c
|
@ -121,7 +121,7 @@ public class InfoResource
|
|||
this.serverInventoryView = serverInventoryView;
|
||||
this.databaseSegmentManager = databaseSegmentManager;
|
||||
this.databaseRuleManager = databaseRuleManager;
|
||||
this.configManager=configManager;
|
||||
this.configManager = configManager;
|
||||
this.indexingServiceClient = indexingServiceClient;
|
||||
}
|
||||
|
||||
|
@ -136,11 +136,17 @@ public class InfoResource
|
|||
}
|
||||
|
||||
@GET
|
||||
@Path("/master/dynamicConfigs")
|
||||
@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());
|
||||
Response.ResponseBuilder builder = Response.status(Response.Status.OK)
|
||||
.entity(
|
||||
configManager.watch(
|
||||
DynamicConfigs.CONFIG_KEY,
|
||||
DynamicConfigs.class
|
||||
).get()
|
||||
);
|
||||
return builder.build();
|
||||
}
|
||||
|
||||
|
@ -389,13 +395,13 @@ public class InfoResource
|
|||
}
|
||||
|
||||
@POST
|
||||
@Path("master/dynamicConfigs")
|
||||
@Path("master/config")
|
||||
@Consumes("application/json")
|
||||
public Response setDynamicConfigs(
|
||||
final DynamicConfigs dynamicConfigs)
|
||||
final DynamicConfigs dynamicConfigs
|
||||
)
|
||||
{
|
||||
if (!configManager.set(DynamicConfigs.CONFIG_KEY, dynamicConfigs))
|
||||
{
|
||||
if (!configManager.set(DynamicConfigs.CONFIG_KEY, dynamicConfigs)) {
|
||||
return Response.status(Response.Status.BAD_REQUEST).build();
|
||||
}
|
||||
return Response.status(Response.Status.OK).build();
|
||||
|
|
|
@ -80,7 +80,6 @@ public class DruidMaster
|
|||
|
||||
private volatile boolean started = false;
|
||||
private volatile boolean master = false;
|
||||
private volatile boolean defaultConfigsSet=false;
|
||||
|
||||
private final DruidMasterConfig config;
|
||||
private final ZkPathsConfig zkPaths;
|
||||
|
@ -466,12 +465,8 @@ public class DruidMaster
|
|||
serverInventoryView.start();
|
||||
|
||||
final List<Pair<? extends MasterRunnable, Duration>> masterRunnables = Lists.newArrayList();
|
||||
if (!defaultConfigsSet)
|
||||
{
|
||||
configManager.watch(DynamicConfigs.CONFIG_KEY, DynamicConfigs.class);
|
||||
configManager.set(DynamicConfigs.CONFIG_KEY, new DynamicConfigs(null,null,null,null));
|
||||
defaultConfigsSet=true;
|
||||
}
|
||||
configManager.set(DynamicConfigs.CONFIG_KEY, (new DynamicConfigs.Builder()).build());
|
||||
masterRunnables.add(Pair.of(new MasterComputeManagerRunnable(), config.getMasterPeriod()));
|
||||
if (indexingServiceClient != null) {
|
||||
|
||||
|
|
|
@ -214,7 +214,7 @@ public class DruidMasterRuntimeParams
|
|||
this.replicationManager = null;
|
||||
this.emitter = null;
|
||||
this.stats = new MasterStats();
|
||||
this.dynamicConfigs = new DynamicConfigs(null,null,null,null);
|
||||
this.dynamicConfigs = new DynamicConfigs.Builder().build();
|
||||
this.balancerReferenceTimestamp = null;
|
||||
}
|
||||
|
||||
|
|
|
@ -30,31 +30,18 @@ public class DynamicConfigs
|
|||
private int maxSegmentsToMove = 5;
|
||||
|
||||
@JsonCreator
|
||||
public DynamicConfigs(@JsonProperty("millisToWaitBeforeDeleting") Integer millisToWaitBeforeDeleting,
|
||||
public DynamicConfigs(@JsonProperty("millisToWaitBeforeDeleting") Long millisToWaitBeforeDeleting,
|
||||
@JsonProperty("mergeBytesLimit") Long mergeBytesLimit,
|
||||
@JsonProperty("mergeSegmentsLimit") Integer mergeSegmentsLimit,
|
||||
@JsonProperty("maxSegmentsToMove") Integer maxSegmentsToMove
|
||||
)
|
||||
{
|
||||
if (maxSegmentsToMove!=null)
|
||||
{
|
||||
this.maxSegmentsToMove=maxSegmentsToMove;
|
||||
}
|
||||
if (millisToWaitBeforeDeleting!=null)
|
||||
{
|
||||
this.millisToWaitBeforeDeleting=millisToWaitBeforeDeleting;
|
||||
}
|
||||
if (mergeSegmentsLimit!=null)
|
||||
{
|
||||
this.mergeSegmentsLimit=mergeSegmentsLimit;
|
||||
}
|
||||
if (mergeBytesLimit!=null)
|
||||
{
|
||||
this.mergeBytesLimit=mergeBytesLimit;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
public static String getConfigKey()
|
||||
{
|
||||
return CONFIG_KEY;
|
||||
|
@ -83,4 +70,59 @@ public class DynamicConfigs
|
|||
{
|
||||
return maxSegmentsToMove;
|
||||
}
|
||||
|
||||
|
||||
public static class Builder
|
||||
{
|
||||
public static final String CONFIG_KEY = "master.dynamicConfigs";
|
||||
private long millisToWaitBeforeDeleting;
|
||||
private long mergeBytesLimit;
|
||||
private int mergeSegmentsLimit;
|
||||
private int maxSegmentsToMove;
|
||||
|
||||
public Builder()
|
||||
{
|
||||
this.millisToWaitBeforeDeleting=15 * 60 * 1000L;
|
||||
this.mergeBytesLimit= 100000000L;
|
||||
this.mergeSegmentsLimit= Integer.MAX_VALUE;
|
||||
this.maxSegmentsToMove = 5;
|
||||
}
|
||||
|
||||
public Builder(long millisToWaitBeforeDeleting, long mergeBytesLimit, int mergeSegmentsLimit, int maxSegmentsToMove)
|
||||
{
|
||||
this.millisToWaitBeforeDeleting = millisToWaitBeforeDeleting;
|
||||
this.mergeBytesLimit = mergeBytesLimit;
|
||||
this.mergeSegmentsLimit = mergeSegmentsLimit;
|
||||
this.maxSegmentsToMove = maxSegmentsToMove;
|
||||
}
|
||||
|
||||
public Builder withMillisToWaitBeforeDeleting(long millisToWaitBeforeDeleting)
|
||||
{
|
||||
this.millisToWaitBeforeDeleting=millisToWaitBeforeDeleting;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withMergeBytesLimit(long mergeBytesLimit)
|
||||
{
|
||||
this.mergeBytesLimit=mergeBytesLimit;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withMergeSegmentsLimit(int mergeSegmentsLimit)
|
||||
{
|
||||
this.mergeSegmentsLimit=mergeSegmentsLimit;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withMaxSegmentsToMove(int maxSegmentsToMove)
|
||||
{
|
||||
this.maxSegmentsToMove=maxSegmentsToMove;
|
||||
return this;
|
||||
}
|
||||
|
||||
public DynamicConfigs build()
|
||||
{
|
||||
return new DynamicConfigs(millisToWaitBeforeDeleting,mergeBytesLimit,mergeSegmentsLimit,maxSegmentsToMove);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -2,33 +2,22 @@
|
|||
<html>
|
||||
<head>
|
||||
<title>Configure Dynamic Master Parameters</title>
|
||||
<link rel="stylesheet" href="js/SlickGrid-master/slick.grid.css" type="text/css"/>
|
||||
<link rel="stylesheet" href="js/SlickGrid-master/css/smoothness/jquery-ui-1.8.16.custom.css" type="text/css"/>
|
||||
<link rel="stylesheet" href="js/SlickGrid-master/examples/examples.css" type="text/css"/>
|
||||
</head>
|
||||
<body>
|
||||
<table width="100%">
|
||||
<tr>
|
||||
<td valign="top" width="50%">
|
||||
<div id="myGrid" style="width:600px;height:500px;"></div>
|
||||
</td>
|
||||
</tr>
|
||||
</table>
|
||||
|
||||
<script src="js/SlickGrid-master/lib/firebugx.js"></script>
|
||||
|
||||
<script src="js/SlickGrid-master/lib/jquery-1.7.min.js"></script>
|
||||
<script src="js/SlickGrid-master/lib/jquery-ui-1.8.16.custom.min.js"></script>
|
||||
<script src="js/SlickGrid-master/lib/jquery.event.drag-2.2.js"></script>
|
||||
|
||||
<script src="js/SlickGrid-master/slick.core.js"></script>
|
||||
<script src="js/SlickGrid-master/plugins/slick.cellrangedecorator.js"></script>
|
||||
<script src="js/SlickGrid-master/plugins/slick.cellrangeselector.js"></script>
|
||||
<script src="js/SlickGrid-master/plugins/slick.cellselectionmodel.js"></script>
|
||||
<script src="js/SlickGrid-master/slick.formatters.js"></script>
|
||||
<script src="js/SlickGrid-master/slick.editors.js"></script>
|
||||
<script src="js/SlickGrid-master/slick.grid.js"></script>
|
||||
|
||||
<script src="js/createDynamicParamsGrid.js"></script>
|
||||
<form action="process_config.html" autocomplete="on" id ="configs">
|
||||
millisToWaitBeforeDeleting:<input type="text" name="millisToWaitBeforeDeleting" id="millis">
|
||||
<br>
|
||||
mergeBytesLimit: <input type="text" name="mergeBytesLimit" id="mergeBytes">
|
||||
<br>
|
||||
mergeSegmentsLimit: <input type="text" name = "mergeSegmentsLimit" id="mergeSegments">
|
||||
<br>
|
||||
maxSegmentsToMove: <input type= "text" name ="maxSegmentsToMove" id ="maxSegments">
|
||||
<br>
|
||||
<button type="button" id="submit"> Submit </button>
|
||||
</form>
|
||||
</body>
|
||||
<script type="text/javascript" src="js/jquery-1.8.3.js"></script>
|
||||
<script type="text/javascript" src="js/dynamicConfigs.js"></script>
|
||||
|
||||
</script>
|
||||
</html>
|
|
@ -1,33 +0,0 @@
|
|||
var grid;
|
||||
var columns = [
|
||||
{id: "millisToWaitBeforeDeleting", name: "millisToWaitBeforeDeleting", field: "millisToWaitBeforeDeleting",editor: Slick.Editors.LongText},
|
||||
{id: "mergeBytesLimit", name: "mergeBytesLimit", field: "mergeBytesLimit",editor: Slick.Editors.LongText},
|
||||
{id: "mergeSegmentsLimit", name: "mergeSegmentsLimit", field: "mergeSegmentsLimit",editor: Slick.Editors.LongText},
|
||||
{id: "maxSegmentsToMove", name: "maxSegmentsToMove", field: "maxSegmentsToMove",editor: Slick.Editors.LongText}
|
||||
];
|
||||
|
||||
var options = {
|
||||
enableCellNavigation: true,
|
||||
enableColumnReorder: false,
|
||||
editable: true,
|
||||
autoEdit: false,
|
||||
fullWidthRows: true
|
||||
};
|
||||
|
||||
$(function () {
|
||||
$.get('../info/master/dynamicConfigs', function (data) {
|
||||
var rowData = [];
|
||||
rowData[0]=data;
|
||||
grid = new Slick.Grid("#myGrid", rowData, columns, options);
|
||||
grid.onCellChange.subscribe(function (e) {
|
||||
var paramJson = JSON.stringify(grid.getData()[0]);
|
||||
$.ajax({
|
||||
url:'../info/master/dynamicConfigs',
|
||||
type:"POST",
|
||||
data:paramJson,
|
||||
contentType:"application/json; charset=utf-8",
|
||||
dataType:"json"
|
||||
});
|
||||
});
|
||||
});
|
||||
});
|
|
@ -0,0 +1,25 @@
|
|||
$(function () {
|
||||
$.get('../info/master/config', function (data) {
|
||||
document.getElementById("millis").value=data["millisToWaitBeforeDeleting"];
|
||||
document.getElementById("mergeBytes").value = data["mergeBytesLimit"];
|
||||
document.getElementById("mergeSegments").value = data["mergeSegmentsLimit"];
|
||||
document.getElementById("maxSegments").value = data["maxSegmentsToMove"];
|
||||
});
|
||||
|
||||
$("#submit").click( function ()
|
||||
{
|
||||
values = {};
|
||||
list = $('form').serializeArray();
|
||||
for (var i=0;i< list.length;i++)
|
||||
{
|
||||
values[list[i]["name"]]=list[i]["value"];
|
||||
}
|
||||
$.ajax({
|
||||
url:'../info/master/config',
|
||||
type:"POST",
|
||||
data: JSON.stringify(values),
|
||||
contentType:"application/json; charset=utf-8",
|
||||
dataType:"json"
|
||||
});
|
||||
});
|
||||
});
|
|
@ -193,14 +193,7 @@ public class DruidMasterBalancerTest
|
|||
)
|
||||
.withAvailableSegments(segments.values())
|
||||
.withDynamicConfigs(
|
||||
new DynamicConfigs(null,null,null,null)
|
||||
{
|
||||
@Override
|
||||
public int getMaxSegmentsToMove()
|
||||
{
|
||||
return MAX_SEGMENTS_TO_MOVE;
|
||||
}
|
||||
}
|
||||
new DynamicConfigs.Builder().withMaxSegmentsToMove(MAX_SEGMENTS_TO_MOVE).build()
|
||||
)
|
||||
.withBalancerReferenceTimestamp(new DateTime("2013-01-01"))
|
||||
.build();
|
||||
|
@ -277,16 +270,20 @@ public class DruidMasterBalancerTest
|
|||
)
|
||||
)
|
||||
)
|
||||
.withLoadManagementPeons(ImmutableMap.<String, LoadQueuePeon>of("1", peon1, "2", peon2, "3", peon3, "4", peon4))
|
||||
.withLoadManagementPeons(
|
||||
ImmutableMap.<String, LoadQueuePeon>of(
|
||||
"1",
|
||||
peon1,
|
||||
"2",
|
||||
peon2,
|
||||
"3",
|
||||
peon3,
|
||||
"4",
|
||||
peon4
|
||||
)
|
||||
)
|
||||
.withAvailableSegments(segments.values())
|
||||
.withDynamicConfigs(new DynamicConfigs(null,null,null,null)
|
||||
{
|
||||
@Override
|
||||
public int getMaxSegmentsToMove()
|
||||
{
|
||||
return MAX_SEGMENTS_TO_MOVE;
|
||||
}
|
||||
})
|
||||
.withDynamicConfigs(new DynamicConfigs.Builder().withMaxSegmentsToMove(MAX_SEGMENTS_TO_MOVE).build())
|
||||
.withBalancerReferenceTimestamp(new DateTime("2013-01-01"))
|
||||
.build();
|
||||
|
||||
|
|
|
@ -177,15 +177,7 @@ public class DruidMasterRuleRunnerTest
|
|||
.withAvailableSegments(availableSegments)
|
||||
.withDatabaseRuleManager(databaseRuleManager)
|
||||
.withSegmentReplicantLookup(SegmentReplicantLookup.make(new DruidCluster()))
|
||||
.withDynamicConfigs(new DynamicConfigs(null,null,null,null)
|
||||
{
|
||||
@Override
|
||||
public int getMaxSegmentsToMove()
|
||||
{
|
||||
return 5;
|
||||
}
|
||||
})
|
||||
.withBalancerReferenceTimestamp(new DateTime("2013-01-01"))
|
||||
.withDynamicConfigs(new DynamicConfigs.Builder().withMaxSegmentsToMove(5).build())
|
||||
.build();
|
||||
|
||||
DruidMasterRuntimeParams afterParams = ruleRunner.run(params);
|
||||
|
@ -530,14 +522,7 @@ public class DruidMasterRuleRunnerTest
|
|||
|
||||
DruidMasterRuntimeParams params = new DruidMasterRuntimeParams.Builder()
|
||||
.withDruidCluster(druidCluster)
|
||||
.withDynamicConfigs( new DynamicConfigs(null,null,null,null)
|
||||
{
|
||||
@Override
|
||||
public long getMillisToWaitBeforeDeleting()
|
||||
{
|
||||
return 0L;
|
||||
}
|
||||
})
|
||||
.withDynamicConfigs(new DynamicConfigs.Builder().withMillisToWaitBeforeDeleting(0L).build())
|
||||
.withAvailableSegments(availableSegments)
|
||||
.withDatabaseRuleManager(databaseRuleManager)
|
||||
.withSegmentReplicantLookup(segmentReplicantLookup)
|
||||
|
@ -611,14 +596,7 @@ public class DruidMasterRuleRunnerTest
|
|||
|
||||
DruidMasterRuntimeParams params = new DruidMasterRuntimeParams.Builder()
|
||||
.withDruidCluster(druidCluster)
|
||||
.withDynamicConfigs( new DynamicConfigs(null,null,null,null)
|
||||
{
|
||||
@Override
|
||||
public long getMillisToWaitBeforeDeleting()
|
||||
{
|
||||
return 0L;
|
||||
}
|
||||
})
|
||||
.withDynamicConfigs(new DynamicConfigs.Builder().withMillisToWaitBeforeDeleting(0L).build())
|
||||
.withAvailableSegments(availableSegments)
|
||||
.withDatabaseRuleManager(databaseRuleManager)
|
||||
.withSegmentReplicantLookup(segmentReplicantLookup)
|
||||
|
@ -699,14 +677,7 @@ public class DruidMasterRuleRunnerTest
|
|||
|
||||
DruidMasterRuntimeParams params = new DruidMasterRuntimeParams.Builder()
|
||||
.withDruidCluster(druidCluster)
|
||||
.withDynamicConfigs( new DynamicConfigs(null,null,null,null)
|
||||
{
|
||||
@Override
|
||||
public long getMillisToWaitBeforeDeleting()
|
||||
{
|
||||
return 0L;
|
||||
}
|
||||
})
|
||||
.withDynamicConfigs(new DynamicConfigs.Builder().withMillisToWaitBeforeDeleting(0L).build())
|
||||
.withAvailableSegments(availableSegments)
|
||||
.withDatabaseRuleManager(databaseRuleManager)
|
||||
.withSegmentReplicantLookup(segmentReplicantLookup)
|
||||
|
@ -783,14 +754,7 @@ public class DruidMasterRuleRunnerTest
|
|||
|
||||
DruidMasterRuntimeParams params = new DruidMasterRuntimeParams.Builder()
|
||||
.withDruidCluster(druidCluster)
|
||||
.withDynamicConfigs( new DynamicConfigs(null,null,null,null)
|
||||
{
|
||||
@Override
|
||||
public long getMillisToWaitBeforeDeleting()
|
||||
{
|
||||
return 0L;
|
||||
}
|
||||
})
|
||||
.withDynamicConfigs(new DynamicConfigs.Builder().withMillisToWaitBeforeDeleting(0L).build())
|
||||
.withAvailableSegments(availableSegments)
|
||||
.withDatabaseRuleManager(databaseRuleManager)
|
||||
.withSegmentReplicantLookup(segmentReplicantLookup)
|
||||
|
@ -879,14 +843,7 @@ public class DruidMasterRuleRunnerTest
|
|||
|
||||
DruidMasterRuntimeParams params = new DruidMasterRuntimeParams.Builder()
|
||||
.withDruidCluster(druidCluster)
|
||||
.withDynamicConfigs( new DynamicConfigs(null,null,null,null)
|
||||
{
|
||||
@Override
|
||||
public long getMillisToWaitBeforeDeleting()
|
||||
{
|
||||
return 0L;
|
||||
}
|
||||
})
|
||||
.withDynamicConfigs(new DynamicConfigs.Builder().withMillisToWaitBeforeDeleting(0L).build())
|
||||
.withAvailableSegments(availableSegments)
|
||||
.withDatabaseRuleManager(databaseRuleManager)
|
||||
.withSegmentReplicantLookup(segmentReplicantLookup)
|
||||
|
@ -1072,14 +1029,7 @@ public class DruidMasterRuleRunnerTest
|
|||
|
||||
DruidMasterRuntimeParams params = new DruidMasterRuntimeParams.Builder()
|
||||
.withDruidCluster(druidCluster)
|
||||
.withDynamicConfigs( new DynamicConfigs(null,null,null,null)
|
||||
{
|
||||
@Override
|
||||
public long getMillisToWaitBeforeDeleting()
|
||||
{
|
||||
return 0L;
|
||||
}
|
||||
})
|
||||
.withDynamicConfigs(new DynamicConfigs.Builder().withMillisToWaitBeforeDeleting(0L).build())
|
||||
.withAvailableSegments(longerAvailableSegments)
|
||||
.withDatabaseRuleManager(databaseRuleManager)
|
||||
.withSegmentReplicantLookup(segmentReplicantLookup)
|
||||
|
|
|
@ -422,19 +422,8 @@ public class DruidMasterSegmentMergerTest
|
|||
final DruidMasterRuntimeParams params = DruidMasterRuntimeParams.newBuilder()
|
||||
.withAvailableSegments(ImmutableSet.copyOf(segments))
|
||||
.withDynamicConfigs(
|
||||
new DynamicConfigs(null,null,null,null)
|
||||
{
|
||||
@Override
|
||||
public long getMergeBytesLimit(){
|
||||
return mergeBytesLimit;
|
||||
}
|
||||
@Override
|
||||
public int getMergeSegmentsLimit(){
|
||||
return mergeSegmentsLimit;
|
||||
}
|
||||
})
|
||||
new DynamicConfigs.Builder().withMergeBytesLimit(mergeBytesLimit).withMergeSegmentsLimit(mergeSegmentsLimit).build())
|
||||
.build();
|
||||
|
||||
merger.run(params);
|
||||
return retVal;
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue