Michael Blow has uploaded a new change for review.
https://asterix-gerrit.ics.uci.edu/1147
Change subject: Access To Node Configuration & Statistics
......................................................................
Access To Node Configuration & Statistics
Change-Id: Ia7549f2bb0b6621886356d50df800d447928aa2c
---
M
asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/ClusterAPIServlet.java
M
asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplicationEntryPoint.java
M
asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/util/AsterixClusterProperties.java
M
hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/HyracksClientInterfaceFunctions.java
M
hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/HyracksClientInterfaceRemoteProxy.java
M
hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/HyracksConnection.java
M
hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/IHyracksClientConnection.java
M
hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/IHyracksClientInterface.java
M
hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/ClusterControllerService.java
M
hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/NodeControllerState.java
M
hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/web/NodesRESTAPIFunction.java
M
hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/GetNodeDetailsJSONWork.java
M
hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/controllers/NodeRegistration.java
A
hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/utils/PidHelper.java
M
hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/NodeControllerService.java
15 files changed, 231 insertions(+), 69 deletions(-)
git pull ssh://asterix-gerrit.ics.uci.edu:29418/asterixdb
refs/changes/47/1147/1
diff --git
a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/ClusterAPIServlet.java
b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/ClusterAPIServlet.java
index c7cf1ea..6717422 100644
---
a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/ClusterAPIServlet.java
+++
b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/ClusterAPIServlet.java
@@ -40,7 +40,7 @@
PrintWriter responseWriter = response.getWriter();
try {
JSONObject responseObject =
AsterixClusterProperties.INSTANCE.getClusterStateDescription();
- responseWriter.write(responseObject.toString());
+ responseWriter.write(responseObject.toString(4));
response.setStatus(HttpServletResponse.SC_OK);
} catch (JSONException e) {
ResultUtil.apiErrorHandler(responseWriter, e);
diff --git
a/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplicationEntryPoint.java
b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplicationEntryPoint.java
index 5b36782..1c59c33 100644
---
a/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplicationEntryPoint.java
+++
b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplicationEntryPoint.java
@@ -72,6 +72,7 @@
import org.eclipse.jetty.server.Server;
import org.eclipse.jetty.servlet.ServletContextHandler;
import org.eclipse.jetty.servlet.ServletHolder;
+import org.eclipse.jetty.servlet.ServletMapping;
public class CCApplicationEntryPoint implements ICCApplicationEntryPoint {
@@ -235,8 +236,13 @@
return queryWebServer;
}
- protected void addServlet(ServletContextHandler context, Servlet servlet,
String path) {
- context.addServlet(new ServletHolder(servlet), path);
+ protected void addServlet(ServletContextHandler context, Servlet servlet,
String... paths) {
+ final ServletHolder holder = new ServletHolder(servlet);
+ context.getServletHandler().addServlet(holder);
+ ServletMapping mapping = new ServletMapping();
+ mapping.setServletName(holder.getName());
+ mapping.setPathSpecs(paths);
+ context.getServletHandler().addServletMapping(mapping);
}
protected void addServlet(ServletContextHandler context, Servlets key) {
diff --git
a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/util/AsterixClusterProperties.java
b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/util/AsterixClusterProperties.java
index 2457ddc..d201d60 100644
---
a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/util/AsterixClusterProperties.java
+++
b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/util/AsterixClusterProperties.java
@@ -668,10 +668,17 @@
public synchronized JSONObject getClusterStateDescription() throws
JSONException {
JSONObject stateDescription = new JSONObject();
- stateDescription.put("State", state.name());
- stateDescription.put("Metadata_Node", currentMetadataNode);
- for (ClusterPartition partition : clusterPartitions.values()) {
- stateDescription.put("partition_" + partition.getPartitionId(),
partition.getActiveNodeId());
+ stateDescription.put("state", state.name());
+ stateDescription.put("metadata_node", currentMetadataNode);
+ for (Map.Entry<String, ClusterPartition[]> entry :
node2PartitionsMap.entrySet()) {
+ JSONObject nodeJSON = new JSONObject();
+ nodeJSON.put("node_id", entry.getKey());
+ List<String> partitions = new ArrayList<>();
+ for (ClusterPartition part : entry.getValue()) {
+ partitions.add("partition_" + part.getPartitionId());
+ }
+ nodeJSON.put("partitions", partitions);
+ stateDescription.accumulate("ncs", nodeJSON);
}
return stateDescription;
}
diff --git
a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/HyracksClientInterfaceFunctions.java
b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/HyracksClientInterfaceFunctions.java
index bf36183..ca0783b 100644
---
a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/HyracksClientInterfaceFunctions.java
+++
b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/HyracksClientInterfaceFunctions.java
@@ -45,7 +45,8 @@
GET_NODE_CONTROLLERS_INFO,
CLI_DEPLOY_BINARY,
CLI_UNDEPLOY_BINARY,
- CLUSTER_SHUTDOWN
+ CLUSTER_SHUTDOWN,
+ GET_NODE_DETAILS_JSON
}
public abstract static class Function implements Serializable {
@@ -294,4 +295,34 @@
}
}
+ public static class GetNodeDetailsJSONFunction extends Function {
+ private static final long serialVersionUID = 1L;
+ private final String nodeId;
+ private final boolean includeStats;
+ private final boolean includeConfig;
+
+ public GetNodeDetailsJSONFunction(String nodeId, boolean includeStats,
boolean includeConfig) {
+ this.nodeId = nodeId;
+ this.includeStats = includeStats;
+ this.includeConfig = includeConfig;
+ }
+
+ public String getNodeId() {
+ return nodeId;
+ }
+
+ public boolean isIncludeStats() {
+ return includeStats;
+ }
+
+ public boolean isIncludeConfig() {
+ return includeConfig;
+ }
+
+ @Override
+ public FunctionId getFunctionId() {
+ return FunctionId.GET_NODE_DETAILS_JSON;
+ }
+ }
+
}
diff --git
a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/HyracksClientInterfaceRemoteProxy.java
b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/HyracksClientInterfaceRemoteProxy.java
index 25b5a0f..3f453e5 100644
---
a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/HyracksClientInterfaceRemoteProxy.java
+++
b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/HyracksClientInterfaceRemoteProxy.java
@@ -131,4 +131,11 @@
throw new IPCException("CC refused to release connection after 9
seconds");
}
}
+
+ @Override
+ public String getNodeDetailsJSON(String nodeId, boolean includeStats,
boolean includeConfig) throws Exception {
+ HyracksClientInterfaceFunctions.GetNodeDetailsJSONFunction gjsf =
+ new
HyracksClientInterfaceFunctions.GetNodeDetailsJSONFunction(nodeId,
includeStats, includeConfig);
+ return (String) rpci.call(ipcHandle, gjsf);
+ }
}
diff --git
a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/HyracksConnection.java
b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/HyracksConnection.java
index 3f1ced6..73813f3 100644
---
a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/HyracksConnection.java
+++
b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/HyracksConnection.java
@@ -187,8 +187,14 @@
public JobInfo getJobInfo(JobId jobId) throws Exception {
return hci.getJobInfo(jobId);
}
+
@Override
public void stopCluster() throws Exception{
hci.stopCluster();
}
+
+ @Override
+ public String getNodeDetailsJSON(String nodeId, boolean includeStats,
boolean includeConfig) throws Exception {
+ return hci.getNodeDetailsJSON(nodeId, includeStats, includeConfig);
+ }
}
diff --git
a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/IHyracksClientConnection.java
b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/IHyracksClientConnection.java
index 824c914..8ee1cc0 100644
---
a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/IHyracksClientConnection.java
+++
b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/IHyracksClientConnection.java
@@ -31,6 +31,7 @@
import org.apache.hyracks.api.job.JobSpecification;
import org.apache.hyracks.api.job.JobStatus;
import org.apache.hyracks.api.topology.ClusterTopology;
+import org.json.JSONObject;
/**
* Interface used by clients to communicate with the Hyracks Cluster
Controller.
@@ -188,4 +189,13 @@
*/
public void stopCluster() throws Exception;
+ /**
+ * Get details of specified node as JSON object
+ * @param nodeId
+ * id the subject node
+ * @param includeStats
+ *@param includeConfig @return {@link JSONObject} containing the node
details
+ * @throws Exception
+ */
+ public String getNodeDetailsJSON(String nodeId, boolean includeStats,
boolean includeConfig) throws Exception;
}
diff --git
a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/IHyracksClientInterface.java
b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/IHyracksClientInterface.java
index b70ef41..4ddb81f 100644
---
a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/IHyracksClientInterface.java
+++
b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/IHyracksClientInterface.java
@@ -56,4 +56,5 @@
public void stopCluster() throws Exception;
+ public String getNodeDetailsJSON(String nodeId, boolean includeStats,
boolean includeConfig) throws Exception;
}
diff --git
a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/ClusterControllerService.java
b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/ClusterControllerService.java
index ae097a6..8dada48 100644
---
a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/ClusterControllerService.java
+++
b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/ClusterControllerService.java
@@ -41,7 +41,6 @@
import org.apache.hyracks.api.client.NodeControllerInfo;
import org.apache.hyracks.api.comm.NetworkAddress;
import org.apache.hyracks.api.context.ICCContext;
-import org.apache.hyracks.api.dataset.DatasetDirectoryRecord;
import org.apache.hyracks.api.dataset.DatasetJobRecord.Status;
import org.apache.hyracks.api.deployment.DeploymentId;
import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -67,6 +66,7 @@
import org.apache.hyracks.control.cc.work.GetJobInfoWork;
import org.apache.hyracks.control.cc.work.GetJobStatusWork;
import org.apache.hyracks.control.cc.work.GetNodeControllersInfoWork;
+import org.apache.hyracks.control.cc.work.GetNodeDetailsJSONWork;
import org.apache.hyracks.control.cc.work.GetResultPartitionLocationsWork;
import org.apache.hyracks.control.cc.work.GetResultStatusWork;
import org.apache.hyracks.control.cc.work.JobStartWork;
@@ -418,6 +418,8 @@
return;
}
+ case CREATE_JOB:
+ break;
case GET_JOB_STATUS: {
HyracksClientInterfaceFunctions.GetJobStatusFunction gjsf =
(HyracksClientInterfaceFunctions.GetJobStatusFunction) fn;
@@ -457,12 +459,14 @@
return;
}
+ case GET_DATASET_RECORD_DESCRIPTOR:
+ break;
case GET_DATASET_RESULT_LOCATIONS: {
HyracksClientInterfaceFunctions.GetDatasetResultLocationsFunction gdrlf =
(HyracksClientInterfaceFunctions.GetDatasetResultLocationsFunction) fn;
workQueue.schedule(new
GetResultPartitionLocationsWork(ClusterControllerService.this,
gdrlf.getJobId(), gdrlf.getResultSetId(),
gdrlf.getKnownRecords(),
- new IPCResponder<DatasetDirectoryRecord[]>(handle,
mid)));
+ new IPCResponder<>(handle, mid)));
return;
}
@@ -476,7 +480,7 @@
case GET_NODE_CONTROLLERS_INFO: {
workQueue.schedule(new
GetNodeControllersInfoWork(ClusterControllerService.this,
- new IPCResponder<Map<String,
NodeControllerInfo>>(handle, mid)));
+ new IPCResponder<>(handle, mid)));
return;
}
@@ -493,7 +497,7 @@
HyracksClientInterfaceFunctions.CliDeployBinaryFunction
dbf =
(HyracksClientInterfaceFunctions.CliDeployBinaryFunction) fn;
workQueue.schedule(new
CliDeployBinaryWork(ClusterControllerService.this, dbf.getBinaryURLs(),
- dbf.getDeploymentId(), new
IPCResponder<DeploymentId>(handle, mid)));
+ dbf.getDeploymentId(), new IPCResponder<>(handle,
mid)));
return;
}
@@ -501,14 +505,21 @@
HyracksClientInterfaceFunctions.CliUnDeployBinaryFunction
udbf =
(HyracksClientInterfaceFunctions.CliUnDeployBinaryFunction) fn;
workQueue.schedule(new
CliUnDeployBinaryWork(ClusterControllerService.this, udbf.getDeploymentId(),
- new IPCResponder<DeploymentId>(handle, mid)));
+ new IPCResponder<>(handle, mid)));
return;
}
case CLUSTER_SHUTDOWN: {
workQueue.schedule(new
ClusterShutdownWork(ClusterControllerService.this,
- new IPCResponder<Boolean>(handle, mid)));
+ new IPCResponder<>(handle, mid)));
return;
}
+
+ case GET_NODE_DETAILS_JSON:
+ HyracksClientInterfaceFunctions.GetNodeDetailsJSONFunction
gndjf =
+
(HyracksClientInterfaceFunctions.GetNodeDetailsJSONFunction) fn;
+ workQueue.schedule(new
GetNodeDetailsJSONWork(ClusterControllerService.this, gndjf.getNodeId(),
+ gndjf.isIncludeStats(), gndjf.isIncludeConfig(),
new IPCResponder<>(handle, mid)));
+ return;
}
try {
handle.send(mid, null, new IllegalArgumentException("Unknown
function " + fn.getFunctionId()));
diff --git
a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/NodeControllerState.java
b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/NodeControllerState.java
index 7fd027b..a848c6e 100644
---
a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/NodeControllerState.java
+++
b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/NodeControllerState.java
@@ -20,7 +20,6 @@
import java.io.File;
import java.util.Arrays;
-import java.util.Collections;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
@@ -76,6 +75,8 @@
private final List<String> inputArguments;
private final Map<String, String> systemProperties;
+
+ private final int pid;
private final HeartbeatSchema hbSchema;
@@ -147,7 +148,7 @@
dataPort = reg.getDataPort();
datasetPort = reg.getDatasetPort();
messagingPort = reg.getMessagingPort();
- activeJobIds = new HashSet<JobId>();
+ activeJobIds = new HashSet<>();
osName = reg.getOSName();
arch = reg.getArch();
@@ -161,6 +162,7 @@
bootClasspath = reg.getBootClasspath();
inputArguments = reg.getInputArguments();
systemProperties = reg.getSystemProperties();
+ pid = reg.getPid();
hbSchema = reg.getHeartbeatSchema();
@@ -203,7 +205,7 @@
rrdPtr = 0;
}
- public void notifyHeartbeat(HeartbeatData hbData) {
+ public synchronized void notifyHeartbeat(HeartbeatData hbData) {
lastHeartbeatDuration = 0;
hbTime[rrdPtr] = System.currentTimeMillis();
if (hbData != null) {
@@ -282,51 +284,57 @@
return o;
}
- public JSONObject toDetailedJSON() throws JSONException {
+ public synchronized JSONObject toDetailedJSON(boolean includeStats,
boolean includeConfig) throws JSONException {
JSONObject o = new JSONObject();
o.put("node-id", ncConfig.nodeId);
- o.put("os-name", osName);
- o.put("arch", arch);
- o.put("os-version", osVersion);
- o.put("num-processors", nProcessors);
- o.put("vm-name", vmName);
- o.put("vm-version", vmVersion);
- o.put("vm-vendor", vmVendor);
- o.put("classpath", new
JSONArray(Arrays.asList(classpath.split(File.pathSeparator))));
- o.put("library-path", new
JSONArray(Arrays.asList(libraryPath.split(File.pathSeparator))));
- o.put("boot-classpath", new
JSONArray(Arrays.asList(bootClasspath.split(File.pathSeparator))));
- o.put("input-arguments", new JSONArray(inputArguments));
- o.put("rrd-ptr", rrdPtr);
- o.put("heartbeat-times", hbTime);
- o.put("heap-init-sizes", heapInitSize);
- o.put("heap-used-sizes", heapUsedSize);
- o.put("heap-committed-sizes", heapCommittedSize);
- o.put("heap-max-sizes", heapMaxSize);
- o.put("nonheap-init-sizes", nonheapInitSize);
- o.put("nonheap-used-sizes", nonheapUsedSize);
- o.put("nonheap-committed-sizes", nonheapCommittedSize);
- o.put("nonheap-max-sizes", nonheapMaxSize);
- o.put("thread-counts", threadCount);
- o.put("peak-thread-counts", peakThreadCount);
- o.put("system-load-averages", systemLoadAverage);
- o.put("gc-names", gcNames);
- o.put("gc-collection-counts", gcCollectionCounts);
- o.put("gc-collection-times", gcCollectionTimes);
- o.put("net-payload-bytes-read", netPayloadBytesRead);
- o.put("net-payload-bytes-written", netPayloadBytesWritten);
- o.put("net-signaling-bytes-read", netSignalingBytesRead);
- o.put("net-signaling-bytes-written", netSignalingBytesWritten);
- o.put("dataset-net-payload-bytes-read", datasetNetPayloadBytesRead);
- o.put("dataset-net-payload-bytes-written",
datasetNetPayloadBytesWritten);
- o.put("dataset-net-signaling-bytes-read",
datasetNetSignalingBytesRead);
- o.put("dataset-net-signaling-bytes-written",
datasetNetSignalingBytesWritten);
- o.put("ipc-messages-sent", ipcMessagesSent);
- o.put("ipc-message-bytes-sent", ipcMessageBytesSent);
- o.put("ipc-messages-received", ipcMessagesReceived);
- o.put("ipc-message-bytes-received", ipcMessageBytesReceived);
- o.put("disk-reads", diskReads);
- o.put("disk-writes", diskWrites);
+ if (includeConfig) {
+ o.put("os-name", osName);
+ o.put("arch", arch);
+ o.put("os-version", osVersion);
+ o.put("num-processors", nProcessors);
+ o.put("vm-name", vmName);
+ o.put("vm-version", vmVersion);
+ o.put("vm-vendor", vmVendor);
+ o.put("classpath", new
JSONArray(Arrays.asList(classpath.split(File.pathSeparator))));
+ o.put("library-path", new
JSONArray(Arrays.asList(libraryPath.split(File.pathSeparator))));
+ o.put("boot-classpath", new
JSONArray(Arrays.asList(bootClasspath.split(File.pathSeparator))));
+ o.put("input-arguments", new JSONArray(inputArguments));
+ o.put("system-properties", new JSONObject(systemProperties));
+ o.put("pid", pid);
+ }
+ if (includeStats) {
+ o.put("rrd-ptr", rrdPtr);
+ o.put("heartbeat-times", hbTime);
+ o.put("heap-init-sizes", heapInitSize);
+ o.put("heap-used-sizes", heapUsedSize);
+ o.put("heap-committed-sizes", heapCommittedSize);
+ o.put("heap-max-sizes", heapMaxSize);
+ o.put("nonheap-init-sizes", nonheapInitSize);
+ o.put("nonheap-used-sizes", nonheapUsedSize);
+ o.put("nonheap-committed-sizes", nonheapCommittedSize);
+ o.put("nonheap-max-sizes", nonheapMaxSize);
+ o.put("thread-counts", threadCount);
+ o.put("peak-thread-counts", peakThreadCount);
+ o.put("system-load-averages", systemLoadAverage);
+ o.put("gc-names", gcNames);
+ o.put("gc-collection-counts", gcCollectionCounts);
+ o.put("gc-collection-times", gcCollectionTimes);
+ o.put("net-payload-bytes-read", netPayloadBytesRead);
+ o.put("net-payload-bytes-written", netPayloadBytesWritten);
+ o.put("net-signaling-bytes-read", netSignalingBytesRead);
+ o.put("net-signaling-bytes-written", netSignalingBytesWritten);
+ o.put("dataset-net-payload-bytes-read",
datasetNetPayloadBytesRead);
+ o.put("dataset-net-payload-bytes-written",
datasetNetPayloadBytesWritten);
+ o.put("dataset-net-signaling-bytes-read",
datasetNetSignalingBytesRead);
+ o.put("dataset-net-signaling-bytes-written",
datasetNetSignalingBytesWritten);
+ o.put("ipc-messages-sent", ipcMessagesSent);
+ o.put("ipc-message-bytes-sent", ipcMessageBytesSent);
+ o.put("ipc-messages-received", ipcMessagesReceived);
+ o.put("ipc-message-bytes-received", ipcMessageBytesReceived);
+ o.put("disk-reads", diskReads);
+ o.put("disk-writes", diskWrites);
+ }
return o;
}
diff --git
a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/web/NodesRESTAPIFunction.java
b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/web/NodesRESTAPIFunction.java
index 8423ea6..58deb55 100644
---
a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/web/NodesRESTAPIFunction.java
+++
b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/web/NodesRESTAPIFunction.java
@@ -43,7 +43,7 @@
result.put("result", gnse.getSummaries());
} else {
String nodeId = arguments[0];
- GetNodeDetailsJSONWork gnde = new
GetNodeDetailsJSONWork(ccs, nodeId);
+ GetNodeDetailsJSONWork gnde = new
GetNodeDetailsJSONWork(ccs, nodeId, true, true);
ccs.getWorkQueue().scheduleAndSync(gnde);
result.put("result", gnde.getDetail());
}
diff --git
a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/GetNodeDetailsJSONWork.java
b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/GetNodeDetailsJSONWork.java
index ca3f1e5..2b5ed47 100644
---
a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/GetNodeDetailsJSONWork.java
+++
b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/GetNodeDetailsJSONWork.java
@@ -18,6 +18,7 @@
*/
package org.apache.hyracks.control.cc.work;
+import org.apache.hyracks.control.common.work.IPCResponder;
import org.json.JSONObject;
import org.apache.hyracks.control.cc.ClusterControllerService;
@@ -27,21 +28,36 @@
public class GetNodeDetailsJSONWork extends SynchronizableWork {
private final ClusterControllerService ccs;
private final String nodeId;
+ private final boolean includeStats;
+ private final boolean includeConfig;
+ private final IPCResponder<String> callback;
private JSONObject detail;
- public GetNodeDetailsJSONWork(ClusterControllerService ccs, String nodeId)
{
+ public GetNodeDetailsJSONWork(ClusterControllerService ccs, String nodeId,
boolean includeStats,
+ boolean includeConfig, IPCResponder<String>
callback) {
this.ccs = ccs;
this.nodeId = nodeId;
+ this.includeStats = includeStats;
+ this.includeConfig = includeConfig;
+ this.callback = callback;
+ }
+
+ public GetNodeDetailsJSONWork(ClusterControllerService ccs, String nodeId,
boolean includeStats,
+ boolean includeConfig) {
+ this(ccs, nodeId, includeStats, includeConfig, null);
}
@Override
protected void doRun() throws Exception {
NodeControllerState ncs = ccs.getNodeMap().get(nodeId);
if (ncs == null) {
- detail = new JSONObject();
- return;
+ detail = null;
+ } else {
+ detail = ncs.toDetailedJSON(includeStats, includeConfig);
}
- detail = ncs.toDetailedJSON();
+ if (callback != null) {
+ callback.setValue(detail == null ? null : detail.toString());
+ }
}
public JSONObject getDetail() {
diff --git
a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/controllers/NodeRegistration.java
b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/controllers/NodeRegistration.java
index bb8022e..e95a004 100644
---
a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/controllers/NodeRegistration.java
+++
b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/controllers/NodeRegistration.java
@@ -67,11 +67,13 @@
private final NetworkAddress messagingPort;
+ private final int pid;
+
public NodeRegistration(InetSocketAddress ncAddress, String nodeId,
NCConfig ncConfig, NetworkAddress dataPort,
- NetworkAddress datasetPort, String osName, String arch, String
osVersion, int nProcessors, String vmName,
- String vmVersion, String vmVendor, String classpath, String
libraryPath, String bootClasspath,
- List<String> inputArguments, Map<String, String> systemProperties,
HeartbeatSchema hbSchema,
- NetworkAddress messagingPort) {
+ NetworkAddress datasetPort, String osName, String
arch, String osVersion, int nProcessors,
+ String vmName, String vmVersion, String vmVendor,
String classpath, String libraryPath,
+ String bootClasspath, List<String> inputArguments,
Map<String, String> systemProperties,
+ HeartbeatSchema hbSchema, NetworkAddress
messagingPort, int pid) {
this.ncAddress = ncAddress;
this.nodeId = nodeId;
this.ncConfig = ncConfig;
@@ -91,6 +93,7 @@
this.systemProperties = systemProperties;
this.hbSchema = hbSchema;
this.messagingPort = messagingPort;
+ this.pid = pid;
}
public InetSocketAddress getNodeControllerAddress() {
@@ -168,4 +171,6 @@
public NetworkAddress getMessagingPort() {
return messagingPort;
}
+
+ public int getPid() { return pid; }
}
diff --git
a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/utils/PidHelper.java
b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/utils/PidHelper.java
new file mode 100644
index 0000000..79642c0
--- /dev/null
+++
b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/utils/PidHelper.java
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF 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 org.apache.hyracks.control.common.utils;
+
+import java.lang.management.ManagementFactory;
+import java.lang.management.RuntimeMXBean;
+import java.lang.reflect.Field;
+import java.lang.reflect.Method;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+public class PidHelper {
+
+ private static final Logger LOGGER =
Logger.getLogger(PidHelper.class.getName());
+
+ private PidHelper() {
+ }
+
+ public static int getPid() {
+ return getPid(ManagementFactory.getRuntimeMXBean());
+ }
+
+ public static int getPid(RuntimeMXBean runtimeMXBean) {
+ try {
+ Field jvmField = runtimeMXBean.getClass().getDeclaredField("jvm");
+ jvmField.setAccessible(true);
+ Object vmManagement = jvmField.get(runtimeMXBean);
+ Method getProcessIdMethod =
vmManagement.getClass().getDeclaredMethod("getProcessId");
+ getProcessIdMethod.setAccessible(true);
+ return (Integer) getProcessIdMethod.invoke(vmManagement);
+ } catch (Exception e) {
+ LOGGER.log(Level.INFO, "Unable to determine PID due to exception",
e);
+ return -1;
+ }
+ }
+}
diff --git
a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/NodeControllerService.java
b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/NodeControllerService.java
index 8373ebe..edadf57 100644
---
a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/NodeControllerService.java
+++
b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/NodeControllerService.java
@@ -65,6 +65,7 @@
import
org.apache.hyracks.control.common.ipc.CCNCFunctions.StateDumpRequestFunction;
import org.apache.hyracks.control.common.ipc.ClusterControllerRemoteProxy;
import org.apache.hyracks.control.common.job.profiling.om.JobProfile;
+import org.apache.hyracks.control.common.utils.PidHelper;
import org.apache.hyracks.control.common.work.FutureValue;
import org.apache.hyracks.control.common.work.WorkQueue;
import org.apache.hyracks.control.nc.application.NCApplicationContext;
@@ -290,7 +291,8 @@
osMXBean.getName(), osMXBean.getArch(), osMXBean.getVersion(),
osMXBean.getAvailableProcessors(),
runtimeMXBean.getVmName(), runtimeMXBean.getVmVersion(),
runtimeMXBean.getVmVendor(),
runtimeMXBean.getClassPath(), runtimeMXBean.getLibraryPath(),
runtimeMXBean.getBootClassPath(),
- runtimeMXBean.getInputArguments(),
runtimeMXBean.getSystemProperties(), hbSchema, meesagingPort));
+ runtimeMXBean.getInputArguments(),
runtimeMXBean.getSystemProperties(), hbSchema, meesagingPort,
+ PidHelper.getPid()));
synchronized (this) {
while (registrationPending) {
--
To view, visit https://asterix-gerrit.ics.uci.edu/1147
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings
Gerrit-MessageType: newchange
Gerrit-Change-Id: Ia7549f2bb0b6621886356d50df800d447928aa2c
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Michael Blow <[email protected]>