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 <mb...@apache.org>

Reply via email to