merge hyracks_dev_next r978:1064
git-svn-id: https://hyracks.googlecode.com/svn/branches/aggregators_dev_next@1065 123451ca-8445-de46-9d55-352943316053
diff --git a/hyracks-algebricks/hyracks-algebricks-tests/src/test/java/edu/uci/ics/hyracks/algebricks/tests/util/AlgebricksHyracksIntegrationUtil.java b/hyracks-algebricks/hyracks-algebricks-tests/src/test/java/edu/uci/ics/hyracks/algebricks/tests/util/AlgebricksHyracksIntegrationUtil.java
index 06b76ef..922b0fd 100644
--- a/hyracks-algebricks/hyracks-algebricks-tests/src/test/java/edu/uci/ics/hyracks/algebricks/tests/util/AlgebricksHyracksIntegrationUtil.java
+++ b/hyracks-algebricks/hyracks-algebricks-tests/src/test/java/edu/uci/ics/hyracks/algebricks/tests/util/AlgebricksHyracksIntegrationUtil.java
@@ -81,9 +81,9 @@
public static void runJob(JobSpecification spec) throws Exception {
JobId jobId = hcc.createJob(AlgebricksConfig.HYRACKS_APP_NAME, spec, EnumSet.of(JobFlag.PROFILE_RUNTIME));
AlgebricksConfig.ALGEBRICKS_LOGGER.info(spec.toJSON().toString());
- cc.startJob(jobId);
+ hcc.start(jobId);
AlgebricksConfig.ALGEBRICKS_LOGGER.info(jobId.toString());
- cc.waitForCompletion(jobId);
+ hcc.waitForCompletion(jobId);
}
}
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/HyracksClientInterfaceRemoteProxy.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/HyracksClientInterfaceRemoteProxy.java
index f74d06e..602288c 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/HyracksClientInterfaceRemoteProxy.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/HyracksClientInterfaceRemoteProxy.java
@@ -21,82 +21,76 @@
import edu.uci.ics.hyracks.api.job.JobId;
import edu.uci.ics.hyracks.api.job.JobStatus;
import edu.uci.ics.hyracks.ipc.api.IIPCHandle;
-import edu.uci.ics.hyracks.ipc.api.SyncRMI;
+import edu.uci.ics.hyracks.ipc.api.RPCInterface;
public class HyracksClientInterfaceRemoteProxy implements IHyracksClientInterface {
private final IIPCHandle ipcHandle;
- public HyracksClientInterfaceRemoteProxy(IIPCHandle ipcHandle) {
+ private final RPCInterface rpci;
+
+ public HyracksClientInterfaceRemoteProxy(IIPCHandle ipcHandle, RPCInterface rpci) {
this.ipcHandle = ipcHandle;
+ this.rpci = rpci;
}
@Override
public ClusterControllerInfo getClusterControllerInfo() throws Exception {
- SyncRMI sync = new SyncRMI();
HyracksClientInterfaceFunctions.GetClusterControllerInfoFunction gccif = new HyracksClientInterfaceFunctions.GetClusterControllerInfoFunction();
- return (ClusterControllerInfo) sync.call(ipcHandle, gccif);
+ return (ClusterControllerInfo) rpci.call(ipcHandle, gccif);
}
@Override
public void createApplication(String appName) throws Exception {
- SyncRMI sync = new SyncRMI();
HyracksClientInterfaceFunctions.CreateApplicationFunction caf = new HyracksClientInterfaceFunctions.CreateApplicationFunction(
appName);
- sync.call(ipcHandle, caf);
+ rpci.call(ipcHandle, caf);
}
@Override
public void startApplication(String appName) throws Exception {
- SyncRMI sync = new SyncRMI();
HyracksClientInterfaceFunctions.StartApplicationFunction saf = new HyracksClientInterfaceFunctions.StartApplicationFunction(
appName);
- sync.call(ipcHandle, saf);
+ rpci.call(ipcHandle, saf);
}
@Override
public void destroyApplication(String appName) throws Exception {
- SyncRMI sync = new SyncRMI();
HyracksClientInterfaceFunctions.DestroyApplicationFunction daf = new HyracksClientInterfaceFunctions.DestroyApplicationFunction(
appName);
- sync.call(ipcHandle, daf);
+ rpci.call(ipcHandle, daf);
}
@Override
public JobId createJob(String appName, byte[] jobSpec, EnumSet<JobFlag> jobFlags) throws Exception {
- SyncRMI sync = new SyncRMI();
HyracksClientInterfaceFunctions.CreateJobFunction cjf = new HyracksClientInterfaceFunctions.CreateJobFunction(
appName, jobSpec, jobFlags);
- return (JobId) sync.call(ipcHandle, cjf);
+ return (JobId) rpci.call(ipcHandle, cjf);
}
@Override
public JobStatus getJobStatus(JobId jobId) throws Exception {
- SyncRMI sync = new SyncRMI();
HyracksClientInterfaceFunctions.GetJobStatusFunction gjsf = new HyracksClientInterfaceFunctions.GetJobStatusFunction(
jobId);
- return (JobStatus) sync.call(ipcHandle, gjsf);
+ return (JobStatus) rpci.call(ipcHandle, gjsf);
}
@Override
public void startJob(JobId jobId) throws Exception {
- SyncRMI sync = new SyncRMI();
HyracksClientInterfaceFunctions.StartJobFunction sjf = new HyracksClientInterfaceFunctions.StartJobFunction(
jobId);
- sync.call(ipcHandle, sjf);
+ rpci.call(ipcHandle, sjf);
}
@Override
public void waitForCompletion(JobId jobId) throws Exception {
- SyncRMI sync = new SyncRMI();
HyracksClientInterfaceFunctions.WaitForCompletionFunction wfcf = new HyracksClientInterfaceFunctions.WaitForCompletionFunction(
jobId);
- sync.call(ipcHandle, wfcf);
+ rpci.call(ipcHandle, wfcf);
}
@Override
public Map<String, NodeControllerInfo> getNodeControllersInfo() throws Exception {
- SyncRMI sync = new SyncRMI();
HyracksClientInterfaceFunctions.GetNodeControllersInfoFunction gncif = new HyracksClientInterfaceFunctions.GetNodeControllersInfoFunction();
- return (Map<String, NodeControllerInfo>) sync.call(ipcHandle, gncif);
+ return (Map<String, NodeControllerInfo>) rpci.call(ipcHandle, gncif);
}
}
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/HyracksConnection.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/HyracksConnection.java
index 1b38cb9..8c9409c 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/HyracksConnection.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/HyracksConnection.java
@@ -32,7 +32,9 @@
import edu.uci.ics.hyracks.api.job.JobStatus;
import edu.uci.ics.hyracks.api.util.JavaSerializationUtils;
import edu.uci.ics.hyracks.ipc.api.IIPCHandle;
+import edu.uci.ics.hyracks.ipc.api.RPCInterface;
import edu.uci.ics.hyracks.ipc.impl.IPCSystem;
+import edu.uci.ics.hyracks.ipc.impl.JavaSerializationBasedPayloadSerializerDeserializer;
/**
* Connection Class used by a Hyracks Client to interact with a Hyracks Cluster
@@ -63,10 +65,11 @@
*/
public HyracksConnection(String ccHost, int ccPort) throws Exception {
this.ccHost = ccHost;
- ipc = new IPCSystem(new InetSocketAddress(0));
+ RPCInterface rpci = new RPCInterface();
+ ipc = new IPCSystem(new InetSocketAddress(0), rpci, new JavaSerializationBasedPayloadSerializerDeserializer());
ipc.start();
IIPCHandle ccIpchandle = ipc.getHandle(new InetSocketAddress(ccHost, ccPort));
- this.hci = new HyracksClientInterfaceRemoteProxy(ccIpchandle);
+ this.hci = new HyracksClientInterfaceRemoteProxy(ccIpchandle, rpci);
ccInfo = hci.getClusterControllerInfo();
}
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/NodeControllerInfo.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/NodeControllerInfo.java
index 1424a63..703f74b 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/NodeControllerInfo.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/NodeControllerInfo.java
@@ -15,20 +15,16 @@
package edu.uci.ics.hyracks.api.client;
import java.io.Serializable;
-import java.net.InetAddress;
public class NodeControllerInfo implements Serializable {
private static final long serialVersionUID = 1L;
private final String nodeId;
- private final InetAddress ipAddress;
-
private final NodeStatus status;
- public NodeControllerInfo(String nodeId, InetAddress ipAddress, NodeStatus status) {
+ public NodeControllerInfo(String nodeId, NodeStatus status) {
this.nodeId = nodeId;
- this.ipAddress = ipAddress;
this.status = status;
}
@@ -36,10 +32,6 @@
return nodeId;
}
- public InetAddress getIpAddress() {
- return ipAddress;
- }
-
public NodeStatus getStatus() {
return status;
}
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/NetworkAddress.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/NetworkAddress.java
index 868221d..d176c3c 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/NetworkAddress.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/NetworkAddress.java
@@ -15,21 +15,20 @@
package edu.uci.ics.hyracks.api.comm;
import java.io.Serializable;
-import java.net.InetAddress;
public final class NetworkAddress implements Serializable {
private static final long serialVersionUID = 1L;
- private final InetAddress ipAddress;
+ private final byte[] ipAddress;
private final int port;
- public NetworkAddress(InetAddress ipAddress, int port) {
+ public NetworkAddress(byte[] ipAddress, int port) {
this.ipAddress = ipAddress;
this.port = port;
}
- public InetAddress getIpAddress() {
+ public byte[] getIpAddress() {
return ipAddress;
}
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/context/ICCContext.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/context/ICCContext.java
index 266ebe9..abd9c20 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/context/ICCContext.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/context/ICCContext.java
@@ -18,5 +18,5 @@
import java.util.Set;
public interface ICCContext {
- public Map<String, Set<String>> getIPAddressNodeMap();
+ public void getIPAddressNodeMap(Map<String, Set<String>> map) throws Exception;
}
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/ActivityId.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/ActivityId.java
index 75cc245..41b4c23 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/ActivityId.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/ActivityId.java
@@ -19,9 +19,9 @@
public final class ActivityId implements Serializable {
private static final long serialVersionUID = 1L;
private final OperatorDescriptorId odId;
- private final long id;
+ private final int id;
- public ActivityId(OperatorDescriptorId odId, long id) {
+ public ActivityId(OperatorDescriptorId odId, int id) {
this.odId = odId;
this.id = id;
}
@@ -30,7 +30,7 @@
return odId;
}
- public long getLocalId() {
+ public int getLocalId() {
return id;
}
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/IConnectorDescriptor.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/IConnectorDescriptor.java
index ec8ab76..c816d26 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/IConnectorDescriptor.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/IConnectorDescriptor.java
@@ -44,7 +44,8 @@
public ConnectorDescriptorId getConnectorId();
/**
- * Factory method to create the send side writer that writes into this connector.
+ * Factory method to create the send side writer that writes into this
+ * connector.
*
* @param ctx
* Context
@@ -66,7 +67,8 @@
throws HyracksDataException;
/**
- * Factory metod to create the receive side reader that reads data from this connector.
+ * Factory metod to create the receive side reader that reads data from this
+ * connector.
*
* @param ctx
* Context
@@ -96,12 +98,23 @@
ICCApplicationContext appCtx);
/**
- * Indicate which consumer partitions may receive data from the given producer partition.
+ * Indicate which consumer partitions may receive data from the given
+ * producer partition.
*/
public void indicateTargetPartitions(int nProducerPartitions, int nConsumerPartitions, int producerIndex,
BitSet targetBitmap);
/**
+ * Gets the display name.
+ */
+ public String getDisplayName();
+
+ /**
+ * Sets the display name.
+ */
+ public void setDisplayName(String displayName);
+
+ /**
* Translate this connector descriptor to JSON.
*
* @return
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/IOperatorDescriptor.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/IOperatorDescriptor.java
index 78847eb..c37a530 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/IOperatorDescriptor.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/IOperatorDescriptor.java
@@ -59,7 +59,8 @@
public RecordDescriptor[] getOutputRecordDescriptors();
/**
- * Contributes the activity graph that describes the behavior of this operator.
+ * Contributes the activity graph that describes the behavior of this
+ * operator.
*
* @param builder
* - graph builder
@@ -78,6 +79,16 @@
ICCApplicationContext appCtx);
/**
+ * Gets the display name.
+ */
+ public String getDisplayName();
+
+ /**
+ * Sets the display name.
+ */
+ public void setDisplayName(String displayName);
+
+ /**
* Translates this operator descriptor to JSON.
*/
public JSONObject toJSON() throws JSONException;
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/ClusterControllerService.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/ClusterControllerService.java
index 4575adc..d3e472b 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/ClusterControllerService.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/ClusterControllerService.java
@@ -16,11 +16,9 @@
import java.io.File;
import java.net.InetSocketAddress;
-import java.util.EnumSet;
import java.util.HashMap;
import java.util.Hashtable;
import java.util.LinkedHashMap;
-import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.Timer;
@@ -31,22 +29,19 @@
import java.util.logging.Logger;
import edu.uci.ics.hyracks.api.client.ClusterControllerInfo;
-import edu.uci.ics.hyracks.api.client.IHyracksClientInterface;
+import edu.uci.ics.hyracks.api.client.HyracksClientInterfaceFunctions;
import edu.uci.ics.hyracks.api.client.NodeControllerInfo;
import edu.uci.ics.hyracks.api.context.ICCContext;
-import edu.uci.ics.hyracks.api.dataflow.TaskAttemptId;
-import edu.uci.ics.hyracks.api.job.JobFlag;
import edu.uci.ics.hyracks.api.job.JobId;
import edu.uci.ics.hyracks.api.job.JobStatus;
import edu.uci.ics.hyracks.control.cc.application.CCApplicationContext;
-import edu.uci.ics.hyracks.control.cc.ipc.HyracksClientInterfaceDelegateIPCI;
-import edu.uci.ics.hyracks.control.cc.job.IJobStatusConditionVariable;
import edu.uci.ics.hyracks.control.cc.job.JobRun;
import edu.uci.ics.hyracks.control.cc.web.WebServer;
import edu.uci.ics.hyracks.control.cc.work.ApplicationCreateWork;
import edu.uci.ics.hyracks.control.cc.work.ApplicationDestroyWork;
import edu.uci.ics.hyracks.control.cc.work.ApplicationStartWork;
-import edu.uci.ics.hyracks.control.cc.work.GetJobStatusConditionVariableWork;
+import edu.uci.ics.hyracks.control.cc.work.ApplicationStateChangeWork;
+import edu.uci.ics.hyracks.control.cc.work.GetIpAddressNodeNameMapWork;
import edu.uci.ics.hyracks.control.cc.work.GetJobStatusWork;
import edu.uci.ics.hyracks.control.cc.work.GetNodeControllersInfoWork;
import edu.uci.ics.hyracks.control.cc.work.JobCreateWork;
@@ -61,29 +56,22 @@
import edu.uci.ics.hyracks.control.cc.work.TaskCompleteWork;
import edu.uci.ics.hyracks.control.cc.work.TaskFailureWork;
import edu.uci.ics.hyracks.control.cc.work.UnregisterNodeWork;
+import edu.uci.ics.hyracks.control.cc.work.WaitForJobCompletionWork;
import edu.uci.ics.hyracks.control.common.AbstractRemoteService;
-import edu.uci.ics.hyracks.control.common.base.IClusterController;
-import edu.uci.ics.hyracks.control.common.base.INodeController;
import edu.uci.ics.hyracks.control.common.context.ServerContext;
import edu.uci.ics.hyracks.control.common.controllers.CCConfig;
-import edu.uci.ics.hyracks.control.common.controllers.NodeParameters;
-import edu.uci.ics.hyracks.control.common.controllers.NodeRegistration;
-import edu.uci.ics.hyracks.control.common.heartbeat.HeartbeatData;
-import edu.uci.ics.hyracks.control.common.ipc.ClusterControllerDelegateIPCI;
-import edu.uci.ics.hyracks.control.common.ipc.NodeControllerRemoteProxy;
-import edu.uci.ics.hyracks.control.common.job.PartitionDescriptor;
-import edu.uci.ics.hyracks.control.common.job.PartitionRequest;
-import edu.uci.ics.hyracks.control.common.job.profiling.om.JobProfile;
-import edu.uci.ics.hyracks.control.common.job.profiling.om.TaskProfile;
+import edu.uci.ics.hyracks.control.common.ipc.CCNCFunctions;
+import edu.uci.ics.hyracks.control.common.ipc.CCNCFunctions.Function;
import edu.uci.ics.hyracks.control.common.logs.LogFile;
-import edu.uci.ics.hyracks.control.common.work.FutureValue;
+import edu.uci.ics.hyracks.control.common.work.IPCResponder;
import edu.uci.ics.hyracks.control.common.work.WorkQueue;
import edu.uci.ics.hyracks.ipc.api.IIPCHandle;
import edu.uci.ics.hyracks.ipc.api.IIPCI;
+import edu.uci.ics.hyracks.ipc.exceptions.IPCException;
import edu.uci.ics.hyracks.ipc.impl.IPCSystem;
+import edu.uci.ics.hyracks.ipc.impl.JavaSerializationBasedPayloadSerializerDeserializer;
-public class ClusterControllerService extends AbstractRemoteService implements IClusterController,
- IHyracksClientInterface {
+public class ClusterControllerService extends AbstractRemoteService {
private static Logger LOGGER = Logger.getLogger(ClusterControllerService.class.getName());
private final CCConfig ccConfig;
@@ -131,11 +119,12 @@
applications = new Hashtable<String, CCApplicationContext>();
serverCtx = new ServerContext(ServerContext.ServerType.CLUSTER_CONTROLLER, new File(ccConfig.ccRoot));
executor = Executors.newCachedThreadPool();
- IIPCI ccIPCI = new ClusterControllerDelegateIPCI(this);
- clusterIPC = new IPCSystem(new InetSocketAddress(ccConfig.clusterNetPort), ccIPCI, executor);
- IIPCI ciIPCI = new HyracksClientInterfaceDelegateIPCI(this);
+ IIPCI ccIPCI = new ClusterControllerIPCI();
+ clusterIPC = new IPCSystem(new InetSocketAddress(ccConfig.clusterNetPort), ccIPCI,
+ new CCNCFunctions.SerializerDeserializer());
+ IIPCI ciIPCI = new HyracksClientInterfaceIPCI();
clientIPC = new IPCSystem(new InetSocketAddress(ccConfig.clientNetIpAddress, ccConfig.clientNetPort), ciIPCI,
- executor);
+ new JavaSerializationBasedPayloadSerializerDeserializer());
webServer = new WebServer(this);
activeRunMap = new HashMap<JobId, JobRun>();
runMapArchive = new LinkedHashMap<JobId, JobRun>() {
@@ -149,8 +138,9 @@
this.timer = new Timer(true);
ccContext = new ICCContext() {
@Override
- public Map<String, Set<String>> getIPAddressNodeMap() {
- return ipAddressNodeNameMap;
+ public void getIPAddressNodeMap(Map<String, Set<String>> map) throws Exception {
+ GetIpAddressNodeNameMapWork ginmw = new GetIpAddressNodeNameMapWork(ClusterControllerService.this, map);
+ workQueue.scheduleAndSync(ginmw);
}
};
sweeper = new DeadNodeSweeper();
@@ -202,6 +192,10 @@
return runMapArchive;
}
+ public Map<String, Set<String>> getIpAddressNodeNameMap() {
+ return ipAddressNodeNameMap;
+ }
+
public LogFile getJobLogFile() {
return jobLog;
}
@@ -226,131 +220,16 @@
return new JobId(jobCounter++);
}
- @Override
- public JobId createJob(String appName, byte[] jobSpec, EnumSet<JobFlag> jobFlags) throws Exception {
- JobId jobId = createJobId();
- JobCreateWork jce = new JobCreateWork(this, jobId, appName, jobSpec, jobFlags);
- workQueue.schedule(jce);
- jce.sync();
- return jobId;
- }
-
- @Override
- public NodeParameters registerNode(NodeRegistration reg) throws Exception {
- InetSocketAddress ncAddress = reg.getNodeControllerAddress();
- String id = reg.getNodeId();
-
- IIPCHandle ncIPCHandle = clusterIPC.getHandle(reg.getNodeControllerAddress());
- INodeController nodeController = new NodeControllerRemoteProxy(ncIPCHandle);
-
- NodeControllerState state = new NodeControllerState(nodeController, reg);
- workQueue.scheduleAndSync(new RegisterNodeWork(this, id, state));
- LOGGER.log(Level.INFO, "Registered INodeController: id = " + id);
- NodeParameters params = new NodeParameters();
- params.setClusterControllerInfo(info);
- params.setHeartbeatPeriod(ccConfig.heartbeatPeriod);
- params.setProfileDumpPeriod(ccConfig.profileDumpPeriod);
- return params;
- }
-
- @Override
- public void unregisterNode(String nodeId) throws Exception {
- workQueue.schedule(new UnregisterNodeWork(this, nodeId));
- }
-
- @Override
- public void notifyTaskComplete(JobId jobId, TaskAttemptId taskId, String nodeId, TaskProfile statistics)
- throws Exception {
- TaskCompleteWork sce = new TaskCompleteWork(this, jobId, taskId, nodeId, statistics);
- workQueue.schedule(sce);
- }
-
- @Override
- public void notifyTaskFailure(JobId jobId, TaskAttemptId taskId, String nodeId, String details) throws Exception {
- TaskFailureWork tfe = new TaskFailureWork(this, jobId, taskId, nodeId, details);
- workQueue.schedule(tfe);
- }
-
- @Override
- public void notifyJobletCleanup(JobId jobId, String nodeId) throws Exception {
- JobletCleanupNotificationWork jcnw = new JobletCleanupNotificationWork(this, jobId, nodeId);
- workQueue.schedule(jcnw);
- }
-
- @Override
- public JobStatus getJobStatus(JobId jobId) throws Exception {
- GetJobStatusWork gse = new GetJobStatusWork(this, jobId);
- workQueue.scheduleAndSync(gse);
- return gse.getStatus();
- }
-
- @Override
- public void startJob(JobId jobId) throws Exception {
- JobStartWork jse = new JobStartWork(this, jobId);
- workQueue.schedule(jse);
- }
-
- @Override
- public void waitForCompletion(JobId jobId) throws Exception {
- GetJobStatusConditionVariableWork e = new GetJobStatusConditionVariableWork(this, jobId);
- workQueue.scheduleAndSync(e);
- IJobStatusConditionVariable var = e.getConditionVariable();
- if (var != null) {
- var.waitForCompletion();
- }
- }
-
- @Override
- public void reportProfile(String id, List<JobProfile> profiles) throws Exception {
- workQueue.schedule(new ReportProfilesWork(this, profiles));
- }
-
- @Override
- public synchronized void nodeHeartbeat(String id, HeartbeatData hbData) throws Exception {
- workQueue.schedule(new NodeHeartbeatWork(this, id, hbData));
- }
-
- @Override
- public void createApplication(String appName) throws Exception {
- FutureValue<Object> fv = new FutureValue<Object>();
- workQueue.schedule(new ApplicationCreateWork(this, appName, fv));
- fv.get();
- }
-
- @Override
- public void destroyApplication(String appName) throws Exception {
- FutureValue<Object> fv = new FutureValue<Object>();
- workQueue.schedule(new ApplicationDestroyWork(this, appName, fv));
- fv.get();
- }
-
- @Override
- public void startApplication(final String appName) throws Exception {
- FutureValue<Object> fv = new FutureValue<Object>();
- workQueue.schedule(new ApplicationStartWork(this, appName, fv));
- fv.get();
- }
-
- @Override
- public ClusterControllerInfo getClusterControllerInfo() throws Exception {
+ public ClusterControllerInfo getClusterControllerInfo() {
return info;
}
- @Override
- public Map<String, NodeControllerInfo> getNodeControllersInfo() throws Exception {
- FutureValue<Map<String, NodeControllerInfo>> fv = new FutureValue<Map<String, NodeControllerInfo>>();
- workQueue.schedule(new GetNodeControllersInfoWork(this, fv));
- return fv.get();
+ public CCConfig getCCConfig() {
+ return ccConfig;
}
- @Override
- public void registerPartitionProvider(PartitionDescriptor partitionDescriptor) {
- workQueue.schedule(new RegisterPartitionAvailibilityWork(this, partitionDescriptor));
- }
-
- @Override
- public void registerPartitionRequest(PartitionRequest partitionRequest) {
- workQueue.schedule(new RegisterPartitionRequestWork(this, partitionRequest));
+ public IPCSystem getClusterIPC() {
+ return clusterIPC;
}
private class DeadNodeSweeper extends TimerTask {
@@ -359,4 +238,156 @@
workQueue.schedule(new RemoveDeadNodesWork(ClusterControllerService.this));
}
}
+
+ private class HyracksClientInterfaceIPCI implements IIPCI {
+ @Override
+ public void deliverIncomingMessage(IIPCHandle handle, long mid, long rmid, Object payload, Exception exception) {
+ HyracksClientInterfaceFunctions.Function fn = (HyracksClientInterfaceFunctions.Function) payload;
+ switch (fn.getFunctionId()) {
+ case GET_CLUSTER_CONTROLLER_INFO: {
+ try {
+ handle.send(mid, info, null);
+ } catch (IPCException e) {
+ e.printStackTrace();
+ }
+ return;
+ }
+
+ case CREATE_APPLICATION: {
+ HyracksClientInterfaceFunctions.CreateApplicationFunction caf = (HyracksClientInterfaceFunctions.CreateApplicationFunction) fn;
+ workQueue.schedule(new ApplicationCreateWork(ClusterControllerService.this, caf.getAppName(),
+ new IPCResponder<Object>(handle, mid)));
+ return;
+ }
+
+ case START_APPLICATION: {
+ HyracksClientInterfaceFunctions.StartApplicationFunction saf = (HyracksClientInterfaceFunctions.StartApplicationFunction) fn;
+ workQueue.schedule(new ApplicationStartWork(ClusterControllerService.this, saf.getAppName(),
+ new IPCResponder<Object>(handle, mid)));
+ return;
+ }
+
+ case DESTROY_APPLICATION: {
+ HyracksClientInterfaceFunctions.DestroyApplicationFunction daf = (HyracksClientInterfaceFunctions.DestroyApplicationFunction) fn;
+ workQueue.schedule(new ApplicationDestroyWork(ClusterControllerService.this, daf.getAppName(),
+ new IPCResponder<Object>(handle, mid)));
+ return;
+ }
+
+ case CREATE_JOB: {
+ HyracksClientInterfaceFunctions.CreateJobFunction cjf = (HyracksClientInterfaceFunctions.CreateJobFunction) fn;
+ JobId jobId = createJobId();
+ workQueue.schedule(new JobCreateWork(ClusterControllerService.this, jobId, cjf.getAppName(), cjf
+ .getJobSpec(), cjf.getJobFlags(), new IPCResponder<JobId>(handle, mid)));
+ return;
+ }
+
+ case GET_JOB_STATUS: {
+ HyracksClientInterfaceFunctions.GetJobStatusFunction gjsf = (HyracksClientInterfaceFunctions.GetJobStatusFunction) fn;
+ workQueue.schedule(new GetJobStatusWork(ClusterControllerService.this, gjsf.getJobId(),
+ new IPCResponder<JobStatus>(handle, mid)));
+ return;
+ }
+
+ case START_JOB: {
+ HyracksClientInterfaceFunctions.StartJobFunction sjf = (HyracksClientInterfaceFunctions.StartJobFunction) fn;
+ workQueue.schedule(new JobStartWork(ClusterControllerService.this, sjf.getJobId(),
+ new IPCResponder<Object>(handle, mid)));
+ return;
+ }
+
+ case WAIT_FOR_COMPLETION: {
+ HyracksClientInterfaceFunctions.WaitForCompletionFunction wfcf = (HyracksClientInterfaceFunctions.WaitForCompletionFunction) fn;
+ workQueue.schedule(new WaitForJobCompletionWork(ClusterControllerService.this, wfcf.getJobId(),
+ new IPCResponder<Object>(handle, mid)));
+ return;
+ }
+
+ case GET_NODE_CONTROLLERS_INFO: {
+ workQueue.schedule(new GetNodeControllersInfoWork(ClusterControllerService.this,
+ new IPCResponder<Map<String, NodeControllerInfo>>(handle, mid)));
+ return;
+ }
+ }
+ try {
+ handle.send(mid, null, new IllegalArgumentException("Unknown function " + fn.getFunctionId()));
+ } catch (IPCException e) {
+ e.printStackTrace();
+ }
+ }
+ }
+
+ private class ClusterControllerIPCI implements IIPCI {
+ @Override
+ public void deliverIncomingMessage(IIPCHandle handle, long mid, long rmid, Object payload, Exception exception) {
+ CCNCFunctions.Function fn = (Function) payload;
+ switch (fn.getFunctionId()) {
+ case REGISTER_NODE: {
+ CCNCFunctions.RegisterNodeFunction rnf = (CCNCFunctions.RegisterNodeFunction) fn;
+ workQueue.schedule(new RegisterNodeWork(ClusterControllerService.this, rnf.getNodeRegistration()));
+ return;
+ }
+
+ case UNREGISTER_NODE: {
+ CCNCFunctions.UnregisterNodeFunction unf = (CCNCFunctions.UnregisterNodeFunction) fn;
+ workQueue.schedule(new UnregisterNodeWork(ClusterControllerService.this, unf.getNodeId()));
+ return;
+ }
+
+ case NODE_HEARTBEAT: {
+ CCNCFunctions.NodeHeartbeatFunction nhf = (CCNCFunctions.NodeHeartbeatFunction) fn;
+ workQueue.schedule(new NodeHeartbeatWork(ClusterControllerService.this, nhf.getNodeId(), nhf
+ .getHeartbeatData()));
+ return;
+ }
+
+ case NOTIFY_JOBLET_CLEANUP: {
+ CCNCFunctions.NotifyJobletCleanupFunction njcf = (CCNCFunctions.NotifyJobletCleanupFunction) fn;
+ workQueue.schedule(new JobletCleanupNotificationWork(ClusterControllerService.this,
+ njcf.getJobId(), njcf.getNodeId()));
+ return;
+ }
+
+ case REPORT_PROFILE: {
+ CCNCFunctions.ReportProfileFunction rpf = (CCNCFunctions.ReportProfileFunction) fn;
+ workQueue.schedule(new ReportProfilesWork(ClusterControllerService.this, rpf.getProfiles()));
+ return;
+ }
+
+ case NOTIFY_TASK_COMPLETE: {
+ CCNCFunctions.NotifyTaskCompleteFunction ntcf = (CCNCFunctions.NotifyTaskCompleteFunction) fn;
+ workQueue.schedule(new TaskCompleteWork(ClusterControllerService.this, ntcf.getJobId(), ntcf
+ .getTaskId(), ntcf.getNodeId(), ntcf.getStatistics()));
+ return;
+ }
+ case NOTIFY_TASK_FAILURE: {
+ CCNCFunctions.NotifyTaskFailureFunction ntff = (CCNCFunctions.NotifyTaskFailureFunction) fn;
+ workQueue.schedule(new TaskFailureWork(ClusterControllerService.this, ntff.getJobId(), ntff
+ .getTaskId(), ntff.getDetails(), ntff.getDetails()));
+ return;
+ }
+
+ case REGISTER_PARTITION_PROVIDER: {
+ CCNCFunctions.RegisterPartitionProviderFunction rppf = (CCNCFunctions.RegisterPartitionProviderFunction) fn;
+ workQueue.schedule(new RegisterPartitionAvailibilityWork(ClusterControllerService.this, rppf
+ .getPartitionDescriptor()));
+ return;
+ }
+
+ case REGISTER_PARTITION_REQUEST: {
+ CCNCFunctions.RegisterPartitionRequestFunction rprf = (CCNCFunctions.RegisterPartitionRequestFunction) fn;
+ workQueue.schedule(new RegisterPartitionRequestWork(ClusterControllerService.this, rprf
+ .getPartitionRequest()));
+ return;
+ }
+
+ case APPLICATION_STATE_CHANGE_RESPONSE: {
+ CCNCFunctions.ApplicationStateChangeResponseFunction astrf = (CCNCFunctions.ApplicationStateChangeResponseFunction) fn;
+ workQueue.schedule(new ApplicationStateChangeWork(ClusterControllerService.this, astrf));
+ return;
+ }
+ }
+ LOGGER.warning("Unknown function: " + fn.getFunctionId());
+ }
+ }
}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/NodeControllerState.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/NodeControllerState.java
index 0b1fea0..2b7563a 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/NodeControllerState.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/NodeControllerState.java
@@ -80,6 +80,14 @@
private final long[][] gcCollectionTimes;
+ private final long[] netPayloadBytesRead;
+
+ private final long[] netPayloadBytesWritten;
+
+ private final long[] netSignalingBytesRead;
+
+ private final long[] netSignalingBytesWritten;
+
private int rrdPtr;
private int lastHeartbeatDuration;
@@ -116,6 +124,10 @@
}
gcCollectionCounts = new long[gcN][RRD_SIZE];
gcCollectionTimes = new long[gcN][RRD_SIZE];
+ netPayloadBytesRead = new long[RRD_SIZE];
+ netPayloadBytesWritten = new long[RRD_SIZE];
+ netSignalingBytesRead = new long[RRD_SIZE];
+ netSignalingBytesWritten = new long[RRD_SIZE];
rrdPtr = 0;
}
@@ -139,6 +151,10 @@
gcCollectionCounts[i][rrdPtr] = hbData.gcCollectionCounts[i];
gcCollectionTimes[i][rrdPtr] = hbData.gcCollectionTimes[i];
}
+ netPayloadBytesRead[rrdPtr] = hbData.netPayloadBytesRead;
+ netPayloadBytesWritten[rrdPtr] = hbData.netPayloadBytesWritten;
+ netSignalingBytesRead[rrdPtr] = hbData.netSignalingBytesRead;
+ netSignalingBytesWritten[rrdPtr] = hbData.netSignalingBytesWritten;
rrdPtr = (rrdPtr + 1) % RRD_SIZE;
}
@@ -199,6 +215,10 @@
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);
return o;
}
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/application/CCApplicationContext.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/application/CCApplicationContext.java
index c1c0161..28f7c59 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/application/CCApplicationContext.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/application/CCApplicationContext.java
@@ -17,7 +17,9 @@
import java.io.IOException;
import java.io.Serializable;
import java.util.ArrayList;
+import java.util.HashSet;
import java.util.List;
+import java.util.Set;
import edu.uci.ics.hyracks.api.application.ICCApplicationContext;
import edu.uci.ics.hyracks.api.application.ICCBootstrap;
@@ -30,10 +32,17 @@
import edu.uci.ics.hyracks.control.cc.job.DeserializingJobSpecificationFactory;
import edu.uci.ics.hyracks.control.common.application.ApplicationContext;
import edu.uci.ics.hyracks.control.common.context.ServerContext;
+import edu.uci.ics.hyracks.control.common.work.IResultCallback;
public class CCApplicationContext extends ApplicationContext implements ICCApplicationContext {
private final ICCContext ccContext;
+ protected final Set<String> initPendingNodeIds;
+ protected final Set<String> deinitPendingNodeIds;
+
+ protected IResultCallback<Object> initializationCallback;
+ protected IResultCallback<Object> deinitializationCallback;
+
private IJobSpecificationFactory jobSpecFactory;
private List<IJobLifecycleListener> jobLifecycleListeners;
@@ -41,6 +50,8 @@
public CCApplicationContext(ServerContext serverCtx, ICCContext ccContext, String appName) throws IOException {
super(serverCtx, appName);
this.ccContext = ccContext;
+ initPendingNodeIds = new HashSet<String>();
+ deinitPendingNodeIds = new HashSet<String>();
jobSpecFactory = DeserializingJobSpecificationFactory.INSTANCE;
jobLifecycleListeners = new ArrayList<IJobLifecycleListener>();
}
@@ -98,4 +109,28 @@
l.notifyJobCreation(jobId, specification);
}
}
+
+ public Set<String> getInitializationPendingNodeIds() {
+ return initPendingNodeIds;
+ }
+
+ public Set<String> getDeinitializationPendingNodeIds() {
+ return deinitPendingNodeIds;
+ }
+
+ public IResultCallback<Object> getInitializationCallback() {
+ return initializationCallback;
+ }
+
+ public void setInitializationCallback(IResultCallback<Object> initializationCallback) {
+ this.initializationCallback = initializationCallback;
+ }
+
+ public IResultCallback<Object> getDeinitializationCallback() {
+ return deinitializationCallback;
+ }
+
+ public void setDeinitializationCallback(IResultCallback<Object> deinitializationCallback) {
+ this.deinitializationCallback = deinitializationCallback;
+ }
}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/ipc/HyracksClientInterfaceDelegateIPCI.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/ipc/HyracksClientInterfaceDelegateIPCI.java
deleted file mode 100644
index b3aa406..0000000
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/ipc/HyracksClientInterfaceDelegateIPCI.java
+++ /dev/null
@@ -1,83 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed 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 from
- *
- * 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 edu.uci.ics.hyracks.control.cc.ipc;
-
-import edu.uci.ics.hyracks.api.client.HyracksClientInterfaceFunctions;
-import edu.uci.ics.hyracks.api.client.IHyracksClientInterface;
-import edu.uci.ics.hyracks.ipc.api.IIPCHandle;
-import edu.uci.ics.hyracks.ipc.api.IIPCI;
-
-public class HyracksClientInterfaceDelegateIPCI implements IIPCI {
- private final IHyracksClientInterface hci;
-
- public HyracksClientInterfaceDelegateIPCI(IHyracksClientInterface hci) {
- this.hci = hci;
- }
-
- @Override
- public Object call(IIPCHandle caller, Object req) throws Exception {
- HyracksClientInterfaceFunctions.Function fn = (HyracksClientInterfaceFunctions.Function) req;
- switch (fn.getFunctionId()) {
- case GET_CLUSTER_CONTROLLER_INFO: {
- return hci.getClusterControllerInfo();
- }
-
- case CREATE_APPLICATION: {
- HyracksClientInterfaceFunctions.CreateApplicationFunction caf = (HyracksClientInterfaceFunctions.CreateApplicationFunction) fn;
- hci.createApplication(caf.getAppName());
- return null;
- }
-
- case START_APPLICATION: {
- HyracksClientInterfaceFunctions.StartApplicationFunction saf = (HyracksClientInterfaceFunctions.StartApplicationFunction) fn;
- hci.startApplication(saf.getAppName());
- return null;
- }
-
- case DESTROY_APPLICATION: {
- HyracksClientInterfaceFunctions.DestroyApplicationFunction daf = (HyracksClientInterfaceFunctions.DestroyApplicationFunction) fn;
- hci.destroyApplication(daf.getAppName());
- return null;
- }
-
- case CREATE_JOB: {
- HyracksClientInterfaceFunctions.CreateJobFunction cjf = (HyracksClientInterfaceFunctions.CreateJobFunction) fn;
- return hci.createJob(cjf.getAppName(), cjf.getJobSpec(), cjf.getJobFlags());
- }
-
- case GET_JOB_STATUS: {
- HyracksClientInterfaceFunctions.GetJobStatusFunction gjsf = (HyracksClientInterfaceFunctions.GetJobStatusFunction) fn;
- return hci.getJobStatus(gjsf.getJobId());
- }
-
- case START_JOB: {
- HyracksClientInterfaceFunctions.StartJobFunction sjf = (HyracksClientInterfaceFunctions.StartJobFunction) fn;
- hci.startJob(sjf.getJobId());
- return null;
- }
-
- case WAIT_FOR_COMPLETION: {
- HyracksClientInterfaceFunctions.WaitForCompletionFunction wfcf = (HyracksClientInterfaceFunctions.WaitForCompletionFunction) fn;
- hci.waitForCompletion(wfcf.getJobId());
- return null;
- }
-
- case GET_NODE_CONTROLLERS_INFO: {
- return hci.getNodeControllersInfo();
- }
- }
- throw new IllegalArgumentException("Unknown function " + fn.getFunctionId());
- }
-}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/RemoteRunner.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/RemoteRunner.java
deleted file mode 100644
index a37c786..0000000
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/RemoteRunner.java
+++ /dev/null
@@ -1,61 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed 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 from
- *
- * 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 edu.uci.ics.hyracks.control.cc.remote;
-
-import java.util.List;
-import java.util.Vector;
-import java.util.concurrent.Semaphore;
-
-import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
-import edu.uci.ics.hyracks.control.cc.NodeControllerState;
-import edu.uci.ics.hyracks.control.common.base.INodeController;
-
-public class RemoteRunner {
- public static <T, R> R runRemote(ClusterControllerService ccs, final RemoteOp<T>[] remoteOps,
- final Accumulator<T, R> accumulator) throws Exception {
- final Semaphore installComplete = new Semaphore(remoteOps.length);
- final List<Exception> errors = new Vector<Exception>();
- for (final RemoteOp<T> remoteOp : remoteOps) {
- NodeControllerState nodeState = ccs.getNodeMap().get(remoteOp.getNodeId());
- final INodeController node = nodeState.getNodeController();
-
- installComplete.acquire();
- Runnable remoteRunner = new Runnable() {
- @Override
- public void run() {
- try {
- T t = remoteOp.execute(node);
- if (accumulator != null) {
- synchronized (accumulator) {
- accumulator.accumulate(t);
- }
- }
- } catch (Exception e) {
- errors.add(e);
- } finally {
- installComplete.release();
- }
- }
- };
-
- ccs.getExecutor().execute(remoteRunner);
- }
- installComplete.acquire(remoteOps.length);
- if (!errors.isEmpty()) {
- throw errors.get(0);
- }
- return accumulator == null ? null : accumulator.getResult();
- }
-}
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/ops/ApplicationDestroyer.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/ops/ApplicationDestroyer.java
deleted file mode 100644
index 52e726c..0000000
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/ops/ApplicationDestroyer.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed 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 from
- *
- * 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 edu.uci.ics.hyracks.control.cc.remote.ops;
-
-import edu.uci.ics.hyracks.control.cc.remote.RemoteOp;
-import edu.uci.ics.hyracks.control.common.base.INodeController;
-
-public class ApplicationDestroyer implements RemoteOp<Void> {
- private String nodeId;
- private String appName;
-
- public ApplicationDestroyer(String nodeId, String appName) {
- this.nodeId = nodeId;
- this.appName = appName;
- }
-
- @Override
- public String getNodeId() {
- return nodeId;
- }
-
- @Override
- public Void execute(INodeController node) throws Exception {
- node.destroyApplication(appName);
- return null;
- }
-
- @Override
- public String toString() {
- return "Destroyed application: " + appName;
- }
-}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/ops/ApplicationStarter.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/ops/ApplicationStarter.java
deleted file mode 100644
index d6a1d26..0000000
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/ops/ApplicationStarter.java
+++ /dev/null
@@ -1,48 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed 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 from
- *
- * 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 edu.uci.ics.hyracks.control.cc.remote.ops;
-
-import edu.uci.ics.hyracks.control.cc.remote.RemoteOp;
-import edu.uci.ics.hyracks.control.common.base.INodeController;
-
-public class ApplicationStarter implements RemoteOp<Void> {
- private String nodeId;
- private String appName;
- private boolean deployHar;
- private byte[] distributedState;
-
- public ApplicationStarter(String nodeId, String appName, boolean deployHar, byte[] distributedState) {
- this.nodeId = nodeId;
- this.appName = appName;
- this.deployHar = deployHar;
- this.distributedState = distributedState;
- }
-
- @Override
- public String getNodeId() {
- return nodeId;
- }
-
- @Override
- public Void execute(INodeController node) throws Exception {
- node.createApplication(appName, deployHar, distributedState);
- return null;
- }
-
- @Override
- public String toString() {
- return "Started application: " + appName;
- }
-}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/ops/JobCompleteNotifier.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/ops/JobCompleteNotifier.java
deleted file mode 100644
index d0f9151..0000000
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/ops/JobCompleteNotifier.java
+++ /dev/null
@@ -1,48 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed 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 from
- *
- * 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 edu.uci.ics.hyracks.control.cc.remote.ops;
-
-import edu.uci.ics.hyracks.api.job.JobId;
-import edu.uci.ics.hyracks.api.job.JobStatus;
-import edu.uci.ics.hyracks.control.cc.remote.RemoteOp;
-import edu.uci.ics.hyracks.control.common.base.INodeController;
-
-public class JobCompleteNotifier implements RemoteOp<Void> {
- private String nodeId;
- private JobId jobId;
- private JobStatus status;
-
- public JobCompleteNotifier(String nodeId, JobId jobId, JobStatus status) {
- this.nodeId = nodeId;
- this.jobId = jobId;
- this.status = status;
- }
-
- @Override
- public Void execute(INodeController node) throws Exception {
- node.cleanUpJoblet(jobId, status);
- return null;
- }
-
- @Override
- public String toString() {
- return jobId + " Cleaning Up";
- }
-
- @Override
- public String getNodeId() {
- return nodeId;
- }
-}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/ActivityClusterPlanner.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/ActivityClusterPlanner.java
index 929462b..e6e6c2f 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/ActivityClusterPlanner.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/ActivityClusterPlanner.java
@@ -49,6 +49,8 @@
import edu.uci.ics.hyracks.control.cc.job.TaskClusterId;
public class ActivityClusterPlanner {
+ private static final boolean USE_CONNECTOR_POLICY_IN_TASK_CLUSTER_CONSTRUCTION = true;
+
private static final Logger LOGGER = Logger.getLogger(ActivityClusterPlanner.class.getName());
private final JobScheduler scheduler;
@@ -64,13 +66,28 @@
JobRun jobRun = scheduler.getJobRun();
Map<ActivityId, ActivityPartitionDetails> pcMap = computePartitionCounts(ac);
+ Map<ActivityId, ActivityPlan> activityPlanMap = buildActivityPlanMap(ac, jobRun, pcMap);
+
+ assignConnectorPolicy(ac, activityPlanMap);
+
+ TaskCluster[] taskClusters = computeTaskClusters(ac, jobRun, activityPlanMap);
+
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Plan for " + ac);
+ LOGGER.info("Built " + taskClusters.length + " Task Clusters");
+ for (TaskCluster tc : taskClusters) {
+ LOGGER.info("Tasks: " + Arrays.toString(tc.getTasks()));
+ }
+ }
+
+ ac.setPlan(new ActivityClusterPlan(taskClusters, activityPlanMap));
+ }
+
+ private Map<ActivityId, ActivityPlan> buildActivityPlanMap(ActivityCluster ac, JobRun jobRun,
+ Map<ActivityId, ActivityPartitionDetails> pcMap) {
Map<ActivityId, ActivityPlan> activityPlanMap = new HashMap<ActivityId, ActivityPlan>();
- Set<ActivityId> activities = ac.getActivities();
-
- Map<TaskId, Set<TaskId>> taskClusterMap = new HashMap<TaskId, Set<TaskId>>();
-
Set<ActivityId> depAnIds = new HashSet<ActivityId>();
- for (ActivityId anId : activities) {
+ for (ActivityId anId : ac.getActivities()) {
depAnIds.clear();
getDependencyActivityIds(depAnIds, anId);
ActivityPartitionDetails apd = pcMap.get(anId);
@@ -94,53 +111,21 @@
tasks[i].getDependencies().add(dTaskId);
dTask.getDependents().add(tid);
}
- Set<TaskId> cluster = new HashSet<TaskId>();
- cluster.add(tid);
- taskClusterMap.put(tid, cluster);
}
activityPlan.setTasks(tasks);
activityPlanMap.put(anId, activityPlan);
}
+ return activityPlanMap;
+ }
- Map<ConnectorDescriptorId, IConnectorPolicy> connectorPolicies = assignConnectorPolicy(ac, activityPlanMap);
- scheduler.getJobRun().getConnectorPolicyMap().putAll(connectorPolicies);
+ private TaskCluster[] computeTaskClusters(ActivityCluster ac, JobRun jobRun,
+ Map<ActivityId, ActivityPlan> activityPlanMap) {
+ Set<ActivityId> activities = ac.getActivities();
+ Map<TaskId, List<Pair<TaskId, ConnectorDescriptorId>>> taskConnectivity = computeTaskConnectivity(jobRun,
+ activityPlanMap, activities);
- Map<TaskId, List<Pair<TaskId, ConnectorDescriptorId>>> taskConnectivity = new HashMap<TaskId, List<Pair<TaskId, ConnectorDescriptorId>>>();
- JobActivityGraph jag = jobRun.getJobActivityGraph();
- BitSet targetBitmap = new BitSet();
- for (ActivityId ac1 : activities) {
- Task[] ac1TaskStates = activityPlanMap.get(ac1).getTasks();
- int nProducers = ac1TaskStates.length;
- List<IConnectorDescriptor> outputConns = jag.getActivityOutputConnectorDescriptors(ac1);
- if (outputConns != null) {
- for (IConnectorDescriptor c : outputConns) {
- ConnectorDescriptorId cdId = c.getConnectorId();
- ActivityId ac2 = jag.getConsumerActivity(cdId);
- Task[] ac2TaskStates = activityPlanMap.get(ac2).getTasks();
- int nConsumers = ac2TaskStates.length;
- for (int i = 0; i < nProducers; ++i) {
- c.indicateTargetPartitions(nProducers, nConsumers, i, targetBitmap);
- List<Pair<TaskId, ConnectorDescriptorId>> cInfoList = taskConnectivity.get(ac1TaskStates[i]
- .getTaskId());
- if (cInfoList == null) {
- cInfoList = new ArrayList<Pair<TaskId, ConnectorDescriptorId>>();
- taskConnectivity.put(ac1TaskStates[i].getTaskId(), cInfoList);
- }
- Set<TaskId> cluster = taskClusterMap.get(ac1TaskStates[i].getTaskId());
- for (int j = targetBitmap.nextSetBit(0); j >= 0; j = targetBitmap.nextSetBit(j + 1)) {
- TaskId targetTID = ac2TaskStates[j].getTaskId();
- cInfoList.add(Pair.<TaskId, ConnectorDescriptorId> of(targetTID, cdId));
- IConnectorPolicy cPolicy = connectorPolicies.get(cdId);
- if (cPolicy.requiresProducerConsumerCoscheduling()) {
- cluster.add(targetTID);
- }
- }
- }
- }
- }
- }
-
- TaskCluster[] taskClusters = buildTaskClusters(ac, activityPlanMap, taskClusterMap);
+ TaskCluster[] taskClusters = USE_CONNECTOR_POLICY_IN_TASK_CLUSTER_CONSTRUCTION ? buildConnectorPolicyAwareTaskClusters(
+ ac, activityPlanMap, taskConnectivity) : buildConnectorPolicyUnawareTaskClusters(ac, activityPlanMap);
for (TaskCluster tc : taskClusters) {
Set<TaskCluster> tcDependencyTaskClusters = tc.getDependencyTaskClusters();
@@ -170,20 +155,88 @@
}
}
}
+ return taskClusters;
+ }
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("Plan for " + ac);
- LOGGER.info("Built " + taskClusters.length + " Task Clusters");
- for (TaskCluster tc : taskClusters) {
- LOGGER.info("Tasks: " + Arrays.toString(tc.getTasks()));
+ private TaskCluster[] buildConnectorPolicyUnawareTaskClusters(ActivityCluster ac,
+ Map<ActivityId, ActivityPlan> activityPlanMap) {
+ List<Task> taskStates = new ArrayList<Task>();
+ for (ActivityId anId : ac.getActivities()) {
+ ActivityPlan ap = activityPlanMap.get(anId);
+ Task[] tasks = ap.getTasks();
+ for (Task t : tasks) {
+ taskStates.add(t);
+ }
+ }
+ TaskCluster tc = new TaskCluster(new TaskClusterId(ac.getActivityClusterId(), 0), ac,
+ taskStates.toArray(new Task[taskStates.size()]));
+ for (Task t : tc.getTasks()) {
+ t.setTaskCluster(tc);
+ }
+ return new TaskCluster[] { tc };
+ }
+
+ private Map<TaskId, List<Pair<TaskId, ConnectorDescriptorId>>> computeTaskConnectivity(JobRun jobRun,
+ Map<ActivityId, ActivityPlan> activityPlanMap, Set<ActivityId> activities) {
+ Map<TaskId, List<Pair<TaskId, ConnectorDescriptorId>>> taskConnectivity = new HashMap<TaskId, List<Pair<TaskId, ConnectorDescriptorId>>>();
+ JobActivityGraph jag = jobRun.getJobActivityGraph();
+ BitSet targetBitmap = new BitSet();
+ for (ActivityId ac1 : activities) {
+ Task[] ac1TaskStates = activityPlanMap.get(ac1).getTasks();
+ int nProducers = ac1TaskStates.length;
+ List<IConnectorDescriptor> outputConns = jag.getActivityOutputConnectorDescriptors(ac1);
+ if (outputConns != null) {
+ for (IConnectorDescriptor c : outputConns) {
+ ConnectorDescriptorId cdId = c.getConnectorId();
+ ActivityId ac2 = jag.getConsumerActivity(cdId);
+ Task[] ac2TaskStates = activityPlanMap.get(ac2).getTasks();
+ int nConsumers = ac2TaskStates.length;
+ for (int i = 0; i < nProducers; ++i) {
+ c.indicateTargetPartitions(nProducers, nConsumers, i, targetBitmap);
+ List<Pair<TaskId, ConnectorDescriptorId>> cInfoList = taskConnectivity.get(ac1TaskStates[i]
+ .getTaskId());
+ if (cInfoList == null) {
+ cInfoList = new ArrayList<Pair<TaskId, ConnectorDescriptorId>>();
+ taskConnectivity.put(ac1TaskStates[i].getTaskId(), cInfoList);
+ }
+ for (int j = targetBitmap.nextSetBit(0); j >= 0; j = targetBitmap.nextSetBit(j + 1)) {
+ TaskId targetTID = ac2TaskStates[j].getTaskId();
+ cInfoList.add(Pair.<TaskId, ConnectorDescriptorId> of(targetTID, cdId));
+ }
+ }
+ }
+ }
+ }
+ return taskConnectivity;
+ }
+
+ private TaskCluster[] buildConnectorPolicyAwareTaskClusters(ActivityCluster ac,
+ Map<ActivityId, ActivityPlan> activityPlanMap,
+ Map<TaskId, List<Pair<TaskId, ConnectorDescriptorId>>> taskConnectivity) {
+ Map<TaskId, Set<TaskId>> taskClusterMap = new HashMap<TaskId, Set<TaskId>>();
+ for (ActivityId anId : ac.getActivities()) {
+ ActivityPlan ap = activityPlanMap.get(anId);
+ Task[] tasks = ap.getTasks();
+ for (Task t : tasks) {
+ Set<TaskId> cluster = new HashSet<TaskId>();
+ TaskId tid = t.getTaskId();
+ cluster.add(tid);
+ taskClusterMap.put(tid, cluster);
}
}
- ac.setPlan(new ActivityClusterPlan(taskClusters, activityPlanMap));
- }
+ JobRun jobRun = ac.getJobRun();
+ Map<ConnectorDescriptorId, IConnectorPolicy> connectorPolicies = jobRun.getConnectorPolicyMap();
+ for (Map.Entry<TaskId, List<Pair<TaskId, ConnectorDescriptorId>>> e : taskConnectivity.entrySet()) {
+ Set<TaskId> cluster = taskClusterMap.get(e.getKey());
+ for (Pair<TaskId, ConnectorDescriptorId> p : e.getValue()) {
+ IConnectorPolicy cPolicy = connectorPolicies.get(p.getRight());
+ if (cPolicy.requiresProducerConsumerCoscheduling()) {
+ cluster.add(p.getLeft());
+ }
+ }
+ }
- private TaskCluster[] buildTaskClusters(ActivityCluster ac, Map<ActivityId, ActivityPlan> activityPlanMap,
- Map<TaskId, Set<TaskId>> taskClusterMap) {
/*
* taskClusterMap contains for every TID x, x -> { coscheduled consumer TIDs U x }
* We compute the transitive closure of this relation to find the largest set of
@@ -273,8 +326,7 @@
}
}
- private Map<ConnectorDescriptorId, IConnectorPolicy> assignConnectorPolicy(ActivityCluster ac,
- Map<ActivityId, ActivityPlan> taskMap) {
+ private void assignConnectorPolicy(ActivityCluster ac, Map<ActivityId, ActivityPlan> taskMap) {
JobActivityGraph jag = scheduler.getJobRun().getJobActivityGraph();
Map<ConnectorDescriptorId, IConnectorPolicy> cPolicyMap = new HashMap<ConnectorDescriptorId, IConnectorPolicy>();
Set<ActivityId> activities = ac.getActivities();
@@ -300,7 +352,7 @@
}
}
}
- return cPolicyMap;
+ ac.getJobRun().getConnectorPolicyMap().putAll(cPolicyMap);
}
private IConnectorPolicy assignConnectorPolicy(IConnectorDescriptor c, int nProducers, int nConsumers, int[] fanouts) {
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/JobScheduler.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/JobScheduler.java
index e2c9ca3..9a443d9 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/JobScheduler.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/JobScheduler.java
@@ -184,8 +184,8 @@
private void startRunnableActivityClusters() throws HyracksException {
Set<TaskCluster> taskClusterRoots = new HashSet<TaskCluster>();
findRunnableTaskClusterRoots(taskClusterRoots, rootActivityClusters);
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("Runnable TC roots: " + taskClusterRoots + ", inProgressTaskClusters: "
+ if (LOGGER.isLoggable(Level.FINE)) {
+ LOGGER.fine("Runnable TC roots: " + taskClusterRoots + ", inProgressTaskClusters: "
+ inProgressTaskClusters);
}
if (taskClusterRoots.isEmpty() && inProgressTaskClusters.isEmpty()) {
@@ -213,19 +213,19 @@
queue.add(new RankedRunnableTaskCluster(priority, tc));
}
}
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("Ranked TCs: " + queue);
+ if (LOGGER.isLoggable(Level.FINE)) {
+ LOGGER.fine("Ranked TCs: " + queue);
}
Map<String, List<TaskAttemptDescriptor>> taskAttemptMap = new HashMap<String, List<TaskAttemptDescriptor>>();
for (RankedRunnableTaskCluster rrtc : queue) {
TaskCluster tc = rrtc.getTaskCluster();
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("Found runnable TC: " + tc);
+ if (LOGGER.isLoggable(Level.FINE)) {
+ LOGGER.fine("Found runnable TC: " + tc);
List<TaskClusterAttempt> attempts = tc.getAttempts();
- LOGGER.info("Attempts so far:" + attempts.size());
+ LOGGER.fine("Attempts so far:" + attempts.size());
for (TaskClusterAttempt tcAttempt : attempts) {
- LOGGER.info("Status: " + tcAttempt.getStatus());
+ LOGGER.fine("Status: " + tcAttempt.getStatus());
}
}
assignTaskLocations(tc, taskAttemptMap);
@@ -245,16 +245,16 @@
* Runnability(Non-schedulable TaskCluster) = {NOT_RUNNABLE, _}
*/
private Runnability assignRunnabilityRank(TaskCluster goal, Map<TaskCluster, Runnability> runnabilityMap) {
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("Computing runnability: " + goal);
+ if (LOGGER.isLoggable(Level.FINE)) {
+ LOGGER.fine("Computing runnability: " + goal);
}
if (runnabilityMap.containsKey(goal)) {
return runnabilityMap.get(goal);
}
TaskClusterAttempt lastAttempt = findLastTaskClusterAttempt(goal);
if (lastAttempt != null) {
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("Last Attempt Status: " + lastAttempt.getStatus());
+ if (LOGGER.isLoggable(Level.FINE)) {
+ LOGGER.fine("Last Attempt Status: " + lastAttempt.getStatus());
}
if (lastAttempt.getStatus() == TaskClusterAttempt.TaskClusterStatus.COMPLETED) {
Runnability runnability = new Runnability(Runnability.Tag.COMPLETED, Integer.MIN_VALUE);
@@ -271,15 +271,15 @@
PartitionMatchMaker pmm = jobRun.getPartitionMatchMaker();
Runnability aggregateRunnability = new Runnability(Runnability.Tag.RUNNABLE, 0);
for (PartitionId pid : goal.getRequiredPartitions()) {
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("Inspecting required partition: " + pid);
+ if (LOGGER.isLoggable(Level.FINE)) {
+ LOGGER.fine("Inspecting required partition: " + pid);
}
Runnability runnability;
ConnectorDescriptorId cdId = pid.getConnectorDescriptorId();
IConnectorPolicy cPolicy = connectorPolicyMap.get(cdId);
PartitionState maxState = pmm.getMaximumAvailableState(pid);
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("Policy: " + cPolicy + " maxState: " + maxState);
+ if (LOGGER.isLoggable(Level.FINE)) {
+ LOGGER.fine("Policy: " + cPolicy + " maxState: " + maxState);
}
if (PartitionState.COMMITTED.equals(maxState)) {
runnability = new Runnability(Runnability.Tag.RUNNABLE, 0);
@@ -309,8 +309,12 @@
}
}
aggregateRunnability = Runnability.getWorstCase(aggregateRunnability, runnability);
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("aggregateRunnability: " + aggregateRunnability);
+ if (aggregateRunnability.getTag() == Runnability.Tag.NOT_RUNNABLE) {
+ // already not runnable -- cannot get better. bail.
+ break;
+ }
+ if (LOGGER.isLoggable(Level.FINE)) {
+ LOGGER.fine("aggregateRunnability: " + aggregateRunnability);
}
}
runnabilityMap.put(goal, aggregateRunnability);
@@ -470,14 +474,14 @@
}
private void abortTaskCluster(TaskClusterAttempt tcAttempt) {
- LOGGER.info("Aborting task cluster: " + tcAttempt.getAttempt());
+ LOGGER.fine("Aborting task cluster: " + tcAttempt.getAttempt());
Set<TaskAttemptId> abortTaskIds = new HashSet<TaskAttemptId>();
Map<String, List<TaskAttemptId>> abortTaskAttemptMap = new HashMap<String, List<TaskAttemptId>>();
for (TaskAttempt ta : tcAttempt.getTaskAttempts()) {
TaskAttemptId taId = ta.getTaskAttemptId();
TaskAttempt.TaskStatus status = ta.getStatus();
abortTaskIds.add(taId);
- LOGGER.info("Checking " + taId + ": " + ta.getStatus());
+ LOGGER.fine("Checking " + taId + ": " + ta.getStatus());
if (status == TaskAttempt.TaskStatus.RUNNING || status == TaskAttempt.TaskStatus.COMPLETED) {
ta.setStatus(TaskAttempt.TaskStatus.ABORTED, null);
ta.setEndTime(System.currentTimeMillis());
@@ -490,12 +494,12 @@
}
}
final JobId jobId = jobRun.getJobId();
- LOGGER.info("Abort map for job: " + jobId + ": " + abortTaskAttemptMap);
+ LOGGER.fine("Abort map for job: " + jobId + ": " + abortTaskAttemptMap);
for (Map.Entry<String, List<TaskAttemptId>> e : abortTaskAttemptMap.entrySet()) {
final NodeControllerState node = ccs.getNodeMap().get(e.getKey());
final List<TaskAttemptId> abortTaskAttempts = e.getValue();
if (node != null) {
- LOGGER.info("Aborting: " + abortTaskAttempts + " at " + e.getKey());
+ LOGGER.fine("Aborting: " + abortTaskAttempts + " at " + e.getKey());
ccs.getExecutor().execute(new Runnable() {
@Override
public void run() {
@@ -609,12 +613,12 @@
*/
public void notifyTaskFailure(TaskAttempt ta, ActivityCluster ac, String details) {
try {
- LOGGER.info("Received failure notification for TaskAttempt " + ta.getTaskAttemptId());
+ LOGGER.fine("Received failure notification for TaskAttempt " + ta.getTaskAttemptId());
TaskAttemptId taId = ta.getTaskAttemptId();
TaskCluster tc = ta.getTask().getTaskCluster();
TaskClusterAttempt lastAttempt = findLastTaskClusterAttempt(tc);
if (lastAttempt != null && taId.getAttempt() == lastAttempt.getAttempt()) {
- LOGGER.info("Marking TaskAttempt " + ta.getTaskAttemptId() + " as failed");
+ LOGGER.fine("Marking TaskAttempt " + ta.getTaskAttemptId() + " as failed");
ta.setStatus(TaskAttempt.TaskStatus.FAILED, details);
abortTaskCluster(lastAttempt);
lastAttempt.setStatus(TaskClusterAttempt.TaskClusterStatus.FAILED);
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/ApplicationCreateWork.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/ApplicationCreateWork.java
index 3c8f7d0..15d6d1f 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/ApplicationCreateWork.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/ApplicationCreateWork.java
@@ -20,34 +20,42 @@
import edu.uci.ics.hyracks.api.exceptions.HyracksException;
import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
import edu.uci.ics.hyracks.control.cc.application.CCApplicationContext;
+import edu.uci.ics.hyracks.control.common.application.ApplicationStatus;
import edu.uci.ics.hyracks.control.common.work.AbstractWork;
-import edu.uci.ics.hyracks.control.common.work.FutureValue;
+import edu.uci.ics.hyracks.control.common.work.IResultCallback;
public class ApplicationCreateWork extends AbstractWork {
private final ClusterControllerService ccs;
private final String appName;
- private FutureValue<Object> fv;
+ private IResultCallback<Object> callback;
- public ApplicationCreateWork(ClusterControllerService ccs, String appName, FutureValue<Object> fv) {
+ public ApplicationCreateWork(ClusterControllerService ccs, String appName, IResultCallback<Object> callback) {
this.ccs = ccs;
this.appName = appName;
- this.fv = fv;
+ this.callback = callback;
}
@Override
public void run() {
- Map<String, CCApplicationContext> applications = ccs.getApplicationMap();
- if (applications.containsKey(appName)) {
- fv.setException(new HyracksException("Duplicate application with name: " + appName + " being created."));
- }
- CCApplicationContext appCtx;
try {
- appCtx = new CCApplicationContext(ccs.getServerContext(), ccs.getCCContext(), appName);
- } catch (IOException e) {
- fv.setException(e);
- return;
+ Map<String, CCApplicationContext> applications = ccs.getApplicationMap();
+ if (applications.containsKey(appName)) {
+ callback.setException(new HyracksException("Duplicate application with name: " + appName
+ + " being created."));
+ return;
+ }
+ CCApplicationContext appCtx;
+ try {
+ appCtx = new CCApplicationContext(ccs.getServerContext(), ccs.getCCContext(), appName);
+ } catch (IOException e) {
+ callback.setException(e);
+ return;
+ }
+ appCtx.setStatus(ApplicationStatus.CREATED);
+ applications.put(appName, appCtx);
+ callback.setValue(null);
+ } catch (Exception e) {
+ callback.setException(e);
}
- applications.put(appName, appCtx);
- fv.setValue(null);
}
}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/ApplicationDestroyWork.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/ApplicationDestroyWork.java
index c6af1b9..ac780c4 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/ApplicationDestroyWork.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/ApplicationDestroyWork.java
@@ -14,62 +14,51 @@
*/
package edu.uci.ics.hyracks.control.cc.work;
-import java.util.ArrayList;
-import java.util.List;
+import java.util.Map;
import edu.uci.ics.hyracks.api.exceptions.HyracksException;
import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
-import edu.uci.ics.hyracks.control.cc.remote.RemoteOp;
-import edu.uci.ics.hyracks.control.cc.remote.RemoteRunner;
-import edu.uci.ics.hyracks.control.cc.remote.ops.ApplicationDestroyer;
-import edu.uci.ics.hyracks.control.common.application.ApplicationContext;
+import edu.uci.ics.hyracks.control.cc.NodeControllerState;
+import edu.uci.ics.hyracks.control.cc.application.CCApplicationContext;
+import edu.uci.ics.hyracks.control.common.application.ApplicationStatus;
+import edu.uci.ics.hyracks.control.common.base.INodeController;
import edu.uci.ics.hyracks.control.common.work.AbstractWork;
-import edu.uci.ics.hyracks.control.common.work.FutureValue;
+import edu.uci.ics.hyracks.control.common.work.IResultCallback;
public class ApplicationDestroyWork extends AbstractWork {
private final ClusterControllerService ccs;
private final String appName;
- private FutureValue<Object> fv;
+ private IResultCallback<Object> callback;
- public ApplicationDestroyWork(ClusterControllerService ccs, String appName, FutureValue<Object> fv) {
+ public ApplicationDestroyWork(ClusterControllerService ccs, String appName, IResultCallback<Object> callback) {
this.ccs = ccs;
this.appName = appName;
- this.fv = fv;
+ this.callback = callback;
}
@Override
public void run() {
- final ApplicationContext appCtx = ccs.getApplicationMap().remove(appName);
- if (appCtx == null) {
- fv.setException(new HyracksException("No application with name: " + appName));
- return;
- }
- List<RemoteOp<Void>> opList = new ArrayList<RemoteOp<Void>>();
- for (final String nodeId : ccs.getNodeMap().keySet()) {
- opList.add(new ApplicationDestroyer(nodeId, appName));
- }
- final RemoteOp[] ops = opList.toArray(new RemoteOp[opList.size()]);
- ccs.getExecutor().execute(new Runnable() {
- @Override
- public void run() {
- try {
- RemoteRunner.runRemote(ccs, ops, null);
- } catch (Exception e) {
- fv.setException(e);
- return;
- }
- ccs.getWorkQueue().schedule(new AbstractWork() {
- @Override
- public void run() {
- try {
- appCtx.deinitialize();
- } catch (Exception e) {
- fv.setException(e);
- }
- fv.setValue(null);
- }
- });
+ try {
+ final CCApplicationContext appCtx = ccs.getApplicationMap().remove(appName);
+ if (appCtx == null) {
+ callback.setException(new HyracksException("No application with name: " + appName));
+ return;
}
- });
+ if (appCtx.getStatus() == ApplicationStatus.IN_DEINITIALIZATION
+ || appCtx.getStatus() == ApplicationStatus.DEINITIALIZED) {
+ return;
+ }
+ Map<String, NodeControllerState> nodeMap = ccs.getNodeMap();
+ appCtx.getDeinitializationPendingNodeIds().addAll(nodeMap.keySet());
+ appCtx.setStatus(ApplicationStatus.IN_DEINITIALIZATION);
+ appCtx.setDeinitializationCallback(callback);
+ for (String nodeId : ccs.getNodeMap().keySet()) {
+ NodeControllerState nodeState = nodeMap.get(nodeId);
+ final INodeController node = nodeState.getNodeController();
+ node.destroyApplication(appName);
+ }
+ } catch (Exception e) {
+ callback.setException(e);
+ }
}
}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/ApplicationStartWork.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/ApplicationStartWork.java
index 2022c7e..e4ad56c 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/ApplicationStartWork.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/ApplicationStartWork.java
@@ -14,60 +14,67 @@
*/
package edu.uci.ics.hyracks.control.cc.work;
-import java.util.ArrayList;
-import java.util.List;
+import java.util.HashMap;
+import java.util.Map;
import edu.uci.ics.hyracks.api.exceptions.HyracksException;
import edu.uci.ics.hyracks.api.util.JavaSerializationUtils;
import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
-import edu.uci.ics.hyracks.control.cc.remote.RemoteOp;
-import edu.uci.ics.hyracks.control.cc.remote.RemoteRunner;
-import edu.uci.ics.hyracks.control.cc.remote.ops.ApplicationStarter;
-import edu.uci.ics.hyracks.control.common.application.ApplicationContext;
+import edu.uci.ics.hyracks.control.cc.NodeControllerState;
+import edu.uci.ics.hyracks.control.cc.application.CCApplicationContext;
+import edu.uci.ics.hyracks.control.common.application.ApplicationStatus;
+import edu.uci.ics.hyracks.control.common.base.INodeController;
import edu.uci.ics.hyracks.control.common.work.AbstractWork;
-import edu.uci.ics.hyracks.control.common.work.FutureValue;
+import edu.uci.ics.hyracks.control.common.work.IResultCallback;
public class ApplicationStartWork extends AbstractWork {
private final ClusterControllerService ccs;
private final String appName;
- private final FutureValue<Object> fv;
+ private final IResultCallback<Object> callback;
- public ApplicationStartWork(ClusterControllerService ccs, String appName, FutureValue<Object> fv) {
+ public ApplicationStartWork(ClusterControllerService ccs, String appName, IResultCallback<Object> callback) {
this.ccs = ccs;
this.appName = appName;
- this.fv = fv;
+ this.callback = callback;
}
@Override
public void run() {
- ApplicationContext appCtx = ccs.getApplicationMap().get(appName);
- if (appCtx == null) {
- fv.setException(new HyracksException("No application with name: " + appName));
- return;
- }
try {
- appCtx.initializeClassPath();
- appCtx.initialize();
- final byte[] distributedState = JavaSerializationUtils.serialize(appCtx.getDistributedState());
- final boolean deployHar = appCtx.containsHar();
- List<RemoteOp<Void>> opList = new ArrayList<RemoteOp<Void>>();
- for (final String nodeId : ccs.getNodeMap().keySet()) {
- opList.add(new ApplicationStarter(nodeId, appName, deployHar, distributedState));
+ final CCApplicationContext appCtx = ccs.getApplicationMap().get(appName);
+ if (appCtx == null) {
+ callback.setException(new HyracksException("No application with name: " + appName));
+ return;
}
- final RemoteOp[] ops = opList.toArray(new RemoteOp[opList.size()]);
+ if (appCtx.getStatus() != ApplicationStatus.CREATED) {
+ callback.setException(new HyracksException("Application in incorrect state for starting: "
+ + appCtx.getStatus()));
+ }
+ final Map<String, NodeControllerState> nodeMapCopy = new HashMap<String, NodeControllerState>(
+ ccs.getNodeMap());
+ appCtx.getInitializationPendingNodeIds().addAll(nodeMapCopy.keySet());
+ appCtx.setStatus(ApplicationStatus.IN_INITIALIZATION);
+ appCtx.setInitializationCallback(callback);
ccs.getExecutor().execute(new Runnable() {
@Override
public void run() {
try {
- RemoteRunner.runRemote(ccs, ops, null);
- fv.setValue(null);
+ appCtx.initializeClassPath();
+ appCtx.initialize();
+ final byte[] distributedState = JavaSerializationUtils.serialize(appCtx.getDistributedState());
+ final boolean deployHar = appCtx.containsHar();
+ for (final String nodeId : nodeMapCopy.keySet()) {
+ NodeControllerState nodeState = nodeMapCopy.get(nodeId);
+ final INodeController node = nodeState.getNodeController();
+ node.createApplication(appName, deployHar, distributedState);
+ }
} catch (Exception e) {
- fv.setException(e);
+ callback.setException(e);
}
}
});
} catch (Exception e) {
- fv.setException(e);
+ callback.setException(e);
}
}
}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/ApplicationStateChangeWork.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/ApplicationStateChangeWork.java
new file mode 100644
index 0000000..f6271fe
--- /dev/null
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/ApplicationStateChangeWork.java
@@ -0,0 +1,85 @@
+package edu.uci.ics.hyracks.control.cc.work;
+
+import java.util.Set;
+import java.util.logging.Logger;
+
+import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
+import edu.uci.ics.hyracks.control.cc.application.CCApplicationContext;
+import edu.uci.ics.hyracks.control.common.application.ApplicationStatus;
+import edu.uci.ics.hyracks.control.common.ipc.CCNCFunctions;
+import edu.uci.ics.hyracks.control.common.work.AbstractWork;
+import edu.uci.ics.hyracks.control.common.work.IResultCallback;
+
+public class ApplicationStateChangeWork extends AbstractWork {
+ private static final Logger LOGGER = Logger.getLogger(ApplicationStateChangeWork.class.getName());
+
+ private final ClusterControllerService ccs;
+ private final CCNCFunctions.ApplicationStateChangeResponseFunction ascrf;
+
+ public ApplicationStateChangeWork(ClusterControllerService ccs,
+ CCNCFunctions.ApplicationStateChangeResponseFunction ascrf) {
+ this.ccs = ccs;
+ this.ascrf = ascrf;
+ }
+
+ @Override
+ public void run() {
+ final CCApplicationContext appCtx = ccs.getApplicationMap().get(ascrf.getApplicationName());
+ if (appCtx == null) {
+ LOGGER.warning("Got ApplicationStateChangeResponse for application " + ascrf.getApplicationName()
+ + " that does not exist");
+ return;
+ }
+ switch (ascrf.getStatus()) {
+ case INITIALIZED: {
+ Set<String> pendingNodeIds = appCtx.getInitializationPendingNodeIds();
+ boolean changed = pendingNodeIds.remove(ascrf.getNodeId());
+ if (!changed) {
+ LOGGER.warning("Got ApplicationStateChangeResponse for application " + ascrf.getApplicationName()
+ + " from unexpected node " + ascrf.getNodeId() + " to state " + ascrf.getStatus());
+ return;
+ }
+ if (pendingNodeIds.isEmpty()) {
+ appCtx.setStatus(ApplicationStatus.INITIALIZED);
+ IResultCallback<Object> callback = appCtx.getInitializationCallback();
+ appCtx.setInitializationCallback(null);
+ callback.setValue(null);
+ }
+ return;
+ }
+
+ case DEINITIALIZED: {
+ Set<String> pendingNodeIds = appCtx.getDeinitializationPendingNodeIds();
+ boolean changed = pendingNodeIds.remove(ascrf.getNodeId());
+ if (!changed) {
+ LOGGER.warning("Got ApplicationStateChangeResponse for application " + ascrf.getApplicationName()
+ + " from unexpected node " + ascrf.getNodeId() + " to state " + ascrf.getStatus());
+ return;
+ }
+ if (pendingNodeIds.isEmpty()) {
+ appCtx.setStatus(ApplicationStatus.DEINITIALIZED);
+ ccs.getExecutor().execute(new Runnable() {
+ @Override
+ public void run() {
+ try {
+ appCtx.deinitialize();
+ } catch (Exception e) {
+ e.printStackTrace();
+ }
+ ccs.getWorkQueue().schedule(new AbstractWork() {
+ @Override
+ public void run() {
+ ccs.getApplicationMap().remove(ascrf.getApplicationName());
+ IResultCallback<Object> callback = appCtx.getDeinitializationCallback();
+ appCtx.setDeinitializationCallback(null);
+ callback.setValue(null);
+ }
+ });
+ }
+ });
+ }
+ return;
+ }
+ }
+ }
+}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/GetJobStatusConditionVariableWork.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/GetIpAddressNodeNameMapWork.java
similarity index 60%
rename from hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/GetJobStatusConditionVariableWork.java
rename to hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/GetIpAddressNodeNameMapWork.java
index ff67928..1f29903 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/GetJobStatusConditionVariableWork.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/GetIpAddressNodeNameMapWork.java
@@ -14,30 +14,23 @@
*/
package edu.uci.ics.hyracks.control.cc.work;
-import edu.uci.ics.hyracks.api.job.JobId;
+import java.util.Map;
+import java.util.Set;
+
import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
-import edu.uci.ics.hyracks.control.cc.job.IJobStatusConditionVariable;
import edu.uci.ics.hyracks.control.common.work.SynchronizableWork;
-public class GetJobStatusConditionVariableWork extends SynchronizableWork {
+public class GetIpAddressNodeNameMapWork extends SynchronizableWork {
private final ClusterControllerService ccs;
- private final JobId jobId;
- private IJobStatusConditionVariable cVar;
+ private Map<String, Set<String>> map;
- public GetJobStatusConditionVariableWork(ClusterControllerService ccs, JobId jobId) {
+ public GetIpAddressNodeNameMapWork(ClusterControllerService ccs, Map<String, Set<String>> map) {
this.ccs = ccs;
- this.jobId = jobId;
+ this.map = map;
}
@Override
protected void doRun() throws Exception {
- cVar = ccs.getActiveRunMap().get(jobId);
- if (cVar == null) {
- cVar = ccs.getRunMapArchive().get(jobId);
- }
- }
-
- public IJobStatusConditionVariable getConditionVariable() {
- return cVar;
+ map.putAll(ccs.getIpAddressNodeNameMap());
}
}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/GetJobStatusWork.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/GetJobStatusWork.java
index 7a8943a..c70e24f 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/GetJobStatusWork.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/GetJobStatusWork.java
@@ -18,28 +18,31 @@
import edu.uci.ics.hyracks.api.job.JobStatus;
import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
import edu.uci.ics.hyracks.control.cc.job.JobRun;
+import edu.uci.ics.hyracks.control.common.work.IResultCallback;
import edu.uci.ics.hyracks.control.common.work.SynchronizableWork;
public class GetJobStatusWork extends SynchronizableWork {
private final ClusterControllerService ccs;
private final JobId jobId;
- private JobStatus status;
+ private final IResultCallback<JobStatus> callback;
- public GetJobStatusWork(ClusterControllerService ccs, JobId jobId) {
+ public GetJobStatusWork(ClusterControllerService ccs, JobId jobId, IResultCallback<JobStatus> callback) {
this.ccs = ccs;
this.jobId = jobId;
+ this.callback = callback;
}
@Override
protected void doRun() throws Exception {
- JobRun run = ccs.getActiveRunMap().get(jobId);
- if (run == null) {
- run = ccs.getRunMapArchive().get(jobId);
+ try {
+ JobRun run = ccs.getActiveRunMap().get(jobId);
+ if (run == null) {
+ run = ccs.getRunMapArchive().get(jobId);
+ }
+ JobStatus status = run == null ? null : run.getStatus();
+ callback.setValue(status);
+ } catch (Exception e) {
+ callback.setException(e);
}
- status = run == null ? null : run.getStatus();
- }
-
- public JobStatus getStatus() {
- return status;
}
}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/GetNodeControllersInfoWork.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/GetNodeControllersInfoWork.java
index 48d9b84..9e8d130 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/GetNodeControllersInfoWork.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/GetNodeControllersInfoWork.java
@@ -22,15 +22,16 @@
import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
import edu.uci.ics.hyracks.control.cc.NodeControllerState;
import edu.uci.ics.hyracks.control.common.work.AbstractWork;
-import edu.uci.ics.hyracks.control.common.work.FutureValue;
+import edu.uci.ics.hyracks.control.common.work.IResultCallback;
public class GetNodeControllersInfoWork extends AbstractWork {
private final ClusterControllerService ccs;
- private FutureValue<Map<String, NodeControllerInfo>> fv;
+ private IResultCallback<Map<String, NodeControllerInfo>> callback;
- public GetNodeControllersInfoWork(ClusterControllerService ccs, FutureValue<Map<String, NodeControllerInfo>> fv) {
+ public GetNodeControllersInfoWork(ClusterControllerService ccs,
+ IResultCallback<Map<String, NodeControllerInfo>> callback) {
this.ccs = ccs;
- this.fv = fv;
+ this.callback = callback;
}
@Override
@@ -38,9 +39,8 @@
Map<String, NodeControllerInfo> result = new LinkedHashMap<String, NodeControllerInfo>();
Map<String, NodeControllerState> nodeMap = ccs.getNodeMap();
for (Map.Entry<String, NodeControllerState> e : nodeMap.entrySet()) {
- result.put(e.getKey(), new NodeControllerInfo(e.getKey(), e.getValue().getDataPort().getIpAddress(),
- NodeStatus.ALIVE));
+ result.put(e.getKey(), new NodeControllerInfo(e.getKey(), NodeStatus.ALIVE));
}
- fv.setValue(result);
+ callback.setValue(result);
}
}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/JobCleanupWork.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/JobCleanupWork.java
index e30a718..07088bb 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/JobCleanupWork.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/JobCleanupWork.java
@@ -20,9 +20,8 @@
import edu.uci.ics.hyracks.api.job.JobId;
import edu.uci.ics.hyracks.api.job.JobStatus;
import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
+import edu.uci.ics.hyracks.control.cc.NodeControllerState;
import edu.uci.ics.hyracks.control.cc.job.JobRun;
-import edu.uci.ics.hyracks.control.cc.remote.RemoteRunner;
-import edu.uci.ics.hyracks.control.cc.remote.ops.JobCompleteNotifier;
import edu.uci.ics.hyracks.control.common.work.AbstractWork;
public class JobCleanupWork extends AbstractWork {
@@ -54,22 +53,13 @@
Set<String> targetNodes = run.getParticipatingNodeIds();
run.getCleanupPendingNodeIds().addAll(targetNodes);
run.setPendingStatus(status, exception);
- final JobCompleteNotifier[] jcns = new JobCompleteNotifier[targetNodes.size()];
- int i = 0;
for (String n : targetNodes) {
- jcns[i++] = new JobCompleteNotifier(n, jobId, status);
- }
- ccs.getExecutor().execute(new Runnable() {
- @Override
- public void run() {
- if (jcns.length > 0) {
- try {
- RemoteRunner.runRemote(ccs, jcns, null);
- } catch (Exception e) {
- e.printStackTrace();
- }
- }
+ NodeControllerState ncs = ccs.getNodeMap().get(n);
+ try {
+ ncs.getNodeController().cleanUpJoblet(jobId, status);
+ } catch (Exception e) {
+ e.printStackTrace();
}
- });
+ }
}
}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/JobCreateWork.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/JobCreateWork.java
index 14947cd..b7cf629 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/JobCreateWork.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/JobCreateWork.java
@@ -30,6 +30,7 @@
import edu.uci.ics.hyracks.control.cc.job.JobRun;
import edu.uci.ics.hyracks.control.cc.job.PlanUtils;
import edu.uci.ics.hyracks.control.cc.scheduler.JobScheduler;
+import edu.uci.ics.hyracks.control.common.work.IResultCallback;
import edu.uci.ics.hyracks.control.common.work.SynchronizableWork;
public class JobCreateWork extends SynchronizableWork {
@@ -38,42 +39,50 @@
private final EnumSet<JobFlag> jobFlags;
private final JobId jobId;
private final String appName;
+ private final IResultCallback<JobId> callback;
public JobCreateWork(ClusterControllerService ccs, JobId jobId, String appName, byte[] jobSpec,
- EnumSet<JobFlag> jobFlags) {
+ EnumSet<JobFlag> jobFlags, IResultCallback<JobId> callback) {
this.jobId = jobId;
this.ccs = ccs;
this.jobSpec = jobSpec;
this.jobFlags = jobFlags;
this.appName = appName;
+ this.callback = callback;
}
@Override
protected void doRun() throws Exception {
- CCApplicationContext appCtx = ccs.getApplicationMap().get(appName);
- if (appCtx == null) {
- throw new HyracksException("No application with id " + appName + " found");
- }
- JobSpecification spec = appCtx.createJobSpecification(jobSpec);
-
- final JobActivityGraphBuilder builder = new JobActivityGraphBuilder();
- builder.init(appName, spec, jobFlags);
- PlanUtils.visit(spec, new IOperatorDescriptorVisitor() {
- @Override
- public void visit(IOperatorDescriptor op) {
- op.contributeActivities(builder);
+ try {
+ CCApplicationContext appCtx = ccs.getApplicationMap().get(appName);
+ if (appCtx == null) {
+ throw new HyracksException("No application with id " + appName + " found");
}
- });
- final JobActivityGraph jag = builder.getActivityGraph();
+ JobSpecification spec = appCtx.createJobSpecification(jobSpec);
- JobRun run = new JobRun(jobId, jag);
+ final JobActivityGraphBuilder builder = new JobActivityGraphBuilder();
+ builder.init(appName, spec, jobFlags);
+ PlanUtils.visit(spec, new IOperatorDescriptorVisitor() {
+ @Override
+ public void visit(IOperatorDescriptor op) {
+ op.contributeActivities(builder);
+ }
+ });
+ final JobActivityGraph jag = builder.getActivityGraph();
- run.setStatus(JobStatus.INITIALIZED, null);
+ JobRun run = new JobRun(jobId, jag);
- ccs.getActiveRunMap().put(jobId, run);
- JobScheduler jrs = new JobScheduler(ccs, run);
- run.setScheduler(jrs);
- appCtx.notifyJobCreation(jobId, spec);
+ run.setStatus(JobStatus.INITIALIZED, null);
+
+ ccs.getActiveRunMap().put(jobId, run);
+ JobScheduler jrs = new JobScheduler(ccs, run);
+ run.setScheduler(jrs);
+ appCtx.notifyJobCreation(jobId, spec);
+ callback.setValue(jobId);
+ } catch (Exception e) {
+ callback.setException(e);
+ return;
+ }
}
public JobId getJobId() {
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/JobStartWork.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/JobStartWork.java
index 855e3e7..dd834ed 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/JobStartWork.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/JobStartWork.java
@@ -18,31 +18,39 @@
import edu.uci.ics.hyracks.api.job.JobStatus;
import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
import edu.uci.ics.hyracks.control.cc.job.JobRun;
+import edu.uci.ics.hyracks.control.common.work.IResultCallback;
import edu.uci.ics.hyracks.control.common.work.SynchronizableWork;
public class JobStartWork extends SynchronizableWork {
private final ClusterControllerService ccs;
private final JobId jobId;
+ private final IResultCallback<Object> callback;
- public JobStartWork(ClusterControllerService ccs, JobId jobId) {
+ public JobStartWork(ClusterControllerService ccs, JobId jobId, IResultCallback<Object> callback) {
this.ccs = ccs;
this.jobId = jobId;
+ this.callback = callback;
}
@Override
protected void doRun() throws Exception {
- JobRun run = ccs.getActiveRunMap().get(jobId);
- if (run == null) {
- throw new Exception("Unable to find job with id = " + jobId);
- }
- if (run.getStatus() != JobStatus.INITIALIZED) {
- throw new Exception("Job already started");
- }
- run.setStatus(JobStatus.RUNNING, null);
try {
- run.getScheduler().startJob();
+ JobRun run = ccs.getActiveRunMap().get(jobId);
+ if (run == null) {
+ throw new Exception("Unable to find job with id = " + jobId);
+ }
+ if (run.getStatus() != JobStatus.INITIALIZED) {
+ throw new Exception("Job already started");
+ }
+ run.setStatus(JobStatus.RUNNING, null);
+ try {
+ run.getScheduler().startJob();
+ } catch (Exception e) {
+ ccs.getWorkQueue().schedule(new JobCleanupWork(ccs, run.getJobId(), JobStatus.FAILURE, e));
+ }
+ callback.setValue(null);
} catch (Exception e) {
- ccs.getWorkQueue().schedule(new JobCleanupWork(ccs, run.getJobId(), JobStatus.FAILURE, e));
+ callback.setException(e);
}
}
}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/RegisterNodeWork.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/RegisterNodeWork.java
index 15daff5..e4e0aae 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/RegisterNodeWork.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/RegisterNodeWork.java
@@ -14,29 +14,65 @@
*/
package edu.uci.ics.hyracks.control.cc.work;
+import java.util.HashSet;
import java.util.Map;
+import java.util.Set;
+import java.util.logging.Level;
+import java.util.logging.Logger;
import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
import edu.uci.ics.hyracks.control.cc.NodeControllerState;
+import edu.uci.ics.hyracks.control.common.base.INodeController;
+import edu.uci.ics.hyracks.control.common.controllers.NodeParameters;
+import edu.uci.ics.hyracks.control.common.controllers.NodeRegistration;
+import edu.uci.ics.hyracks.control.common.ipc.CCNCFunctions;
+import edu.uci.ics.hyracks.control.common.ipc.NodeControllerRemoteProxy;
import edu.uci.ics.hyracks.control.common.work.SynchronizableWork;
+import edu.uci.ics.hyracks.ipc.api.IIPCHandle;
public class RegisterNodeWork extends SynchronizableWork {
- private final ClusterControllerService ccs;
- private final String nodeId;
- private final NodeControllerState state;
+ private static final Logger LOGGER = Logger.getLogger(RegisterNodeWork.class.getName());
- public RegisterNodeWork(ClusterControllerService ccs, String nodeId, NodeControllerState state) {
+ private final ClusterControllerService ccs;
+ private final NodeRegistration reg;
+
+ public RegisterNodeWork(ClusterControllerService ccs, NodeRegistration reg) {
this.ccs = ccs;
- this.nodeId = nodeId;
- this.state = state;
+ this.reg = reg;
}
@Override
protected void doRun() throws Exception {
- Map<String, NodeControllerState> nodeMap = ccs.getNodeMap();
- if (nodeMap.containsKey(nodeId)) {
- throw new Exception("Node with this name already registered.");
+ String id = reg.getNodeId();
+
+ IIPCHandle ncIPCHandle = ccs.getClusterIPC().getHandle(reg.getNodeControllerAddress());
+ CCNCFunctions.NodeRegistrationResult result = null;
+ try {
+ INodeController nodeController = new NodeControllerRemoteProxy(ncIPCHandle);
+
+ NodeControllerState state = new NodeControllerState(nodeController, reg);
+ Map<String, NodeControllerState> nodeMap = ccs.getNodeMap();
+ if (nodeMap.containsKey(id)) {
+ throw new Exception("Node with this name already registered.");
+ }
+ nodeMap.put(id, state);
+ Map<String, Set<String>> ipAddressNodeNameMap = ccs.getIpAddressNodeNameMap();
+ String ipAddress = state.getNCConfig().dataIPAddress;
+ Set<String> nodes = ipAddressNodeNameMap.get(ipAddress);
+ if (nodes == null) {
+ nodes = new HashSet<String>();
+ ipAddressNodeNameMap.put(ipAddress, nodes);
+ }
+ nodes.add(id);
+ LOGGER.log(Level.INFO, "Registered INodeController: id = " + id);
+ NodeParameters params = new NodeParameters();
+ params.setClusterControllerInfo(ccs.getClusterControllerInfo());
+ params.setHeartbeatPeriod(ccs.getCCConfig().heartbeatPeriod);
+ params.setProfileDumpPeriod(ccs.getCCConfig().profileDumpPeriod);
+ result = new CCNCFunctions.NodeRegistrationResult(params, null);
+ } catch (Exception e) {
+ result = new CCNCFunctions.NodeRegistrationResult(null, e);
}
- nodeMap.put(nodeId, state);
+ ncIPCHandle.send(-1, result, null);
}
}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/WaitForJobCompletionWork.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/WaitForJobCompletionWork.java
new file mode 100644
index 0000000..6cfe025
--- /dev/null
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/WaitForJobCompletionWork.java
@@ -0,0 +1,64 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ *
+ * 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 edu.uci.ics.hyracks.control.cc.work;
+
+import edu.uci.ics.hyracks.api.job.JobId;
+import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
+import edu.uci.ics.hyracks.control.cc.job.IJobStatusConditionVariable;
+import edu.uci.ics.hyracks.control.common.work.IResultCallback;
+import edu.uci.ics.hyracks.control.common.work.SynchronizableWork;
+
+public class WaitForJobCompletionWork extends SynchronizableWork {
+ private final ClusterControllerService ccs;
+ private final JobId jobId;
+ private final IResultCallback<Object> callback;
+
+ public WaitForJobCompletionWork(ClusterControllerService ccs, JobId jobId, IResultCallback<Object> callback) {
+ this.ccs = ccs;
+ this.jobId = jobId;
+ this.callback = callback;
+ }
+
+ @Override
+ protected void doRun() throws Exception {
+ final IJobStatusConditionVariable cRunningVar = ccs.getActiveRunMap().get(jobId);
+ if (cRunningVar != null) {
+ ccs.getExecutor().execute(new Runnable() {
+ @Override
+ public void run() {
+ try {
+ cRunningVar.waitForCompletion();
+ callback.setValue(null);
+ } catch (Exception e) {
+ callback.setException(e);
+ }
+ }
+ });
+ } else {
+ final IJobStatusConditionVariable cArchivedVar = ccs.getRunMapArchive().get(jobId);
+ ccs.getExecutor().execute(new Runnable() {
+ @Override
+ public void run() {
+ try {
+ cArchivedVar.waitForCompletion();
+ callback.setValue(null);
+ } catch (Exception e) {
+ callback.setException(e);
+ }
+ }
+ });
+ }
+ }
+}
\ No newline at end of file
diff --git a/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/AbstractRemoteService.java b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/AbstractRemoteService.java
index 42d0ecb..21e3e9d 100644
--- a/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/AbstractRemoteService.java
+++ b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/AbstractRemoteService.java
@@ -19,4 +19,4 @@
public abstract class AbstractRemoteService implements IService {
public AbstractRemoteService() {
}
-}
+}
\ No newline at end of file
diff --git a/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/application/ApplicationContext.java b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/application/ApplicationContext.java
index d2a06be..9f29fbd 100644
--- a/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/application/ApplicationContext.java
+++ b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/application/ApplicationContext.java
@@ -59,7 +59,6 @@
this.serverCtx = serverCtx;
this.appName = appName;
this.applicationRootDir = new File(new File(serverCtx.getBaseDir(), APPLICATION_ROOT), appName);
- status = ApplicationStatus.CREATED;
FileUtils.deleteDirectory(applicationRootDir);
applicationRootDir.mkdirs();
}
@@ -87,9 +86,6 @@
}
public void initialize() throws Exception {
- if (status != ApplicationStatus.CREATED) {
- throw new IllegalStateException();
- }
if (deploymentDescriptor != null) {
String bootstrapClass = null;
switch (serverCtx.getServerType()) {
@@ -107,7 +103,6 @@
start();
}
}
- status = ApplicationStatus.INITIALIZED;
}
protected abstract void start() throws Exception;
@@ -168,7 +163,6 @@
}
public void deinitialize() throws Exception {
- status = ApplicationStatus.DEINITIALIZED;
stop();
File expandedFolder = getExpandedFolder();
FileUtils.deleteDirectory(expandedFolder);
@@ -203,4 +197,12 @@
public ClassLoader getClassLoader() {
return classLoader;
}
-}
+
+ public void setStatus(ApplicationStatus status) {
+ this.status = status;
+ }
+
+ public ApplicationStatus getStatus() {
+ return status;
+ }
+}
\ No newline at end of file
diff --git a/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/application/ApplicationStatus.java b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/application/ApplicationStatus.java
index 8a97ad6..b45c4f1 100644
--- a/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/application/ApplicationStatus.java
+++ b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/application/ApplicationStatus.java
@@ -16,6 +16,8 @@
public enum ApplicationStatus {
CREATED,
+ IN_INITIALIZATION,
INITIALIZED,
+ IN_DEINITIALIZATION,
DEINITIALIZED
}
\ No newline at end of file
diff --git a/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/base/IClusterController.java b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/base/IClusterController.java
index 40e9347..a03f5c9 100644
--- a/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/base/IClusterController.java
+++ b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/base/IClusterController.java
@@ -18,7 +18,7 @@
import edu.uci.ics.hyracks.api.dataflow.TaskAttemptId;
import edu.uci.ics.hyracks.api.job.JobId;
-import edu.uci.ics.hyracks.control.common.controllers.NodeParameters;
+import edu.uci.ics.hyracks.control.common.application.ApplicationStatus;
import edu.uci.ics.hyracks.control.common.controllers.NodeRegistration;
import edu.uci.ics.hyracks.control.common.heartbeat.HeartbeatData;
import edu.uci.ics.hyracks.control.common.job.PartitionDescriptor;
@@ -27,7 +27,7 @@
import edu.uci.ics.hyracks.control.common.job.profiling.om.TaskProfile;
public interface IClusterController {
- public NodeParameters registerNode(NodeRegistration reg) throws Exception;
+ public void registerNode(NodeRegistration reg) throws Exception;
public void unregisterNode(String nodeId) throws Exception;
@@ -45,4 +45,6 @@
public void registerPartitionProvider(PartitionDescriptor partitionDescriptor) throws Exception;
public void registerPartitionRequest(PartitionRequest partitionRequest) throws Exception;
+
+ public void notifyApplicationStateChange(String nodeId, String appName, ApplicationStatus status) throws Exception;
}
\ No newline at end of file
diff --git a/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/controllers/NCConfig.java b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/controllers/NCConfig.java
index c55f34a..574f552 100644
--- a/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/controllers/NCConfig.java
+++ b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/controllers/NCConfig.java
@@ -52,6 +52,9 @@
@Option(name = "-dcache-client-path", usage = "Sets the path to store the files retrieved from the DCache server (default /tmp/dcache-client)")
public String dcacheClientPath = "/tmp/dcache-client";
+ @Option(name = "-net-thread-count", usage = "Number of threads to use for Network I/O (default: 1)")
+ public int nNetThreads = 1;
+
public void toCommandLine(List<String> cList) {
cList.add("-cc-host");
cList.add(ccHost);
@@ -75,5 +78,7 @@
}
cList.add("-dcache-client-path");
cList.add(dcacheClientPath);
+ cList.add("-net-thread-count");
+ cList.add(String.valueOf(nNetThreads));
}
}
\ No newline at end of file
diff --git a/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/heartbeat/HeartbeatData.java b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/heartbeat/HeartbeatData.java
index cd9086f..8ca0fa4 100644
--- a/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/heartbeat/HeartbeatData.java
+++ b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/heartbeat/HeartbeatData.java
@@ -33,4 +33,8 @@
public double systemLoadAverage;
public long[] gcCollectionCounts;
public long[] gcCollectionTimes;
+ public long netPayloadBytesRead;
+ public long netPayloadBytesWritten;
+ public long netSignalingBytesRead;
+ public long netSignalingBytesWritten;
}
\ No newline at end of file
diff --git a/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/ipc/CCNCFunctions.java b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/ipc/CCNCFunctions.java
new file mode 100644
index 0000000..9ea4636
--- /dev/null
+++ b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/ipc/CCNCFunctions.java
@@ -0,0 +1,792 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ *
+ * 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 edu.uci.ics.hyracks.control.common.ipc;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.io.Serializable;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.Map;
+
+import edu.uci.ics.hyracks.api.comm.NetworkAddress;
+import edu.uci.ics.hyracks.api.dataflow.ActivityId;
+import edu.uci.ics.hyracks.api.dataflow.ConnectorDescriptorId;
+import edu.uci.ics.hyracks.api.dataflow.OperatorDescriptorId;
+import edu.uci.ics.hyracks.api.dataflow.TaskAttemptId;
+import edu.uci.ics.hyracks.api.dataflow.TaskId;
+import edu.uci.ics.hyracks.api.dataflow.connectors.IConnectorPolicy;
+import edu.uci.ics.hyracks.api.job.JobId;
+import edu.uci.ics.hyracks.api.job.JobStatus;
+import edu.uci.ics.hyracks.api.partitions.PartitionId;
+import edu.uci.ics.hyracks.control.common.application.ApplicationStatus;
+import edu.uci.ics.hyracks.control.common.controllers.NodeParameters;
+import edu.uci.ics.hyracks.control.common.controllers.NodeRegistration;
+import edu.uci.ics.hyracks.control.common.heartbeat.HeartbeatData;
+import edu.uci.ics.hyracks.control.common.job.PartitionDescriptor;
+import edu.uci.ics.hyracks.control.common.job.PartitionRequest;
+import edu.uci.ics.hyracks.control.common.job.PartitionState;
+import edu.uci.ics.hyracks.control.common.job.TaskAttemptDescriptor;
+import edu.uci.ics.hyracks.control.common.job.profiling.om.JobProfile;
+import edu.uci.ics.hyracks.control.common.job.profiling.om.TaskProfile;
+import edu.uci.ics.hyracks.ipc.api.IPayloadSerializerDeserializer;
+import edu.uci.ics.hyracks.ipc.impl.JavaSerializationBasedPayloadSerializerDeserializer;
+
+public class CCNCFunctions {
+ private static final int FID_CODE_SIZE = 1;
+
+ public enum FunctionId {
+ REGISTER_NODE,
+ UNREGISTER_NODE,
+ NOTIFY_JOBLET_CLEANUP,
+ NOTIFY_TASK_COMPLETE,
+ NOTIFY_TASK_FAILURE,
+ NODE_HEARTBEAT,
+ REPORT_PROFILE,
+ REGISTER_PARTITION_PROVIDER,
+ REGISTER_PARTITION_REQUEST,
+ APPLICATION_STATE_CHANGE_RESPONSE,
+
+ NODE_REGISTRATION_RESULT,
+ START_TASKS,
+ ABORT_TASKS,
+ CLEANUP_JOBLET,
+ CREATE_APPLICATION,
+ DESTROY_APPLICATION,
+ REPORT_PARTITION_AVAILABILITY,
+
+ OTHER
+ }
+
+ public static abstract class Function implements Serializable {
+ private static final long serialVersionUID = 1L;
+
+ public abstract FunctionId getFunctionId();
+ }
+
+ public static class RegisterNodeFunction extends Function {
+ private static final long serialVersionUID = 1L;
+
+ private final NodeRegistration reg;
+
+ public RegisterNodeFunction(NodeRegistration reg) {
+ this.reg = reg;
+ }
+
+ @Override
+ public FunctionId getFunctionId() {
+ return FunctionId.REGISTER_NODE;
+ }
+
+ public NodeRegistration getNodeRegistration() {
+ return reg;
+ }
+ }
+
+ public static class UnregisterNodeFunction extends Function {
+ private static final long serialVersionUID = 1L;
+
+ private final String nodeId;
+
+ public UnregisterNodeFunction(String nodeId) {
+ this.nodeId = nodeId;
+ }
+
+ @Override
+ public FunctionId getFunctionId() {
+ return FunctionId.UNREGISTER_NODE;
+ }
+
+ public String getNodeId() {
+ return nodeId;
+ }
+ }
+
+ public static class NotifyTaskCompleteFunction extends Function {
+ private static final long serialVersionUID = 1L;
+
+ private final JobId jobId;
+ private final TaskAttemptId taskId;
+ private final String nodeId;
+ private final TaskProfile statistics;
+
+ public NotifyTaskCompleteFunction(JobId jobId, TaskAttemptId taskId, String nodeId, TaskProfile statistics) {
+ this.jobId = jobId;
+ this.taskId = taskId;
+ this.nodeId = nodeId;
+ this.statistics = statistics;
+ }
+
+ @Override
+ public FunctionId getFunctionId() {
+ return FunctionId.NOTIFY_TASK_COMPLETE;
+ }
+
+ public JobId getJobId() {
+ return jobId;
+ }
+
+ public TaskAttemptId getTaskId() {
+ return taskId;
+ }
+
+ public String getNodeId() {
+ return nodeId;
+ }
+
+ public TaskProfile getStatistics() {
+ return statistics;
+ }
+ }
+
+ public static class NotifyTaskFailureFunction extends Function {
+ private static final long serialVersionUID = 1L;
+
+ private final JobId jobId;
+ private final TaskAttemptId taskId;
+ private final String nodeId;
+ private final String details;
+
+ public NotifyTaskFailureFunction(JobId jobId, TaskAttemptId taskId, String nodeId, String details) {
+ this.jobId = jobId;
+ this.taskId = taskId;
+ this.nodeId = nodeId;
+ this.details = details;
+ }
+
+ @Override
+ public FunctionId getFunctionId() {
+ return FunctionId.NOTIFY_TASK_FAILURE;
+ }
+
+ public JobId getJobId() {
+ return jobId;
+ }
+
+ public TaskAttemptId getTaskId() {
+ return taskId;
+ }
+
+ public String getNodeId() {
+ return nodeId;
+ }
+
+ public String getDetails() {
+ return details;
+ }
+ }
+
+ public static class NotifyJobletCleanupFunction extends Function {
+ private static final long serialVersionUID = 1L;
+
+ private final JobId jobId;
+ private final String nodeId;
+
+ public NotifyJobletCleanupFunction(JobId jobId, String nodeId) {
+ this.jobId = jobId;
+ this.nodeId = nodeId;
+ }
+
+ @Override
+ public FunctionId getFunctionId() {
+ return FunctionId.NOTIFY_JOBLET_CLEANUP;
+ }
+
+ public JobId getJobId() {
+ return jobId;
+ }
+
+ public String getNodeId() {
+ return nodeId;
+ }
+ }
+
+ public static class NodeHeartbeatFunction extends Function {
+ private static final long serialVersionUID = 1L;
+
+ private final String nodeId;
+ private final HeartbeatData hbData;
+
+ public NodeHeartbeatFunction(String nodeId, HeartbeatData hbData) {
+ this.nodeId = nodeId;
+ this.hbData = hbData;
+ }
+
+ @Override
+ public FunctionId getFunctionId() {
+ return FunctionId.NODE_HEARTBEAT;
+ }
+
+ public String getNodeId() {
+ return nodeId;
+ }
+
+ public HeartbeatData getHeartbeatData() {
+ return hbData;
+ }
+ }
+
+ public static class ReportProfileFunction extends Function {
+ private static final long serialVersionUID = 1L;
+
+ private final String nodeId;
+ private final List<JobProfile> profiles;
+
+ public ReportProfileFunction(String nodeId, List<JobProfile> profiles) {
+ this.nodeId = nodeId;
+ this.profiles = profiles;
+ }
+
+ @Override
+ public FunctionId getFunctionId() {
+ return FunctionId.REPORT_PROFILE;
+ }
+
+ public String getNodeId() {
+ return nodeId;
+ }
+
+ public List<JobProfile> getProfiles() {
+ return profiles;
+ }
+ }
+
+ public static class RegisterPartitionProviderFunction extends Function {
+ private static final long serialVersionUID = 1L;
+
+ private final PartitionDescriptor partitionDescriptor;
+
+ public RegisterPartitionProviderFunction(PartitionDescriptor partitionDescriptor) {
+ this.partitionDescriptor = partitionDescriptor;
+ }
+
+ @Override
+ public FunctionId getFunctionId() {
+ return FunctionId.REGISTER_PARTITION_PROVIDER;
+ }
+
+ public PartitionDescriptor getPartitionDescriptor() {
+ return partitionDescriptor;
+ }
+
+ public static Object deserialize(ByteBuffer buffer, int length) throws Exception {
+ ByteArrayInputStream bais = new ByteArrayInputStream(buffer.array(), buffer.position(), length);
+ DataInputStream dis = new DataInputStream(bais);
+
+ // Read PartitionId
+ PartitionId pid = readPartitionId(dis);
+
+ // Read nodeId
+ String nodeId = dis.readUTF();
+
+ // Read TaskAttemptId
+ TaskAttemptId taId = readTaskAttemptId(dis);
+
+ // Read reusable flag
+ boolean reusable = dis.readBoolean();
+
+ // Read Partition State
+ PartitionState state = readPartitionState(dis);
+
+ PartitionDescriptor pd = new PartitionDescriptor(pid, nodeId, taId, reusable);
+ pd.setState(state);
+ return new RegisterPartitionProviderFunction(pd);
+ }
+
+ public static void serialize(OutputStream out, Object object) throws Exception {
+ RegisterPartitionProviderFunction fn = (RegisterPartitionProviderFunction) object;
+
+ DataOutputStream dos = new DataOutputStream(out);
+
+ PartitionDescriptor pd = fn.getPartitionDescriptor();
+
+ // Write PartitionId
+ writePartitionId(dos, pd.getPartitionId());
+
+ // Write nodeId
+ dos.writeUTF(pd.getNodeId());
+
+ // Write TaskAttemptId
+ writeTaskAttemptId(dos, pd.getProducingTaskAttemptId());
+
+ // Write reusable flag
+ dos.writeBoolean(pd.isReusable());
+
+ // Write Partition State
+ writePartitionState(dos, pd.getState());
+ }
+ }
+
+ public static class RegisterPartitionRequestFunction extends Function {
+ private static final long serialVersionUID = 1L;
+
+ private final PartitionRequest partitionRequest;
+
+ public RegisterPartitionRequestFunction(PartitionRequest partitionRequest) {
+ this.partitionRequest = partitionRequest;
+ }
+
+ @Override
+ public FunctionId getFunctionId() {
+ return FunctionId.REGISTER_PARTITION_REQUEST;
+ }
+
+ public PartitionRequest getPartitionRequest() {
+ return partitionRequest;
+ }
+
+ public static Object deserialize(ByteBuffer buffer, int length) throws Exception {
+ ByteArrayInputStream bais = new ByteArrayInputStream(buffer.array(), buffer.position(), length);
+ DataInputStream dis = new DataInputStream(bais);
+
+ // Read PartitionId
+ PartitionId pid = readPartitionId(dis);
+
+ // Read nodeId
+ String nodeId = dis.readUTF();
+
+ // Read TaskAttemptId
+ TaskAttemptId taId = readTaskAttemptId(dis);
+
+ // Read Partition State
+ PartitionState state = readPartitionState(dis);
+
+ PartitionRequest pr = new PartitionRequest(pid, nodeId, taId, state);
+ return new RegisterPartitionRequestFunction(pr);
+ }
+
+ public static void serialize(OutputStream out, Object object) throws Exception {
+ RegisterPartitionRequestFunction fn = (RegisterPartitionRequestFunction) object;
+
+ DataOutputStream dos = new DataOutputStream(out);
+
+ PartitionRequest pr = fn.getPartitionRequest();
+
+ // Write PartitionId
+ writePartitionId(dos, pr.getPartitionId());
+
+ // Write nodeId
+ dos.writeUTF(pr.getNodeId());
+
+ // Write TaskAttemptId
+ writeTaskAttemptId(dos, pr.getRequestingTaskAttemptId());
+
+ // Write Partition State
+ writePartitionState(dos, pr.getMinimumState());
+ }
+ }
+
+ public static class ApplicationStateChangeResponseFunction extends Function {
+ private static final long serialVersionUID = 1L;
+
+ private final String nodeId;
+ private final String appName;
+ private final ApplicationStatus status;
+
+ public ApplicationStateChangeResponseFunction(String nodeId, String appName, ApplicationStatus status) {
+ this.nodeId = nodeId;
+ this.appName = appName;
+ this.status = status;
+ }
+
+ @Override
+ public FunctionId getFunctionId() {
+ return FunctionId.APPLICATION_STATE_CHANGE_RESPONSE;
+ }
+
+ public String getNodeId() {
+ return nodeId;
+ }
+
+ public String getApplicationName() {
+ return appName;
+ }
+
+ public ApplicationStatus getStatus() {
+ return status;
+ }
+ }
+
+ public static class NodeRegistrationResult extends Function {
+ private static final long serialVersionUID = 1L;
+
+ private final NodeParameters params;
+
+ private final Exception exception;
+
+ public NodeRegistrationResult(NodeParameters params, Exception exception) {
+ this.params = params;
+ this.exception = exception;
+ }
+
+ @Override
+ public FunctionId getFunctionId() {
+ return FunctionId.NODE_REGISTRATION_RESULT;
+ }
+
+ public NodeParameters getNodeParameters() {
+ return params;
+ }
+
+ public Exception getException() {
+ return exception;
+ }
+ }
+
+ public static class StartTasksFunction extends Function {
+ private static final long serialVersionUID = 1L;
+
+ private final String appName;
+ private final JobId jobId;
+ private final byte[] planBytes;
+ private final List<TaskAttemptDescriptor> taskDescriptors;
+ private final Map<ConnectorDescriptorId, IConnectorPolicy> connectorPolicies;
+
+ public StartTasksFunction(String appName, JobId jobId, byte[] planBytes,
+ List<TaskAttemptDescriptor> taskDescriptors,
+ Map<ConnectorDescriptorId, IConnectorPolicy> connectorPolicies) {
+ this.appName = appName;
+ this.jobId = jobId;
+ this.planBytes = planBytes;
+ this.taskDescriptors = taskDescriptors;
+ this.connectorPolicies = connectorPolicies;
+ }
+
+ @Override
+ public FunctionId getFunctionId() {
+ return FunctionId.START_TASKS;
+ }
+
+ public String getAppName() {
+ return appName;
+ }
+
+ public JobId getJobId() {
+ return jobId;
+ }
+
+ public byte[] getPlanBytes() {
+ return planBytes;
+ }
+
+ public List<TaskAttemptDescriptor> getTaskDescriptors() {
+ return taskDescriptors;
+ }
+
+ public Map<ConnectorDescriptorId, IConnectorPolicy> getConnectorPolicies() {
+ return connectorPolicies;
+ }
+ }
+
+ public static class AbortTasksFunction extends Function {
+ private static final long serialVersionUID = 1L;
+
+ private final JobId jobId;
+ private final List<TaskAttemptId> tasks;
+
+ public AbortTasksFunction(JobId jobId, List<TaskAttemptId> tasks) {
+ this.jobId = jobId;
+ this.tasks = tasks;
+ }
+
+ @Override
+ public FunctionId getFunctionId() {
+ return FunctionId.ABORT_TASKS;
+ }
+
+ public JobId getJobId() {
+ return jobId;
+ }
+
+ public List<TaskAttemptId> getTasks() {
+ return tasks;
+ }
+ }
+
+ public static class CleanupJobletFunction extends Function {
+ private static final long serialVersionUID = 1L;
+
+ private final JobId jobId;
+ private final JobStatus status;
+
+ public CleanupJobletFunction(JobId jobId, JobStatus status) {
+ this.jobId = jobId;
+ this.status = status;
+ }
+
+ @Override
+ public FunctionId getFunctionId() {
+ return FunctionId.CLEANUP_JOBLET;
+ }
+
+ public JobId getJobId() {
+ return jobId;
+ }
+
+ public JobStatus getStatus() {
+ return status;
+ }
+ }
+
+ public static class CreateApplicationFunction extends Function {
+ private static final long serialVersionUID = 1L;
+
+ private final String appName;
+ private final boolean deployHar;
+ private final byte[] serializedDistributedState;
+
+ public CreateApplicationFunction(String appName, boolean deployHar, byte[] serializedDistributedState) {
+ this.appName = appName;
+ this.deployHar = deployHar;
+ this.serializedDistributedState = serializedDistributedState;
+ }
+
+ @Override
+ public FunctionId getFunctionId() {
+ return FunctionId.CREATE_APPLICATION;
+ }
+
+ public String getAppName() {
+ return appName;
+ }
+
+ public boolean isDeployHar() {
+ return deployHar;
+ }
+
+ public byte[] getSerializedDistributedState() {
+ return serializedDistributedState;
+ }
+ }
+
+ public static class DestroyApplicationFunction extends Function {
+ private static final long serialVersionUID = 1L;
+
+ private final String appName;
+
+ public DestroyApplicationFunction(String appName) {
+ this.appName = appName;
+ }
+
+ @Override
+ public FunctionId getFunctionId() {
+ return FunctionId.DESTROY_APPLICATION;
+ }
+
+ public String getAppName() {
+ return appName;
+ }
+ }
+
+ public static class ReportPartitionAvailabilityFunction extends Function {
+ private static final long serialVersionUID = 1L;
+
+ private final PartitionId pid;
+ private final NetworkAddress networkAddress;
+
+ public ReportPartitionAvailabilityFunction(PartitionId pid, NetworkAddress networkAddress) {
+ this.pid = pid;
+ this.networkAddress = networkAddress;
+ }
+
+ @Override
+ public FunctionId getFunctionId() {
+ return FunctionId.REPORT_PARTITION_AVAILABILITY;
+ }
+
+ public PartitionId getPartitionId() {
+ return pid;
+ }
+
+ public NetworkAddress getNetworkAddress() {
+ return networkAddress;
+ }
+
+ public static Object deserialize(ByteBuffer buffer, int length) throws Exception {
+ ByteArrayInputStream bais = new ByteArrayInputStream(buffer.array(), buffer.position(), length);
+ DataInputStream dis = new DataInputStream(bais);
+
+ // Read PartitionId
+ PartitionId pid = readPartitionId(dis);
+
+ // Read NetworkAddress
+ NetworkAddress networkAddress = readNetworkAddress(dis);
+
+ return new ReportPartitionAvailabilityFunction(pid, networkAddress);
+ }
+
+ public static void serialize(OutputStream out, Object object) throws Exception {
+ ReportPartitionAvailabilityFunction fn = (ReportPartitionAvailabilityFunction) object;
+
+ DataOutputStream dos = new DataOutputStream(out);
+
+ // Write PartitionId
+ writePartitionId(dos, fn.getPartitionId());
+
+ // Write NetworkAddress
+ writeNetworkAddress(dos, fn.getNetworkAddress());
+ }
+ }
+
+ public static class SerializerDeserializer implements IPayloadSerializerDeserializer {
+ private final JavaSerializationBasedPayloadSerializerDeserializer javaSerde;
+
+ public SerializerDeserializer() {
+ javaSerde = new JavaSerializationBasedPayloadSerializerDeserializer();
+ }
+
+ @Override
+ public Object deserializeObject(ByteBuffer buffer, int length) throws Exception {
+ if (length < FID_CODE_SIZE) {
+ throw new IllegalStateException("Message size too small: " + length);
+ }
+ byte fid = buffer.get();
+ return deserialize(fid, buffer, length - FID_CODE_SIZE);
+ }
+
+ @Override
+ public Exception deserializeException(ByteBuffer buffer, int length) throws Exception {
+ if (length < FID_CODE_SIZE) {
+ throw new IllegalStateException("Message size too small: " + length);
+ }
+ byte fid = buffer.get();
+ if (fid != FunctionId.OTHER.ordinal()) {
+ throw new IllegalStateException("Expected FID for OTHER, found: " + fid);
+ }
+ return (Exception) deserialize(fid, buffer, length - FID_CODE_SIZE);
+ }
+
+ @Override
+ public byte[] serializeObject(Object object) throws Exception {
+ if (object instanceof Function) {
+ Function fn = (Function) object;
+ return serialize(object, (byte) fn.getFunctionId().ordinal());
+ } else {
+ return serialize(object, (byte) FunctionId.OTHER.ordinal());
+ }
+ }
+
+ @Override
+ public byte[] serializeException(Exception object) throws Exception {
+ return serialize(object, (byte) FunctionId.OTHER.ordinal());
+ }
+
+ private byte[] serialize(Object object, byte fid) throws Exception {
+ ByteArrayOutputStream baos = new ByteArrayOutputStream();
+ baos.write(fid);
+ serialize(baos, object, fid);
+ JavaSerializationBasedPayloadSerializerDeserializer.serialize(baos, object);
+ baos.close();
+ return baos.toByteArray();
+ }
+
+ private void serialize(OutputStream out, Object object, byte fid) throws Exception {
+ switch (FunctionId.values()[fid]) {
+ case REGISTER_PARTITION_PROVIDER:
+ RegisterPartitionProviderFunction.serialize(out, object);
+ return;
+
+ case REGISTER_PARTITION_REQUEST:
+ RegisterPartitionRequestFunction.serialize(out, object);
+ return;
+
+ case REPORT_PARTITION_AVAILABILITY:
+ ReportPartitionAvailabilityFunction.serialize(out, object);
+ return;
+ }
+ JavaSerializationBasedPayloadSerializerDeserializer.serialize(out, object);
+ }
+
+ private Object deserialize(byte fid, ByteBuffer buffer, int length) throws Exception {
+ switch (FunctionId.values()[fid]) {
+ case REGISTER_PARTITION_PROVIDER:
+ return RegisterPartitionProviderFunction.deserialize(buffer, length);
+
+ case REGISTER_PARTITION_REQUEST:
+ return RegisterPartitionRequestFunction.deserialize(buffer, length);
+
+ case REPORT_PARTITION_AVAILABILITY:
+ return ReportPartitionAvailabilityFunction.deserialize(buffer, length);
+ }
+
+ return javaSerde.deserializeObject(buffer, length);
+ }
+ }
+
+ private static PartitionId readPartitionId(DataInputStream dis) throws IOException {
+ long jobId = dis.readLong();
+ int cdid = dis.readInt();
+ int senderIndex = dis.readInt();
+ int receiverIndex = dis.readInt();
+ PartitionId pid = new PartitionId(new JobId(jobId), new ConnectorDescriptorId(cdid), senderIndex, receiverIndex);
+ return pid;
+ }
+
+ private static void writePartitionId(DataOutputStream dos, PartitionId pid) throws IOException {
+ dos.writeLong(pid.getJobId().getId());
+ dos.writeInt(pid.getConnectorDescriptorId().getId());
+ dos.writeInt(pid.getSenderIndex());
+ dos.writeInt(pid.getReceiverIndex());
+ }
+
+ private static TaskAttemptId readTaskAttemptId(DataInputStream dis) throws IOException {
+ int odid = dis.readInt();
+ int aid = dis.readInt();
+ int partition = dis.readInt();
+ int attempt = dis.readInt();
+ TaskAttemptId taId = new TaskAttemptId(new TaskId(new ActivityId(new OperatorDescriptorId(odid), aid),
+ partition), attempt);
+ return taId;
+ }
+
+ private static void writeTaskAttemptId(DataOutputStream dos, TaskAttemptId taId) throws IOException {
+ TaskId tid = taId.getTaskId();
+ ActivityId aid = tid.getActivityId();
+ OperatorDescriptorId odId = aid.getOperatorDescriptorId();
+ dos.writeInt(odId.getId());
+ dos.writeInt(aid.getLocalId());
+ dos.writeInt(tid.getPartition());
+ dos.writeInt(taId.getAttempt());
+ }
+
+ private static PartitionState readPartitionState(DataInputStream dis) throws IOException {
+ PartitionState state = PartitionState.values()[dis.readInt()];
+ return state;
+ }
+
+ private static void writePartitionState(DataOutputStream dos, PartitionState state) throws IOException {
+ dos.writeInt(state.ordinal());
+ }
+
+ private static NetworkAddress readNetworkAddress(DataInputStream dis) throws IOException {
+ int bLen = dis.readInt();
+ byte[] ipAddress = new byte[bLen];
+ dis.read(ipAddress);
+ int port = dis.readInt();
+ NetworkAddress networkAddress = new NetworkAddress(ipAddress, port);
+ return networkAddress;
+ }
+
+ private static void writeNetworkAddress(DataOutputStream dos, NetworkAddress networkAddress) throws IOException {
+ byte[] ipAddress = networkAddress.getIpAddress();
+ dos.writeInt(ipAddress.length);
+ dos.write(ipAddress);
+ dos.writeInt(networkAddress.getPort());
+ }
+}
\ No newline at end of file
diff --git a/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/ipc/ClusterControllerDelegateIPCI.java b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/ipc/ClusterControllerDelegateIPCI.java
deleted file mode 100644
index e35f962..0000000
--- a/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/ipc/ClusterControllerDelegateIPCI.java
+++ /dev/null
@@ -1,87 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed 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 from
- *
- * 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 edu.uci.ics.hyracks.control.common.ipc;
-
-import edu.uci.ics.hyracks.control.common.base.IClusterController;
-import edu.uci.ics.hyracks.control.common.ipc.ClusterControllerFunctions.Function;
-import edu.uci.ics.hyracks.ipc.api.IIPCHandle;
-import edu.uci.ics.hyracks.ipc.api.IIPCI;
-
-public class ClusterControllerDelegateIPCI implements IIPCI {
- private final IClusterController cc;
-
- public ClusterControllerDelegateIPCI(IClusterController cc) {
- this.cc = cc;
- }
-
- @Override
- public Object call(IIPCHandle caller, Object req) throws Exception {
- ClusterControllerFunctions.Function fn = (Function) req;
- switch (fn.getFunctionId()) {
- case REGISTER_NODE: {
- ClusterControllerFunctions.RegisterNodeFunction rnf = (ClusterControllerFunctions.RegisterNodeFunction) fn;
- return cc.registerNode(rnf.getNodeRegistration());
- }
-
- case UNREGISTER_NODE: {
- ClusterControllerFunctions.UnregisterNodeFunction unf = (ClusterControllerFunctions.UnregisterNodeFunction) fn;
- cc.unregisterNode(unf.getNodeId());
- return null;
- }
-
- case NODE_HEARTBEAT: {
- ClusterControllerFunctions.NodeHeartbeatFunction nhf = (ClusterControllerFunctions.NodeHeartbeatFunction) fn;
- cc.nodeHeartbeat(nhf.getNodeId(), nhf.getHeartbeatData());
- return null;
- }
-
- case NOTIFY_JOBLET_CLEANUP: {
- ClusterControllerFunctions.NotifyJobletCleanupFunction njcf = (ClusterControllerFunctions.NotifyJobletCleanupFunction) fn;
- cc.notifyJobletCleanup(njcf.getJobId(), njcf.getNodeId());
- return null;
- }
-
- case REPORT_PROFILE: {
- ClusterControllerFunctions.ReportProfileFunction rpf = (ClusterControllerFunctions.ReportProfileFunction) fn;
- cc.reportProfile(rpf.getNodeId(), rpf.getProfiles());
- return null;
- }
-
- case NOTIFY_TASK_COMPLETE: {
- ClusterControllerFunctions.NotifyTaskCompleteFunction ntcf = (ClusterControllerFunctions.NotifyTaskCompleteFunction) fn;
- cc.notifyTaskComplete(ntcf.getJobId(), ntcf.getTaskId(), ntcf.getNodeId(), ntcf.getStatistics());
- return null;
- }
- case NOTIFY_TASK_FAILURE: {
- ClusterControllerFunctions.NotifyTaskFailureFunction ntff = (ClusterControllerFunctions.NotifyTaskFailureFunction) fn;
- cc.notifyTaskFailure(ntff.getJobId(), ntff.getTaskId(), ntff.getDetails(), ntff.getDetails());
- return null;
- }
-
- case REGISTER_PARTITION_PROVIDER: {
- ClusterControllerFunctions.RegisterPartitionProviderFunction rppf = (ClusterControllerFunctions.RegisterPartitionProviderFunction) fn;
- cc.registerPartitionProvider(rppf.getPartitionDescriptor());
- return null;
- }
-
- case REGISTER_PARTITION_REQUEST: {
- ClusterControllerFunctions.RegisterPartitionRequestFunction rprf = (ClusterControllerFunctions.RegisterPartitionRequestFunction) fn;
- cc.registerPartitionRequest(rprf.getPartitionRequest());
- return null;
- }
- }
- throw new IllegalArgumentException("Unknown function: " + fn.getFunctionId());
- }
-}
\ No newline at end of file
diff --git a/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/ipc/ClusterControllerFunctions.java b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/ipc/ClusterControllerFunctions.java
deleted file mode 100644
index 4c76357..0000000
--- a/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/ipc/ClusterControllerFunctions.java
+++ /dev/null
@@ -1,272 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed 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 from
- *
- * 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 edu.uci.ics.hyracks.control.common.ipc;
-
-import java.io.Serializable;
-import java.util.List;
-
-import edu.uci.ics.hyracks.api.dataflow.TaskAttemptId;
-import edu.uci.ics.hyracks.api.job.JobId;
-import edu.uci.ics.hyracks.control.common.controllers.NodeRegistration;
-import edu.uci.ics.hyracks.control.common.heartbeat.HeartbeatData;
-import edu.uci.ics.hyracks.control.common.job.PartitionDescriptor;
-import edu.uci.ics.hyracks.control.common.job.PartitionRequest;
-import edu.uci.ics.hyracks.control.common.job.profiling.om.JobProfile;
-import edu.uci.ics.hyracks.control.common.job.profiling.om.TaskProfile;
-
-public class ClusterControllerFunctions {
- public enum FunctionId {
- REGISTER_NODE,
- UNREGISTER_NODE,
- NOTIFY_TASK_COMPLETE,
- NOTIFY_TASK_FAILURE,
- NOTIFY_JOBLET_CLEANUP,
- NODE_HEARTBEAT,
- REPORT_PROFILE,
- REGISTER_PARTITION_PROVIDER,
- REGISTER_PARTITION_REQUEST,
- }
-
- public static abstract class Function implements Serializable {
- private static final long serialVersionUID = 1L;
-
- public abstract FunctionId getFunctionId();
- }
-
- public static class RegisterNodeFunction extends Function {
- private static final long serialVersionUID = 1L;
-
- private final NodeRegistration reg;
-
- public RegisterNodeFunction(NodeRegistration reg) {
- this.reg = reg;
- }
-
- @Override
- public FunctionId getFunctionId() {
- return FunctionId.REGISTER_NODE;
- }
-
- public NodeRegistration getNodeRegistration() {
- return reg;
- }
- }
-
- public static class UnregisterNodeFunction extends Function {
- private static final long serialVersionUID = 1L;
-
- private final String nodeId;
-
- public UnregisterNodeFunction(String nodeId) {
- this.nodeId = nodeId;
- }
-
- @Override
- public FunctionId getFunctionId() {
- return FunctionId.UNREGISTER_NODE;
- }
-
- public String getNodeId() {
- return nodeId;
- }
- }
-
- public static class NotifyTaskCompleteFunction extends Function {
- private static final long serialVersionUID = 1L;
-
- private final JobId jobId;
- private final TaskAttemptId taskId;
- private final String nodeId;
- private final TaskProfile statistics;
-
- public NotifyTaskCompleteFunction(JobId jobId, TaskAttemptId taskId, String nodeId, TaskProfile statistics) {
- this.jobId = jobId;
- this.taskId = taskId;
- this.nodeId = nodeId;
- this.statistics = statistics;
- }
-
- @Override
- public FunctionId getFunctionId() {
- return FunctionId.NOTIFY_TASK_COMPLETE;
- }
-
- public JobId getJobId() {
- return jobId;
- }
-
- public TaskAttemptId getTaskId() {
- return taskId;
- }
-
- public String getNodeId() {
- return nodeId;
- }
-
- public TaskProfile getStatistics() {
- return statistics;
- }
- }
-
- public static class NotifyTaskFailureFunction extends Function {
- private static final long serialVersionUID = 1L;
-
- private final JobId jobId;
- private final TaskAttemptId taskId;
- private final String nodeId;
- private final String details;
-
- public NotifyTaskFailureFunction(JobId jobId, TaskAttemptId taskId, String nodeId, String details) {
- this.jobId = jobId;
- this.taskId = taskId;
- this.nodeId = nodeId;
- this.details = details;
- }
-
- @Override
- public FunctionId getFunctionId() {
- return FunctionId.NOTIFY_TASK_FAILURE;
- }
-
- public JobId getJobId() {
- return jobId;
- }
-
- public TaskAttemptId getTaskId() {
- return taskId;
- }
-
- public String getNodeId() {
- return nodeId;
- }
-
- public String getDetails() {
- return details;
- }
- }
-
- public static class NotifyJobletCleanupFunction extends Function {
- private static final long serialVersionUID = 1L;
-
- private final JobId jobId;
- private final String nodeId;
-
- public NotifyJobletCleanupFunction(JobId jobId, String nodeId) {
- this.jobId = jobId;
- this.nodeId = nodeId;
- }
-
- @Override
- public FunctionId getFunctionId() {
- return FunctionId.NOTIFY_JOBLET_CLEANUP;
- }
-
- public JobId getJobId() {
- return jobId;
- }
-
- public String getNodeId() {
- return nodeId;
- }
- }
-
- public static class NodeHeartbeatFunction extends Function {
- private static final long serialVersionUID = 1L;
-
- private final String nodeId;
- private final HeartbeatData hbData;
-
- public NodeHeartbeatFunction(String nodeId, HeartbeatData hbData) {
- this.nodeId = nodeId;
- this.hbData = hbData;
- }
-
- @Override
- public FunctionId getFunctionId() {
- return FunctionId.NODE_HEARTBEAT;
- }
-
- public String getNodeId() {
- return nodeId;
- }
-
- public HeartbeatData getHeartbeatData() {
- return hbData;
- }
- }
-
- public static class ReportProfileFunction extends Function {
- private static final long serialVersionUID = 1L;
-
- private final String nodeId;
- private final List<JobProfile> profiles;
-
- public ReportProfileFunction(String nodeId, List<JobProfile> profiles) {
- this.nodeId = nodeId;
- this.profiles = profiles;
- }
-
- @Override
- public FunctionId getFunctionId() {
- return FunctionId.REPORT_PROFILE;
- }
-
- public String getNodeId() {
- return nodeId;
- }
-
- public List<JobProfile> getProfiles() {
- return profiles;
- }
- }
-
- public static class RegisterPartitionProviderFunction extends Function {
- private static final long serialVersionUID = 1L;
-
- private final PartitionDescriptor partitionDescriptor;
-
- public RegisterPartitionProviderFunction(PartitionDescriptor partitionDescriptor) {
- this.partitionDescriptor = partitionDescriptor;
- }
-
- @Override
- public FunctionId getFunctionId() {
- return FunctionId.REGISTER_PARTITION_PROVIDER;
- }
-
- public PartitionDescriptor getPartitionDescriptor() {
- return partitionDescriptor;
- }
- }
-
- public static class RegisterPartitionRequestFunction extends Function {
- private static final long serialVersionUID = 1L;
-
- private final PartitionRequest partitionRequest;
-
- public RegisterPartitionRequestFunction(PartitionRequest partitionRequest) {
- this.partitionRequest = partitionRequest;
- }
-
- @Override
- public FunctionId getFunctionId() {
- return FunctionId.REGISTER_PARTITION_REQUEST;
- }
-
- public PartitionRequest getPartitionRequest() {
- return partitionRequest;
- }
- }
-}
\ No newline at end of file
diff --git a/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/ipc/ClusterControllerRemoteProxy.java b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/ipc/ClusterControllerRemoteProxy.java
index 0aeab72..a0dabdd 100644
--- a/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/ipc/ClusterControllerRemoteProxy.java
+++ b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/ipc/ClusterControllerRemoteProxy.java
@@ -18,8 +18,8 @@
import edu.uci.ics.hyracks.api.dataflow.TaskAttemptId;
import edu.uci.ics.hyracks.api.job.JobId;
+import edu.uci.ics.hyracks.control.common.application.ApplicationStatus;
import edu.uci.ics.hyracks.control.common.base.IClusterController;
-import edu.uci.ics.hyracks.control.common.controllers.NodeParameters;
import edu.uci.ics.hyracks.control.common.controllers.NodeRegistration;
import edu.uci.ics.hyracks.control.common.heartbeat.HeartbeatData;
import edu.uci.ics.hyracks.control.common.job.PartitionDescriptor;
@@ -27,7 +27,6 @@
import edu.uci.ics.hyracks.control.common.job.profiling.om.JobProfile;
import edu.uci.ics.hyracks.control.common.job.profiling.om.TaskProfile;
import edu.uci.ics.hyracks.ipc.api.IIPCHandle;
-import edu.uci.ics.hyracks.ipc.api.SyncRMI;
public class ClusterControllerRemoteProxy implements IClusterController {
private final IIPCHandle ipcHandle;
@@ -37,75 +36,72 @@
}
@Override
- public NodeParameters registerNode(NodeRegistration reg) throws Exception {
- SyncRMI sync = new SyncRMI();
- ClusterControllerFunctions.RegisterNodeFunction fn = new ClusterControllerFunctions.RegisterNodeFunction(reg);
- NodeParameters result = (NodeParameters) sync.call(ipcHandle, fn);
- return result;
+ public void registerNode(NodeRegistration reg) throws Exception {
+ CCNCFunctions.RegisterNodeFunction fn = new CCNCFunctions.RegisterNodeFunction(reg);
+ ipcHandle.send(-1, fn, null);
}
@Override
public void unregisterNode(String nodeId) throws Exception {
- SyncRMI sync = new SyncRMI();
- ClusterControllerFunctions.UnregisterNodeFunction fn = new ClusterControllerFunctions.UnregisterNodeFunction(
+ CCNCFunctions.UnregisterNodeFunction fn = new CCNCFunctions.UnregisterNodeFunction(
nodeId);
- sync.call(ipcHandle, fn);
+ ipcHandle.send(-1, fn, null);
}
@Override
public void notifyTaskComplete(JobId jobId, TaskAttemptId taskId, String nodeId, TaskProfile statistics)
throws Exception {
- SyncRMI sync = new SyncRMI();
- ClusterControllerFunctions.NotifyTaskCompleteFunction fn = new ClusterControllerFunctions.NotifyTaskCompleteFunction(
+ CCNCFunctions.NotifyTaskCompleteFunction fn = new CCNCFunctions.NotifyTaskCompleteFunction(
jobId, taskId, nodeId, statistics);
- sync.call(ipcHandle, fn);
+ ipcHandle.send(-1, fn, null);
}
@Override
public void notifyTaskFailure(JobId jobId, TaskAttemptId taskId, String nodeId, String details) throws Exception {
- SyncRMI sync = new SyncRMI();
- ClusterControllerFunctions.NotifyTaskFailureFunction fn = new ClusterControllerFunctions.NotifyTaskFailureFunction(
+ CCNCFunctions.NotifyTaskFailureFunction fn = new CCNCFunctions.NotifyTaskFailureFunction(
jobId, taskId, nodeId, details);
- sync.call(ipcHandle, fn);
+ ipcHandle.send(-1, fn, null);
}
@Override
public void notifyJobletCleanup(JobId jobId, String nodeId) throws Exception {
- SyncRMI sync = new SyncRMI();
- ClusterControllerFunctions.NotifyJobletCleanupFunction fn = new ClusterControllerFunctions.NotifyJobletCleanupFunction(
+ CCNCFunctions.NotifyJobletCleanupFunction fn = new CCNCFunctions.NotifyJobletCleanupFunction(
jobId, nodeId);
- sync.call(ipcHandle, fn);
+ ipcHandle.send(-1, fn, null);
}
@Override
public void nodeHeartbeat(String id, HeartbeatData hbData) throws Exception {
- SyncRMI sync = new SyncRMI();
- ClusterControllerFunctions.NodeHeartbeatFunction fn = new ClusterControllerFunctions.NodeHeartbeatFunction(id,
+ CCNCFunctions.NodeHeartbeatFunction fn = new CCNCFunctions.NodeHeartbeatFunction(id,
hbData);
- sync.call(ipcHandle, fn);
+ ipcHandle.send(-1, fn, null);
}
@Override
public void reportProfile(String id, List<JobProfile> profiles) throws Exception {
- SyncRMI sync = new SyncRMI();
- ClusterControllerFunctions.ReportProfileFunction fn = new ClusterControllerFunctions.ReportProfileFunction(id,
+ CCNCFunctions.ReportProfileFunction fn = new CCNCFunctions.ReportProfileFunction(id,
profiles);
- sync.call(ipcHandle, fn);
+ ipcHandle.send(-1, fn, null);
}
@Override
public void registerPartitionProvider(PartitionDescriptor partitionDescriptor) throws Exception {
- SyncRMI sync = new SyncRMI();
- ClusterControllerFunctions.RegisterPartitionProviderFunction fn = new ClusterControllerFunctions.RegisterPartitionProviderFunction(
+ CCNCFunctions.RegisterPartitionProviderFunction fn = new CCNCFunctions.RegisterPartitionProviderFunction(
partitionDescriptor);
- sync.call(ipcHandle, fn);
+ ipcHandle.send(-1, fn, null);
}
@Override
public void registerPartitionRequest(PartitionRequest partitionRequest) throws Exception {
- SyncRMI sync = new SyncRMI();
- ClusterControllerFunctions.RegisterPartitionRequestFunction fn = new ClusterControllerFunctions.RegisterPartitionRequestFunction(
+ CCNCFunctions.RegisterPartitionRequestFunction fn = new CCNCFunctions.RegisterPartitionRequestFunction(
partitionRequest);
- sync.call(ipcHandle, fn);
+ ipcHandle.send(-1, fn, null);
+ }
+
+ @Override
+ public void notifyApplicationStateChange(String nodeId, String appName, ApplicationStatus status) throws Exception {
+ CCNCFunctions.ApplicationStateChangeResponseFunction fn = new CCNCFunctions.ApplicationStateChangeResponseFunction(
+ nodeId, appName, status);
+ ipcHandle.send(-1, fn, null);
}
}
\ No newline at end of file
diff --git a/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/ipc/NodeControllerDelegateIPCI.java b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/ipc/NodeControllerDelegateIPCI.java
deleted file mode 100644
index f3e51ec..0000000
--- a/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/ipc/NodeControllerDelegateIPCI.java
+++ /dev/null
@@ -1,71 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed 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 from
- *
- * 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 edu.uci.ics.hyracks.control.common.ipc;
-
-import edu.uci.ics.hyracks.control.common.base.INodeController;
-import edu.uci.ics.hyracks.ipc.api.IIPCHandle;
-import edu.uci.ics.hyracks.ipc.api.IIPCI;
-
-public class NodeControllerDelegateIPCI implements IIPCI {
- private final INodeController nc;
-
- public NodeControllerDelegateIPCI(INodeController nc) {
- this.nc = nc;
- }
-
- @Override
- public Object call(IIPCHandle caller, Object req) throws Exception {
- NodeControllerFunctions.Function fn = (NodeControllerFunctions.Function) req;
- switch (fn.getFunctionId()) {
- case START_TASKS: {
- NodeControllerFunctions.StartTasksFunction stf = (NodeControllerFunctions.StartTasksFunction) fn;
- nc.startTasks(stf.getAppName(), stf.getJobId(), stf.getPlanBytes(), stf.getTaskDescriptors(),
- stf.getConnectorPolicies());
- return null;
- }
-
- case ABORT_TASKS: {
- NodeControllerFunctions.AbortTasksFunction atf = (NodeControllerFunctions.AbortTasksFunction) fn;
- nc.abortTasks(atf.getJobId(), atf.getTasks());
- return null;
- }
-
- case CLEANUP_JOBLET: {
- NodeControllerFunctions.CleanupJobletFunction cjf = (NodeControllerFunctions.CleanupJobletFunction) fn;
- nc.cleanUpJoblet(cjf.getJobId(), cjf.getStatus());
- return null;
- }
-
- case CREATE_APPLICATION: {
- NodeControllerFunctions.CreateApplicationFunction caf = (NodeControllerFunctions.CreateApplicationFunction) fn;
- nc.createApplication(caf.getAppName(), caf.isDeployHar(), caf.getSerializedDistributedState());
- return null;
- }
-
- case DESTROY_APPLICATION: {
- NodeControllerFunctions.DestroyApplicationFunction daf = (NodeControllerFunctions.DestroyApplicationFunction) fn;
- nc.destroyApplication(daf.getAppName());
- return null;
- }
-
- case REPORT_PARTITION_AVAILABILITY: {
- NodeControllerFunctions.ReportPartitionAvailabilityFunction rpaf = (NodeControllerFunctions.ReportPartitionAvailabilityFunction) fn;
- nc.reportPartitionAvailability(rpaf.getPartitionId(), rpaf.getNetworkAddress());
- return null;
- }
- }
- throw new IllegalArgumentException("Unknown function: " + fn.getFunctionId());
- }
-}
\ No newline at end of file
diff --git a/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/ipc/NodeControllerFunctions.java b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/ipc/NodeControllerFunctions.java
deleted file mode 100644
index 0d39c1b..0000000
--- a/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/ipc/NodeControllerFunctions.java
+++ /dev/null
@@ -1,215 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed 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 from
- *
- * 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 edu.uci.ics.hyracks.control.common.ipc;
-
-import java.io.Serializable;
-import java.util.List;
-import java.util.Map;
-
-import edu.uci.ics.hyracks.api.comm.NetworkAddress;
-import edu.uci.ics.hyracks.api.dataflow.ConnectorDescriptorId;
-import edu.uci.ics.hyracks.api.dataflow.TaskAttemptId;
-import edu.uci.ics.hyracks.api.dataflow.connectors.IConnectorPolicy;
-import edu.uci.ics.hyracks.api.job.JobId;
-import edu.uci.ics.hyracks.api.job.JobStatus;
-import edu.uci.ics.hyracks.api.partitions.PartitionId;
-import edu.uci.ics.hyracks.control.common.job.TaskAttemptDescriptor;
-
-public class NodeControllerFunctions {
- public enum FunctionId {
- START_TASKS,
- ABORT_TASKS,
- CLEANUP_JOBLET,
- CREATE_APPLICATION,
- DESTROY_APPLICATION,
- REPORT_PARTITION_AVAILABILITY
- }
-
- public static abstract class Function implements Serializable {
- private static final long serialVersionUID = 1L;
-
- public abstract FunctionId getFunctionId();
- }
-
- public static class StartTasksFunction extends Function {
- private static final long serialVersionUID = 1L;
-
- private final String appName;
- private final JobId jobId;
- private final byte[] planBytes;
- private final List<TaskAttemptDescriptor> taskDescriptors;
- private final Map<ConnectorDescriptorId, IConnectorPolicy> connectorPolicies;
-
- public StartTasksFunction(String appName, JobId jobId, byte[] planBytes,
- List<TaskAttemptDescriptor> taskDescriptors,
- Map<ConnectorDescriptorId, IConnectorPolicy> connectorPolicies) {
- this.appName = appName;
- this.jobId = jobId;
- this.planBytes = planBytes;
- this.taskDescriptors = taskDescriptors;
- this.connectorPolicies = connectorPolicies;
- }
-
- @Override
- public FunctionId getFunctionId() {
- return FunctionId.START_TASKS;
- }
-
- public String getAppName() {
- return appName;
- }
-
- public JobId getJobId() {
- return jobId;
- }
-
- public byte[] getPlanBytes() {
- return planBytes;
- }
-
- public List<TaskAttemptDescriptor> getTaskDescriptors() {
- return taskDescriptors;
- }
-
- public Map<ConnectorDescriptorId, IConnectorPolicy> getConnectorPolicies() {
- return connectorPolicies;
- }
- }
-
- public static class AbortTasksFunction extends Function {
- private static final long serialVersionUID = 1L;
-
- private final JobId jobId;
- private final List<TaskAttemptId> tasks;
-
- public AbortTasksFunction(JobId jobId, List<TaskAttemptId> tasks) {
- this.jobId = jobId;
- this.tasks = tasks;
- }
-
- @Override
- public FunctionId getFunctionId() {
- return FunctionId.ABORT_TASKS;
- }
-
- public JobId getJobId() {
- return jobId;
- }
-
- public List<TaskAttemptId> getTasks() {
- return tasks;
- }
- }
-
- public static class CleanupJobletFunction extends Function {
- private static final long serialVersionUID = 1L;
-
- private final JobId jobId;
- private final JobStatus status;
-
- public CleanupJobletFunction(JobId jobId, JobStatus status) {
- this.jobId = jobId;
- this.status = status;
- }
-
- @Override
- public FunctionId getFunctionId() {
- return FunctionId.CLEANUP_JOBLET;
- }
-
- public JobId getJobId() {
- return jobId;
- }
-
- public JobStatus getStatus() {
- return status;
- }
- }
-
- public static class CreateApplicationFunction extends Function {
- private static final long serialVersionUID = 1L;
-
- private final String appName;
- private final boolean deployHar;
- private final byte[] serializedDistributedState;
-
- public CreateApplicationFunction(String appName, boolean deployHar, byte[] serializedDistributedState) {
- this.appName = appName;
- this.deployHar = deployHar;
- this.serializedDistributedState = serializedDistributedState;
- }
-
- @Override
- public FunctionId getFunctionId() {
- return FunctionId.CREATE_APPLICATION;
- }
-
- public String getAppName() {
- return appName;
- }
-
- public boolean isDeployHar() {
- return deployHar;
- }
-
- public byte[] getSerializedDistributedState() {
- return serializedDistributedState;
- }
- }
-
- public static class DestroyApplicationFunction extends Function {
- private static final long serialVersionUID = 1L;
-
- private final String appName;
-
- public DestroyApplicationFunction(String appName) {
- this.appName = appName;
- }
-
- @Override
- public FunctionId getFunctionId() {
- return FunctionId.DESTROY_APPLICATION;
- }
-
- public String getAppName() {
- return appName;
- }
- }
-
- public static class ReportPartitionAvailabilityFunction extends Function {
- private static final long serialVersionUID = 1L;
-
- private final PartitionId pid;
- private final NetworkAddress networkAddress;
-
- public ReportPartitionAvailabilityFunction(PartitionId pid, NetworkAddress networkAddress) {
- this.pid = pid;
- this.networkAddress = networkAddress;
- }
-
- @Override
- public FunctionId getFunctionId() {
- return FunctionId.REPORT_PARTITION_AVAILABILITY;
- }
-
- public PartitionId getPartitionId() {
- return pid;
- }
-
- public NetworkAddress getNetworkAddress() {
- return networkAddress;
- }
- }
-}
\ No newline at end of file
diff --git a/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/ipc/NodeControllerRemoteProxy.java b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/ipc/NodeControllerRemoteProxy.java
index ccd9468..961a9e3 100644
--- a/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/ipc/NodeControllerRemoteProxy.java
+++ b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/ipc/NodeControllerRemoteProxy.java
@@ -27,7 +27,6 @@
import edu.uci.ics.hyracks.control.common.base.INodeController;
import edu.uci.ics.hyracks.control.common.job.TaskAttemptDescriptor;
import edu.uci.ics.hyracks.ipc.api.IIPCHandle;
-import edu.uci.ics.hyracks.ipc.api.SyncRMI;
public class NodeControllerRemoteProxy implements INodeController {
private final IIPCHandle ipcHandle;
@@ -39,48 +38,43 @@
@Override
public void startTasks(String appName, JobId jobId, byte[] planBytes, List<TaskAttemptDescriptor> taskDescriptors,
Map<ConnectorDescriptorId, IConnectorPolicy> connectorPolicies) throws Exception {
- SyncRMI sync = new SyncRMI();
- NodeControllerFunctions.StartTasksFunction stf = new NodeControllerFunctions.StartTasksFunction(appName, jobId,
+ CCNCFunctions.StartTasksFunction stf = new CCNCFunctions.StartTasksFunction(appName, jobId,
planBytes, taskDescriptors, connectorPolicies);
- sync.call(ipcHandle, stf);
+ ipcHandle.send(-1, stf, null);
}
@Override
public void abortTasks(JobId jobId, List<TaskAttemptId> tasks) throws Exception {
- SyncRMI sync = new SyncRMI();
- NodeControllerFunctions.AbortTasksFunction atf = new NodeControllerFunctions.AbortTasksFunction(jobId, tasks);
- sync.call(ipcHandle, atf);
+ CCNCFunctions.AbortTasksFunction atf = new CCNCFunctions.AbortTasksFunction(jobId, tasks);
+ ipcHandle.send(-1, atf, null);
}
@Override
public void cleanUpJoblet(JobId jobId, JobStatus status) throws Exception {
- SyncRMI sync = new SyncRMI();
- NodeControllerFunctions.CleanupJobletFunction cjf = new NodeControllerFunctions.CleanupJobletFunction(jobId,
+ CCNCFunctions.CleanupJobletFunction cjf = new CCNCFunctions.CleanupJobletFunction(jobId,
status);
- sync.call(ipcHandle, cjf);
+ ipcHandle.send(-1, cjf, null);
}
@Override
public void createApplication(String appName, boolean deployHar, byte[] serializedDistributedState)
throws Exception {
- SyncRMI sync = new SyncRMI();
- NodeControllerFunctions.CreateApplicationFunction caf = new NodeControllerFunctions.CreateApplicationFunction(
+ CCNCFunctions.CreateApplicationFunction caf = new CCNCFunctions.CreateApplicationFunction(
appName, deployHar, serializedDistributedState);
- sync.call(ipcHandle, caf);
+ ipcHandle.send(-1, caf, null);
}
@Override
public void destroyApplication(String appName) throws Exception {
- SyncRMI sync = new SyncRMI();
- NodeControllerFunctions.DestroyApplicationFunction daf = new NodeControllerFunctions.DestroyApplicationFunction(
+ CCNCFunctions.DestroyApplicationFunction daf = new CCNCFunctions.DestroyApplicationFunction(
appName);
- sync.call(ipcHandle, daf);
+ ipcHandle.send(-1, daf, null);
}
@Override
public void reportPartitionAvailability(PartitionId pid, NetworkAddress networkAddress) throws Exception {
- NodeControllerFunctions.ReportPartitionAvailabilityFunction rpaf = new NodeControllerFunctions.ReportPartitionAvailabilityFunction(
+ CCNCFunctions.ReportPartitionAvailabilityFunction rpaf = new CCNCFunctions.ReportPartitionAvailabilityFunction(
pid, networkAddress);
- ipcHandle.send(rpaf, null);
+ ipcHandle.send(-1, rpaf, null);
}
}
\ No newline at end of file
diff --git a/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/work/FutureValue.java b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/work/FutureValue.java
index 00565b6..7eb4ff6 100644
--- a/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/work/FutureValue.java
+++ b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/work/FutureValue.java
@@ -14,7 +14,7 @@
*/
package edu.uci.ics.hyracks.control.common.work;
-public class FutureValue<T> {
+public class FutureValue<T> implements IResultCallback<T> {
private boolean done;
private T value;
@@ -27,6 +27,7 @@
e = null;
}
+ @Override
public synchronized void setValue(T value) {
done = true;
this.value = value;
@@ -34,6 +35,7 @@
notifyAll();
}
+ @Override
public synchronized void setException(Exception e) {
done = true;
this.e = e;
diff --git a/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/work/IPCResponder.java b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/work/IPCResponder.java
new file mode 100644
index 0000000..dcea864
--- /dev/null
+++ b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/work/IPCResponder.java
@@ -0,0 +1,33 @@
+package edu.uci.ics.hyracks.control.common.work;
+
+import edu.uci.ics.hyracks.ipc.api.IIPCHandle;
+import edu.uci.ics.hyracks.ipc.exceptions.IPCException;
+
+public class IPCResponder<T> implements IResultCallback<T> {
+ private final IIPCHandle handle;
+
+ private final long rmid;
+
+ public IPCResponder(IIPCHandle handle, long rmid) {
+ this.handle = handle;
+ this.rmid = rmid;
+ }
+
+ @Override
+ public void setValue(T result) {
+ try {
+ handle.send(rmid, result, null);
+ } catch (IPCException e) {
+ e.printStackTrace();
+ }
+ }
+
+ @Override
+ public void setException(Exception e) {
+ try {
+ handle.send(rmid, null, e);
+ } catch (IPCException e1) {
+ e1.printStackTrace();
+ }
+ }
+}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/Accumulator.java b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/work/IResultCallback.java
similarity index 78%
copy from hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/Accumulator.java
copy to hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/work/IResultCallback.java
index c728b0b..80c3d76 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/Accumulator.java
+++ b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/work/IResultCallback.java
@@ -12,10 +12,10 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package edu.uci.ics.hyracks.control.cc.remote;
+package edu.uci.ics.hyracks.control.common.work;
-public interface Accumulator<T, R> {
- public void accumulate(T o);
+public interface IResultCallback<T> {
+ public void setValue(T result);
- public R getResult();
+ public void setException(Exception e);
}
\ No newline at end of file
diff --git a/hyracks-control-nc/pom.xml b/hyracks-control-nc/pom.xml
index 64a409c..8c56958 100644
--- a/hyracks-control-nc/pom.xml
+++ b/hyracks-control-nc/pom.xml
@@ -1,9 +1,6 @@
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
<modelVersion>4.0.0</modelVersion>
- <groupId>edu.uci.ics.hyracks</groupId>
<artifactId>hyracks-control-nc</artifactId>
- <version>0.2.0-SNAPSHOT</version>
-
<parent>
<groupId>edu.uci.ics.hyracks</groupId>
<artifactId>hyracks</artifactId>
@@ -37,6 +34,11 @@
<type>jar</type>
<scope>compile</scope>
</dependency>
+ <dependency>
+ <groupId>edu.uci.ics.hyracks</groupId>
+ <artifactId>hyracks-net</artifactId>
+ <version>0.2.0-SNAPSHOT</version>
+ </dependency>
</dependencies>
<reporting>
<plugins>
diff --git a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/NodeControllerService.java b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/NodeControllerService.java
index d869515..0f512a7 100644
--- a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/NodeControllerService.java
+++ b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/NodeControllerService.java
@@ -39,33 +39,25 @@
import java.util.regex.Matcher;
import java.util.regex.Pattern;
-import edu.uci.ics.hyracks.api.comm.NetworkAddress;
import edu.uci.ics.hyracks.api.context.IHyracksRootContext;
-import edu.uci.ics.hyracks.api.dataflow.ConnectorDescriptorId;
-import edu.uci.ics.hyracks.api.dataflow.TaskAttemptId;
-import edu.uci.ics.hyracks.api.dataflow.connectors.IConnectorPolicy;
import edu.uci.ics.hyracks.api.io.IODeviceHandle;
import edu.uci.ics.hyracks.api.job.JobId;
-import edu.uci.ics.hyracks.api.job.JobStatus;
-import edu.uci.ics.hyracks.api.partitions.PartitionId;
import edu.uci.ics.hyracks.control.common.AbstractRemoteService;
import edu.uci.ics.hyracks.control.common.base.IClusterController;
-import edu.uci.ics.hyracks.control.common.base.INodeController;
import edu.uci.ics.hyracks.control.common.context.ServerContext;
import edu.uci.ics.hyracks.control.common.controllers.NCConfig;
import edu.uci.ics.hyracks.control.common.controllers.NodeParameters;
import edu.uci.ics.hyracks.control.common.controllers.NodeRegistration;
import edu.uci.ics.hyracks.control.common.heartbeat.HeartbeatData;
import edu.uci.ics.hyracks.control.common.heartbeat.HeartbeatSchema;
+import edu.uci.ics.hyracks.control.common.ipc.CCNCFunctions;
import edu.uci.ics.hyracks.control.common.ipc.ClusterControllerRemoteProxy;
-import edu.uci.ics.hyracks.control.common.ipc.NodeControllerDelegateIPCI;
-import edu.uci.ics.hyracks.control.common.job.TaskAttemptDescriptor;
import edu.uci.ics.hyracks.control.common.job.profiling.om.JobProfile;
import edu.uci.ics.hyracks.control.common.work.FutureValue;
import edu.uci.ics.hyracks.control.common.work.WorkQueue;
import edu.uci.ics.hyracks.control.nc.application.NCApplicationContext;
import edu.uci.ics.hyracks.control.nc.io.IOManager;
-import edu.uci.ics.hyracks.control.nc.net.ConnectionManager;
+import edu.uci.ics.hyracks.control.nc.net.NetworkManager;
import edu.uci.ics.hyracks.control.nc.partitions.PartitionManager;
import edu.uci.ics.hyracks.control.nc.runtime.RootHyracksContext;
import edu.uci.ics.hyracks.control.nc.work.AbortTasksWork;
@@ -76,9 +68,11 @@
import edu.uci.ics.hyracks.control.nc.work.ReportPartitionAvailabilityWork;
import edu.uci.ics.hyracks.control.nc.work.StartTasksWork;
import edu.uci.ics.hyracks.ipc.api.IIPCHandle;
+import edu.uci.ics.hyracks.ipc.api.IIPCI;
import edu.uci.ics.hyracks.ipc.impl.IPCSystem;
+import edu.uci.ics.hyracks.net.protocols.muxdemux.PerformanceCounters;
-public class NodeControllerService extends AbstractRemoteService implements INodeController {
+public class NodeControllerService extends AbstractRemoteService {
private static Logger LOGGER = Logger.getLogger(NodeControllerService.class.getName());
private NCConfig ncConfig;
@@ -91,12 +85,16 @@
private final PartitionManager partitionManager;
- private final ConnectionManager connectionManager;
+ private final NetworkManager netManager;
private final WorkQueue queue;
private final Timer timer;
+ private boolean registrationPending;
+
+ private Exception registrationException;
+
private IClusterController ccs;
private final Map<JobId, Joblet> jobletMap;
@@ -125,15 +123,15 @@
this.ncConfig = ncConfig;
id = ncConfig.nodeId;
executor = Executors.newCachedThreadPool();
- NodeControllerDelegateIPCI ipci = new NodeControllerDelegateIPCI(this);
- ipc = new IPCSystem(new InetSocketAddress(ncConfig.clusterNetIPAddress, 0), ipci, executor);
+ NodeControllerIPCI ipci = new NodeControllerIPCI();
+ ipc = new IPCSystem(new InetSocketAddress(ncConfig.clusterNetIPAddress, 0), ipci,
+ new CCNCFunctions.SerializerDeserializer());
this.ctx = new RootHyracksContext(ncConfig.frameSize, new IOManager(getDevices(ncConfig.ioDevices), executor));
if (id == null) {
throw new Exception("id not set");
}
- connectionManager = new ConnectionManager(ctx, getIpAddress(ncConfig));
partitionManager = new PartitionManager(this);
- connectionManager.setPartitionRequestListener(partitionManager);
+ netManager = new NetworkManager(ctx, getIpAddress(ncConfig), partitionManager, ncConfig.nNetThreads);
queue = new WorkQueue();
jobletMap = new Hashtable<JobId, Joblet>();
@@ -146,6 +144,7 @@
threadMXBean = ManagementFactory.getThreadMXBean();
runtimeMXBean = ManagementFactory.getRuntimeMXBean();
osMXBean = ManagementFactory.getOperatingSystemMXBean();
+ registrationPending = true;
}
public IHyracksRootContext getRootContext() {
@@ -162,11 +161,18 @@
return devices;
}
+ private synchronized void setNodeRegistrationResult(NodeParameters parameters, Exception exception) {
+ this.nodeParameters = parameters;
+ this.registrationException = exception;
+ this.registrationPending = false;
+ notifyAll();
+ }
+
@Override
public void start() throws Exception {
LOGGER.log(Level.INFO, "Starting NodeControllerService");
ipc.start();
- connectionManager.start();
+ netManager.start();
IIPCHandle ccIPCHandle = ipc.getHandle(new InetSocketAddress(ncConfig.ccHost, ncConfig.ccPort));
this.ccs = new ClusterControllerRemoteProxy(ccIPCHandle);
HeartbeatSchema.GarbageCollectorInfo[] gcInfos = new HeartbeatSchema.GarbageCollectorInfo[gcMXBeans.size()];
@@ -174,9 +180,19 @@
gcInfos[i] = new HeartbeatSchema.GarbageCollectorInfo(gcMXBeans.get(i).getName());
}
HeartbeatSchema hbSchema = new HeartbeatSchema(gcInfos);
- this.nodeParameters = ccs.registerNode(new NodeRegistration(ipc.getSocketAddress(), id, ncConfig,
- connectionManager.getNetworkAddress(), osMXBean.getName(), osMXBean.getArch(), osMXBean.getVersion(),
- osMXBean.getAvailableProcessors(), hbSchema));
+ ccs.registerNode(new NodeRegistration(ipc.getSocketAddress(), id, ncConfig, netManager.getNetworkAddress(),
+ osMXBean.getName(), osMXBean.getArch(), osMXBean.getVersion(), osMXBean.getAvailableProcessors(),
+ hbSchema));
+
+ synchronized (this) {
+ while (registrationPending) {
+ wait();
+ }
+ }
+ if (registrationException != null) {
+ throw registrationException;
+ }
+
queue.start();
heartbeatTask = new HeartbeatTask(ccs);
@@ -197,7 +213,7 @@
LOGGER.log(Level.INFO, "Stopping NodeControllerService");
partitionManager.close();
heartbeatTask.cancel();
- connectionManager.stop();
+ netManager.stop();
queue.stop();
LOGGER.log(Level.INFO, "Stopped NodeControllerService");
}
@@ -218,8 +234,8 @@
return jobletMap;
}
- public ConnectionManager getConnectionManager() {
- return connectionManager;
+ public NetworkManager getNetworkManager() {
+ return netManager;
}
public PartitionManager getPartitionManager() {
@@ -238,53 +254,10 @@
return executor;
}
- @Override
- public void startTasks(String appName, final JobId jobId, byte[] jagBytes,
- List<TaskAttemptDescriptor> taskDescriptors,
- Map<ConnectorDescriptorId, IConnectorPolicy> connectorPoliciesMap) throws Exception {
- StartTasksWork stw = new StartTasksWork(this, appName, jobId, jagBytes, taskDescriptors, connectorPoliciesMap);
- queue.schedule(stw);
- }
-
- @Override
- public void cleanUpJoblet(JobId jobId, JobStatus status) throws Exception {
- CleanupJobletWork cjw = new CleanupJobletWork(this, jobId, status);
- queue.schedule(cjw);
- }
-
public NCConfig getConfiguration() throws Exception {
return ncConfig;
}
- @Override
- public synchronized void abortTasks(JobId jobId, List<TaskAttemptId> tasks) throws Exception {
- AbortTasksWork atw = new AbortTasksWork(this, jobId, tasks);
- queue.schedule(atw);
- }
-
- @Override
- public void createApplication(String appName, boolean deployHar, byte[] serializedDistributedState)
- throws Exception {
- FutureValue<Object> fv = new FutureValue<Object>();
- CreateApplicationWork caw = new CreateApplicationWork(this, appName, deployHar, serializedDistributedState, fv);
- queue.schedule(caw);
- fv.get();
- }
-
- @Override
- public void destroyApplication(String appName) throws Exception {
- FutureValue<Object> fv = new FutureValue<Object>();
- DestroyApplicationWork daw = new DestroyApplicationWork(this, appName, fv);
- queue.schedule(daw);
- fv.get();
- }
-
- @Override
- public void reportPartitionAvailability(PartitionId pid, NetworkAddress networkAddress) throws Exception {
- ReportPartitionAvailabilityWork rpaw = new ReportPartitionAvailabilityWork(this, pid, networkAddress);
- queue.scheduleAndSync(rpaw);
- }
-
private static InetAddress getIpAddress(NCConfig ncConfig) throws Exception {
String ipaddrStr = ncConfig.dataIPAddress;
ipaddrStr = ipaddrStr.trim();
@@ -336,6 +309,11 @@
hbData.gcCollectionCounts[i] = gcMXBean.getCollectionCount();
hbData.gcCollectionTimes[i] = gcMXBean.getCollectionTime();
}
+ PerformanceCounters netPC = netManager.getPerformanceCounters();
+ hbData.netPayloadBytesRead = netPC.getPayloadBytesRead();
+ hbData.netPayloadBytesWritten = netPC.getPayloadBytesWritten();
+ hbData.netSignalingBytesRead = netPC.getSignalingBytesRead();
+ hbData.netSignalingBytesWritten = netPC.getSignalingBytesWritten();
try {
cc.nodeHeartbeat(id, hbData);
} catch (Exception e) {
@@ -366,4 +344,59 @@
}
}
}
+
+ private final class NodeControllerIPCI implements IIPCI {
+ @Override
+ public void deliverIncomingMessage(IIPCHandle handle, long mid, long rmid, Object payload, Exception exception) {
+ CCNCFunctions.Function fn = (CCNCFunctions.Function) payload;
+ switch (fn.getFunctionId()) {
+ case START_TASKS: {
+ CCNCFunctions.StartTasksFunction stf = (CCNCFunctions.StartTasksFunction) fn;
+ queue.schedule(new StartTasksWork(NodeControllerService.this, stf.getAppName(), stf.getJobId(), stf
+ .getPlanBytes(), stf.getTaskDescriptors(), stf.getConnectorPolicies()));
+ return;
+ }
+
+ case ABORT_TASKS: {
+ CCNCFunctions.AbortTasksFunction atf = (CCNCFunctions.AbortTasksFunction) fn;
+ queue.schedule(new AbortTasksWork(NodeControllerService.this, atf.getJobId(), atf.getTasks()));
+ return;
+ }
+
+ case CLEANUP_JOBLET: {
+ CCNCFunctions.CleanupJobletFunction cjf = (CCNCFunctions.CleanupJobletFunction) fn;
+ queue.schedule(new CleanupJobletWork(NodeControllerService.this, cjf.getJobId(), cjf.getStatus()));
+ return;
+ }
+
+ case CREATE_APPLICATION: {
+ CCNCFunctions.CreateApplicationFunction caf = (CCNCFunctions.CreateApplicationFunction) fn;
+ queue.schedule(new CreateApplicationWork(NodeControllerService.this, caf.getAppName(), caf
+ .isDeployHar(), caf.getSerializedDistributedState()));
+ return;
+ }
+
+ case DESTROY_APPLICATION: {
+ CCNCFunctions.DestroyApplicationFunction daf = (CCNCFunctions.DestroyApplicationFunction) fn;
+ queue.schedule(new DestroyApplicationWork(NodeControllerService.this, daf.getAppName()));
+ return;
+ }
+
+ case REPORT_PARTITION_AVAILABILITY: {
+ CCNCFunctions.ReportPartitionAvailabilityFunction rpaf = (CCNCFunctions.ReportPartitionAvailabilityFunction) fn;
+ queue.schedule(new ReportPartitionAvailabilityWork(NodeControllerService.this, rpaf
+ .getPartitionId(), rpaf.getNetworkAddress()));
+ return;
+ }
+
+ case NODE_REGISTRATION_RESULT: {
+ CCNCFunctions.NodeRegistrationResult nrrf = (CCNCFunctions.NodeRegistrationResult) fn;
+ setNodeRegistrationResult(nrrf.getNodeParameters(), nrrf.getException());
+ return;
+ }
+ }
+ throw new IllegalArgumentException("Unknown function: " + fn.getFunctionId());
+
+ }
+ }
}
\ No newline at end of file
diff --git a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/ConnectionManager.java b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/ConnectionManager.java
deleted file mode 100644
index 6e38ef7..0000000
--- a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/ConnectionManager.java
+++ /dev/null
@@ -1,254 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed 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 from
- *
- * 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 edu.uci.ics.hyracks.control.nc.net;
-
-import java.io.IOException;
-import java.net.InetAddress;
-import java.net.InetSocketAddress;
-import java.net.ServerSocket;
-import java.nio.ByteBuffer;
-import java.nio.channels.AsynchronousCloseException;
-import java.nio.channels.SelectionKey;
-import java.nio.channels.Selector;
-import java.nio.channels.ServerSocketChannel;
-import java.nio.channels.SocketChannel;
-import java.util.ArrayList;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Set;
-import java.util.logging.Level;
-import java.util.logging.Logger;
-
-import edu.uci.ics.hyracks.api.comm.NetworkAddress;
-import edu.uci.ics.hyracks.api.context.IHyracksRootContext;
-import edu.uci.ics.hyracks.api.dataflow.ConnectorDescriptorId;
-import edu.uci.ics.hyracks.api.exceptions.HyracksException;
-import edu.uci.ics.hyracks.api.job.JobId;
-import edu.uci.ics.hyracks.api.partitions.PartitionId;
-import edu.uci.ics.hyracks.control.nc.partitions.IPartitionRequestListener;
-
-public class ConnectionManager {
- private static final Logger LOGGER = Logger.getLogger(ConnectionManager.class.getName());
-
- static final int INITIAL_MESSAGE_SIZE = 20;
-
- private final IHyracksRootContext ctx;
-
- private IPartitionRequestListener partitionRequestListener;
-
- private final ServerSocketChannel serverChannel;
-
- private volatile boolean stopped;
-
- private final ConnectionListenerThread connectionListener;
-
- private final DataListenerThread dataListener;
-
- private final NetworkAddress networkAddress;
-
- public ConnectionManager(IHyracksRootContext ctx, InetAddress inetAddress) throws IOException {
- this.ctx = ctx;
- serverChannel = ServerSocketChannel.open();
- ServerSocket serverSocket = serverChannel.socket();
- serverSocket.bind(new InetSocketAddress(inetAddress, 0), 0);
- serverSocket.setReuseAddress(true);
- stopped = false;
- connectionListener = new ConnectionListenerThread();
- dataListener = new DataListenerThread();
- networkAddress = new NetworkAddress(serverSocket.getInetAddress(), serverSocket.getLocalPort());
-
- }
-
- public void setPartitionRequestListener(IPartitionRequestListener partitionRequestListener) {
- this.partitionRequestListener = partitionRequestListener;
- }
-
- public void start() {
- connectionListener.start();
- dataListener.start();
- }
-
- public void stop() {
- try {
- stopped = true;
- serverChannel.close();
- } catch (IOException e) {
- e.printStackTrace();
- }
- }
-
- public void connect(INetworkChannel channel) throws IOException {
- dataListener.addOutgoingConnection(channel);
- }
-
- private final class ConnectionListenerThread extends Thread {
- public ConnectionListenerThread() {
- super("Hyracks NC Connection Listener");
- setDaemon(true);
- setPriority(MAX_PRIORITY);
- }
-
- @Override
- public void run() {
- while (!stopped) {
- try {
- SocketChannel sc = serverChannel.accept();
- dataListener.addIncomingConnection(sc);
- } catch (AsynchronousCloseException e) {
- // do nothing
- if (!stopped) {
- e.printStackTrace();
- }
- } catch (IOException e) {
- e.printStackTrace();
- }
- }
- }
- }
-
- private final class DataListenerThread extends Thread {
- private Selector selector;
-
- private final List<SocketChannel> pendingIncomingConnections;
- private final Set<SocketChannel> pendingNegotiations;
- private final List<INetworkChannel> pendingOutgoingConnections;
-
- public DataListenerThread() {
- super("Hyracks Data Listener Thread");
- setDaemon(true);
- try {
- selector = Selector.open();
- } catch (IOException e) {
- throw new RuntimeException(e);
- }
- pendingIncomingConnections = new ArrayList<SocketChannel>();
- pendingNegotiations = new HashSet<SocketChannel>();
- pendingOutgoingConnections = new ArrayList<INetworkChannel>();
- }
-
- synchronized void addIncomingConnection(SocketChannel sc) throws IOException {
- pendingIncomingConnections.add(sc);
- selector.wakeup();
- }
-
- synchronized void addOutgoingConnection(INetworkChannel channel) throws IOException {
- pendingOutgoingConnections.add(channel);
- selector.wakeup();
- }
-
- @Override
- public void run() {
- while (!stopped) {
- try {
- if (LOGGER.isLoggable(Level.FINE)) {
- LOGGER.fine("Starting Select");
- }
- int n = selector.select();
- synchronized (this) {
- if (!pendingIncomingConnections.isEmpty()) {
- for (SocketChannel sc : pendingIncomingConnections) {
- sc.configureBlocking(false);
- sc.socket().setReuseAddress(true);
- SelectionKey scKey = sc.register(selector, SelectionKey.OP_READ);
- ByteBuffer buffer = ByteBuffer.allocate(INITIAL_MESSAGE_SIZE);
- scKey.attach(buffer);
- pendingNegotiations.add(sc);
- }
- pendingIncomingConnections.clear();
- }
- if (!pendingOutgoingConnections.isEmpty()) {
- for (INetworkChannel nc : pendingOutgoingConnections) {
- SocketChannel sc = SocketChannel.open();
- sc.configureBlocking(false);
- sc.socket().setReuseAddress(true);
- SelectionKey scKey = sc.register(selector, 0);
- scKey.attach(nc);
- nc.setSelectionKey(scKey);
- nc.notifyConnectionManagerRegistration();
- }
- pendingOutgoingConnections.clear();
- }
- if (LOGGER.isLoggable(Level.FINE)) {
- LOGGER.fine("Selector: " + n);
- }
- if (n > 0) {
- for (Iterator<SelectionKey> i = selector.selectedKeys().iterator(); i.hasNext();) {
- SelectionKey key = i.next();
- i.remove();
- SocketChannel sc = (SocketChannel) key.channel();
- if (pendingNegotiations.contains(sc)) {
- if (key.isReadable()) {
- ByteBuffer buffer = (ByteBuffer) key.attachment();
- sc.read(buffer);
- buffer.flip();
- if (buffer.remaining() >= INITIAL_MESSAGE_SIZE) {
- PartitionId pid = readInitialMessage(buffer);
- pendingNegotiations.remove(sc);
- key.interestOps(0);
- NetworkOutputChannel channel = new NetworkOutputChannel(ctx, 5);
- channel.setSelectionKey(key);
- key.attach(channel);
- try {
- partitionRequestListener.registerPartitionRequest(pid, channel);
- } catch (HyracksException e) {
- key.cancel();
- sc.close();
- channel.abort();
- }
- } else {
- buffer.compact();
- }
- }
- } else {
- INetworkChannel channel = (INetworkChannel) key.attachment();
- boolean close = false;
- boolean error = false;
- try {
- close = channel.dispatchNetworkEvent();
- } catch (IOException e) {
- e.printStackTrace();
- error = true;
- }
- if (close || error) {
- key.cancel();
- sc.close();
- if (error) {
- channel.abort();
- }
- }
- }
- }
- }
- }
- } catch (Exception e) {
- e.printStackTrace();
- }
- }
- }
-
- private PartitionId readInitialMessage(ByteBuffer buffer) {
- JobId jobId = new JobId(buffer.getLong());
- ConnectorDescriptorId cdid = new ConnectorDescriptorId(buffer.getInt());
- int senderIndex = buffer.getInt();
- int receiverIndex = buffer.getInt();
- return new PartitionId(jobId, cdid, senderIndex, receiverIndex);
- }
- }
-
- public NetworkAddress getNetworkAddress() {
- return networkAddress;
- }
-}
\ No newline at end of file
diff --git a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/INetworkChannel.java b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/INetworkChannel.java
deleted file mode 100644
index 61cd91f..0000000
--- a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/INetworkChannel.java
+++ /dev/null
@@ -1,19 +0,0 @@
-package edu.uci.ics.hyracks.control.nc.net;
-
-import java.io.IOException;
-import java.net.SocketAddress;
-import java.nio.channels.SelectionKey;
-
-public interface INetworkChannel {
- public boolean dispatchNetworkEvent() throws IOException;
-
- public void setSelectionKey(SelectionKey key);
-
- public SelectionKey getSelectionKey();
-
- public SocketAddress getRemoteAddress();
-
- public void abort();
-
- public void notifyConnectionManagerRegistration() throws IOException;
-}
\ No newline at end of file
diff --git a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/NetworkInputChannel.java b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/NetworkInputChannel.java
index 23cf514..ae2cd37 100644
--- a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/NetworkInputChannel.java
+++ b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/NetworkInputChannel.java
@@ -14,11 +14,8 @@
*/
package edu.uci.ics.hyracks.control.nc.net;
-import java.io.IOException;
import java.net.SocketAddress;
import java.nio.ByteBuffer;
-import java.nio.channels.SelectionKey;
-import java.nio.channels.SocketChannel;
import java.util.ArrayDeque;
import java.util.Queue;
import java.util.logging.Level;
@@ -26,53 +23,43 @@
import edu.uci.ics.hyracks.api.channels.IInputChannel;
import edu.uci.ics.hyracks.api.channels.IInputChannelMonitor;
-import edu.uci.ics.hyracks.api.comm.FrameHelper;
import edu.uci.ics.hyracks.api.context.IHyracksRootContext;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.api.exceptions.HyracksException;
import edu.uci.ics.hyracks.api.partitions.PartitionId;
+import edu.uci.ics.hyracks.net.buffers.IBufferAcceptor;
+import edu.uci.ics.hyracks.net.buffers.ICloseableBufferAcceptor;
+import edu.uci.ics.hyracks.net.protocols.muxdemux.ChannelControlBlock;
-public class NetworkInputChannel implements IInputChannel, INetworkChannel {
+public class NetworkInputChannel implements IInputChannel {
private static final Logger LOGGER = Logger.getLogger(NetworkInputChannel.class.getName());
- private final ConnectionManager connectionManager;
+ private IHyracksRootContext ctx;
+
+ private final NetworkManager netManager;
private final SocketAddress remoteAddress;
private final PartitionId partitionId;
- private final Queue<ByteBuffer> emptyQueue;
-
private final Queue<ByteBuffer> fullQueue;
- private SocketChannel socketChannel;
+ private final int nBuffers;
- private SelectionKey key;
-
- private ByteBuffer currentBuffer;
-
- private boolean eos;
-
- private boolean aborted;
+ private ChannelControlBlock ccb;
private IInputChannelMonitor monitor;
private Object attachment;
- private ByteBuffer writeBuffer;
-
- public NetworkInputChannel(IHyracksRootContext ctx, ConnectionManager connectionManager,
- SocketAddress remoteAddress, PartitionId partitionId, int nBuffers) {
- this.connectionManager = connectionManager;
+ public NetworkInputChannel(IHyracksRootContext ctx, NetworkManager netManager, SocketAddress remoteAddress,
+ PartitionId partitionId, int nBuffers) {
+ this.ctx = ctx;
+ this.netManager = netManager;
this.remoteAddress = remoteAddress;
this.partitionId = partitionId;
- this.emptyQueue = new ArrayDeque<ByteBuffer>(nBuffers);
- for (int i = 0; i < nBuffers; ++i) {
- emptyQueue.add(ctx.allocateFrame());
- }
fullQueue = new ArrayDeque<ByteBuffer>(nBuffers);
- aborted = false;
- eos = false;
+ this.nBuffers = nBuffers;
}
@Override
@@ -96,29 +83,34 @@
}
@Override
- public synchronized void recycleBuffer(ByteBuffer buffer) {
+ public void recycleBuffer(ByteBuffer buffer) {
buffer.clear();
- emptyQueue.add(buffer);
- if (!eos && !aborted) {
- int ops = key.interestOps();
- if ((ops & SelectionKey.OP_READ) == 0) {
- key.interestOps(ops | SelectionKey.OP_READ);
- key.selector().wakeup();
- if (currentBuffer == null) {
- currentBuffer = emptyQueue.poll();
- }
- }
- }
+ ccb.getReadInterface().getEmptyBufferAcceptor().accept(buffer);
}
@Override
public void open() throws HyracksDataException {
- currentBuffer = emptyQueue.poll();
try {
- connectionManager.connect(this);
- } catch (IOException e) {
+ ccb = netManager.connect(remoteAddress);
+ } catch (Exception e) {
throw new HyracksDataException(e);
}
+ ccb.getReadInterface().setFullBufferAcceptor(new ReadFullBufferAcceptor());
+ ccb.getWriteInterface().setEmptyBufferAcceptor(new WriteEmptyBufferAcceptor());
+ for (int i = 0; i < nBuffers; ++i) {
+ ccb.getReadInterface().getEmptyBufferAcceptor().accept(ctx.allocateFrame());
+ }
+ ByteBuffer writeBuffer = ByteBuffer.allocate(NetworkManager.INITIAL_MESSAGE_SIZE);
+ writeBuffer.putLong(partitionId.getJobId().getId());
+ writeBuffer.putInt(partitionId.getConnectorDescriptorId().getId());
+ writeBuffer.putInt(partitionId.getSenderIndex());
+ writeBuffer.putInt(partitionId.getReceiverIndex());
+ writeBuffer.flip();
+ if (LOGGER.isLoggable(Level.FINE)) {
+ LOGGER.fine("Sending partition request: " + partitionId + " on channel: " + ccb);
+ }
+ ccb.getWriteInterface().getFullBufferAcceptor().accept(writeBuffer);
+ ccb.getWriteInterface().getFullBufferAcceptor().close();
}
@Override
@@ -126,110 +118,28 @@
}
- @Override
- public synchronized boolean dispatchNetworkEvent() throws IOException {
- if (aborted) {
- eos = true;
- monitor.notifyFailure(this);
- return true;
+ private class ReadFullBufferAcceptor implements ICloseableBufferAcceptor {
+ @Override
+ public void accept(ByteBuffer buffer) {
+ fullQueue.add(buffer);
+ monitor.notifyDataAvailability(NetworkInputChannel.this, 1);
}
- if (key.isConnectable()) {
- if (socketChannel.finishConnect()) {
- key.interestOps(key.interestOps() & ~SelectionKey.OP_CONNECT);
- prepareForWrite();
- }
- } else if (key.isWritable()) {
- socketChannel.write(writeBuffer);
- if (writeBuffer.remaining() == 0) {
- key.interestOps(SelectionKey.OP_READ);
- }
- } else if (key.isReadable()) {
- if (LOGGER.isLoggable(Level.FINER)) {
- LOGGER.finer("Before read: " + currentBuffer.position() + " " + currentBuffer.limit());
- }
- int bytesRead = socketChannel.read(currentBuffer);
- if (bytesRead < 0) {
- eos = true;
- monitor.notifyEndOfStream(this);
- return true;
- }
- if (LOGGER.isLoggable(Level.FINER)) {
- LOGGER.finer("After read: " + currentBuffer.position() + " " + currentBuffer.limit());
- }
- currentBuffer.flip();
- int dataLen = currentBuffer.remaining();
- if (dataLen >= currentBuffer.capacity() || aborted()) {
- if (LOGGER.isLoggable(Level.FINEST)) {
- LOGGER.finest("NetworkInputChannel: frame received: sender = " + partitionId.getSenderIndex());
- }
- if (currentBuffer.getInt(FrameHelper.getTupleCountOffset(currentBuffer.capacity())) == 0) {
- eos = true;
- monitor.notifyEndOfStream(this);
- return true;
- }
- fullQueue.add(currentBuffer);
- currentBuffer = emptyQueue.poll();
- if (currentBuffer == null && key.isValid()) {
- int ops = key.interestOps();
- key.interestOps(ops & ~SelectionKey.OP_READ);
- }
- monitor.notifyDataAvailability(this, 1);
- return false;
- }
- currentBuffer.compact();
+
+ @Override
+ public void close() {
+ monitor.notifyEndOfStream(NetworkInputChannel.this);
}
- return false;
+
+ @Override
+ public void error(int ecode) {
+ monitor.notifyFailure(NetworkInputChannel.this);
+ }
}
- private void prepareForConnect() {
- key.interestOps(SelectionKey.OP_CONNECT);
- }
-
- private void prepareForWrite() {
- writeBuffer = ByteBuffer.allocate(ConnectionManager.INITIAL_MESSAGE_SIZE);
- writeBuffer.putLong(partitionId.getJobId().getId());
- writeBuffer.putInt(partitionId.getConnectorDescriptorId().getId());
- writeBuffer.putInt(partitionId.getSenderIndex());
- writeBuffer.putInt(partitionId.getReceiverIndex());
- writeBuffer.flip();
-
- key.interestOps(SelectionKey.OP_WRITE);
- }
-
- @Override
- public void setSelectionKey(SelectionKey key) {
- this.key = key;
- socketChannel = (SocketChannel) key.channel();
- }
-
- @Override
- public SocketAddress getRemoteAddress() {
- return remoteAddress;
- }
-
- @Override
- public SelectionKey getSelectionKey() {
- return key;
- }
-
- public PartitionId getPartitionId() {
- return partitionId;
- }
-
- public void abort() {
- aborted = true;
- }
-
- public boolean aborted() {
- return aborted;
- }
-
- @Override
- public void notifyConnectionManagerRegistration() throws IOException {
- if (socketChannel.connect(remoteAddress)) {
- prepareForWrite();
- } else {
- prepareForConnect();
+ private class WriteEmptyBufferAcceptor implements IBufferAcceptor {
+ @Override
+ public void accept(ByteBuffer buffer) {
+ // do nothing
}
}
}
\ No newline at end of file
diff --git a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/NetworkManager.java b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/NetworkManager.java
new file mode 100644
index 0000000..1a0a820
--- /dev/null
+++ b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/NetworkManager.java
@@ -0,0 +1,134 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ *
+ * 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 edu.uci.ics.hyracks.control.nc.net;
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.net.SocketAddress;
+import java.nio.ByteBuffer;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import edu.uci.ics.hyracks.api.comm.NetworkAddress;
+import edu.uci.ics.hyracks.api.context.IHyracksRootContext;
+import edu.uci.ics.hyracks.api.dataflow.ConnectorDescriptorId;
+import edu.uci.ics.hyracks.api.exceptions.HyracksException;
+import edu.uci.ics.hyracks.api.job.JobId;
+import edu.uci.ics.hyracks.api.partitions.PartitionId;
+import edu.uci.ics.hyracks.control.nc.partitions.IPartitionRequestListener;
+import edu.uci.ics.hyracks.net.buffers.ICloseableBufferAcceptor;
+import edu.uci.ics.hyracks.net.exceptions.NetException;
+import edu.uci.ics.hyracks.net.protocols.muxdemux.ChannelControlBlock;
+import edu.uci.ics.hyracks.net.protocols.muxdemux.IChannelOpenListener;
+import edu.uci.ics.hyracks.net.protocols.muxdemux.MultiplexedConnection;
+import edu.uci.ics.hyracks.net.protocols.muxdemux.MuxDemux;
+import edu.uci.ics.hyracks.net.protocols.muxdemux.PerformanceCounters;
+
+public class NetworkManager {
+ private static final Logger LOGGER = Logger.getLogger(NetworkManager.class.getName());
+
+ static final int INITIAL_MESSAGE_SIZE = 20;
+
+ private final IHyracksRootContext ctx;
+
+ private final IPartitionRequestListener partitionRequestListener;
+
+ private final MuxDemux md;
+
+ private NetworkAddress networkAddress;
+
+ public NetworkManager(IHyracksRootContext ctx, InetAddress inetAddress,
+ IPartitionRequestListener partitionRequestListener, int nThreads) throws IOException {
+ this.ctx = ctx;
+ this.partitionRequestListener = partitionRequestListener;
+ md = new MuxDemux(new InetSocketAddress(inetAddress, 0), new ChannelOpenListener(), nThreads);
+ }
+
+ public void start() throws IOException {
+ md.start();
+ InetSocketAddress sockAddr = md.getLocalAddress();
+ networkAddress = new NetworkAddress(sockAddr.getAddress().getAddress(), sockAddr.getPort());
+ }
+
+ public NetworkAddress getNetworkAddress() {
+ return networkAddress;
+ }
+
+ public void stop() {
+
+ }
+
+ public ChannelControlBlock connect(SocketAddress remoteAddress) throws InterruptedException, NetException {
+ MultiplexedConnection mConn = md.connect((InetSocketAddress) remoteAddress);
+ return mConn.openChannel();
+ }
+
+ private class ChannelOpenListener implements IChannelOpenListener {
+ @Override
+ public void channelOpened(ChannelControlBlock channel) {
+ channel.getReadInterface().setFullBufferAcceptor(new InitialBufferAcceptor(channel));
+ channel.getReadInterface().getEmptyBufferAcceptor().accept(ByteBuffer.allocate(INITIAL_MESSAGE_SIZE));
+ }
+ }
+
+ private class InitialBufferAcceptor implements ICloseableBufferAcceptor {
+ private final ChannelControlBlock ccb;
+
+ private NetworkOutputChannel noc;
+
+ public InitialBufferAcceptor(ChannelControlBlock ccb) {
+ this.ccb = ccb;
+ }
+
+ @Override
+ public void accept(ByteBuffer buffer) {
+ PartitionId pid = readInitialMessage(buffer);
+ if (LOGGER.isLoggable(Level.FINE)) {
+ LOGGER.fine("Received initial partition request: " + pid + " on channel: " + ccb);
+ }
+ noc = new NetworkOutputChannel(ctx, ccb, 5);
+ try {
+ partitionRequestListener.registerPartitionRequest(pid, noc);
+ } catch (HyracksException e) {
+ noc.abort();
+ }
+ }
+
+ @Override
+ public void close() {
+
+ }
+
+ @Override
+ public void error(int ecode) {
+ if (noc != null) {
+ noc.abort();
+ }
+ }
+ }
+
+ private static PartitionId readInitialMessage(ByteBuffer buffer) {
+ JobId jobId = new JobId(buffer.getLong());
+ ConnectorDescriptorId cdid = new ConnectorDescriptorId(buffer.getInt());
+ int senderIndex = buffer.getInt();
+ int receiverIndex = buffer.getInt();
+ return new PartitionId(jobId, cdid, senderIndex, receiverIndex);
+ }
+
+ public PerformanceCounters getPerformanceCounters() {
+ return md.getPerformanceCounters();
+ }
+}
\ No newline at end of file
diff --git a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/NetworkOutputChannel.java b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/NetworkOutputChannel.java
index 31ce924..8d5f475 100644
--- a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/NetworkOutputChannel.java
+++ b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/NetworkOutputChannel.java
@@ -14,124 +14,44 @@
*/
package edu.uci.ics.hyracks.control.nc.net;
-import java.io.IOException;
-import java.net.SocketAddress;
import java.nio.ByteBuffer;
-import java.nio.channels.SelectionKey;
-import java.nio.channels.SocketChannel;
import java.util.ArrayDeque;
import java.util.Queue;
-import edu.uci.ics.hyracks.api.comm.FrameHelper;
import edu.uci.ics.hyracks.api.comm.IFrameWriter;
import edu.uci.ics.hyracks.api.context.IHyracksRootContext;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.net.buffers.IBufferAcceptor;
+import edu.uci.ics.hyracks.net.protocols.muxdemux.ChannelControlBlock;
-public class NetworkOutputChannel implements INetworkChannel, IFrameWriter {
- private final IHyracksRootContext ctx;
+public class NetworkOutputChannel implements IFrameWriter {
+ private final ChannelControlBlock ccb;
private final Queue<ByteBuffer> emptyQueue;
- private final Queue<ByteBuffer> fullQueue;
-
- private SelectionKey key;
-
private boolean aborted;
- private boolean eos;
-
- private boolean eosSent;
-
- private boolean failed;
-
- private ByteBuffer currentBuffer;
-
- public NetworkOutputChannel(IHyracksRootContext ctx, int nBuffers) {
- this.ctx = ctx;
+ public NetworkOutputChannel(IHyracksRootContext ctx, ChannelControlBlock ccb, int nBuffers) {
+ this.ccb = ccb;
emptyQueue = new ArrayDeque<ByteBuffer>(nBuffers);
for (int i = 0; i < nBuffers; ++i) {
- emptyQueue.add(ctx.allocateFrame());
+ emptyQueue.add(ByteBuffer.allocateDirect(ctx.getFrameSize()));
}
- fullQueue = new ArrayDeque<ByteBuffer>(nBuffers);
- }
-
- @Override
- public synchronized boolean dispatchNetworkEvent() throws IOException {
- if (failed || aborted) {
- eos = true;
- return true;
- } else if (key.isWritable()) {
- while (true) {
- if (currentBuffer == null) {
- if (eosSent) {
- return true;
- }
- currentBuffer = fullQueue.poll();
- if (currentBuffer == null) {
- if (eos) {
- currentBuffer = emptyQueue.poll();
- currentBuffer.clear();
- currentBuffer.putInt(FrameHelper.getTupleCountOffset(ctx.getFrameSize()), 0);
- eosSent = true;
- } else {
- key.interestOps(key.interestOps() & ~SelectionKey.OP_WRITE);
- return false;
- }
- }
- }
- int bytesWritten = ((SocketChannel) key.channel()).write(currentBuffer);
- if (bytesWritten < 0) {
- eos = true;
- return true;
- }
- if (currentBuffer.remaining() == 0) {
- emptyQueue.add(currentBuffer);
- notifyAll();
- currentBuffer = null;
- if (eosSent) {
- return true;
- }
- } else {
- return false;
- }
- }
- }
- return false;
- }
-
- @Override
- public void setSelectionKey(SelectionKey key) {
- this.key = key;
- }
-
- @Override
- public SelectionKey getSelectionKey() {
- return key;
- }
-
- @Override
- public SocketAddress getRemoteAddress() {
- return ((SocketChannel) key.channel()).socket().getRemoteSocketAddress();
- }
-
- @Override
- public synchronized void abort() {
- aborted = true;
+ ccb.getWriteInterface().setEmptyBufferAcceptor(new WriteEmptyBufferAcceptor());
}
@Override
public void open() throws HyracksDataException {
- currentBuffer = null;
}
@Override
public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
ByteBuffer destBuffer = null;
synchronized (this) {
- if (aborted) {
- throw new HyracksDataException("Connection has been aborted");
- }
while (true) {
+ if (aborted) {
+ throw new HyracksDataException("Connection has been aborted");
+ }
destBuffer = emptyQueue.poll();
if (destBuffer != null) {
break;
@@ -148,26 +68,34 @@
destBuffer.clear();
destBuffer.put(buffer);
destBuffer.flip();
- synchronized (this) {
- fullQueue.add(destBuffer);
- }
- key.interestOps(SelectionKey.OP_WRITE);
- key.selector().wakeup();
+ ccb.getWriteInterface().getFullBufferAcceptor().accept(destBuffer);
}
@Override
public void fail() throws HyracksDataException {
- failed = true;
+ ccb.getWriteInterface().getFullBufferAcceptor().error(1);
}
@Override
- public synchronized void close() throws HyracksDataException {
- eos = true;
- key.interestOps(SelectionKey.OP_WRITE);
- key.selector().wakeup();
+ public void close() throws HyracksDataException {
+ ccb.getWriteInterface().getFullBufferAcceptor().close();
}
- @Override
- public void notifyConnectionManagerRegistration() throws IOException {
+ void abort() {
+ ccb.getWriteInterface().getFullBufferAcceptor().error(1);
+ synchronized (NetworkOutputChannel.this) {
+ aborted = true;
+ NetworkOutputChannel.this.notifyAll();
+ }
+ }
+
+ private class WriteEmptyBufferAcceptor implements IBufferAcceptor {
+ @Override
+ public void accept(ByteBuffer buffer) {
+ synchronized (NetworkOutputChannel.this) {
+ emptyQueue.add(buffer);
+ NetworkOutputChannel.this.notifyAll();
+ }
+ }
}
}
\ No newline at end of file
diff --git a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/PipelinedPartition.java b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/PipelinedPartition.java
index 9403736..44e61f1 100644
--- a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/PipelinedPartition.java
+++ b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/PipelinedPartition.java
@@ -32,6 +32,8 @@
private IFrameWriter delegate;
+ private boolean pendingConnection;
+
private boolean failed;
public PipelinedPartition(PartitionManager manager, PartitionId pid, TaskAttemptId taId) {
@@ -57,32 +59,44 @@
}
@Override
- public synchronized void open() throws HyracksDataException {
+ public void open() throws HyracksDataException {
manager.registerPartition(pid, taId, this, PartitionState.STARTED);
failed = false;
- while (delegate == null) {
- try {
- wait();
- } catch (InterruptedException e) {
- throw new HyracksDataException(e);
- }
- }
- delegate.open();
+ pendingConnection = true;
}
@Override
public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+ ensureConnected();
delegate.nextFrame(buffer);
}
+ private void ensureConnected() throws HyracksDataException {
+ if (pendingConnection) {
+ synchronized (this) {
+ while (delegate == null) {
+ try {
+ wait();
+ } catch (InterruptedException e) {
+ throw new HyracksDataException(e);
+ }
+ }
+ }
+ delegate.open();
+ }
+ pendingConnection = false;
+ }
+
@Override
public void fail() throws HyracksDataException {
+ ensureConnected();
failed = true;
delegate.fail();
}
@Override
public void close() throws HyracksDataException {
+ ensureConnected();
if (!failed) {
manager.updatePartitionState(pid, taId, this, PartitionState.COMMITTED);
}
diff --git a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/CreateApplicationWork.java b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/CreateApplicationWork.java
index 5b60a08..eb982df 100644
--- a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/CreateApplicationWork.java
+++ b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/CreateApplicationWork.java
@@ -27,9 +27,9 @@
import org.apache.http.impl.client.DefaultHttpClient;
import edu.uci.ics.hyracks.api.exceptions.HyracksException;
+import edu.uci.ics.hyracks.control.common.application.ApplicationStatus;
import edu.uci.ics.hyracks.control.common.controllers.NCConfig;
import edu.uci.ics.hyracks.control.common.controllers.NodeParameters;
-import edu.uci.ics.hyracks.control.common.work.FutureValue;
import edu.uci.ics.hyracks.control.common.work.SynchronizableWork;
import edu.uci.ics.hyracks.control.nc.NodeControllerService;
import edu.uci.ics.hyracks.control.nc.application.NCApplicationContext;
@@ -45,15 +45,12 @@
private final byte[] serializedDistributedState;
- private final FutureValue<Object> fv;
-
public CreateApplicationWork(NodeControllerService ncs, String appName, boolean deployHar,
- byte[] serializedDistributedState, FutureValue<Object> fv) {
+ byte[] serializedDistributedState) {
this.ncs = ncs;
this.appName = appName;
this.deployHar = deployHar;
this.serializedDistributedState = serializedDistributedState;
- this.fv = fv;
}
@Override
@@ -85,9 +82,10 @@
appCtx.initializeClassPath();
appCtx.setDistributedState((Serializable) appCtx.deserialize(serializedDistributedState));
appCtx.initialize();
- fv.setValue(null);
+ ncs.getClusterController()
+ .notifyApplicationStateChange(ncs.getId(), appName, ApplicationStatus.INITIALIZED);
} catch (Exception e) {
- fv.setException(e);
+ LOGGER.warning("Error creating application: " + e.getMessage());
}
}
}
\ No newline at end of file
diff --git a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/DestroyApplicationWork.java b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/DestroyApplicationWork.java
index 8ac0b5c..cfe00f6 100644
--- a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/DestroyApplicationWork.java
+++ b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/DestroyApplicationWork.java
@@ -18,7 +18,7 @@
import java.util.logging.Logger;
import edu.uci.ics.hyracks.control.common.application.ApplicationContext;
-import edu.uci.ics.hyracks.control.common.work.FutureValue;
+import edu.uci.ics.hyracks.control.common.application.ApplicationStatus;
import edu.uci.ics.hyracks.control.common.work.SynchronizableWork;
import edu.uci.ics.hyracks.control.nc.NodeControllerService;
import edu.uci.ics.hyracks.control.nc.application.NCApplicationContext;
@@ -30,12 +30,9 @@
private final String appName;
- private FutureValue<Object> fv;
-
- public DestroyApplicationWork(NodeControllerService ncs, String appName, FutureValue<Object> fv) {
+ public DestroyApplicationWork(NodeControllerService ncs, String appName) {
this.ncs = ncs;
this.appName = appName;
- this.fv = fv;
}
@Override
@@ -46,9 +43,9 @@
if (appCtx != null) {
appCtx.deinitialize();
}
- fv.setValue(null);
} catch (Exception e) {
- fv.setException(e);
+ LOGGER.warning("Error destroying application: " + e.getMessage());
}
+ ncs.getClusterController().notifyApplicationStateChange(ncs.getId(), appName, ApplicationStatus.DEINITIALIZED);
}
}
\ No newline at end of file
diff --git a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/ReportPartitionAvailabilityWork.java b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/ReportPartitionAvailabilityWork.java
index bfa21c9..9734567 100644
--- a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/ReportPartitionAvailabilityWork.java
+++ b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/ReportPartitionAvailabilityWork.java
@@ -14,6 +14,7 @@
*/
package edu.uci.ics.hyracks.control.nc.work;
+import java.net.InetAddress;
import java.net.InetSocketAddress;
import java.util.Map;
import java.util.logging.Logger;
@@ -48,8 +49,8 @@
Joblet ji = jobletMap.get(pid.getJobId());
if (ji != null) {
PartitionChannel channel = new PartitionChannel(pid, new NetworkInputChannel(ncs.getRootContext(),
- ncs.getConnectionManager(), new InetSocketAddress(networkAddress.getIpAddress(),
- networkAddress.getPort()), pid, 1));
+ ncs.getNetworkManager(), new InetSocketAddress(InetAddress.getByAddress(networkAddress
+ .getIpAddress()), networkAddress.getPort()), pid, 5));
ji.reportPartitionAvailability(channel);
}
}
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/base/AbstractConnectorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/base/AbstractConnectorDescriptor.java
index 4ffc060..37154bc 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/base/AbstractConnectorDescriptor.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/base/AbstractConnectorDescriptor.java
@@ -28,21 +28,33 @@
private static final long serialVersionUID = 1L;
protected final ConnectorDescriptorId id;
+ protected String displayName;
+
public AbstractConnectorDescriptor(JobSpecification spec) {
this.id = spec.createConnectorDescriptor();
spec.getConnectorMap().put(id, this);
+ displayName = getClass().getName() + "[" + id + "]";
}
public ConnectorDescriptorId getConnectorId() {
return id;
}
+ public String getDisplayName() {
+ return displayName;
+ }
+
+ public void setDisplayName(String displayName) {
+ this.displayName = displayName;
+ }
+
@Override
public JSONObject toJSON() throws JSONException {
JSONObject jconn = new JSONObject();
jconn.put("id", getConnectorId().getId());
jconn.put("java-class", getClass().getName());
+ jconn.put("display-name", displayName);
return jconn;
}
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/base/AbstractOperatorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/base/AbstractOperatorDescriptor.java
index 1bb13ca..1a60290 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/base/AbstractOperatorDescriptor.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/base/AbstractOperatorDescriptor.java
@@ -38,12 +38,15 @@
protected final int outputArity;
+ protected String displayName;
+
public AbstractOperatorDescriptor(JobSpecification spec, int inputArity, int outputArity) {
odId = spec.createOperatorDescriptorId();
this.inputArity = inputArity;
this.outputArity = outputArity;
recordDescriptors = new RecordDescriptor[outputArity];
spec.getOperatorMap().put(getOperatorId(), this);
+ displayName = getClass().getName() + "[" + odId + "]";
}
@Override
@@ -66,6 +69,14 @@
return recordDescriptors;
}
+ public String getDisplayName() {
+ return displayName;
+ }
+
+ public void setDisplayName(String displayName) {
+ this.displayName = displayName;
+ }
+
@Override
public void contributeSchedulingConstraints(IConstraintAcceptor constraintAcceptor, JobActivityGraph plan,
ICCApplicationContext appCtx) {
@@ -79,6 +90,7 @@
jop.put("java-class", getClass().getName());
jop.put("in-arity", getInputArity());
jop.put("out-arity", getOutputArity());
+ jop.put("display-name", displayName);
return jop;
}
}
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/collectors/NonDeterministicPartitionCollector.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/collectors/NonDeterministicPartitionCollector.java
index 2b37c7b..d5f4fd8 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/collectors/NonDeterministicPartitionCollector.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/collectors/NonDeterministicPartitionCollector.java
@@ -18,6 +18,8 @@
import java.util.ArrayList;
import java.util.BitSet;
import java.util.Collection;
+import java.util.logging.Level;
+import java.util.logging.Logger;
import edu.uci.ics.hyracks.api.channels.IInputChannel;
import edu.uci.ics.hyracks.api.channels.IInputChannelMonitor;
@@ -31,6 +33,8 @@
import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
public class NonDeterministicPartitionCollector extends AbstractPartitionCollector {
+ private static final Logger LOGGER = Logger.getLogger(NonDeterministicPartitionCollector.class.getName());
+
private final FrameReader reader;
private final BitSet expectedPartitions;
@@ -173,6 +177,10 @@
int senderIndex = pid.getSenderIndex();
failSenders.set(senderIndex);
eosSenders.set(senderIndex);
+ if (LOGGER.isLoggable(Level.FINE)) {
+ LOGGER.fine("Failure: " + connectorId + " sender: " + senderIndex + " receiver: "
+ + receiverIndex);
+ }
NonDeterministicPartitionCollector.this.notifyAll();
}
}
@@ -184,6 +192,10 @@
int senderIndex = pid.getSenderIndex();
availableFrameCounts[senderIndex] += nFrames;
frameAvailability.set(senderIndex);
+ if (LOGGER.isLoggable(Level.FINE)) {
+ LOGGER.fine("Data available: " + connectorId + " sender: " + senderIndex + " receiver: "
+ + receiverIndex);
+ }
NonDeterministicPartitionCollector.this.notifyAll();
}
}
@@ -194,6 +206,10 @@
PartitionId pid = (PartitionId) channel.getAttachment();
int senderIndex = pid.getSenderIndex();
eosSenders.set(senderIndex);
+ if (LOGGER.isLoggable(Level.FINE)) {
+ LOGGER.fine("EOS: " + connectorId + " sender: " + senderIndex + " receiver: "
+ + receiverIndex);
+ }
NonDeterministicPartitionCollector.this.notifyAll();
}
}
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/ExternalGroupOperatorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/ExternalGroupOperatorDescriptor.java
index ca17779..41da49a 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/ExternalGroupOperatorDescriptor.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/ExternalGroupOperatorDescriptor.java
@@ -428,6 +428,10 @@
*/
tupleBuilder.reset();
+
+ for(int k = 0; k < storedKeys.length; k++){
+ tupleBuilder.addField(fta, tupleIndex, storedKeys[k]);
+ }
aggregator.init(tupleBuilder, fta, tupleIndex, aggregateState);
diff --git a/hyracks-examples/hadoop-compat-example/hadoopcompatapp/pom.xml b/hyracks-examples/hadoop-compat-example/hadoopcompatapp/pom.xml
index 15d31b6..fe71db4 100644
--- a/hyracks-examples/hadoop-compat-example/hadoopcompatapp/pom.xml
+++ b/hyracks-examples/hadoop-compat-example/hadoopcompatapp/pom.xml
@@ -77,7 +77,7 @@
<plugin>
<groupId>edu.uci.ics.hyracks</groupId>
<artifactId>hyracks-maven-plugin</artifactId>
- <version>0.0.1</version>
+ <version>0.0.2</version>
<configuration>
<hyracksServerHome>${basedir}/../../../hyracks-server/target/hyracks-server-${project.version}-binary-assembly</hyracksServerHome>
<hyracksCLIHome>${basedir}/../../../hyracks-cli/target/hyracks-cli-${project.version}-binary-assembly</hyracksCLIHome>
diff --git a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AbstractIntegrationTest.java b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AbstractIntegrationTest.java
index fac141c..4ed8361 100644
--- a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AbstractIntegrationTest.java
+++ b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AbstractIntegrationTest.java
@@ -116,7 +116,7 @@
if (LOGGER.isLoggable(Level.INFO)) {
LOGGER.info(jobId.toString());
}
- cc.waitForCompletion(jobId);
+ hcc.waitForCompletion(jobId);
dumpOutputFiles();
}
diff --git a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AggregationTests.java b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AggregationTest.java
similarity index 99%
rename from hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AggregationTests.java
rename to hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AggregationTest.java
index 1c5c014..2813a62 100644
--- a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AggregationTests.java
+++ b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AggregationTest.java
@@ -64,7 +64,7 @@
/**
*
*/
-public class AggregationTests extends AbstractIntegrationTest {
+public class AggregationTest extends AbstractIntegrationTest {
final IFileSplitProvider splitProvider = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC2_ID,
new FileReference(new File("data/tpch0.001/lineitem.tbl"))) });
diff --git a/hyracks-examples/text-example/textapp/pom.xml b/hyracks-examples/text-example/textapp/pom.xml
index 3b3b9fd..913a95a 100644
--- a/hyracks-examples/text-example/textapp/pom.xml
+++ b/hyracks-examples/text-example/textapp/pom.xml
@@ -77,7 +77,7 @@
<plugin>
<groupId>edu.uci.ics.hyracks</groupId>
<artifactId>hyracks-maven-plugin</artifactId>
- <version>0.0.1</version>
+ <version>0.0.2</version>
<configuration>
<hyracksServerHome>${basedir}/../../../hyracks-server/target/hyracks-server-${project.version}-binary-assembly</hyracksServerHome>
<hyracksCLIHome>${basedir}/../../../hyracks-cli/target/hyracks-cli-${project.version}-binary-assembly</hyracksCLIHome>
diff --git a/hyracks-hadoop-compat/src/main/java/edu/uci/ics/hyracks/hadoop/compat/client/HyracksClient.java b/hyracks-hadoop-compat/src/main/java/edu/uci/ics/hyracks/hadoop/compat/client/HyracksClient.java
index 81e9f84..eadf679 100644
--- a/hyracks-hadoop-compat/src/main/java/edu/uci/ics/hyracks/hadoop/compat/client/HyracksClient.java
+++ b/hyracks-hadoop-compat/src/main/java/edu/uci/ics/hyracks/hadoop/compat/client/HyracksClient.java
@@ -25,7 +25,7 @@
private void initialize(Properties properties) throws Exception {
String clusterController = (String) properties.get(ConfigurationConstants.clusterControllerHost);
- connection = new HyracksConnection(clusterController, 1099);
+ connection = new HyracksConnection(clusterController, 1098);
systemLibs = new HashSet<String>();
for (String systemLib : ConfigurationConstants.systemLibs) {
String systemLibPath = properties.getProperty(systemLib);
diff --git a/hyracks-ipc/src/main/java/edu/uci/ics/hyracks/ipc/api/IIPCHandle.java b/hyracks-ipc/src/main/java/edu/uci/ics/hyracks/ipc/api/IIPCHandle.java
index a4adf23..8a3630f 100644
--- a/hyracks-ipc/src/main/java/edu/uci/ics/hyracks/ipc/api/IIPCHandle.java
+++ b/hyracks-ipc/src/main/java/edu/uci/ics/hyracks/ipc/api/IIPCHandle.java
@@ -21,7 +21,7 @@
public interface IIPCHandle {
public InetSocketAddress getRemoteAddress();
- public void send(Object request, IResponseCallback callback) throws IPCException;
+ public long send(long requestId, Object payload, Exception exception) throws IPCException;
public void setAttachment(Object attachment);
diff --git a/hyracks-ipc/src/main/java/edu/uci/ics/hyracks/ipc/api/IIPCI.java b/hyracks-ipc/src/main/java/edu/uci/ics/hyracks/ipc/api/IIPCI.java
index ba9f343..24ab943 100644
--- a/hyracks-ipc/src/main/java/edu/uci/ics/hyracks/ipc/api/IIPCI.java
+++ b/hyracks-ipc/src/main/java/edu/uci/ics/hyracks/ipc/api/IIPCI.java
@@ -15,5 +15,5 @@
package edu.uci.ics.hyracks.ipc.api;
public interface IIPCI {
- public Object call(IIPCHandle caller, Object req) throws Exception;
+ public void deliverIncomingMessage(IIPCHandle handle, long mid, long rmid, Object payload, Exception exception);
}
\ No newline at end of file
diff --git a/hyracks-ipc/src/main/java/edu/uci/ics/hyracks/ipc/api/IPayloadSerializerDeserializer.java b/hyracks-ipc/src/main/java/edu/uci/ics/hyracks/ipc/api/IPayloadSerializerDeserializer.java
new file mode 100644
index 0000000..62648ff
--- /dev/null
+++ b/hyracks-ipc/src/main/java/edu/uci/ics/hyracks/ipc/api/IPayloadSerializerDeserializer.java
@@ -0,0 +1,27 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ *
+ * 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 edu.uci.ics.hyracks.ipc.api;
+
+import java.nio.ByteBuffer;
+
+public interface IPayloadSerializerDeserializer {
+ public Object deserializeObject(ByteBuffer buffer, int length) throws Exception;
+
+ public Exception deserializeException(ByteBuffer buffer, int length) throws Exception;
+
+ public byte[] serializeObject(Object object) throws Exception;
+
+ public byte[] serializeException(Exception object) throws Exception;
+}
\ No newline at end of file
diff --git a/hyracks-ipc/src/main/java/edu/uci/ics/hyracks/ipc/api/RPCInterface.java b/hyracks-ipc/src/main/java/edu/uci/ics/hyracks/ipc/api/RPCInterface.java
new file mode 100644
index 0000000..3340516
--- /dev/null
+++ b/hyracks-ipc/src/main/java/edu/uci/ics/hyracks/ipc/api/RPCInterface.java
@@ -0,0 +1,72 @@
+package edu.uci.ics.hyracks.ipc.api;
+
+import java.util.HashMap;
+import java.util.Map;
+
+public class RPCInterface implements IIPCI {
+ private final Map<Long, Request> reqMap;
+
+ public RPCInterface() {
+ reqMap = new HashMap<Long, RPCInterface.Request>();
+ }
+
+ public Object call(IIPCHandle handle, Object request) throws Exception {
+ Request req;
+ synchronized (this) {
+ req = new Request();
+ long mid = handle.send(-1, request, null);
+ reqMap.put(mid, req);
+ }
+ return req.getResponse();
+ }
+
+ @Override
+ public void deliverIncomingMessage(IIPCHandle handle, long mid, long rmid, Object payload, Exception exception) {
+ Request req;
+ synchronized (this) {
+ req = reqMap.remove(rmid);
+ }
+ assert req != null;
+ if (exception != null) {
+ req.setException(exception);
+ } else {
+ req.setResult(payload);
+ }
+ }
+
+ private static class Request {
+ private boolean pending;
+
+ private Object result;
+
+ private Exception exception;
+
+ Request() {
+ pending = true;
+ result = null;
+ exception = null;
+ }
+
+ synchronized void setResult(Object result) {
+ this.pending = false;
+ this.result = result;
+ notifyAll();
+ }
+
+ synchronized void setException(Exception exception) {
+ this.pending = false;
+ this.exception = exception;
+ notifyAll();
+ }
+
+ synchronized Object getResponse() throws Exception {
+ while (pending) {
+ wait();
+ }
+ if (exception != null) {
+ throw exception;
+ }
+ return result;
+ }
+ }
+}
\ No newline at end of file
diff --git a/hyracks-ipc/src/main/java/edu/uci/ics/hyracks/ipc/api/SyncRMI.java b/hyracks-ipc/src/main/java/edu/uci/ics/hyracks/ipc/api/SyncRMI.java
deleted file mode 100644
index 180b1dd..0000000
--- a/hyracks-ipc/src/main/java/edu/uci/ics/hyracks/ipc/api/SyncRMI.java
+++ /dev/null
@@ -1,48 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed 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 from
- *
- * 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 edu.uci.ics.hyracks.ipc.api;
-
-public final class SyncRMI implements IResponseCallback {
- private boolean pending;
-
- private Object response;
-
- private Exception exception;
-
- public SyncRMI() {
- }
-
- @Override
- public synchronized void callback(IIPCHandle handle, Object response, Exception exception) {
- pending = false;
- this.response = response;
- this.exception = exception;
- notifyAll();
- }
-
- public synchronized Object call(IIPCHandle handle, Object request) throws Exception {
- pending = true;
- response = null;
- exception = null;
- handle.send(request, this);
- while (pending) {
- wait();
- }
- if (exception != null) {
- throw exception;
- }
- return response;
- }
-}
\ No newline at end of file
diff --git a/hyracks-ipc/src/main/java/edu/uci/ics/hyracks/ipc/impl/IPCConnectionManager.java b/hyracks-ipc/src/main/java/edu/uci/ics/hyracks/ipc/impl/IPCConnectionManager.java
index cbbe718..053ac6b 100644
--- a/hyracks-ipc/src/main/java/edu/uci/ics/hyracks/ipc/impl/IPCConnectionManager.java
+++ b/hyracks-ipc/src/main/java/edu/uci/ics/hyracks/ipc/impl/IPCConnectionManager.java
@@ -43,13 +43,13 @@
private final Map<InetSocketAddress, IPCHandle> ipcHandleMap;
- private final List<IPCHandle>[] pendingConnections;
+ private final List<IPCHandle> pendingConnections;
- private final List<Message>[] sendList;
+ private final List<IPCHandle> workingPendingConnections;
- private int writerIndex;
+ private final List<Message> sendList;
- private int readerIndex;
+ private final List<Message> workingSendList;
private final InetSocketAddress address;
@@ -59,15 +59,16 @@
this.system = system;
this.networkThread = new NetworkThread();
this.serverSocketChannel = ServerSocketChannel.open();
+ serverSocketChannel.socket().setReuseAddress(true);
serverSocketChannel.configureBlocking(false);
ServerSocket socket = serverSocketChannel.socket();
socket.bind(socketAddress);
address = new InetSocketAddress(socket.getInetAddress(), socket.getLocalPort());
ipcHandleMap = new HashMap<InetSocketAddress, IPCHandle>();
- pendingConnections = new ArrayList[] { new ArrayList<IPCHandle>(), new ArrayList<IPCHandle>() };
- sendList = new ArrayList[] { new ArrayList<Message>(), new ArrayList<Message>() };
- writerIndex = 0;
- readerIndex = 1;
+ pendingConnections = new ArrayList<IPCHandle>();
+ workingPendingConnections = new ArrayList<IPCHandle>();
+ sendList = new ArrayList<Message>();
+ workingSendList = new ArrayList<Message>();
}
InetSocketAddress getAddress() {
@@ -90,7 +91,7 @@
handle = ipcHandleMap.get(remoteAddress);
if (handle == null) {
handle = new IPCHandle(system, remoteAddress);
- pendingConnections[writerIndex].add(handle);
+ pendingConnections.add(handle);
networkThread.selector.wakeup();
}
}
@@ -103,14 +104,22 @@
}
synchronized void write(Message msg) {
- sendList[writerIndex].add(msg);
+ if (LOGGER.isLoggable(Level.FINE)) {
+ LOGGER.fine("Enqueued message: " + msg);
+ }
+ sendList.add(msg);
networkThread.selector.wakeup();
}
- private synchronized void swapReadersAndWriters() {
- int temp = readerIndex;
- readerIndex = writerIndex;
- writerIndex = temp;
+ private synchronized void collectOutstandingWork() {
+ if (!pendingConnections.isEmpty()) {
+ workingPendingConnections.addAll(pendingConnections);
+ pendingConnections.clear();
+ }
+ if (!sendList.isEmpty()) {
+ workingSendList.addAll(sendList);
+ sendList.clear();
+ }
}
private Message createInitialReqMessage(IPCHandle handle) {
@@ -161,9 +170,9 @@
LOGGER.fine("Starting Select");
}
int n = selector.select();
- swapReadersAndWriters();
- if (!pendingConnections[readerIndex].isEmpty()) {
- for (IPCHandle handle : pendingConnections[readerIndex]) {
+ collectOutstandingWork();
+ if (!workingPendingConnections.isEmpty()) {
+ for (IPCHandle handle : workingPendingConnections) {
SocketChannel channel = SocketChannel.open();
channel.configureBlocking(false);
SelectionKey cKey = null;
@@ -177,11 +186,14 @@
handle.setKey(cKey);
cKey.attach(handle);
}
- pendingConnections[readerIndex].clear();
+ workingPendingConnections.clear();
}
- if (!sendList[readerIndex].isEmpty()) {
- for (Iterator<Message> i = sendList[readerIndex].iterator(); i.hasNext();) {
+ if (!workingSendList.isEmpty()) {
+ for (Iterator<Message> i = workingSendList.iterator(); i.hasNext();) {
Message msg = i.next();
+ if (LOGGER.isLoggable(Level.FINE)) {
+ LOGGER.fine("Processing send of message: " + msg);
+ }
IPCHandle handle = msg.getIPCHandle();
if (handle.getState() == HandleState.CLOSED) {
i.remove();
@@ -196,7 +208,7 @@
SelectionKey key = handle.getKey();
key.interestOps(key.interestOps() | SelectionKey.OP_WRITE);
} else {
- if (buffer.position() == 0) {
+ if (!buffer.hasRemaining()) {
handle.resizeOutBuffer();
continue;
}
@@ -239,7 +251,10 @@
} else if (!writeBuffer.hasRemaining()) {
key.interestOps(key.interestOps() & ~SelectionKey.OP_WRITE);
}
- handle.clearFull();
+ if (handle.full()) {
+ handle.clearFull();
+ selector.wakeup();
+ }
} else if (key.isAcceptable()) {
assert sc == serverSocketChannel;
SocketChannel channel = serverSocketChannel.accept();
@@ -261,7 +276,7 @@
}
}
}
- } catch (IOException e) {
+ } catch (Exception e) {
e.printStackTrace();
}
}
diff --git a/hyracks-ipc/src/main/java/edu/uci/ics/hyracks/ipc/impl/IPCHandle.java b/hyracks-ipc/src/main/java/edu/uci/ics/hyracks/ipc/impl/IPCHandle.java
index 481a0b0..3d3bc7a 100644
--- a/hyracks-ipc/src/main/java/edu/uci/ics/hyracks/ipc/impl/IPCHandle.java
+++ b/hyracks-ipc/src/main/java/edu/uci/ics/hyracks/ipc/impl/IPCHandle.java
@@ -17,11 +17,8 @@
import java.net.InetSocketAddress;
import java.nio.ByteBuffer;
import java.nio.channels.SelectionKey;
-import java.util.HashMap;
-import java.util.Map;
import edu.uci.ics.hyracks.ipc.api.IIPCHandle;
-import edu.uci.ics.hyracks.ipc.api.IResponseCallback;
import edu.uci.ics.hyracks.ipc.exceptions.IPCException;
final class IPCHandle implements IIPCHandle {
@@ -31,8 +28,6 @@
private InetSocketAddress remoteAddress;
- private final Map<Long, IResponseCallback> pendingRequestMap;
-
private HandleState state;
private SelectionKey key;
@@ -48,7 +43,6 @@
IPCHandle(IPCSystem system, InetSocketAddress remoteAddress) {
this.system = system;
this.remoteAddress = remoteAddress;
- pendingRequestMap = new HashMap<Long, IResponseCallback>();
inBuffer = ByteBuffer.allocate(DEFAULT_BUFFER_SIZE);
outBuffer = ByteBuffer.allocate(DEFAULT_BUFFER_SIZE);
outBuffer.flip();
@@ -60,24 +54,32 @@
return remoteAddress;
}
+ IPCSystem getIPCSystem() {
+ return system;
+ }
+
void setRemoteAddress(InetSocketAddress remoteAddress) {
this.remoteAddress = remoteAddress;
}
@Override
- public synchronized void send(Object req, IResponseCallback callback) throws IPCException {
+ public synchronized long send(long requestId, Object req, Exception exception) throws IPCException {
if (state != HandleState.CONNECTED) {
throw new IPCException("Handle is not in Connected state");
}
Message msg = new Message(this);
long mid = system.createMessageId();
msg.setMessageId(mid);
- msg.setRequestMessageId(-1);
- msg.setPayload(req);
- if (callback != null) {
- pendingRequestMap.put(mid, callback);
+ msg.setRequestMessageId(requestId);
+ if (exception != null) {
+ msg.setFlag(Message.ERROR);
+ msg.setPayload(exception);
+ } else {
+ msg.setFlag(Message.NORMAL);
+ msg.setPayload(req);
}
system.getConnectionManager().write(msg);
+ return mid;
}
@Override
@@ -110,7 +112,7 @@
this.state = state;
notifyAll();
}
-
+
synchronized void waitTillConnected() throws InterruptedException {
while (!isConnected()) {
wait();
@@ -127,9 +129,6 @@
synchronized void close() {
setState(HandleState.CLOSED);
- for (IResponseCallback cb : pendingRequestMap.values()) {
- cb.callback(this, null, new IPCException("IPC Handle Closed"));
- }
}
synchronized void processIncomingMessages() {
@@ -157,19 +156,7 @@
}
continue;
}
- long requestMessageId = message.getRequestMessageId();
- if (requestMessageId < 0) {
- system.deliverIncomingMessage(message);
- } else {
- Long rid = Long.valueOf(requestMessageId);
- IResponseCallback cb = pendingRequestMap.remove(rid);
- if (cb != null) {
- byte flag = message.getFlag();
- Object payload = flag == Message.ERROR ? null : message.getPayload();
- Exception exception = (Exception) (flag == Message.ERROR ? message.getPayload() : null);
- cb.callback(this, payload, exception);
- }
- }
+ system.deliverIncomingMessage(message);
}
inBuffer.compact();
}
@@ -178,14 +165,12 @@
inBuffer.flip();
ByteBuffer readBuffer = ByteBuffer.allocate(inBuffer.capacity() * 2);
readBuffer.put(inBuffer);
- readBuffer.compact();
inBuffer = readBuffer;
}
void resizeOutBuffer() {
ByteBuffer writeBuffer = ByteBuffer.allocate(outBuffer.capacity() * 2);
writeBuffer.put(outBuffer);
- writeBuffer.compact();
writeBuffer.flip();
outBuffer = writeBuffer;
}
diff --git a/hyracks-ipc/src/main/java/edu/uci/ics/hyracks/ipc/impl/IPCSystem.java b/hyracks-ipc/src/main/java/edu/uci/ics/hyracks/ipc/impl/IPCSystem.java
index 9eef8ff..6c9c82c 100644
--- a/hyracks-ipc/src/main/java/edu/uci/ics/hyracks/ipc/impl/IPCSystem.java
+++ b/hyracks-ipc/src/main/java/edu/uci/ics/hyracks/ipc/impl/IPCSystem.java
@@ -16,11 +16,11 @@
import java.io.IOException;
import java.net.InetSocketAddress;
-import java.util.concurrent.Executor;
import java.util.concurrent.atomic.AtomicLong;
import edu.uci.ics.hyracks.ipc.api.IIPCHandle;
import edu.uci.ics.hyracks.ipc.api.IIPCI;
+import edu.uci.ics.hyracks.ipc.api.IPayloadSerializerDeserializer;
import edu.uci.ics.hyracks.ipc.exceptions.IPCException;
public class IPCSystem {
@@ -28,18 +28,15 @@
private final IIPCI ipci;
- private final Executor executor;
+ private final IPayloadSerializerDeserializer serde;
private final AtomicLong midFactory;
- public IPCSystem(InetSocketAddress socketAddress) throws IOException {
- this(socketAddress, null, null);
- }
-
- public IPCSystem(InetSocketAddress socketAddress, IIPCI ipci, Executor executor) throws IOException {
+ public IPCSystem(InetSocketAddress socketAddress, IIPCI ipci, IPayloadSerializerDeserializer serde)
+ throws IOException {
cMgr = new IPCConnectionManager(this, socketAddress);
this.ipci = ipci;
- this.executor = executor;
+ this.serde = serde;
midFactory = new AtomicLong();
}
@@ -61,30 +58,25 @@
}
}
+ IPayloadSerializerDeserializer getSerializerDeserializer() {
+ return serde;
+ }
+
long createMessageId() {
return midFactory.incrementAndGet();
}
void deliverIncomingMessage(final Message message) {
- assert message.getFlag() == Message.NORMAL;
- executor.execute(new Runnable() {
- @Override
- public void run() {
- IPCHandle handle = message.getIPCHandle();
- Message response = new Message(handle);
- response.setMessageId(createMessageId());
- response.setRequestMessageId(message.getMessageId());
- response.setFlag(Message.NORMAL);
- try {
- Object result = ipci.call(handle, message.getPayload());
- response.setPayload(result);
- } catch (Exception e) {
- response.setFlag(Message.ERROR);
- response.setPayload(e);
- }
- cMgr.write(response);
- }
- });
+ long mid = message.getMessageId();
+ long rmid = message.getRequestMessageId();
+ Object payload = null;
+ Exception exception = null;
+ if (message.getFlag() == Message.ERROR) {
+ exception = (Exception) message.getPayload();
+ } else {
+ payload = message.getPayload();
+ }
+ ipci.deliverIncomingMessage(message.getIPCHandle(), mid, rmid, payload, exception);
}
IPCConnectionManager getConnectionManager() {
diff --git a/hyracks-ipc/src/main/java/edu/uci/ics/hyracks/ipc/impl/JavaSerializationBasedPayloadSerializerDeserializer.java b/hyracks-ipc/src/main/java/edu/uci/ics/hyracks/ipc/impl/JavaSerializationBasedPayloadSerializerDeserializer.java
new file mode 100644
index 0000000..fdf8e92
--- /dev/null
+++ b/hyracks-ipc/src/main/java/edu/uci/ics/hyracks/ipc/impl/JavaSerializationBasedPayloadSerializerDeserializer.java
@@ -0,0 +1,53 @@
+package edu.uci.ics.hyracks.ipc.impl;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.io.OutputStream;
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.ipc.api.IPayloadSerializerDeserializer;
+
+public class JavaSerializationBasedPayloadSerializerDeserializer implements IPayloadSerializerDeserializer {
+ @Override
+ public Object deserializeObject(ByteBuffer buffer, int length) throws Exception {
+ return deserialize(buffer, length);
+ }
+
+ @Override
+ public Exception deserializeException(ByteBuffer buffer, int length) throws Exception {
+ return (Exception) deserialize(buffer, length);
+ }
+
+ @Override
+ public byte[] serializeObject(Object object) throws Exception {
+ return serialize(object);
+ }
+
+ @Override
+ public byte[] serializeException(Exception exception) throws Exception {
+ return serialize(exception);
+ }
+
+ public static void serialize(OutputStream out, Object object) throws Exception {
+ ObjectOutputStream oos = new ObjectOutputStream(out);
+ oos.writeObject(object);
+ oos.flush();
+ }
+
+ private Object deserialize(ByteBuffer buffer, int length) throws Exception {
+ ObjectInputStream ois = new ObjectInputStream(new ByteArrayInputStream(buffer.array(), buffer.position(),
+ length));
+ Object object = ois.readObject();
+ ois.close();
+ return object;
+ }
+
+ private byte[] serialize(Object object) throws Exception {
+ ByteArrayOutputStream baos = new ByteArrayOutputStream();
+ serialize(baos, object);
+ baos.close();
+ return baos.toByteArray();
+ }
+}
\ No newline at end of file
diff --git a/hyracks-ipc/src/main/java/edu/uci/ics/hyracks/ipc/impl/Message.java b/hyracks-ipc/src/main/java/edu/uci/ics/hyracks/ipc/impl/Message.java
index ab3428e..6bb3156 100644
--- a/hyracks-ipc/src/main/java/edu/uci/ics/hyracks/ipc/impl/Message.java
+++ b/hyracks-ipc/src/main/java/edu/uci/ics/hyracks/ipc/impl/Message.java
@@ -14,13 +14,10 @@
*/
package edu.uci.ics.hyracks.ipc.impl;
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.IOException;
-import java.io.ObjectInputStream;
-import java.io.ObjectOutputStream;
import java.nio.ByteBuffer;
+import edu.uci.ics.hyracks.ipc.api.IPayloadSerializerDeserializer;
+
class Message {
private static final int MSG_SIZE_SIZE = 4;
@@ -92,29 +89,26 @@
return buffer.remaining() >= msgSize + MSG_SIZE_SIZE;
}
- void read(ByteBuffer buffer) throws IOException, ClassNotFoundException {
+ void read(ByteBuffer buffer) throws Exception {
assert hasMessage(buffer);
int msgSize = buffer.getInt();
messageId = buffer.getLong();
requestMessageId = buffer.getLong();
flag = buffer.get();
int finalPosition = buffer.position() + msgSize - HEADER_SIZE;
+ int length = msgSize - HEADER_SIZE;
try {
- ObjectInputStream ois = new ObjectInputStream(new ByteArrayInputStream(buffer.array(), buffer.position(),
- msgSize - HEADER_SIZE));
- payload = ois.readObject();
- ois.close();
+ IPayloadSerializerDeserializer serde = ipcHandle.getIPCSystem().getSerializerDeserializer();
+ payload = flag == ERROR ? serde.deserializeException(buffer, length) : serde.deserializeObject(buffer,
+ length);
} finally {
buffer.position(finalPosition);
}
}
- boolean write(ByteBuffer buffer) throws IOException {
- ByteArrayOutputStream baos = new ByteArrayOutputStream();
- ObjectOutputStream oos = new ObjectOutputStream(baos);
- oos.writeObject(payload);
- oos.close();
- byte[] bytes = baos.toByteArray();
+ boolean write(ByteBuffer buffer) throws Exception {
+ IPayloadSerializerDeserializer serde = ipcHandle.getIPCSystem().getSerializerDeserializer();
+ byte[] bytes = flag == ERROR ? serde.serializeException((Exception) payload) : serde.serializeObject(payload);
if (buffer.remaining() >= MSG_SIZE_SIZE + HEADER_SIZE + bytes.length) {
buffer.putInt(HEADER_SIZE + bytes.length);
buffer.putLong(messageId);
@@ -125,4 +119,9 @@
}
return false;
}
+
+ @Override
+ public String toString() {
+ return "MSG[" + messageId + ":" + requestMessageId + ":" + flag + ":" + payload + "]";
+ }
}
\ No newline at end of file
diff --git a/hyracks-ipc/src/test/java/edu/uci/ics/hyracks/ipc/tests/IPCTest.java b/hyracks-ipc/src/test/java/edu/uci/ics/hyracks/ipc/tests/IPCTest.java
new file mode 100644
index 0000000..5b2f660
--- /dev/null
+++ b/hyracks-ipc/src/test/java/edu/uci/ics/hyracks/ipc/tests/IPCTest.java
@@ -0,0 +1,92 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ *
+ * 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 edu.uci.ics.hyracks.ipc.tests;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Executors;
+
+import junit.framework.Assert;
+
+import org.junit.Test;
+
+import edu.uci.ics.hyracks.ipc.api.IIPCHandle;
+import edu.uci.ics.hyracks.ipc.api.IIPCI;
+import edu.uci.ics.hyracks.ipc.api.RPCInterface;
+import edu.uci.ics.hyracks.ipc.exceptions.IPCException;
+import edu.uci.ics.hyracks.ipc.impl.IPCSystem;
+import edu.uci.ics.hyracks.ipc.impl.JavaSerializationBasedPayloadSerializerDeserializer;
+
+public class IPCTest {
+ @Test
+ public void test() throws Exception {
+ IPCSystem server = createServerIPCSystem();
+ server.start();
+ InetSocketAddress serverAddr = server.getSocketAddress();
+
+ RPCInterface rpci = new RPCInterface();
+ IPCSystem client = createClientIPCSystem(rpci);
+ client.start();
+
+ IIPCHandle handle = client.getHandle(serverAddr);
+
+ for (int i = 0; i < 100; ++i) {
+ Assert.assertEquals(rpci.call(handle, Integer.valueOf(i)), Integer.valueOf(2 * i));
+ }
+
+ try {
+ rpci.call(handle, "Foo");
+ Assert.assertTrue(false);
+ } catch (Exception e) {
+ Assert.assertTrue(true);
+ }
+ }
+
+ private IPCSystem createServerIPCSystem() throws IOException {
+ final Executor executor = Executors.newCachedThreadPool();
+ IIPCI ipci = new IIPCI() {
+ @Override
+ public void deliverIncomingMessage(final IIPCHandle handle, final long mid, long rmid,
+ final Object payload, Exception exception) {
+ executor.execute(new Runnable() {
+ @Override
+ public void run() {
+ Object result = null;
+ Exception exception = null;
+ try {
+ Integer i = (Integer) payload;
+ result = i.intValue() * 2;
+ } catch (Exception e) {
+ exception = e;
+ }
+ try {
+ handle.send(mid, result, exception);
+ } catch (IPCException e) {
+ e.printStackTrace();
+ }
+ }
+ });
+ }
+ };
+ return new IPCSystem(new InetSocketAddress("127.0.0.1", 0), ipci,
+ new JavaSerializationBasedPayloadSerializerDeserializer());
+ }
+
+ private IPCSystem createClientIPCSystem(RPCInterface rpci) throws IOException {
+ return new IPCSystem(new InetSocketAddress("127.0.0.1", 0), rpci,
+ new JavaSerializationBasedPayloadSerializerDeserializer());
+ }
+}
\ No newline at end of file
diff --git a/hyracks-ipc/src/test/java/edu/uci/ics/ipc/tests/IPCTest.java b/hyracks-ipc/src/test/java/edu/uci/ics/ipc/tests/IPCTest.java
deleted file mode 100644
index 1fc1f6f..0000000
--- a/hyracks-ipc/src/test/java/edu/uci/ics/ipc/tests/IPCTest.java
+++ /dev/null
@@ -1,80 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed 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 from
- *
- * 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 edu.uci.ics.ipc.tests;
-
-import java.io.IOException;
-import java.net.InetSocketAddress;
-import java.util.concurrent.Executor;
-import java.util.concurrent.Executors;
-
-import junit.framework.Assert;
-
-import org.junit.Test;
-
-import edu.uci.ics.hyracks.ipc.api.IIPCHandle;
-import edu.uci.ics.hyracks.ipc.api.IIPCI;
-import edu.uci.ics.hyracks.ipc.api.SyncRMI;
-import edu.uci.ics.hyracks.ipc.impl.IPCSystem;
-
-public class IPCTest {
- @Test
- public void test() throws Exception {
- IPCSystem server = createServerIPCSystem();
- server.start();
- InetSocketAddress serverAddr = server.getSocketAddress();
-
- IPCSystem client = createClientIPCSystem();
- client.start();
-
- IIPCHandle handle = client.getHandle(serverAddr);
-
- SyncRMI rmi = new SyncRMI();
- for (int i = 0; i < 100; ++i) {
- Assert.assertEquals(rmi.call(handle, Integer.valueOf(i)), Integer.valueOf(2 * i));
- }
-
- IIPCHandle rHandle = server.getHandle(client.getSocketAddress());
-
- try {
- rmi.call(rHandle, "Foo");
- Assert.assertTrue(false);
- } catch (Exception e) {
- Assert.assertTrue(true);
- }
- }
-
- private IPCSystem createServerIPCSystem() throws IOException {
- Executor executor = Executors.newCachedThreadPool();
- IIPCI ipci = new IIPCI() {
- @Override
- public Object call(IIPCHandle caller, Object req) throws Exception {
- Integer i = (Integer) req;
- return i.intValue() * 2;
- }
- };
- return new IPCSystem(new InetSocketAddress("127.0.0.1", 0), ipci, executor);
- }
-
- private IPCSystem createClientIPCSystem() throws IOException {
- Executor executor = Executors.newCachedThreadPool();
- IIPCI ipci = new IIPCI() {
- @Override
- public Object call(IIPCHandle caller, Object req) throws Exception {
- throw new IllegalStateException();
- }
- };
- return new IPCSystem(new InetSocketAddress("127.0.0.1", 0), ipci, executor);
- }
-}
\ No newline at end of file
diff --git a/hyracks-net/pom.xml b/hyracks-net/pom.xml
new file mode 100644
index 0000000..12004f7
--- /dev/null
+++ b/hyracks-net/pom.xml
@@ -0,0 +1,31 @@
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+ <modelVersion>4.0.0</modelVersion>
+ <artifactId>hyracks-net</artifactId>
+ <parent>
+ <groupId>edu.uci.ics.hyracks</groupId>
+ <artifactId>hyracks</artifactId>
+ <version>0.2.0-SNAPSHOT</version>
+ </parent>
+
+ <build>
+ <plugins>
+ <plugin>
+ <groupId>org.apache.maven.plugins</groupId>
+ <artifactId>maven-compiler-plugin</artifactId>
+ <version>2.0.2</version>
+ <configuration>
+ <source>1.6</source>
+ <target>1.6</target>
+ </configuration>
+ </plugin>
+ </plugins>
+ </build>
+ <dependencies>
+ <dependency>
+ <groupId>junit</groupId>
+ <artifactId>junit</artifactId>
+ <version>4.8.1</version>
+ <scope>test</scope>
+ </dependency>
+ </dependencies>
+</project>
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/Accumulator.java b/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/buffers/IBufferAcceptor.java
similarity index 80%
copy from hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/Accumulator.java
copy to hyracks-net/src/main/java/edu/uci/ics/hyracks/net/buffers/IBufferAcceptor.java
index c728b0b..2f27bf0 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/Accumulator.java
+++ b/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/buffers/IBufferAcceptor.java
@@ -12,10 +12,10 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package edu.uci.ics.hyracks.control.cc.remote;
+package edu.uci.ics.hyracks.net.buffers;
-public interface Accumulator<T, R> {
- public void accumulate(T o);
+import java.nio.ByteBuffer;
- public R getResult();
+public interface IBufferAcceptor {
+ public void accept(ByteBuffer buffer);
}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/Accumulator.java b/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/buffers/ICloseableBufferAcceptor.java
similarity index 78%
copy from hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/Accumulator.java
copy to hyracks-net/src/main/java/edu/uci/ics/hyracks/net/buffers/ICloseableBufferAcceptor.java
index c728b0b..c395ac9 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/Accumulator.java
+++ b/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/buffers/ICloseableBufferAcceptor.java
@@ -12,10 +12,10 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package edu.uci.ics.hyracks.control.cc.remote;
+package edu.uci.ics.hyracks.net.buffers;
-public interface Accumulator<T, R> {
- public void accumulate(T o);
+public interface ICloseableBufferAcceptor extends IBufferAcceptor {
+ public void close();
- public R getResult();
+ public void error(int ecode);
}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/RemoteOp.java b/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/exceptions/NetException.java
similarity index 60%
copy from hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/RemoteOp.java
copy to hyracks-net/src/main/java/edu/uci/ics/hyracks/net/exceptions/NetException.java
index ec15186..ecd0373 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/RemoteOp.java
+++ b/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/exceptions/NetException.java
@@ -12,12 +12,23 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package edu.uci.ics.hyracks.control.cc.remote;
+package edu.uci.ics.hyracks.net.exceptions;
-import edu.uci.ics.hyracks.control.common.base.INodeController;
+public class NetException extends Exception {
+ private static final long serialVersionUID = 1L;
-public interface RemoteOp<T> {
- public String getNodeId();
+ public NetException() {
+ }
- public T execute(INodeController node) throws Exception;
+ public NetException(String message, Throwable cause) {
+ super(message, cause);
+ }
+
+ public NetException(String message) {
+ super(message);
+ }
+
+ public NetException(Throwable cause) {
+ super(cause);
+ }
}
\ No newline at end of file
diff --git a/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/muxdemux/ChannelControlBlock.java b/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/muxdemux/ChannelControlBlock.java
new file mode 100644
index 0000000..7f6853b
--- /dev/null
+++ b/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/muxdemux/ChannelControlBlock.java
@@ -0,0 +1,330 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ *
+ * 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 edu.uci.ics.hyracks.net.protocols.muxdemux;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.SocketChannel;
+import java.util.LinkedList;
+import java.util.Queue;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import edu.uci.ics.hyracks.net.buffers.IBufferAcceptor;
+import edu.uci.ics.hyracks.net.buffers.ICloseableBufferAcceptor;
+import edu.uci.ics.hyracks.net.exceptions.NetException;
+
+public class ChannelControlBlock {
+ private static final Logger LOGGER = Logger.getLogger(ChannelControlBlock.class.getName());
+
+ private final ChannelSet cSet;
+
+ private final int channelId;
+
+ private final ReadInterface ri;
+
+ private final WriteInterface wi;
+
+ private final AtomicBoolean localClose;
+
+ private final AtomicBoolean localCloseAck;
+
+ private final AtomicBoolean remoteClose;
+
+ ChannelControlBlock(ChannelSet cSet, int channelId) {
+ this.cSet = cSet;
+ this.channelId = channelId;
+ this.ri = new ReadInterface();
+ this.wi = new WriteInterface();
+ localClose = new AtomicBoolean();
+ localCloseAck = new AtomicBoolean();
+ remoteClose = new AtomicBoolean();
+ }
+
+ int getChannelId() {
+ return channelId;
+ }
+
+ public IChannelReadInterface getReadInterface() {
+ return ri;
+ }
+
+ public IChannelWriteInterface getWriteInterface() {
+ return wi;
+ }
+
+ private final class ReadInterface implements IChannelReadInterface {
+ private final Queue<ByteBuffer> riEmptyQueue;
+
+ private final IBufferAcceptor eba = new IBufferAcceptor() {
+ @Override
+ public void accept(ByteBuffer buffer) {
+ if (remoteClose.get()) {
+ return;
+ }
+ int delta;
+ synchronized (ChannelControlBlock.this) {
+ riEmptyQueue.add(buffer);
+ delta = buffer.remaining();
+ }
+ credits.addAndGet(delta);
+ if (delta != 0) {
+ cSet.markPendingCredits(channelId);
+ }
+ }
+ };
+
+ private ICloseableBufferAcceptor fba;
+
+ private final AtomicInteger credits;
+
+ private ByteBuffer currentReadBuffer;
+
+ ReadInterface() {
+ riEmptyQueue = new LinkedList<ByteBuffer>();
+ credits = new AtomicInteger();
+ }
+
+ @Override
+ public void setFullBufferAcceptor(ICloseableBufferAcceptor fullBufferAcceptor) {
+ fba = fullBufferAcceptor;
+ }
+
+ @Override
+ public IBufferAcceptor getEmptyBufferAcceptor() {
+ return eba;
+ }
+
+ int read(SocketChannel sc, int size) throws IOException, NetException {
+ while (true) {
+ if (size <= 0) {
+ return size;
+ }
+ if (ri.currentReadBuffer == null) {
+ ri.currentReadBuffer = ri.riEmptyQueue.poll();
+ assert ri.currentReadBuffer != null;
+ }
+ int rSize = Math.min(size, ri.currentReadBuffer.remaining());
+ if (rSize > 0) {
+ ri.currentReadBuffer.limit(ri.currentReadBuffer.position() + rSize);
+ int len;
+ try {
+ len = sc.read(ri.currentReadBuffer);
+ if (len < 0) {
+ throw new NetException("Socket Closed");
+ }
+ } finally {
+ ri.currentReadBuffer.limit(ri.currentReadBuffer.capacity());
+ }
+ size -= len;
+ if (len < rSize) {
+ return size;
+ }
+ } else {
+ return size;
+ }
+ if (ri.currentReadBuffer.remaining() <= 0) {
+ flush();
+ }
+ }
+ }
+
+ void flush() {
+ if (currentReadBuffer != null) {
+ currentReadBuffer.flip();
+ fba.accept(ri.currentReadBuffer);
+ currentReadBuffer = null;
+ }
+ }
+ }
+
+ private final class WriteInterface implements IChannelWriteInterface {
+ private final Queue<ByteBuffer> wiFullQueue;
+
+ private int channelWriteEventCount;
+
+ private final ICloseableBufferAcceptor fba = new ICloseableBufferAcceptor() {
+ @Override
+ public void accept(ByteBuffer buffer) {
+ synchronized (ChannelControlBlock.this) {
+ wiFullQueue.add(buffer);
+ incrementLocalWriteEventCount();
+ }
+ }
+
+ @Override
+ public void close() {
+ synchronized (ChannelControlBlock.this) {
+ if (eos) {
+ if (LOGGER.isLoggable(Level.WARNING)) {
+ LOGGER.warning("Received duplicate close() on channel: " + channelId);
+ }
+ return;
+ }
+ eos = true;
+ incrementLocalWriteEventCount();
+ }
+ }
+
+ @Override
+ public void error(int ecode) {
+ synchronized (ChannelControlBlock.this) {
+ WriteInterface.this.ecode = ecode;
+ incrementLocalWriteEventCount();
+ }
+ }
+ };
+
+ private IBufferAcceptor eba;
+
+ private final AtomicInteger credits;
+
+ private boolean eos;
+
+ private boolean eosSent;
+
+ private int ecode;
+
+ private boolean ecodeSent;
+
+ private ByteBuffer currentWriteBuffer;
+
+ WriteInterface() {
+ wiFullQueue = new LinkedList<ByteBuffer>();
+ credits = new AtomicInteger();
+ eos = false;
+ eosSent = false;
+ ecode = -1;
+ ecodeSent = false;
+ }
+
+ @Override
+ public void setEmptyBufferAcceptor(IBufferAcceptor emptyBufferAcceptor) {
+ eba = emptyBufferAcceptor;
+ }
+
+ @Override
+ public ICloseableBufferAcceptor getFullBufferAcceptor() {
+ return fba;
+ }
+
+ void write(MultiplexedConnection.WriterState writerState) throws NetException {
+ if (currentWriteBuffer == null) {
+ currentWriteBuffer = wiFullQueue.poll();
+ }
+ if (currentWriteBuffer != null) {
+ int size = Math.min(currentWriteBuffer.remaining(), credits.get());
+ if (size > 0) {
+ credits.addAndGet(-size);
+ writerState.command.setChannelId(channelId);
+ writerState.command.setCommandType(MuxDemuxCommand.CommandType.DATA);
+ writerState.command.setData(size);
+ writerState.reset(currentWriteBuffer, size, ChannelControlBlock.this);
+ }
+ } else if (ecode >= 0 && !ecodeSent) {
+ decrementLocalWriteEventCount();
+ writerState.command.setChannelId(channelId);
+ writerState.command.setCommandType(MuxDemuxCommand.CommandType.ERROR);
+ writerState.command.setData(ecode);
+ writerState.reset(null, 0, null);
+ ecodeSent = true;
+ localClose.set(true);
+ } else if (wi.eos && !wi.eosSent) {
+ decrementLocalWriteEventCount();
+ writerState.command.setChannelId(channelId);
+ writerState.command.setCommandType(MuxDemuxCommand.CommandType.CLOSE_CHANNEL);
+ writerState.command.setData(0);
+ writerState.reset(null, 0, null);
+ eosSent = true;
+ localClose.set(true);
+ }
+ }
+
+ void writeComplete() {
+ if (currentWriteBuffer.remaining() <= 0) {
+ currentWriteBuffer.clear();
+ eba.accept(currentWriteBuffer);
+ decrementLocalWriteEventCount();
+ currentWriteBuffer = null;
+ }
+ }
+
+ void incrementLocalWriteEventCount() {
+ ++channelWriteEventCount;
+ if (channelWriteEventCount == 1) {
+ cSet.markPendingWrite(channelId);
+ }
+ }
+
+ void decrementLocalWriteEventCount() {
+ --channelWriteEventCount;
+ if (channelWriteEventCount == 0) {
+ cSet.unmarkPendingWrite(channelId);
+ }
+ }
+ }
+
+ synchronized void write(MultiplexedConnection.WriterState writerState) throws NetException {
+ wi.write(writerState);
+ }
+
+ synchronized void writeComplete() {
+ wi.writeComplete();
+ }
+
+ synchronized int read(SocketChannel sc, int size) throws IOException, NetException {
+ return ri.read(sc, size);
+ }
+
+ void addReadCredits(int delta) {
+ ri.credits.addAndGet(delta);
+ }
+
+ int getAndResetReadCredits() {
+ return ri.credits.getAndSet(0);
+ }
+
+ void addWriteCredits(int delta) {
+ wi.credits.addAndGet(delta);
+ }
+
+ synchronized void reportRemoteEOS() {
+ ri.flush();
+ ri.fba.close();
+ remoteClose.set(true);
+ }
+
+ synchronized void reportLocalEOSAck() {
+ localCloseAck.set(true);
+ }
+
+ synchronized void reportRemoteError(int ecode) {
+ ri.flush();
+ ri.fba.error(ecode);
+ remoteClose.set(true);
+ }
+
+ boolean completelyClosed() {
+ return localCloseAck.get() && remoteClose.get();
+ }
+
+ @Override
+ public String toString() {
+ return "Channel:" + channelId + "[localClose: " + localClose + " localCloseAck: " + localCloseAck
+ + " remoteClose: " + remoteClose + " readCredits: " + ri.credits + " writeCredits: " + wi.credits + "]";
+ }
+}
\ No newline at end of file
diff --git a/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/muxdemux/ChannelSet.java b/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/muxdemux/ChannelSet.java
new file mode 100644
index 0000000..9411d42
--- /dev/null
+++ b/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/muxdemux/ChannelSet.java
@@ -0,0 +1,206 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ *
+ * 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 edu.uci.ics.hyracks.net.protocols.muxdemux;
+
+import java.util.Arrays;
+import java.util.BitSet;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import edu.uci.ics.hyracks.net.exceptions.NetException;
+
+public class ChannelSet {
+ private static final Logger LOGGER = Logger.getLogger(ChannelSet.class.getName());
+
+ private static final int MAX_OPEN_CHANNELS = 1024;
+
+ private static final int INITIAL_SIZE = 16;
+
+ private final MultiplexedConnection mConn;
+
+ private ChannelControlBlock[] ccbArray;
+
+ private final BitSet allocationBitmap;
+
+ private final BitSet pendingChannelWriteBitmap;
+
+ private final BitSet pendingChannelCreditsBitmap;
+
+ private final BitSet pendingChannelSynBitmap;
+
+ private final BitSet pendingEOSAckBitmap;
+
+ private int openChannelCount;
+
+ private final IEventCounter pendingWriteEventsCounter;
+
+ ChannelSet(MultiplexedConnection mConn, IEventCounter pendingWriteEventsCounter) {
+ this.mConn = mConn;
+ ccbArray = new ChannelControlBlock[INITIAL_SIZE];
+ allocationBitmap = new BitSet();
+ pendingChannelWriteBitmap = new BitSet();
+ pendingChannelCreditsBitmap = new BitSet();
+ pendingChannelSynBitmap = new BitSet();
+ pendingEOSAckBitmap = new BitSet();
+ this.pendingWriteEventsCounter = pendingWriteEventsCounter;
+ openChannelCount = 0;
+ }
+
+ ChannelControlBlock allocateChannel() throws NetException {
+ synchronized (mConn) {
+ int idx = allocationBitmap.nextClearBit(0);
+ if (idx < 0 || idx == ccbArray.length) {
+ cleanupClosedChannels();
+ idx = allocationBitmap.nextClearBit(0);
+ if (idx < 0 || idx == ccbArray.length) {
+ idx = ccbArray.length;
+ }
+ }
+ return createChannel(idx);
+ }
+ }
+
+ private void cleanupClosedChannels() {
+ for (int i = 0; i < ccbArray.length; ++i) {
+ ChannelControlBlock ccb = ccbArray[i];
+ if (ccb != null) {
+ if (ccb.completelyClosed()) {
+ if (LOGGER.isLoggable(Level.FINE)) {
+ LOGGER.fine("Cleaning free channel: " + ccb);
+ }
+ freeChannel(ccb);
+ }
+ }
+ }
+ }
+
+ ChannelControlBlock registerChannel(int channelId) throws NetException {
+ synchronized (mConn) {
+ return createChannel(channelId);
+ }
+ }
+
+ private void freeChannel(ChannelControlBlock channel) {
+ int idx = channel.getChannelId();
+ ccbArray[idx] = null;
+ allocationBitmap.clear(idx);
+ --openChannelCount;
+ }
+
+ ChannelControlBlock getCCB(int channelId) {
+ return ccbArray[channelId];
+ }
+
+ BitSet getPendingChannelWriteBitmap() {
+ return pendingChannelWriteBitmap;
+ }
+
+ BitSet getPendingChannelCreditsBitmap() {
+ return pendingChannelCreditsBitmap;
+ }
+
+ BitSet getPendingChannelSynBitmap() {
+ return pendingChannelSynBitmap;
+ }
+
+ BitSet getPendingEOSAckBitmap() {
+ return pendingEOSAckBitmap;
+ }
+
+ int getOpenChannelCount() {
+ return openChannelCount;
+ }
+
+ void initiateChannelSyn(int channelId) {
+ synchronized (mConn) {
+ assert !pendingChannelSynBitmap.get(channelId);
+ pendingChannelSynBitmap.set(channelId);
+ pendingWriteEventsCounter.increment();
+ }
+ }
+
+ void markPendingCredits(int channelId) {
+ synchronized (mConn) {
+ if (!pendingChannelCreditsBitmap.get(channelId)) {
+ pendingChannelCreditsBitmap.set(channelId);
+ pendingWriteEventsCounter.increment();
+ }
+ }
+ }
+
+ void unmarkPendingCredits(int channelId) {
+ synchronized (mConn) {
+ if (pendingChannelCreditsBitmap.get(channelId)) {
+ pendingChannelCreditsBitmap.clear(channelId);
+ pendingWriteEventsCounter.decrement();
+ }
+ }
+ }
+
+ void markPendingWrite(int channelId) {
+ synchronized (mConn) {
+ assert !pendingChannelWriteBitmap.get(channelId);
+ pendingChannelWriteBitmap.set(channelId);
+ pendingWriteEventsCounter.increment();
+ }
+ }
+
+ void unmarkPendingWrite(int channelId) {
+ synchronized (mConn) {
+ assert pendingChannelWriteBitmap.get(channelId);
+ pendingChannelWriteBitmap.clear(channelId);
+ pendingWriteEventsCounter.decrement();
+ }
+ }
+
+ void markEOSAck(int channelId) {
+ synchronized (mConn) {
+ assert !pendingEOSAckBitmap.get(channelId);
+ pendingEOSAckBitmap.set(channelId);
+ pendingWriteEventsCounter.increment();
+ }
+ }
+
+ private ChannelControlBlock createChannel(int idx) throws NetException {
+ if (idx >= ccbArray.length) {
+ expand(idx);
+ }
+ if (idx > MAX_OPEN_CHANNELS) {
+ throw new NetException("More than " + MAX_OPEN_CHANNELS + " opened concurrently");
+ }
+ if (ccbArray[idx] != null) {
+ assert ccbArray[idx].completelyClosed();
+ if (ccbArray[idx].completelyClosed()) {
+ if (LOGGER.isLoggable(Level.FINE)) {
+ LOGGER.fine("Cleaning free channel: " + ccbArray[idx]);
+ }
+ freeChannel(ccbArray[idx]);
+ }
+ }
+ assert idx < ccbArray.length;
+ assert !allocationBitmap.get(idx);
+ ChannelControlBlock channel = new ChannelControlBlock(this, idx);
+ ccbArray[idx] = channel;
+ allocationBitmap.set(idx);
+ ++openChannelCount;
+ return channel;
+ }
+
+ private void expand(int idx) {
+ while (idx >= ccbArray.length) {
+ ccbArray = Arrays.copyOf(ccbArray, ccbArray.length * 2);
+ }
+ }
+}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/Accumulator.java b/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/muxdemux/IChannelOpenListener.java
similarity index 80%
copy from hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/Accumulator.java
copy to hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/muxdemux/IChannelOpenListener.java
index c728b0b..0fc9b2a 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/Accumulator.java
+++ b/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/muxdemux/IChannelOpenListener.java
@@ -12,10 +12,8 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package edu.uci.ics.hyracks.control.cc.remote;
+package edu.uci.ics.hyracks.net.protocols.muxdemux;
-public interface Accumulator<T, R> {
- public void accumulate(T o);
-
- public R getResult();
+public interface IChannelOpenListener {
+ public void channelOpened(ChannelControlBlock channel);
}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/RemoteOp.java b/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/muxdemux/IChannelReadInterface.java
similarity index 64%
copy from hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/RemoteOp.java
copy to hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/muxdemux/IChannelReadInterface.java
index ec15186..468a617 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/RemoteOp.java
+++ b/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/muxdemux/IChannelReadInterface.java
@@ -12,12 +12,13 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package edu.uci.ics.hyracks.control.cc.remote;
+package edu.uci.ics.hyracks.net.protocols.muxdemux;
-import edu.uci.ics.hyracks.control.common.base.INodeController;
+import edu.uci.ics.hyracks.net.buffers.IBufferAcceptor;
+import edu.uci.ics.hyracks.net.buffers.ICloseableBufferAcceptor;
-public interface RemoteOp<T> {
- public String getNodeId();
+public interface IChannelReadInterface {
+ public void setFullBufferAcceptor(ICloseableBufferAcceptor fullBufferAcceptor);
- public T execute(INodeController node) throws Exception;
+ public IBufferAcceptor getEmptyBufferAcceptor();
}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/RemoteOp.java b/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/muxdemux/IChannelWriteInterface.java
similarity index 63%
copy from hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/RemoteOp.java
copy to hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/muxdemux/IChannelWriteInterface.java
index ec15186..1e53d71 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/RemoteOp.java
+++ b/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/muxdemux/IChannelWriteInterface.java
@@ -12,12 +12,13 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package edu.uci.ics.hyracks.control.cc.remote;
+package edu.uci.ics.hyracks.net.protocols.muxdemux;
-import edu.uci.ics.hyracks.control.common.base.INodeController;
+import edu.uci.ics.hyracks.net.buffers.IBufferAcceptor;
+import edu.uci.ics.hyracks.net.buffers.ICloseableBufferAcceptor;
-public interface RemoteOp<T> {
- public String getNodeId();
+public interface IChannelWriteInterface {
+ public void setEmptyBufferAcceptor(IBufferAcceptor emptyBufferAcceptor);
- public T execute(INodeController node) throws Exception;
+ public ICloseableBufferAcceptor getFullBufferAcceptor();
}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/Accumulator.java b/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/muxdemux/IEventCounter.java
similarity index 81%
rename from hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/Accumulator.java
rename to hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/muxdemux/IEventCounter.java
index c728b0b..148078c 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/Accumulator.java
+++ b/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/muxdemux/IEventCounter.java
@@ -12,10 +12,10 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package edu.uci.ics.hyracks.control.cc.remote;
+package edu.uci.ics.hyracks.net.protocols.muxdemux;
-public interface Accumulator<T, R> {
- public void accumulate(T o);
+public interface IEventCounter {
+ public void increment();
- public R getResult();
+ public void decrement();
}
\ No newline at end of file
diff --git a/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/muxdemux/MultiplexedConnection.java b/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/muxdemux/MultiplexedConnection.java
new file mode 100644
index 0000000..1d71fac
--- /dev/null
+++ b/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/muxdemux/MultiplexedConnection.java
@@ -0,0 +1,353 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ *
+ * 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 edu.uci.ics.hyracks.net.protocols.muxdemux;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.SelectionKey;
+import java.nio.channels.SocketChannel;
+import java.util.BitSet;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import edu.uci.ics.hyracks.net.exceptions.NetException;
+import edu.uci.ics.hyracks.net.protocols.tcp.ITCPConnectionEventListener;
+import edu.uci.ics.hyracks.net.protocols.tcp.TCPConnection;
+
+public class MultiplexedConnection implements ITCPConnectionEventListener {
+ private static final Logger LOGGER = Logger.getLogger(MultiplexedConnection.class.getName());
+
+ private static final int MAX_CHUNKS_READ_PER_CYCLE = 4;
+
+ private final MuxDemux muxDemux;
+
+ private final IEventCounter pendingWriteEventsCounter;
+
+ private final ChannelSet cSet;
+
+ private final ReaderState readerState;
+
+ private final WriterState writerState;
+
+ private TCPConnection tcpConnection;
+
+ private int lastChannelWritten;
+
+ public MultiplexedConnection(MuxDemux muxDemux) {
+ this.muxDemux = muxDemux;
+ pendingWriteEventsCounter = new IEventCounter() {
+ private int counter;
+
+ @Override
+ public synchronized void increment() {
+ ++counter;
+ if (counter == 1) {
+ tcpConnection.enable(SelectionKey.OP_WRITE);
+ }
+ }
+
+ @Override
+ public synchronized void decrement() {
+ --counter;
+ if (counter == 0) {
+ tcpConnection.disable(SelectionKey.OP_WRITE);
+ }
+ if (counter < 0) {
+ throw new IllegalStateException();
+ }
+ }
+ };
+ cSet = new ChannelSet(this, pendingWriteEventsCounter);
+ readerState = new ReaderState();
+ writerState = new WriterState();
+ lastChannelWritten = -1;
+ }
+
+ synchronized void setTCPConnection(TCPConnection tcpConnection) {
+ this.tcpConnection = tcpConnection;
+ tcpConnection.enable(SelectionKey.OP_READ);
+ notifyAll();
+ }
+
+ synchronized void waitUntilConnected() throws InterruptedException {
+ while (tcpConnection == null) {
+ wait();
+ }
+ }
+
+ @Override
+ public void notifyIOReady(TCPConnection connection, boolean readable, boolean writable) throws IOException,
+ NetException {
+ if (readable) {
+ driveReaderStateMachine();
+ }
+ if (writable) {
+ driveWriterStateMachine();
+ }
+ }
+
+ public ChannelControlBlock openChannel() throws NetException, InterruptedException {
+ ChannelControlBlock channel = cSet.allocateChannel();
+ int channelId = channel.getChannelId();
+ cSet.initiateChannelSyn(channelId);
+ return channel;
+ }
+
+ class WriterState {
+ private final ByteBuffer writeBuffer;
+
+ final MuxDemuxCommand command;
+
+ private ByteBuffer pendingBuffer;
+
+ private int pendingWriteSize;
+
+ private ChannelControlBlock ccb;
+
+ public WriterState() {
+ writeBuffer = ByteBuffer.allocateDirect(MuxDemuxCommand.COMMAND_SIZE);
+ writeBuffer.flip();
+ command = new MuxDemuxCommand();
+ ccb = null;
+ }
+
+ boolean writePending() {
+ return writeBuffer.remaining() > 0 || (pendingBuffer != null && pendingWriteSize > 0);
+ }
+
+ void reset(ByteBuffer pendingBuffer, int pendingWriteSize, ChannelControlBlock ccb) {
+ writeBuffer.clear();
+ command.write(writeBuffer);
+ writeBuffer.flip();
+ this.pendingBuffer = pendingBuffer;
+ this.pendingWriteSize = pendingWriteSize;
+ this.ccb = ccb;
+ }
+
+ boolean performPendingWrite(SocketChannel sc) throws IOException {
+ int len = writeBuffer.remaining();
+ if (len > 0) {
+ int written = sc.write(writeBuffer);
+ muxDemux.getPerformanceCounters().addSignalingBytesWritten(written);
+ if (written < len) {
+ return false;
+ }
+ }
+ if (pendingBuffer != null) {
+ if (pendingWriteSize > 0) {
+ assert pendingWriteSize <= pendingBuffer.remaining();
+ int oldLimit = pendingBuffer.limit();
+ try {
+ pendingBuffer.limit(pendingWriteSize + pendingBuffer.position());
+ int written = sc.write(pendingBuffer);
+ muxDemux.getPerformanceCounters().addPayloadBytesWritten(written);
+ pendingWriteSize -= written;
+ } finally {
+ pendingBuffer.limit(oldLimit);
+ }
+ }
+ if (pendingWriteSize > 0) {
+ return false;
+ }
+ pendingBuffer = null;
+ pendingWriteSize = 0;
+ }
+ if (ccb != null) {
+ ccb.writeComplete();
+ ccb = null;
+ }
+ return true;
+ }
+ }
+
+ void driveWriterStateMachine() throws IOException, NetException {
+ SocketChannel sc = tcpConnection.getSocketChannel();
+ if (writerState.writePending()) {
+ if (!writerState.performPendingWrite(sc)) {
+ return;
+ }
+ pendingWriteEventsCounter.decrement();
+ }
+ int numCycles;
+
+ synchronized (MultiplexedConnection.this) {
+ numCycles = cSet.getOpenChannelCount();
+ }
+
+ for (int i = 0; i < numCycles; ++i) {
+ ChannelControlBlock writeCCB = null;
+ synchronized (MultiplexedConnection.this) {
+ BitSet pendingChannelSynBitmap = cSet.getPendingChannelSynBitmap();
+ for (int j = pendingChannelSynBitmap.nextSetBit(0); j >= 0; j = pendingChannelSynBitmap.nextSetBit(j)) {
+ pendingChannelSynBitmap.clear(j);
+ pendingWriteEventsCounter.decrement();
+ writerState.command.setChannelId(j);
+ writerState.command.setCommandType(MuxDemuxCommand.CommandType.OPEN_CHANNEL);
+ writerState.command.setData(0);
+ writerState.reset(null, 0, null);
+ if (!writerState.performPendingWrite(sc)) {
+ return;
+ }
+ }
+ BitSet pendingChannelCreditsBitmap = cSet.getPendingChannelCreditsBitmap();
+ for (int j = pendingChannelCreditsBitmap.nextSetBit(0); j >= 0; j = pendingChannelCreditsBitmap
+ .nextSetBit(j)) {
+ pendingChannelCreditsBitmap.clear(j);
+ pendingWriteEventsCounter.decrement();
+ writerState.command.setChannelId(j);
+ writerState.command.setCommandType(MuxDemuxCommand.CommandType.ADD_CREDITS);
+ ChannelControlBlock ccb = cSet.getCCB(j);
+ int credits = ccb.getAndResetReadCredits();
+ writerState.command.setData(credits);
+ writerState.reset(null, 0, null);
+ if (!writerState.performPendingWrite(sc)) {
+ return;
+ }
+ }
+ BitSet pendingEOSAckBitmap = cSet.getPendingEOSAckBitmap();
+ for (int j = pendingEOSAckBitmap.nextSetBit(0); j >= 0; j = pendingEOSAckBitmap.nextSetBit(j)) {
+ pendingEOSAckBitmap.clear(j);
+ pendingWriteEventsCounter.decrement();
+ writerState.command.setChannelId(j);
+ writerState.command.setCommandType(MuxDemuxCommand.CommandType.CLOSE_CHANNEL_ACK);
+ writerState.command.setData(0);
+ writerState.reset(null, 0, null);
+ if (!writerState.performPendingWrite(sc)) {
+ return;
+ }
+ }
+ BitSet pendingChannelWriteBitmap = cSet.getPendingChannelWriteBitmap();
+ lastChannelWritten = pendingChannelWriteBitmap.nextSetBit(lastChannelWritten + 1);
+ if (lastChannelWritten < 0) {
+ lastChannelWritten = pendingChannelWriteBitmap.nextSetBit(0);
+ if (lastChannelWritten < 0) {
+ return;
+ }
+ }
+ writeCCB = cSet.getCCB(lastChannelWritten);
+ }
+ writeCCB.write(writerState);
+ if (writerState.writePending()) {
+ pendingWriteEventsCounter.increment();
+ if (!writerState.performPendingWrite(sc)) {
+ return;
+ }
+ pendingWriteEventsCounter.decrement();
+ }
+ }
+ }
+
+ class ReaderState {
+ private final ByteBuffer readBuffer;
+
+ final MuxDemuxCommand command;
+
+ private int pendingReadSize;
+
+ private ChannelControlBlock ccb;
+
+ ReaderState() {
+ readBuffer = ByteBuffer.allocateDirect(MuxDemuxCommand.COMMAND_SIZE);
+ command = new MuxDemuxCommand();
+ }
+
+ void reset() {
+ readBuffer.clear();
+ pendingReadSize = 0;
+ ccb = null;
+ }
+
+ private ChannelControlBlock getCCBInCommand() {
+ synchronized (MultiplexedConnection.this) {
+ return cSet.getCCB(command.getChannelId());
+ }
+ }
+ }
+
+ void driveReaderStateMachine() throws IOException, NetException {
+ SocketChannel sc = tcpConnection.getSocketChannel();
+ int chunksRead = 0;
+ while (chunksRead < MAX_CHUNKS_READ_PER_CYCLE) {
+ if (readerState.readBuffer.remaining() > 0) {
+ int read = sc.read(readerState.readBuffer);
+ if (read < 0) {
+ throw new NetException("Socket Closed");
+ }
+ muxDemux.getPerformanceCounters().addSignalingBytesRead(read);
+ if (readerState.readBuffer.remaining() > 0) {
+ return;
+ }
+ readerState.readBuffer.flip();
+ readerState.command.read(readerState.readBuffer);
+ if (LOGGER.isLoggable(Level.FINE)) {
+ LOGGER.fine("Received command: " + readerState.command);
+ }
+ ChannelControlBlock ccb = null;
+ switch (readerState.command.getCommandType()) {
+ case ADD_CREDITS: {
+ ccb = readerState.getCCBInCommand();
+ ccb.addWriteCredits(readerState.command.getData());
+ break;
+ }
+ case CLOSE_CHANNEL: {
+ ccb = readerState.getCCBInCommand();
+ ccb.reportRemoteEOS();
+ int channelId = ccb.getChannelId();
+ cSet.markEOSAck(channelId);
+ cSet.unmarkPendingCredits(channelId);
+ break;
+ }
+ case CLOSE_CHANNEL_ACK: {
+ ccb = readerState.getCCBInCommand();
+ ccb.reportLocalEOSAck();
+ break;
+ }
+ case DATA: {
+ ccb = readerState.getCCBInCommand();
+ readerState.pendingReadSize = readerState.command.getData();
+ readerState.ccb = ccb;
+ break;
+ }
+ case ERROR: {
+ ccb = readerState.getCCBInCommand();
+ ccb.reportRemoteError(readerState.command.getData());
+ int channelId = ccb.getChannelId();
+ cSet.markEOSAck(channelId);
+ cSet.unmarkPendingCredits(channelId);
+ break;
+ }
+ case OPEN_CHANNEL: {
+ int channelId = readerState.command.getChannelId();
+ ccb = cSet.registerChannel(channelId);
+ muxDemux.getChannelOpenListener().channelOpened(ccb);
+ }
+ }
+ if (LOGGER.isLoggable(Level.FINE)) {
+ LOGGER.fine("Applied command: " + readerState.command + " on " + ccb);
+ }
+ }
+ if (readerState.pendingReadSize > 0) {
+ ++chunksRead;
+ int newPendingReadSize = readerState.ccb.read(sc, readerState.pendingReadSize);
+ muxDemux.getPerformanceCounters().addPayloadBytesRead(readerState.pendingReadSize - newPendingReadSize);
+ readerState.pendingReadSize = newPendingReadSize;
+ if (readerState.pendingReadSize > 0) {
+ return;
+ }
+ }
+ readerState.reset();
+ }
+ }
+}
\ No newline at end of file
diff --git a/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/muxdemux/MuxDemux.java b/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/muxdemux/MuxDemux.java
new file mode 100644
index 0000000..b041e2c
--- /dev/null
+++ b/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/muxdemux/MuxDemux.java
@@ -0,0 +1,94 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ *
+ * 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 edu.uci.ics.hyracks.net.protocols.muxdemux;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.HashMap;
+import java.util.Map;
+
+import edu.uci.ics.hyracks.net.protocols.tcp.ITCPConnectionListener;
+import edu.uci.ics.hyracks.net.protocols.tcp.TCPConnection;
+import edu.uci.ics.hyracks.net.protocols.tcp.TCPEndpoint;
+
+public class MuxDemux {
+ private final InetSocketAddress localAddress;
+
+ private final IChannelOpenListener channelOpenListener;
+
+ private final Map<InetSocketAddress, MultiplexedConnection> connectionMap;
+
+ private final TCPEndpoint tcpEndpoint;
+
+ private final PerformanceCounters perfCounters;
+
+ public MuxDemux(InetSocketAddress localAddress, IChannelOpenListener listener, int nThreads) {
+ this.localAddress = localAddress;
+ this.channelOpenListener = listener;
+ connectionMap = new HashMap<InetSocketAddress, MultiplexedConnection>();
+ this.tcpEndpoint = new TCPEndpoint(new ITCPConnectionListener() {
+ @Override
+ public void connectionEstablished(TCPConnection connection) {
+ MultiplexedConnection mConn;
+ synchronized (MuxDemux.this) {
+ mConn = connectionMap.get(connection.getRemoteAddress());
+ }
+ assert mConn != null;
+ mConn.setTCPConnection(connection);
+ connection.setEventListener(mConn);
+ connection.setAttachment(mConn);
+ }
+
+ @Override
+ public void acceptedConnection(TCPConnection connection) {
+ MultiplexedConnection mConn = new MultiplexedConnection(MuxDemux.this);
+ mConn.setTCPConnection(connection);
+ connection.setEventListener(mConn);
+ connection.setAttachment(mConn);
+ }
+ }, nThreads);
+ perfCounters = new PerformanceCounters();
+ }
+
+ public void start() throws IOException {
+ tcpEndpoint.start(localAddress);
+ }
+
+ public MultiplexedConnection connect(InetSocketAddress remoteAddress) throws InterruptedException {
+ MultiplexedConnection mConn = null;
+ synchronized (this) {
+ mConn = connectionMap.get(remoteAddress);
+ if (mConn == null) {
+ mConn = new MultiplexedConnection(this);
+ connectionMap.put(remoteAddress, mConn);
+ tcpEndpoint.initiateConnection(remoteAddress);
+ }
+ }
+ mConn.waitUntilConnected();
+ return mConn;
+ }
+
+ IChannelOpenListener getChannelOpenListener() {
+ return channelOpenListener;
+ }
+
+ public InetSocketAddress getLocalAddress() {
+ return tcpEndpoint.getLocalAddress();
+ }
+
+ public PerformanceCounters getPerformanceCounters() {
+ return perfCounters;
+ }
+}
\ No newline at end of file
diff --git a/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/muxdemux/MuxDemuxCommand.java b/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/muxdemux/MuxDemuxCommand.java
new file mode 100644
index 0000000..9124aad
--- /dev/null
+++ b/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/muxdemux/MuxDemuxCommand.java
@@ -0,0 +1,75 @@
+package edu.uci.ics.hyracks.net.protocols.muxdemux;
+
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.net.exceptions.NetException;
+
+class MuxDemuxCommand {
+ static final int MAX_CHANNEL_ID = 0x3ff;
+
+ static final int COMMAND_SIZE = 4;
+
+ static final int MAX_DATA_VALUE = 0x7ffff;
+
+ enum CommandType {
+ OPEN_CHANNEL,
+ CLOSE_CHANNEL,
+ CLOSE_CHANNEL_ACK,
+ ERROR,
+ ADD_CREDITS,
+ DATA,
+ }
+
+ private int channelId;
+
+ private CommandType type;
+
+ private int data;
+
+ public int getChannelId() {
+ return channelId;
+ }
+
+ public void setChannelId(int channelId) throws NetException {
+ if (channelId > MAX_CHANNEL_ID) {
+ throw new NetException("channelId " + channelId + " exceeds " + MAX_CHANNEL_ID);
+ }
+ this.channelId = channelId;
+ }
+
+ public CommandType getCommandType() {
+ return type;
+ }
+
+ public void setCommandType(CommandType type) {
+ this.type = type;
+ }
+
+ public int getData() {
+ return data;
+ }
+
+ public void setData(int data) throws NetException {
+ if (channelId > MAX_DATA_VALUE) {
+ throw new NetException("data " + data + " exceeds " + MAX_DATA_VALUE);
+ }
+ this.data = data;
+ }
+
+ public void write(ByteBuffer buffer) {
+ int cmd = (channelId << 22) | (type.ordinal() << 19) | (data & 0x7ffff);
+ buffer.putInt(cmd);
+ }
+
+ public void read(ByteBuffer buffer) {
+ int cmd = buffer.getInt();
+ channelId = (cmd >> 22) & 0x3ff;
+ type = CommandType.values()[(cmd >> 19) & 0x7];
+ data = cmd & 0x7ffff;
+ }
+
+ @Override
+ public String toString() {
+ return channelId + ":" + type + ":" + data;
+ }
+}
\ No newline at end of file
diff --git a/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/muxdemux/PerformanceCounters.java b/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/muxdemux/PerformanceCounters.java
new file mode 100644
index 0000000..a203f06
--- /dev/null
+++ b/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/muxdemux/PerformanceCounters.java
@@ -0,0 +1,66 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ *
+ * 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 edu.uci.ics.hyracks.net.protocols.muxdemux;
+
+import java.util.concurrent.atomic.AtomicLong;
+
+public class PerformanceCounters {
+ private final AtomicLong payloadBytesRead;
+
+ private final AtomicLong payloadBytesWritten;
+
+ private final AtomicLong signalingBytesRead;
+
+ private final AtomicLong signalingBytesWritten;
+
+ public PerformanceCounters() {
+ payloadBytesRead = new AtomicLong();
+ payloadBytesWritten = new AtomicLong();
+ signalingBytesRead = new AtomicLong();
+ signalingBytesWritten = new AtomicLong();
+ }
+
+ public void addPayloadBytesRead(long delta) {
+ payloadBytesRead.addAndGet(delta);
+ }
+
+ public long getPayloadBytesRead() {
+ return payloadBytesRead.get();
+ }
+
+ public void addPayloadBytesWritten(long delta) {
+ payloadBytesWritten.addAndGet(delta);
+ }
+
+ public long getPayloadBytesWritten() {
+ return payloadBytesWritten.get();
+ }
+
+ public void addSignalingBytesRead(long delta) {
+ signalingBytesRead.addAndGet(delta);
+ }
+
+ public long getSignalingBytesRead() {
+ return signalingBytesRead.get();
+ }
+
+ public void addSignalingBytesWritten(long delta) {
+ signalingBytesWritten.addAndGet(delta);
+ }
+
+ public long getSignalingBytesWritten() {
+ return signalingBytesWritten.get();
+ }
+}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/RemoteOp.java b/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/tcp/ITCPConnectionEventListener.java
similarity index 66%
rename from hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/RemoteOp.java
rename to hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/tcp/ITCPConnectionEventListener.java
index ec15186..607bf31 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/RemoteOp.java
+++ b/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/tcp/ITCPConnectionEventListener.java
@@ -12,12 +12,13 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package edu.uci.ics.hyracks.control.cc.remote;
+package edu.uci.ics.hyracks.net.protocols.tcp;
-import edu.uci.ics.hyracks.control.common.base.INodeController;
+import java.io.IOException;
-public interface RemoteOp<T> {
- public String getNodeId();
+import edu.uci.ics.hyracks.net.exceptions.NetException;
- public T execute(INodeController node) throws Exception;
+public interface ITCPConnectionEventListener {
+ public void notifyIOReady(TCPConnection connection, boolean readable, boolean writable) throws IOException,
+ NetException;
}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/Accumulator.java b/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/tcp/ITCPConnectionListener.java
similarity index 74%
copy from hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/Accumulator.java
copy to hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/tcp/ITCPConnectionListener.java
index c728b0b..cdaabf4 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/Accumulator.java
+++ b/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/tcp/ITCPConnectionListener.java
@@ -12,10 +12,10 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package edu.uci.ics.hyracks.control.cc.remote;
+package edu.uci.ics.hyracks.net.protocols.tcp;
-public interface Accumulator<T, R> {
- public void accumulate(T o);
+public interface ITCPConnectionListener {
+ public void acceptedConnection(TCPConnection connection);
- public R getResult();
+ public void connectionEstablished(TCPConnection connection);
}
\ No newline at end of file
diff --git a/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/tcp/TCPConnection.java b/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/tcp/TCPConnection.java
new file mode 100644
index 0000000..210508b
--- /dev/null
+++ b/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/tcp/TCPConnection.java
@@ -0,0 +1,93 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ *
+ * 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 edu.uci.ics.hyracks.net.protocols.tcp;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.nio.channels.SelectionKey;
+import java.nio.channels.Selector;
+import java.nio.channels.SocketChannel;
+
+public class TCPConnection {
+ private final TCPEndpoint endpoint;
+
+ private final SocketChannel channel;
+
+ private final SelectionKey key;
+
+ private final Selector selector;
+
+ private ITCPConnectionEventListener eventListener;
+
+ private Object attachment;
+
+ public TCPConnection(TCPEndpoint endpoint, SocketChannel channel, SelectionKey key, Selector selector) {
+ this.endpoint = endpoint;
+ this.channel = channel;
+ this.key = key;
+ this.selector = selector;
+ }
+
+ public TCPEndpoint getEndpoint() {
+ return endpoint;
+ }
+
+ public SocketChannel getSocketChannel() {
+ return channel;
+ }
+
+ public InetSocketAddress getLocalAddress() {
+ return (InetSocketAddress) channel.socket().getLocalSocketAddress();
+ }
+
+ public InetSocketAddress getRemoteAddress() {
+ return (InetSocketAddress) channel.socket().getRemoteSocketAddress();
+ }
+
+ public void enable(int ops) {
+ key.interestOps(key.interestOps() | ops);
+ selector.wakeup();
+ }
+
+ public void disable(int ops) {
+ key.interestOps(key.interestOps() & ~(ops));
+ selector.wakeup();
+ }
+
+ public ITCPConnectionEventListener getEventListener() {
+ return eventListener;
+ }
+
+ public void setEventListener(ITCPConnectionEventListener eventListener) {
+ this.eventListener = eventListener;
+ }
+
+ public Object getAttachment() {
+ return attachment;
+ }
+
+ public void setAttachment(Object attachment) {
+ this.attachment = attachment;
+ }
+
+ public void close() {
+ key.cancel();
+ try {
+ channel.close();
+ } catch (IOException e) {
+ e.printStackTrace();
+ }
+ }
+}
\ No newline at end of file
diff --git a/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/tcp/TCPEndpoint.java b/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/tcp/TCPEndpoint.java
new file mode 100644
index 0000000..d73ba21
--- /dev/null
+++ b/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/tcp/TCPEndpoint.java
@@ -0,0 +1,198 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ *
+ * 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 edu.uci.ics.hyracks.net.protocols.tcp;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.net.ServerSocket;
+import java.nio.channels.SelectableChannel;
+import java.nio.channels.SelectionKey;
+import java.nio.channels.Selector;
+import java.nio.channels.ServerSocketChannel;
+import java.nio.channels.SocketChannel;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+
+public class TCPEndpoint {
+ private final ITCPConnectionListener connectionListener;
+
+ private final int nThreads;
+
+ private ServerSocketChannel serverSocketChannel;
+
+ private InetSocketAddress localAddress;
+
+ private IOThread[] ioThreads;
+
+ private int nextThread;
+
+ public TCPEndpoint(ITCPConnectionListener connectionListener, int nThreads) {
+ this.connectionListener = connectionListener;
+ this.nThreads = nThreads;
+ }
+
+ public void start(InetSocketAddress localAddress) throws IOException {
+ serverSocketChannel = ServerSocketChannel.open();
+ ServerSocket serverSocket = serverSocketChannel.socket();
+ serverSocket.bind(localAddress);
+ this.localAddress = (InetSocketAddress) serverSocket.getLocalSocketAddress();
+ ioThreads = new IOThread[nThreads];
+ for (int i = 0; i < ioThreads.length; ++i) {
+ ioThreads[i] = new IOThread();
+ }
+ ioThreads[0].registerServerSocket(serverSocketChannel);
+ for (int i = 0; i < ioThreads.length; ++i) {
+ ioThreads[i].start();
+ }
+ }
+
+ private synchronized int getNextThread() {
+ int result = nextThread;
+ nextThread = (nextThread + 1) % nThreads;
+ return result;
+ }
+
+ public void initiateConnection(InetSocketAddress remoteAddress) {
+ int targetThread = getNextThread();
+ ioThreads[targetThread].initiateConnection(remoteAddress);
+ }
+
+ private void distributeIncomingConnection(SocketChannel channel) {
+ int targetThread = getNextThread();
+ ioThreads[targetThread].addIncomingConnection(channel);
+ }
+
+ public InetSocketAddress getLocalAddress() {
+ return localAddress;
+ }
+
+ private class IOThread extends Thread {
+ private final List<InetSocketAddress> pendingConnections;
+
+ private final List<InetSocketAddress> workingPendingConnections;
+
+ private final List<SocketChannel> incomingConnections;
+
+ private final List<SocketChannel> workingIncomingConnections;
+
+ private Selector selector;
+
+ public IOThread() throws IOException {
+ super("TCPEndpoint IO Thread");
+ setPriority(MAX_PRIORITY);
+ this.pendingConnections = new ArrayList<InetSocketAddress>();
+ this.workingPendingConnections = new ArrayList<InetSocketAddress>();
+ this.incomingConnections = new ArrayList<SocketChannel>();
+ this.workingIncomingConnections = new ArrayList<SocketChannel>();
+ selector = Selector.open();
+ }
+
+ @Override
+ public void run() {
+ while (true) {
+ try {
+ int n = selector.select();
+ collectOutstandingWork();
+ if (!workingPendingConnections.isEmpty()) {
+ for (InetSocketAddress address : workingPendingConnections) {
+ SocketChannel channel = SocketChannel.open();
+ channel.configureBlocking(false);
+ if (!channel.connect(address)) {
+ channel.register(selector, SelectionKey.OP_CONNECT);
+ } else {
+ SelectionKey key = channel.register(selector, 0);
+ createConnection(key, channel);
+ }
+ }
+ workingPendingConnections.clear();
+ }
+ if (!workingIncomingConnections.isEmpty()) {
+ for (SocketChannel channel : workingIncomingConnections) {
+ channel.configureBlocking(false);
+ SelectionKey sKey = channel.register(selector, 0);
+ TCPConnection connection = new TCPConnection(TCPEndpoint.this, channel, sKey, selector);
+ sKey.attach(connection);
+ synchronized (connectionListener) {
+ connectionListener.acceptedConnection(connection);
+ }
+ }
+ workingIncomingConnections.clear();
+ }
+ if (n > 0) {
+ Iterator<SelectionKey> i = selector.selectedKeys().iterator();
+ while (i.hasNext()) {
+ SelectionKey key = i.next();
+ i.remove();
+ SelectableChannel sc = key.channel();
+ boolean readable = key.isReadable();
+ boolean writable = key.isWritable();
+
+ if (readable || writable) {
+ TCPConnection connection = (TCPConnection) key.attachment();
+ connection.getEventListener().notifyIOReady(connection, readable, writable);
+ }
+ if (key.isAcceptable()) {
+ assert sc == serverSocketChannel;
+ SocketChannel channel = serverSocketChannel.accept();
+ distributeIncomingConnection(channel);
+ } else if (key.isConnectable()) {
+ SocketChannel channel = (SocketChannel) sc;
+ if (channel.finishConnect()) {
+ createConnection(key, channel);
+ }
+ }
+ }
+ }
+ } catch (Exception e) {
+ e.printStackTrace();
+ }
+ }
+ }
+
+ private void createConnection(SelectionKey key, SocketChannel channel) {
+ TCPConnection connection = new TCPConnection(TCPEndpoint.this, channel, key, selector);
+ key.attach(connection);
+ key.interestOps(0);
+ connectionListener.connectionEstablished(connection);
+ }
+
+ synchronized void initiateConnection(InetSocketAddress remoteAddress) {
+ pendingConnections.add(remoteAddress);
+ selector.wakeup();
+ }
+
+ synchronized void addIncomingConnection(SocketChannel channel) {
+ incomingConnections.add(channel);
+ selector.wakeup();
+ }
+
+ void registerServerSocket(ServerSocketChannel serverSocketChannel) throws IOException {
+ serverSocketChannel.configureBlocking(false);
+ serverSocketChannel.register(selector, SelectionKey.OP_ACCEPT);
+ }
+
+ private synchronized void collectOutstandingWork() {
+ if (!pendingConnections.isEmpty()) {
+ workingPendingConnections.addAll(pendingConnections);
+ pendingConnections.clear();
+ }
+ if (!incomingConnections.isEmpty()) {
+ workingIncomingConnections.addAll(incomingConnections);
+ incomingConnections.clear();
+ }
+ }
+ }
+}
\ No newline at end of file
diff --git a/hyracks-net/src/test/java/edu/uci/ics/hyracks/net/tests/NetTest.java b/hyracks-net/src/test/java/edu/uci/ics/hyracks/net/tests/NetTest.java
new file mode 100644
index 0000000..31bc2df
--- /dev/null
+++ b/hyracks-net/src/test/java/edu/uci/ics/hyracks/net/tests/NetTest.java
@@ -0,0 +1,213 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ *
+ * 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 edu.uci.ics.hyracks.net.tests;
+
+import java.net.InetSocketAddress;
+import java.nio.ByteBuffer;
+import java.util.LinkedList;
+import java.util.Queue;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import junit.framework.Assert;
+
+import org.junit.Test;
+
+import edu.uci.ics.hyracks.net.buffers.IBufferAcceptor;
+import edu.uci.ics.hyracks.net.buffers.ICloseableBufferAcceptor;
+import edu.uci.ics.hyracks.net.protocols.muxdemux.ChannelControlBlock;
+import edu.uci.ics.hyracks.net.protocols.muxdemux.IChannelOpenListener;
+import edu.uci.ics.hyracks.net.protocols.muxdemux.MultiplexedConnection;
+import edu.uci.ics.hyracks.net.protocols.muxdemux.MuxDemux;
+
+public class NetTest {
+ @Test
+ public void test() throws Exception {
+ AtomicBoolean failFlag = new AtomicBoolean();
+
+ MuxDemux md1 = createMuxDemux("md1", failFlag);
+ md1.start();
+ MuxDemux md2 = createMuxDemux("md2", failFlag);
+ md2.start();
+ InetSocketAddress md2Address = md2.getLocalAddress();
+
+ MultiplexedConnection md1md2 = md1.connect(md2Address);
+
+ Thread t1 = createThread(md1md2, 1);
+ Thread t2 = createThread(md1md2, -1);
+ t1.start();
+ t2.start();
+
+ t1.join();
+ t2.join();
+
+ Assert.assertFalse("Failure flag was set to true", failFlag.get());
+ }
+
+ private Thread createThread(final MultiplexedConnection md1md2, final int factor) {
+ return new Thread() {
+ @Override
+ public void run() {
+ try {
+ ChannelControlBlock md1md2c1 = md1md2.openChannel();
+
+ final Semaphore sem = new Semaphore(1);
+ sem.acquire();
+ md1md2c1.getWriteInterface().setEmptyBufferAcceptor(new IBufferAcceptor() {
+ @Override
+ public void accept(ByteBuffer buffer) {
+ }
+ });
+
+ md1md2c1.getReadInterface().setFullBufferAcceptor(new ICloseableBufferAcceptor() {
+ @Override
+ public void accept(ByteBuffer buffer) {
+ }
+
+ @Override
+ public void error(int ecode) {
+ }
+
+ @Override
+ public void close() {
+ sem.release();
+ }
+ });
+
+ ICloseableBufferAcceptor fba = md1md2c1.getWriteInterface().getFullBufferAcceptor();
+ for (int i = 0; i < 10000; ++i) {
+ ByteBuffer buffer = ByteBuffer.allocate(1024);
+ for (int j = 0; j < 256; ++j) {
+ buffer.putInt(factor * (i + j));
+ }
+ buffer.flip();
+ fba.accept(buffer);
+ }
+ fba.close();
+ sem.acquire();
+ } catch (Exception e) {
+ e.printStackTrace();
+ }
+ }
+ };
+
+ }
+
+ private MuxDemux createMuxDemux(final String label, final AtomicBoolean failFlag) {
+ IChannelOpenListener md1OpenListener = new IChannelOpenListener() {
+ @Override
+ public void channelOpened(final ChannelControlBlock channel) {
+ final ChannelIO cio = new ChannelIO(label, channel);
+ channel.getReadInterface().setFullBufferAcceptor(cio.rifba);
+ channel.getWriteInterface().setEmptyBufferAcceptor(cio.wieba);
+
+ final IBufferAcceptor rieba = channel.getReadInterface().getEmptyBufferAcceptor();
+ for (int i = 0; i < 50; ++i) {
+ rieba.accept(ByteBuffer.allocate(1024));
+ }
+ new Thread() {
+ private int prevTotal = 0;
+
+ @Override
+ public void run() {
+ while (true) {
+ ByteBuffer fbuf = null;
+ synchronized (channel) {
+ while (!cio.eos && cio.ecode == 0 && cio.rifq.isEmpty()) {
+ try {
+ channel.wait();
+ } catch (InterruptedException e) {
+ e.printStackTrace();
+ }
+ }
+ if (!cio.rifq.isEmpty()) {
+ fbuf = cio.rifq.poll();
+ } else if (cio.ecode != 0) {
+ throw new RuntimeException("Error: " + cio.ecode);
+ } else if (cio.eos) {
+ channel.getWriteInterface().getFullBufferAcceptor().close();
+ return;
+ }
+ }
+ int counter = 0;
+ while (fbuf.remaining() > 0) {
+ counter += fbuf.getInt();
+ }
+ if (prevTotal != 0) {
+ if (Math.abs(counter - prevTotal) != 256) {
+ failFlag.set(true);
+ }
+ }
+ prevTotal = counter;
+ fbuf.compact();
+ rieba.accept(fbuf);
+ }
+ }
+ }.start();
+ }
+ };
+ return new MuxDemux(new InetSocketAddress("127.0.0.1", 0), md1OpenListener, 1);
+ }
+
+ private class ChannelIO {
+ private ChannelControlBlock channel;
+
+ private Queue<ByteBuffer> rifq;
+
+ private Queue<ByteBuffer> wieq;
+
+ private boolean eos;
+
+ private int ecode;
+
+ private ICloseableBufferAcceptor rifba;
+
+ private IBufferAcceptor wieba;
+
+ public ChannelIO(final String label, ChannelControlBlock channel) {
+ this.channel = channel;
+ this.rifq = new LinkedList<ByteBuffer>();
+ this.wieq = new LinkedList<ByteBuffer>();
+
+ rifba = new ICloseableBufferAcceptor() {
+ @Override
+ public void accept(ByteBuffer buffer) {
+ rifq.add(buffer);
+ ChannelIO.this.channel.notifyAll();
+ }
+
+ @Override
+ public void error(int ecode) {
+ ChannelIO.this.ecode = ecode;
+ ChannelIO.this.channel.notifyAll();
+ }
+
+ @Override
+ public void close() {
+ eos = true;
+ ChannelIO.this.channel.notifyAll();
+ }
+ };
+
+ wieba = new IBufferAcceptor() {
+ @Override
+ public void accept(ByteBuffer buffer) {
+ wieq.add(buffer);
+ ChannelIO.this.channel.notifyAll();
+ }
+ };
+ }
+ }
+}
\ No newline at end of file
diff --git a/pom.xml b/pom.xml
index 890f42d..0ab0646 100644
--- a/pom.xml
+++ b/pom.xml
@@ -25,7 +25,7 @@
<artifactId>maven-surefire-plugin</artifactId>
<configuration>
<forkMode>pertest</forkMode>
- <argLine>-enableassertions -Djava.util.logging.config.file=${user.home}/logging.properties</argLine>
+ <argLine>-enableassertions -Djava.util.logging.config.file=${user.home}/logging.properties -Xdebug -Xrunjdwp:transport=dt_socket,server=y,address=8000,suspend=n</argLine>
</configuration>
</plugin>
</plugins>
@@ -86,6 +86,7 @@
<module>hyracks-dataflow-hadoop</module>
<module>hyracks-control-common</module>
<module>hyracks-control-cc</module>
+ <module>hyracks-net</module>
<module>hyracks-control-nc</module>
<module>hyracks-data</module>
<module>hyracks-cli</module>