mirror of https://github.com/apache/druid.git
added web console for changing dynamicConfigs
This commit is contained in:
parent
31ffed466c
commit
b0d9889c83
|
@ -140,7 +140,7 @@ public class InfoResource
|
|||
@Produces("application/json")
|
||||
public Response getDynamicConfigs()
|
||||
{
|
||||
Response.ResponseBuilder builder = Response.status(Response.Status.OK);
|
||||
Response.ResponseBuilder builder = Response.status(Response.Status.OK).entity(configManager.watch(DynamicConfigs.CONFIG_KEY,DynamicConfigs.class).get());
|
||||
return builder.build();
|
||||
}
|
||||
|
||||
|
@ -390,7 +390,7 @@ public class InfoResource
|
|||
|
||||
@POST
|
||||
@Path("master/setDynamicConfigs")
|
||||
@Produces("application/json")
|
||||
@Consumes("application/json")
|
||||
public Response setDynamicConfigs(
|
||||
final DynamicConfigs dynamicConfigs)
|
||||
{
|
||||
|
|
|
@ -274,7 +274,8 @@ public class MasterMain
|
|||
databaseRuleManager,
|
||||
master,
|
||||
jsonMapper,
|
||||
indexingServiceClient
|
||||
indexingServiceClient,
|
||||
configManager
|
||||
)
|
||||
);
|
||||
|
||||
|
|
|
@ -25,6 +25,7 @@ import com.google.inject.Provides;
|
|||
import com.google.inject.util.Providers;
|
||||
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;
|
||||
|
@ -43,6 +44,7 @@ public class MasterServletModule extends JerseyServletModule
|
|||
private final DruidMaster master;
|
||||
private final ObjectMapper jsonMapper;
|
||||
private final IndexingServiceClient indexingServiceClient;
|
||||
private final JacksonConfigManager configManager;
|
||||
|
||||
public MasterServletModule(
|
||||
InventoryView serverInventoryView,
|
||||
|
@ -50,7 +52,8 @@ public class MasterServletModule extends JerseyServletModule
|
|||
DatabaseRuleManager databaseRuleManager,
|
||||
DruidMaster master,
|
||||
ObjectMapper jsonMapper,
|
||||
IndexingServiceClient indexingServiceClient
|
||||
IndexingServiceClient indexingServiceClient,
|
||||
JacksonConfigManager configManager
|
||||
)
|
||||
{
|
||||
this.serverInventoryView = serverInventoryView;
|
||||
|
@ -59,6 +62,7 @@ public class MasterServletModule extends JerseyServletModule
|
|||
this.master = master;
|
||||
this.jsonMapper = jsonMapper;
|
||||
this.indexingServiceClient = indexingServiceClient;
|
||||
this.configManager = configManager;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -70,6 +74,7 @@ public class MasterServletModule extends JerseyServletModule
|
|||
bind(DatabaseSegmentManager.class).toInstance(segmentInventoryManager);
|
||||
bind(DatabaseRuleManager.class).toInstance(databaseRuleManager);
|
||||
bind(DruidMaster.class).toInstance(master);
|
||||
bind(JacksonConfigManager.class).toInstance(configManager);
|
||||
if (indexingServiceClient == null) {
|
||||
bind(IndexingServiceClient.class).toProvider(Providers.<IndexingServiceClient>of(null));
|
||||
}
|
||||
|
|
|
@ -468,7 +468,8 @@ public class DruidMaster
|
|||
final List<Pair<? extends MasterRunnable, Duration>> masterRunnables = Lists.newArrayList();
|
||||
if (!defaultConfigsSet)
|
||||
{
|
||||
configManager.set(DynamicConfigs.CONFIG_KEY, new DynamicConfigs());
|
||||
configManager.watch(DynamicConfigs.CONFIG_KEY, DynamicConfigs.class);
|
||||
configManager.set(DynamicConfigs.CONFIG_KEY, new DynamicConfigs(null,null,null,null));
|
||||
defaultConfigsSet=true;
|
||||
}
|
||||
masterRunnables.add(Pair.of(new MasterComputeManagerRunnable(), config.getMasterPeriod()));
|
||||
|
@ -760,6 +761,7 @@ public class DruidMaster
|
|||
.withLoadManagementPeons(loadManagementPeons)
|
||||
.withSegmentReplicantLookup(segmentReplicantLookup)
|
||||
.withBalancerReferenceTimestamp(DateTime.now())
|
||||
.withDynamicConfigs(configManager.watch(DynamicConfigs.CONFIG_KEY,DynamicConfigs.class).get())
|
||||
.build();
|
||||
}
|
||||
},
|
||||
|
|
|
@ -214,7 +214,7 @@ public class DruidMasterRuntimeParams
|
|||
this.replicationManager = null;
|
||||
this.emitter = null;
|
||||
this.stats = new MasterStats();
|
||||
this.dynamicConfigs = new DynamicConfigs();
|
||||
this.dynamicConfigs = new DynamicConfigs(null,null,null,null);
|
||||
this.balancerReferenceTimestamp = null;
|
||||
}
|
||||
|
||||
|
|
|
@ -54,30 +54,31 @@ public class DynamicConfigs
|
|||
}
|
||||
}
|
||||
|
||||
public DynamicConfigs()
|
||||
{
|
||||
}
|
||||
|
||||
public static String getConfigKey()
|
||||
{
|
||||
return CONFIG_KEY;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public long getMillisToWaitBeforeDeleting()
|
||||
{
|
||||
return millisToWaitBeforeDeleting;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public long getMergeBytesLimit()
|
||||
{
|
||||
return mergeBytesLimit;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public int getMergeSegmentsLimit()
|
||||
{
|
||||
return mergeSegmentsLimit;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public int getMaxSegmentsToMove()
|
||||
{
|
||||
return maxSegmentsToMove;
|
||||
|
|
|
@ -0,0 +1,68 @@
|
|||
<!DOCTYPE html>
|
||||
<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>
|
||||
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/setDynamicConfigs',
|
||||
type:"POST",
|
||||
data:paramJson,
|
||||
contentType:"application/json; charset=utf-8",
|
||||
dataType:"json"
|
||||
});
|
||||
});
|
||||
});
|
||||
});
|
||||
</script>
|
||||
</body>
|
||||
</html>
|
|
@ -37,6 +37,9 @@
|
|||
<div>
|
||||
<a href="rules.html">Configure Compute Node Rules</a>
|
||||
</div>
|
||||
<div>
|
||||
<a href="dynamicConfigs.html">Configure Dynamic Master Parameters</a>
|
||||
</div>
|
||||
<div>
|
||||
<a href="enable.html">Enable/Disable Datasources</a>
|
||||
</div>
|
||||
|
|
|
@ -193,7 +193,7 @@ public class DruidMasterBalancerTest
|
|||
)
|
||||
.withAvailableSegments(segments.values())
|
||||
.withDynamicConfigs(
|
||||
new DynamicConfigs()
|
||||
new DynamicConfigs(null,null,null,null)
|
||||
{
|
||||
@Override
|
||||
public int getMaxSegmentsToMove()
|
||||
|
@ -279,7 +279,7 @@ public class DruidMasterBalancerTest
|
|||
)
|
||||
.withLoadManagementPeons(ImmutableMap.<String, LoadQueuePeon>of("1", peon1, "2", peon2, "3", peon3, "4", peon4))
|
||||
.withAvailableSegments(segments.values())
|
||||
.withDynamicConfigs(new DynamicConfigs()
|
||||
.withDynamicConfigs(new DynamicConfigs(null,null,null,null)
|
||||
{
|
||||
@Override
|
||||
public int getMaxSegmentsToMove()
|
||||
|
|
|
@ -177,7 +177,7 @@ public class DruidMasterRuleRunnerTest
|
|||
.withAvailableSegments(availableSegments)
|
||||
.withDatabaseRuleManager(databaseRuleManager)
|
||||
.withSegmentReplicantLookup(SegmentReplicantLookup.make(new DruidCluster()))
|
||||
.withDynamicConfigs(new DynamicConfigs()
|
||||
.withDynamicConfigs(new DynamicConfigs(null,null,null,null)
|
||||
{
|
||||
@Override
|
||||
public int getMaxSegmentsToMove()
|
||||
|
@ -530,7 +530,7 @@ public class DruidMasterRuleRunnerTest
|
|||
|
||||
DruidMasterRuntimeParams params = new DruidMasterRuntimeParams.Builder()
|
||||
.withDruidCluster(druidCluster)
|
||||
.withDynamicConfigs( new DynamicConfigs()
|
||||
.withDynamicConfigs( new DynamicConfigs(null,null,null,null)
|
||||
{
|
||||
@Override
|
||||
public long getMillisToWaitBeforeDeleting()
|
||||
|
@ -611,7 +611,7 @@ public class DruidMasterRuleRunnerTest
|
|||
|
||||
DruidMasterRuntimeParams params = new DruidMasterRuntimeParams.Builder()
|
||||
.withDruidCluster(druidCluster)
|
||||
.withDynamicConfigs( new DynamicConfigs()
|
||||
.withDynamicConfigs( new DynamicConfigs(null,null,null,null)
|
||||
{
|
||||
@Override
|
||||
public long getMillisToWaitBeforeDeleting()
|
||||
|
@ -699,7 +699,7 @@ public class DruidMasterRuleRunnerTest
|
|||
|
||||
DruidMasterRuntimeParams params = new DruidMasterRuntimeParams.Builder()
|
||||
.withDruidCluster(druidCluster)
|
||||
.withDynamicConfigs( new DynamicConfigs()
|
||||
.withDynamicConfigs( new DynamicConfigs(null,null,null,null)
|
||||
{
|
||||
@Override
|
||||
public long getMillisToWaitBeforeDeleting()
|
||||
|
@ -783,7 +783,7 @@ public class DruidMasterRuleRunnerTest
|
|||
|
||||
DruidMasterRuntimeParams params = new DruidMasterRuntimeParams.Builder()
|
||||
.withDruidCluster(druidCluster)
|
||||
.withDynamicConfigs( new DynamicConfigs()
|
||||
.withDynamicConfigs( new DynamicConfigs(null,null,null,null)
|
||||
{
|
||||
@Override
|
||||
public long getMillisToWaitBeforeDeleting()
|
||||
|
@ -879,7 +879,7 @@ public class DruidMasterRuleRunnerTest
|
|||
|
||||
DruidMasterRuntimeParams params = new DruidMasterRuntimeParams.Builder()
|
||||
.withDruidCluster(druidCluster)
|
||||
.withDynamicConfigs( new DynamicConfigs()
|
||||
.withDynamicConfigs( new DynamicConfigs(null,null,null,null)
|
||||
{
|
||||
@Override
|
||||
public long getMillisToWaitBeforeDeleting()
|
||||
|
@ -1072,7 +1072,7 @@ public class DruidMasterRuleRunnerTest
|
|||
|
||||
DruidMasterRuntimeParams params = new DruidMasterRuntimeParams.Builder()
|
||||
.withDruidCluster(druidCluster)
|
||||
.withDynamicConfigs( new DynamicConfigs()
|
||||
.withDynamicConfigs( new DynamicConfigs(null,null,null,null)
|
||||
{
|
||||
@Override
|
||||
public long getMillisToWaitBeforeDeleting()
|
||||
|
|
|
@ -419,11 +419,10 @@ public class DruidMasterSegmentMergerTest
|
|||
|
||||
final AtomicReference<MergerWhitelist> whitelistRef = new AtomicReference<MergerWhitelist>(null);
|
||||
final DruidMasterSegmentMerger merger = new DruidMasterSegmentMerger(indexingServiceClient, whitelistRef);
|
||||
final DynamicConfigs dynamicConfigs = new DynamicConfigs(null,mergeBytesLimit,mergeSegmentsLimit,null);
|
||||
final DruidMasterRuntimeParams params = DruidMasterRuntimeParams.newBuilder()
|
||||
.withAvailableSegments(ImmutableSet.copyOf(segments))
|
||||
.withDynamicConfigs(
|
||||
new DynamicConfigs()
|
||||
new DynamicConfigs(null,null,null,null)
|
||||
{
|
||||
@Override
|
||||
public long getMergeBytesLimit(){
|
||||
|
|
Loading…
Reference in New Issue