Added application support. Added examples. Added CLI
git-svn-id: https://hyracks.googlecode.com/svn/trunk@57 123451ca-8445-de46-9d55-352943316053
diff --git a/hyracks/hyracks-api/pom.xml b/hyracks/hyracks-api/pom.xml
index 9bec30c..bd02957 100644
--- a/hyracks/hyracks-api/pom.xml
+++ b/hyracks/hyracks-api/pom.xml
@@ -24,5 +24,19 @@
<type>jar</type>
<scope>compile</scope>
</dependency>
+ <dependency>
+ <groupId>org.apache.httpcomponents</groupId>
+ <artifactId>httpclient</artifactId>
+ <version>4.1-alpha2</version>
+ <type>jar</type>
+ <scope>compile</scope>
+ </dependency>
+ <dependency>
+ <groupId>args4j</groupId>
+ <artifactId>args4j</artifactId>
+ <version>2.0.12</version>
+ <type>jar</type>
+ <scope>compile</scope>
+ </dependency>
</dependencies>
</project>
diff --git a/hyracks/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/service/ILifeCycle.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/application/IApplicationContext.java
similarity index 78%
copy from hyracks/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/service/ILifeCycle.java
copy to hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/application/IApplicationContext.java
index c9f5abb..4ba5869 100644
--- a/hyracks/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/service/ILifeCycle.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/application/IApplicationContext.java
@@ -12,10 +12,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package edu.uci.ics.hyracks.control.common.service;
+package edu.uci.ics.hyracks.api.application;
-public interface ILifeCycle {
- public void start() throws Exception;
-
- public void stop() throws Exception;
+public interface IApplicationContext {
}
\ No newline at end of file
diff --git a/hyracks/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/service/ILifeCycle.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/application/IBootstrap.java
similarity index 82%
rename from hyracks/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/service/ILifeCycle.java
rename to hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/application/IBootstrap.java
index c9f5abb..ceae504 100644
--- a/hyracks/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/service/ILifeCycle.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/application/IBootstrap.java
@@ -12,10 +12,12 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package edu.uci.ics.hyracks.control.common.service;
+package edu.uci.ics.hyracks.api.application;
-public interface ILifeCycle {
+public interface IBootstrap {
public void start() throws Exception;
public void stop() throws Exception;
+
+ public void setApplicationContext(IApplicationContext appCtx);
}
\ No newline at end of file
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/AbstractHyracksConnection.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/AbstractHyracksConnection.java
index 2bb44e0..5c17085 100644
--- a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/AbstractHyracksConnection.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/AbstractHyracksConnection.java
@@ -14,30 +14,53 @@
*/
package edu.uci.ics.hyracks.api.client;
+import java.io.ByteArrayOutputStream;
+import java.io.File;
+import java.io.IOException;
+import java.io.ObjectOutputStream;
import java.net.InetAddress;
import java.util.EnumSet;
import java.util.Map;
import java.util.UUID;
+import org.apache.http.HttpResponse;
+import org.apache.http.client.HttpClient;
+import org.apache.http.client.methods.HttpPut;
+import org.apache.http.entity.FileEntity;
+import org.apache.http.impl.client.DefaultHttpClient;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksException;
import edu.uci.ics.hyracks.api.job.JobFlag;
import edu.uci.ics.hyracks.api.job.JobSpecification;
import edu.uci.ics.hyracks.api.job.JobStatus;
import edu.uci.ics.hyracks.api.job.statistics.JobStatistics;
abstract class AbstractHyracksConnection implements IHyracksClientConnection {
- private IHyracksClientInterface hci;
+ private final String ccHost;
- public AbstractHyracksConnection(IHyracksClientInterface hci) {
+ private final IHyracksClientInterface hci;
+
+ private final ClusterControllerInfo ccInfo;
+
+ public AbstractHyracksConnection(String ccHost, IHyracksClientInterface hci) throws Exception {
+ this.ccHost = ccHost;
this.hci = hci;
+ ccInfo = hci.getClusterControllerInfo();
}
@Override
- public void createApplication(String appName) throws Exception {
+ public void createApplication(String appName, File harFile) throws Exception {
hci.createApplication(appName);
- }
-
- @Override
- public void startApplication(String appName) throws Exception {
+ if (harFile != null) {
+ HttpClient hc = new DefaultHttpClient();
+ HttpPut put = new HttpPut("http://" + ccHost + ":" + ccInfo.getWebPort() + "/applications/" + appName);
+ put.setEntity(new FileEntity(harFile, "application/octet-stream"));
+ HttpResponse response = hc.execute(put);
+ if (response.getStatusLine().getStatusCode() != 200) {
+ hci.destroyApplication(appName);
+ throw new HyracksException(response.getStatusLine().toString());
+ }
+ }
hci.startApplication(appName);
}
@@ -47,13 +70,20 @@
}
@Override
- public UUID createJob(JobSpecification jobSpec) throws Exception {
- return hci.createJob(jobSpec);
+ public UUID createJob(String appName, JobSpecification jobSpec) throws Exception {
+ return createJob(appName, jobSpec, EnumSet.noneOf(JobFlag.class));
}
@Override
- public UUID createJob(JobSpecification jobSpec, EnumSet<JobFlag> jobFlags) throws Exception {
- return hci.createJob(jobSpec, jobFlags);
+ public UUID createJob(String appName, JobSpecification jobSpec, EnumSet<JobFlag> jobFlags) throws Exception {
+ return hci.createJob(appName, serialize(jobSpec), jobFlags);
+ }
+
+ private byte[] serialize(JobSpecification jobSpec) throws IOException {
+ ByteArrayOutputStream baos = new ByteArrayOutputStream();
+ ObjectOutputStream oos = new ObjectOutputStream(baos);
+ oos.writeObject(jobSpec);
+ return baos.toByteArray();
}
@Override
diff --git a/hyracks/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/NodeCapability.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/ClusterControllerInfo.java
similarity index 70%
copy from hyracks/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/NodeCapability.java
copy to hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/ClusterControllerInfo.java
index d78bd74..315f9dc 100644
--- a/hyracks/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/NodeCapability.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/ClusterControllerInfo.java
@@ -12,20 +12,23 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package edu.uci.ics.hyracks.control.common;
+package edu.uci.ics.hyracks.api.client;
import java.io.Serializable;
-public class NodeCapability implements Serializable {
+public class ClusterControllerInfo implements Serializable {
private static final long serialVersionUID = 1L;
- private int cpuCount;
+ private int webPort;
- public int getCPUCount() {
- return cpuCount;
+ public ClusterControllerInfo() {
}
- public void setCPUCount(int cpuCount) {
- this.cpuCount = cpuCount;
+ public void setWebPort(int webPort) {
+ this.webPort = webPort;
+ }
+
+ public int getWebPort() {
+ return webPort;
}
}
\ No newline at end of file
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/HyracksLocalConnection.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/HyracksLocalConnection.java
index df1ff2b..b41e92c 100644
--- a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/HyracksLocalConnection.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/HyracksLocalConnection.java
@@ -15,7 +15,7 @@
package edu.uci.ics.hyracks.api.client;
public final class HyracksLocalConnection extends AbstractHyracksConnection {
- public HyracksLocalConnection(IHyracksClientInterface hci) {
- super(hci);
+ public HyracksLocalConnection(IHyracksClientInterface hci) throws Exception {
+ super("localhost", hci);
}
}
\ No newline at end of file
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/HyracksRMIConnection.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/HyracksRMIConnection.java
index 18c354b..930126d 100644
--- a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/HyracksRMIConnection.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/HyracksRMIConnection.java
@@ -20,8 +20,8 @@
import java.rmi.registry.Registry;
public final class HyracksRMIConnection extends AbstractHyracksConnection {
- public HyracksRMIConnection(String host, int port) throws RemoteException, NotBoundException {
- super(lookupHCI(host, port));
+ public HyracksRMIConnection(String host, int port) throws Exception {
+ super(host, lookupHCI(host, port));
}
private static IHyracksClientInterface lookupHCI(String host, int port) throws RemoteException, NotBoundException {
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/IHyracksClientConnection.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/IHyracksClientConnection.java
index ef11fd0..11978e3 100644
--- a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/IHyracksClientConnection.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/IHyracksClientConnection.java
@@ -14,6 +14,7 @@
*/
package edu.uci.ics.hyracks.api.client;
+import java.io.File;
import java.net.InetAddress;
import java.util.EnumSet;
import java.util.Map;
@@ -25,15 +26,13 @@
import edu.uci.ics.hyracks.api.job.statistics.JobStatistics;
public interface IHyracksClientConnection {
- public void createApplication(String appName) throws Exception;
-
- public void startApplication(String appName) throws Exception;
+ public void createApplication(String appName, File harFile) throws Exception;
public void destroyApplication(String appName) throws Exception;
- public UUID createJob(JobSpecification jobSpec) throws Exception;
+ public UUID createJob(String appName, JobSpecification jobSpec) throws Exception;
- public UUID createJob(JobSpecification jobSpec, EnumSet<JobFlag> jobFlags) throws Exception;
+ public UUID createJob(String appName, JobSpecification jobSpec, EnumSet<JobFlag> jobFlags) throws Exception;
public JobStatus getJobStatus(UUID jobId) throws Exception;
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/IHyracksClientInterface.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/IHyracksClientInterface.java
index 229299f..fcd114a 100644
--- a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/IHyracksClientInterface.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/IHyracksClientInterface.java
@@ -21,20 +21,19 @@
import java.util.UUID;
import edu.uci.ics.hyracks.api.job.JobFlag;
-import edu.uci.ics.hyracks.api.job.JobSpecification;
import edu.uci.ics.hyracks.api.job.JobStatus;
import edu.uci.ics.hyracks.api.job.statistics.JobStatistics;
public interface IHyracksClientInterface extends Remote {
+ public ClusterControllerInfo getClusterControllerInfo() throws Exception;
+
public void createApplication(String appName) throws Exception;
public void startApplication(String appName) throws Exception;
public void destroyApplication(String appName) throws Exception;
- public UUID createJob(JobSpecification jobSpec) throws Exception;
-
- public UUID createJob(JobSpecification jobSpec, EnumSet<JobFlag> jobFlags) throws Exception;
+ public UUID createJob(String appName, byte[] jobSpec, EnumSet<JobFlag> jobFlags) throws Exception;
public JobStatus getJobStatus(UUID jobId) throws Exception;
diff --git a/hyracks/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/comm/Endpoint.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/Endpoint.java
similarity index 96%
rename from hyracks/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/comm/Endpoint.java
rename to hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/Endpoint.java
index d879d99..c0d23c1 100644
--- a/hyracks/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/comm/Endpoint.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/Endpoint.java
@@ -12,7 +12,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package edu.uci.ics.hyracks.control.common.comm;
+package edu.uci.ics.hyracks.api.comm;
import java.io.Serializable;
import java.util.UUID;
diff --git a/hyracks/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/comm/NetworkAddress.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/NetworkAddress.java
similarity index 96%
rename from hyracks/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/comm/NetworkAddress.java
rename to hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/NetworkAddress.java
index 0b7ed04..868221d 100644
--- a/hyracks/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/comm/NetworkAddress.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/NetworkAddress.java
@@ -12,7 +12,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package edu.uci.ics.hyracks.control.common.comm;
+package edu.uci.ics.hyracks.api.comm;
import java.io.Serializable;
import java.net.InetAddress;
diff --git a/hyracks/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/api/CCConfig.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/control/CCConfig.java
similarity index 95%
rename from hyracks/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/api/CCConfig.java
rename to hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/control/CCConfig.java
index d4106cb..f965278 100644
--- a/hyracks/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/api/CCConfig.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/control/CCConfig.java
@@ -12,7 +12,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package edu.uci.ics.hyracks.control.common.api;
+package edu.uci.ics.hyracks.api.control;
import org.kohsuke.args4j.Option;
@@ -20,7 +20,7 @@
@Option(name = "-port", usage = "Sets the port to listen for connections from node controllers (default 1099)")
public int port = 1099;
- @Option(name = "-http-port", usage = "Sets the http port for the admin console")
+ @Option(name = "-http-port", usage = "Sets the http port for the Cluster Controller")
public int httpPort;
@Option(name = "-heartbeat-period", usage = "Sets the time duration between two heartbeats from each node controller in milliseconds (default: 10000)")
diff --git a/hyracks/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/api/IClusterController.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/control/IClusterController.java
similarity index 83%
rename from hyracks/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/api/IClusterController.java
rename to hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/control/IClusterController.java
index 3b6efdd..3b6e8aa 100644
--- a/hyracks/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/api/IClusterController.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/control/IClusterController.java
@@ -12,16 +12,14 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package edu.uci.ics.hyracks.control.common.api;
+package edu.uci.ics.hyracks.api.control;
import java.rmi.Remote;
import java.util.UUID;
-import edu.uci.ics.hyracks.api.client.IHyracksClientInterface;
import edu.uci.ics.hyracks.api.job.statistics.StageletStatistics;
-import edu.uci.ics.hyracks.control.common.NodeParameters;
-public interface IClusterController extends Remote, IHyracksClientInterface {
+public interface IClusterController extends Remote {
public NodeParameters registerNode(INodeController nodeController) throws Exception;
public void unregisterNode(INodeController nodeController) throws Exception;
diff --git a/hyracks/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/api/INodeController.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/control/INodeController.java
similarity index 68%
rename from hyracks/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/api/INodeController.java
rename to hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/control/INodeController.java
index 31e2716..cd8c067 100644
--- a/hyracks/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/api/INodeController.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/control/INodeController.java
@@ -12,19 +12,17 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package edu.uci.ics.hyracks.control.common.api;
+package edu.uci.ics.hyracks.api.control;
import java.rmi.Remote;
import java.util.Map;
import java.util.Set;
import java.util.UUID;
+import edu.uci.ics.hyracks.api.comm.Endpoint;
import edu.uci.ics.hyracks.api.dataflow.ActivityNodeId;
import edu.uci.ics.hyracks.api.dataflow.OperatorDescriptorId;
import edu.uci.ics.hyracks.api.dataflow.PortInstanceId;
-import edu.uci.ics.hyracks.control.common.NodeCapability;
-import edu.uci.ics.hyracks.control.common.comm.Endpoint;
-import edu.uci.ics.hyracks.control.common.job.JobPlan;
public interface INodeController extends Remote {
public String getId() throws Exception;
@@ -33,12 +31,13 @@
public NodeCapability getNodeCapability() throws Exception;
- public Map<PortInstanceId, Endpoint> initializeJobletPhase1(UUID jobId, JobPlan plan, UUID stageId, int attempt,
- Map<ActivityNodeId, Set<Integer>> tasks, Map<OperatorDescriptorId, Set<Integer>> opPartitions) throws Exception;
+ public Map<PortInstanceId, Endpoint> initializeJobletPhase1(String appName, UUID jobId, byte[] plan, UUID stageId,
+ int attempt, Map<ActivityNodeId, Set<Integer>> tasks, Map<OperatorDescriptorId, Set<Integer>> opPartitions)
+ throws Exception;
- public void initializeJobletPhase2(UUID jobId, JobPlan plan, UUID stageId, Map<ActivityNodeId, Set<Integer>> tasks,
- Map<OperatorDescriptorId, Set<Integer>> opPartitions, Map<PortInstanceId, Endpoint> globalPortMap)
- throws Exception;
+ public void initializeJobletPhase2(String appName, UUID jobId, byte[] plan, UUID stageId,
+ Map<ActivityNodeId, Set<Integer>> tasks, Map<OperatorDescriptorId, Set<Integer>> opPartitions,
+ Map<PortInstanceId, Endpoint> globalPortMap) throws Exception;
public void commitJobletInitialization(UUID jobId, UUID stageId) throws Exception;
@@ -49,4 +48,8 @@
public void startStage(UUID jobId, UUID stageId) throws Exception;
public void notifyRegistration(IClusterController ccs) throws Exception;
+
+ public void createApplication(String appName, boolean deployHar) throws Exception;
+
+ public void destroyApplication(String appName) throws Exception;
}
\ No newline at end of file
diff --git a/hyracks/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/api/NCConfig.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/control/NCConfig.java
similarity index 97%
rename from hyracks/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/api/NCConfig.java
rename to hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/control/NCConfig.java
index 0037117..8d1920f 100644
--- a/hyracks/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/api/NCConfig.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/control/NCConfig.java
@@ -12,7 +12,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package edu.uci.ics.hyracks.control.common.api;
+package edu.uci.ics.hyracks.api.control;
import java.io.Serializable;
diff --git a/hyracks/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/NodeCapability.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/control/NodeCapability.java
similarity index 95%
rename from hyracks/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/NodeCapability.java
rename to hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/control/NodeCapability.java
index d78bd74..88efc1f 100644
--- a/hyracks/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/NodeCapability.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/control/NodeCapability.java
@@ -12,7 +12,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package edu.uci.ics.hyracks.control.common;
+package edu.uci.ics.hyracks.api.control;
import java.io.Serializable;
diff --git a/hyracks/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/NodeParameters.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/control/NodeParameters.java
similarity index 73%
rename from hyracks/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/NodeParameters.java
rename to hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/control/NodeParameters.java
index 7e6f3e9..f21a375 100644
--- a/hyracks/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/NodeParameters.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/control/NodeParameters.java
@@ -12,15 +12,27 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package edu.uci.ics.hyracks.control.common;
+package edu.uci.ics.hyracks.api.control;
import java.io.Serializable;
+import edu.uci.ics.hyracks.api.client.ClusterControllerInfo;
+
public class NodeParameters implements Serializable {
private static final long serialVersionUID = 1L;
+ private ClusterControllerInfo ccInfo;
+
private int heartbeatPeriod;
+ public ClusterControllerInfo getClusterControllerInfo() {
+ return ccInfo;
+ }
+
+ public void setClusterControllerInfo(ClusterControllerInfo ccInfo) {
+ this.ccInfo = ccInfo;
+ }
+
public int getHeartbeatPeriod() {
return heartbeatPeriod;
}
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/exceptions/HyracksException.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/exceptions/HyracksException.java
new file mode 100644
index 0000000..450708b
--- /dev/null
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/exceptions/HyracksException.java
@@ -0,0 +1,20 @@
+package edu.uci.ics.hyracks.api.exceptions;
+
+public class HyracksException extends Exception {
+ private static final long serialVersionUID = 1L;
+
+ public HyracksException() {
+ }
+
+ public HyracksException(String message) {
+ super(message);
+ }
+
+ public HyracksException(Throwable cause) {
+ super(cause);
+ }
+
+ public HyracksException(String message, Throwable cause) {
+ super(message, cause);
+ }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/JobPlan.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/JobPlan.java
similarity index 98%
rename from hyracks/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/JobPlan.java
rename to hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/JobPlan.java
index 016a6d4..5d8ef1d 100644
--- a/hyracks/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/JobPlan.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/JobPlan.java
@@ -12,7 +12,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package edu.uci.ics.hyracks.control.common.job;
+package edu.uci.ics.hyracks.api.job;
import java.io.Serializable;
import java.util.ArrayList;
diff --git a/hyracks/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/JobStage.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/JobStage.java
similarity index 97%
rename from hyracks/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/JobStage.java
rename to hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/JobStage.java
index 2514866..41187f1 100644
--- a/hyracks/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/JobStage.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/JobStage.java
@@ -12,7 +12,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package edu.uci.ics.hyracks.control.common.job;
+package edu.uci.ics.hyracks.api.job;
import java.io.Serializable;
import java.util.HashSet;
diff --git a/hyracks/hyracks-cli/.classpath b/hyracks/hyracks-cli/.classpath
new file mode 100644
index 0000000..ba0bb5a
--- /dev/null
+++ b/hyracks/hyracks-cli/.classpath
@@ -0,0 +1,8 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<classpath>
+ <classpathentry kind="src" output="target/classes" path="src/main/java"/>
+ <classpathentry kind="src" path="target/generated-sources/javacc"/>
+ <classpathentry kind="con" path="org.eclipse.jdt.launching.JRE_CONTAINER/org.eclipse.jdt.internal.debug.ui.launcher.StandardVMType/JavaSE-1.6"/>
+ <classpathentry kind="con" path="org.maven.ide.eclipse.MAVEN2_CLASSPATH_CONTAINER"/>
+ <classpathentry kind="output" path="target/classes"/>
+</classpath>
diff --git a/hyracks/hyracks-cli/.project b/hyracks/hyracks-cli/.project
new file mode 100644
index 0000000..9741f63
--- /dev/null
+++ b/hyracks/hyracks-cli/.project
@@ -0,0 +1,23 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<projectDescription>
+ <name>hyracks-cli</name>
+ <comment></comment>
+ <projects>
+ </projects>
+ <buildSpec>
+ <buildCommand>
+ <name>org.eclipse.jdt.core.javabuilder</name>
+ <arguments>
+ </arguments>
+ </buildCommand>
+ <buildCommand>
+ <name>org.maven.ide.eclipse.maven2Builder</name>
+ <arguments>
+ </arguments>
+ </buildCommand>
+ </buildSpec>
+ <natures>
+ <nature>org.maven.ide.eclipse.maven2Nature</nature>
+ <nature>org.eclipse.jdt.core.javanature</nature>
+ </natures>
+</projectDescription>
diff --git a/hyracks/hyracks-cli/.settings/org.eclipse.jdt.core.prefs b/hyracks/hyracks-cli/.settings/org.eclipse.jdt.core.prefs
new file mode 100644
index 0000000..f7947fa
--- /dev/null
+++ b/hyracks/hyracks-cli/.settings/org.eclipse.jdt.core.prefs
@@ -0,0 +1,6 @@
+#Thu Aug 05 10:16:23 PDT 2010
+eclipse.preferences.version=1
+org.eclipse.jdt.core.compiler.codegen.targetPlatform=1.6
+org.eclipse.jdt.core.compiler.compliance=1.6
+org.eclipse.jdt.core.compiler.problem.forbiddenReference=warning
+org.eclipse.jdt.core.compiler.source=1.6
diff --git a/hyracks/hyracks-cli/.settings/org.maven.ide.eclipse.prefs b/hyracks/hyracks-cli/.settings/org.maven.ide.eclipse.prefs
new file mode 100644
index 0000000..a4b7f25
--- /dev/null
+++ b/hyracks/hyracks-cli/.settings/org.maven.ide.eclipse.prefs
@@ -0,0 +1,9 @@
+#Thu Aug 05 10:16:22 PDT 2010
+activeProfiles=
+eclipse.preferences.version=1
+fullBuildGoals=process-test-resources
+includeModules=false
+resolveWorkspaceProjects=true
+resourceFilterGoals=process-resources resources\:testResources
+skipCompilerPlugin=true
+version=1
diff --git a/hyracks/hyracks-cli/pom.xml b/hyracks/hyracks-cli/pom.xml
new file mode 100644
index 0000000..a0adab9
--- /dev/null
+++ b/hyracks/hyracks-cli/pom.xml
@@ -0,0 +1,72 @@
+<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-cli</artifactId>
+ <version>0.1.0</version>
+ <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>
+ <plugin>
+ <groupId>org.codehaus.mojo</groupId>
+ <artifactId>javacc-maven-plugin</artifactId>
+ <version>2.6</version>
+ <executions>
+ <execution>
+ <id>javacc</id>
+ <goals>
+ <goal>javacc</goal>
+ </goals>
+ <configuration>
+ <isStatic>false</isStatic>
+ </configuration>
+ </execution>
+ </executions>
+ </plugin>
+ <plugin>
+ <groupId>org.codehaus.mojo</groupId>
+ <artifactId>appassembler-maven-plugin</artifactId>
+ <executions>
+ <execution>
+ <configuration>
+ <programs>
+ <program>
+ <mainClass>edu.uci.ics.hyracks.cli.Main</mainClass>
+ <name>hyrackscli</name>
+ </program>
+ </programs>
+ <repositoryLayout>flat</repositoryLayout>
+ <repositoryName>lib</repositoryName>
+ </configuration>
+ <phase>package</phase>
+ <goals>
+ <goal>assemble</goal>
+ </goals>
+ </execution>
+ </executions>
+ </plugin>
+ </plugins>
+ </build>
+ <dependencies>
+ <dependency>
+ <groupId>jline</groupId>
+ <artifactId>jline</artifactId>
+ <version>0.9.94</version>
+ <type>jar</type>
+ <scope>compile</scope>
+ </dependency>
+ <dependency>
+ <groupId>edu.uci.ics.hyracks</groupId>
+ <artifactId>hyracks-api</artifactId>
+ <version>0.1.0</version>
+ <scope>compile</scope>
+ </dependency>
+ </dependencies>
+</project>
diff --git a/hyracks/hyracks-cli/src/main/java/edu/uci/ics/hyracks/cli/CLI.java b/hyracks/hyracks-cli/src/main/java/edu/uci/ics/hyracks/cli/CLI.java
new file mode 100644
index 0000000..73f4bc9
--- /dev/null
+++ b/hyracks/hyracks-cli/src/main/java/edu/uci/ics/hyracks/cli/CLI.java
@@ -0,0 +1,83 @@
+/*
+ * 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.cli;
+
+import java.io.IOException;
+import java.io.StringReader;
+import java.util.List;
+
+import jline.ConsoleReader;
+import edu.uci.ics.hyracks.api.client.IHyracksClientConnection;
+import edu.uci.ics.hyracks.cli.commands.Command;
+
+public class CLI {
+ private static final String HYRACKS_PROMPT = "hyracks> ";
+ private static final String HYRAX_CONTINUE_PROMPT = "> ";
+ private final ConsoleReader reader;
+ private IHyracksClientConnection connection;
+
+ public CLI(String[] args) throws IOException {
+ reader = new ConsoleReader();
+ connection = null;
+ }
+
+ public void run() throws IOException {
+ boolean eof = false;
+ while (true) {
+ String prompt = HYRACKS_PROMPT;
+ StringBuffer command = new StringBuffer();
+ while (true) {
+ String line = reader.readLine(prompt);
+ if (line == null) {
+ eof = true;
+ break;
+ }
+ prompt = HYRAX_CONTINUE_PROMPT;
+ line = line.trim();
+ command.append(line);
+ if ("".equals(line)) {
+ break;
+ }
+ if (line.endsWith(";")) {
+ break;
+ }
+ }
+ if (eof) {
+ break;
+ }
+ try {
+ execute(command);
+ } catch (Exception e) {
+ e.printStackTrace();
+ }
+ }
+ }
+
+ private void execute(StringBuffer command) throws Exception {
+ CLIParser parser = new CLIParser(new StringReader(command.toString()));
+ List<Command> cmds = parser.Commands();
+ for (Command cmd : cmds) {
+ cmd.run(this);
+ }
+ }
+
+ public void setConnection(IHyracksClientConnection connection) {
+ this.connection = connection;
+ }
+
+ public IHyracksClientConnection getConnection() {
+ return connection;
+ }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/service/ILifeCycle.java b/hyracks/hyracks-cli/src/main/java/edu/uci/ics/hyracks/cli/Main.java
similarity index 78%
copy from hyracks/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/service/ILifeCycle.java
copy to hyracks/hyracks-cli/src/main/java/edu/uci/ics/hyracks/cli/Main.java
index c9f5abb..5f7ef0b 100644
--- a/hyracks/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/service/ILifeCycle.java
+++ b/hyracks/hyracks-cli/src/main/java/edu/uci/ics/hyracks/cli/Main.java
@@ -12,10 +12,11 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package edu.uci.ics.hyracks.control.common.service;
+package edu.uci.ics.hyracks.cli;
-public interface ILifeCycle {
- public void start() throws Exception;
-
- public void stop() throws Exception;
+public class Main {
+ public static void main(String[] args) throws Exception {
+ CLI cli = new CLI(args);
+ cli.run();
+ }
}
\ No newline at end of file
diff --git a/hyracks/hyracks-cli/src/main/java/edu/uci/ics/hyracks/cli/commands/Command.java b/hyracks/hyracks-cli/src/main/java/edu/uci/ics/hyracks/cli/commands/Command.java
new file mode 100644
index 0000000..4e3fedd
--- /dev/null
+++ b/hyracks/hyracks-cli/src/main/java/edu/uci/ics/hyracks/cli/commands/Command.java
@@ -0,0 +1,7 @@
+package edu.uci.ics.hyracks.cli.commands;
+
+import edu.uci.ics.hyracks.cli.CLI;
+
+public abstract class Command {
+ public abstract void run(CLI cli) throws Exception;
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-cli/src/main/java/edu/uci/ics/hyracks/cli/commands/ConnectCommand.java b/hyracks/hyracks-cli/src/main/java/edu/uci/ics/hyracks/cli/commands/ConnectCommand.java
new file mode 100644
index 0000000..57034be
--- /dev/null
+++ b/hyracks/hyracks-cli/src/main/java/edu/uci/ics/hyracks/cli/commands/ConnectCommand.java
@@ -0,0 +1,28 @@
+package edu.uci.ics.hyracks.cli.commands;
+
+import edu.uci.ics.hyracks.api.client.HyracksRMIConnection;
+import edu.uci.ics.hyracks.api.client.IHyracksClientConnection;
+import edu.uci.ics.hyracks.cli.CLI;
+
+public class ConnectCommand extends Command {
+ private String host;
+
+ private int port;
+
+ public ConnectCommand(String hostPortStr) {
+ int idx = hostPortStr.indexOf(':');
+ host = hostPortStr;
+ port = 1099;
+ if (idx != -1) {
+ host = hostPortStr.substring(0, idx);
+ port = Integer.valueOf(hostPortStr.substring(idx + 1));
+ }
+ }
+
+ @Override
+ public void run(CLI cli) throws Exception {
+ System.err.println("Connecting to host: " + host + ", port: " + port);
+ IHyracksClientConnection conn = new HyracksRMIConnection(host, port);
+ cli.setConnection(conn);
+ }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-cli/src/main/java/edu/uci/ics/hyracks/cli/commands/CreateApplicationCommand.java b/hyracks/hyracks-cli/src/main/java/edu/uci/ics/hyracks/cli/commands/CreateApplicationCommand.java
new file mode 100644
index 0000000..7164e12
--- /dev/null
+++ b/hyracks/hyracks-cli/src/main/java/edu/uci/ics/hyracks/cli/commands/CreateApplicationCommand.java
@@ -0,0 +1,27 @@
+package edu.uci.ics.hyracks.cli.commands;
+
+import java.io.File;
+
+import edu.uci.ics.hyracks.api.client.IHyracksClientConnection;
+import edu.uci.ics.hyracks.cli.CLI;
+
+public class CreateApplicationCommand extends Command {
+ private String appName;
+
+ private File harFile;
+
+ public CreateApplicationCommand(String appName, File harFile) {
+ this.appName = appName;
+ this.harFile = harFile;
+ }
+
+ @Override
+ public void run(CLI cli) throws Exception {
+ IHyracksClientConnection hcc = cli.getConnection();
+ if (hcc == null) {
+ throw new RuntimeException("Not connected to Hyracks Cluster Controller");
+ }
+ System.err.println("Creating application: " + appName + " with har: " + harFile.getAbsolutePath());
+ hcc.createApplication(appName, harFile);
+ }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-cli/src/main/java/edu/uci/ics/hyracks/cli/commands/DestroyApplicationCommand.java b/hyracks/hyracks-cli/src/main/java/edu/uci/ics/hyracks/cli/commands/DestroyApplicationCommand.java
new file mode 100644
index 0000000..2d473dc
--- /dev/null
+++ b/hyracks/hyracks-cli/src/main/java/edu/uci/ics/hyracks/cli/commands/DestroyApplicationCommand.java
@@ -0,0 +1,22 @@
+package edu.uci.ics.hyracks.cli.commands;
+
+import edu.uci.ics.hyracks.api.client.IHyracksClientConnection;
+import edu.uci.ics.hyracks.cli.CLI;
+
+public class DestroyApplicationCommand extends Command {
+ private String appName;
+
+ public DestroyApplicationCommand(String appName) {
+ this.appName = appName;
+ }
+
+ @Override
+ public void run(CLI cli) throws Exception {
+ IHyracksClientConnection hcc = cli.getConnection();
+ if (hcc == null) {
+ throw new RuntimeException("Not connected to Hyracks Cluster Controller");
+ }
+ System.err.println("Destroying application: " + appName);
+ hcc.destroyApplication(appName);
+ }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-cli/src/main/java/edu/uci/ics/hyracks/cli/commands/DisconnectCommand.java b/hyracks/hyracks-cli/src/main/java/edu/uci/ics/hyracks/cli/commands/DisconnectCommand.java
new file mode 100644
index 0000000..460d2c3
--- /dev/null
+++ b/hyracks/hyracks-cli/src/main/java/edu/uci/ics/hyracks/cli/commands/DisconnectCommand.java
@@ -0,0 +1,11 @@
+package edu.uci.ics.hyracks.cli.commands;
+
+import edu.uci.ics.hyracks.cli.CLI;
+
+public class DisconnectCommand extends Command {
+ @Override
+ public void run(CLI cli) throws Exception {
+ System.err.println("Disconnecting...");
+ cli.setConnection(null);
+ }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-cli/src/main/javacc/cli.jj b/hyracks/hyracks-cli/src/main/javacc/cli.jj
new file mode 100644
index 0000000..0f386cb
--- /dev/null
+++ b/hyracks/hyracks-cli/src/main/javacc/cli.jj
@@ -0,0 +1,131 @@
+/*
+ * 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.
+ */
+options {
+ STATIC = false;
+ IGNORE_CASE = true;
+}
+
+PARSER_BEGIN(CLIParser)
+
+package edu.uci.ics.hyracks.cli;
+
+import java.io.*;
+import java.util.*;
+import edu.uci.ics.hyracks.cli.commands.*;
+
+class CLIParser {
+ private String unquote(String s) {
+ return s.substring(1, s.length() - 1);
+ }
+}
+
+PARSER_END(CLIParser)
+
+List<Command> Commands():
+{
+ List<Command> result = new ArrayList<Command>();
+ Command cmd = null;
+} {
+ (
+ (
+ cmd = ConnectCommand()
+ | cmd = DisconnectCommand()
+ | cmd = CreateApplicationCommand()
+ | cmd = DestroyApplicationCommand()
+ ) ";" {
+ result.add(cmd);
+ }
+ )* <EOF> {
+ return result;
+ }
+}
+
+protected Command ConnectCommand():
+{
+ Token t;
+} {
+ "connect" "to" t = <StringLiteral> {
+ return new ConnectCommand(unquote(t.image));
+ }
+}
+
+protected Command CreateApplicationCommand():
+{
+ Token a;
+ Token f;
+} {
+ "create" "application" a = <Identifier> f = <StringLiteral> {
+ return new CreateApplicationCommand(a.image, new File(unquote(f.image)));
+ }
+}
+
+protected Command DestroyApplicationCommand():
+{
+ Token a;
+} {
+ "destroy" "application" a = <Identifier> {
+ return new DestroyApplicationCommand(a.image);
+ }
+}
+
+protected Command DisconnectCommand():
+{
+ Token t;
+} {
+ "disconnect" {
+ return new DisconnectCommand();
+ }
+}
+
+<DEFAULT>
+TOKEN : {
+ <StringLiteral: (("\"" (~["\"", "\n"])* "\"") | ("'" (~["'", "\n"])* "'"))>
+ | <Identifier: <Letter> (<Letter> | <Digit> | <Extender>)*>
+}
+
+SPECIAL_TOKEN :
+{
+ < WhitespaceChar : ["\t", "\r", "\n", " "] >
+}
+
+TOKEN :
+{
+ < #Letter : (<BaseChar> | <Ideographic>) >
+}
+
+TOKEN :
+{
+ < #BaseChar : ["\u0041" - "\u005a", "\u0061" - "\u007a", "\u00c0" - "\u00d6", "\u00d8" - "\u00f6", "\u00f8" - "\u00ff", "\u0100" - "\u0131", "\u0134" - "\u013e", "\u0141" - "\u0148", "\u014a" - "\u017e", "\u0180" - "\u01c3", "\u01cd" - "\u01f0", "\u01f4" - "\u01f5", "\u01fa" - "\u0217", "\u0250" - "\u02a8", "\u02bb" - "\u02c1", "\u0386", "\u0388" - "\u038a", "\u038c", "\u038e" - "\u03a1", "\u03a3" - "\u03ce", "\u03d0" - "\u03d6", "\u03da", "\u03dc", "\u03de", "\u03e0", "\u03e2" - "\u03f3", "\u0401" - "\u040c", "\u040e" - "\u044f", "\u0451" - "\u045c", "\u045e" - "\u0481", "\u0490" - "\u04c4", "\u04c7" - "\u04c8", "\u04cb" - "\u04cc", "\u04d0" - "\u04eb", "\u04ee" - "\u04f5", "\u04f8" - "\u04f9", "\u0531" - "\u0556", "\u0559", "\u0561" - "\u0586", "\u05d0" - "\u05ea", "\u05f0" - "\u05f2", "\u0621" - "\u063a", "\u0641" - "\u064a", "\u0671" - "\u06b7", "\u06ba" - "\u06be", "\u06c0" - "\u06ce", "\u06d0" - "\u06d3", "\u06d5", "\u06e5" - "\u06e6", "\u0905" - "\u0939", "\u093d", "\u0958" - "\u0961", "\u0985" - "\u098c", "\u098f" - "\u0990", "\u0993" - "\u09a8", "\u09aa" - "\u09b0", "\u09b2", "\u09b6" - "\u09b9", "\u09dc" - "\u09dd", "\u09df" - "\u09e1", "\u09f0" - "\u09f1", "\u0a05" - "\u0a0a", "\u0a0f" - "\u0a10", "\u0a13" - "\u0a28", "\u0a2a" - "\u0a30", "\u0a32" - "\u0a33", "\u0a35" - "\u0a36", "\u0a38" - "\u0a39", "\u0a59" - "\u0a5c", "\u0a5e", "\u0a72" - "\u0a74", "\u0a85" - "\u0a8b", "\u0a8d", "\u0a8f" - "\u0a91", "\u0a93" - "\u0aa8", "\u0aaa" - "\u0ab0", "\u0ab2" - "\u0ab3", "\u0ab5" - "\u0ab9", "\u0abd", "\u0ae0", "\u0b05" - "\u0b0c", "\u0b0f" - "\u0b10", "\u0b13" - "\u0b28", "\u0b2a" - "\u0b30", "\u0b32" - "\u0b33", "\u0b36" - "\u0b39", "\u0b3d", "\u0b5c" - "\u0b5d", "\u0b5f" - "\u0b61", "\u0b85" - "\u0b8a", "\u0b8e" - "\u0b90", "\u0b92" - "\u0b95", "\u0b99" - "\u0b9a", "\u0b9c", "\u0b9e" - "\u0b9f", "\u0ba3" - "\u0ba4", "\u0ba8" - "\u0baa", "\u0bae" - "\u0bb5", "\u0bb7" - "\u0bb9", "\u0c05" - "\u0c0c", "\u0c0e" - "\u0c10", "\u0c12" - "\u0c28", "\u0c2a" - "\u0c33", "\u0c35" - "\u0c39", "\u0c60" - "\u0c61", "\u0c85" - "\u0c8c", "\u0c8e" - "\u0c90", "\u0c92" - "\u0ca8", "\u0caa" - "\u0cb3", "\u0cb5" - "\u0cb9", "\u0cde", "\u0ce0" - "\u0ce1", "\u0d05" - "\u0d0c", "\u0d0e" - "\u0d10", "\u0d12" - "\u0d28", "\u0d2a" - "\u0d39", "\u0d60" - "\u0d61", "\u0e01" - "\u0e2e", "\u0e30", "\u0e32" - "\u0e33", "\u0e40" - "\u0e45", "\u0e81" - "\u0e82", "\u0e84", "\u0e87" - "\u0e88", "\u0e8a", "\u0e8d", "\u0e94" - "\u0e97", "\u0e99" - "\u0e9f", "\u0ea1" - "\u0ea3", "\u0ea5", "\u0ea7", "\u0eaa" - "\u0eab", "\u0ead" - "\u0eae", "\u0eb0", "\u0eb2" - "\u0eb3", "\u0ebd", "\u0ec0" - "\u0ec4", "\u0f40" - "\u0f47", "\u0f49" - "\u0f69", "\u10a0" - "\u10c5", "\u10d0" - "\u10f6", "\u1100", "\u1102" - "\u1103", "\u1105" - "\u1107", "\u1109", "\u110b" - "\u110c", "\u110e" - "\u1112", "\u113c", "\u113e", "\u1140", "\u114c", "\u114e", "\u1150", "\u1154" - "\u1155", "\u1159", "\u115f" - "\u1161", "\u1163", "\u1165", "\u1167", "\u1169", "\u116d" - "\u116e", "\u1172" - "\u1173", "\u1175", "\u119e", "\u11a8", "\u11ab", "\u11ae" - "\u11af", "\u11b7" - "\u11b8", "\u11ba", "\u11bc" - "\u11c2", "\u11eb", "\u11f0", "\u11f9", "\u1e00" - "\u1e9b", "\u1ea0" - "\u1ef9", "\u1f00" - "\u1f15", "\u1f18" - "\u1f1d", "\u1f20" - "\u1f45", "\u1f48" - "\u1f4d", "\u1f50" - "\u1f57", "\u1f59", "\u1f5b", "\u1f5d", "\u1f5f" - "\u1f7d", "\u1f80" - "\u1fb4", "\u1fb6" - "\u1fbc", "\u1fbe", "\u1fc2" - "\u1fc4", "\u1fc6" - "\u1fcc", "\u1fd0" - "\u1fd3", "\u1fd6" - "\u1fdb", "\u1fe0" - "\u1fec", "\u1ff2" - "\u1ff4", "\u1ff6" - "\u1ffc", "\u2126", "\u212a" - "\u212b", "\u212e", "\u2180" - "\u2182", "\u3041" - "\u3094", "\u30a1" - "\u30fa", "\u3105" - "\u312c", "\uac00" - "\ud7a3"] >
+}
+
+TOKEN :
+{
+ < #Ideographic : ["\u4e00" - "\u9fa5", "\u3007", "\u3021" - "\u3029"] >
+}
+
+TOKEN :
+{
+ < #CombiningChar : ["\u0300" - "\u0345", "\u0360" - "\u0361", "\u0483" - "\u0486", "\u0591" - "\u05a1", "\u05a3" - "\u05b9", "\u05bb" - "\u05bd", "\u05bf", "\u05c1" - "\u05c2", "\u05c4", "\u064b" - "\u0652", "\u0670", "\u06d6" - "\u06dc", "\u06dd" - "\u06df", "\u06e0" - "\u06e4", "\u06e7" - "\u06e8", "\u06ea" - "\u06ed", "\u0901" - "\u0903", "\u093c", "\u093e" - "\u094c", "\u094d", "\u0951" - "\u0954", "\u0962" - "\u0963", "\u0981" - "\u0983", "\u09bc", "\u09be", "\u09bf", "\u09c0" - "\u09c4", "\u09c7" - "\u09c8", "\u09cb" - "\u09cd", "\u09d7", "\u09e2" - "\u09e3", "\u0a02", "\u0a3c", "\u0a3e", "\u0a3f", "\u0a40" - "\u0a42", "\u0a47" - "\u0a48", "\u0a4b" - "\u0a4d", "\u0a70" - "\u0a71", "\u0a81" - "\u0a83", "\u0abc", "\u0abe" - "\u0ac5", "\u0ac7" - "\u0ac9", "\u0acb" - "\u0acd", "\u0b01" - "\u0b03", "\u0b3c", "\u0b3e" - "\u0b43", "\u0b47" - "\u0b48", "\u0b4b" - "\u0b4d", "\u0b56" - "\u0b57", "\u0b82" - "\u0b83", "\u0bbe" - "\u0bc2", "\u0bc6" - "\u0bc8", "\u0bca" - "\u0bcd", "\u0bd7", "\u0c01" - "\u0c03", "\u0c3e" - "\u0c44", "\u0c46" - "\u0c48", "\u0c4a" - "\u0c4d", "\u0c55" - "\u0c56", "\u0c82" - "\u0c83", "\u0cbe" - "\u0cc4", "\u0cc6" - "\u0cc8", "\u0cca" - "\u0ccd", "\u0cd5" - "\u0cd6", "\u0d02" - "\u0d03", "\u0d3e" - "\u0d43", "\u0d46" - "\u0d48", "\u0d4a" - "\u0d4d", "\u0d57", "\u0e31", "\u0e34" - "\u0e3a", "\u0e47" - "\u0e4e", "\u0eb1", "\u0eb4" - "\u0eb9", "\u0ebb" - "\u0ebc", "\u0ec8" - "\u0ecd", "\u0f18" - "\u0f19", "\u0f35", "\u0f37", "\u0f39", "\u0f3e", "\u0f3f", "\u0f71" - "\u0f84", "\u0f86" - "\u0f8b", "\u0f90" - "\u0f95", "\u0f97", "\u0f99" - "\u0fad", "\u0fb1" - "\u0fb7", "\u0fb9", "\u20d0" - "\u20dc", "\u20e1", "\u302a" - "\u302f", "\u3099", "\u309a"] >
+}
+
+TOKEN :
+{
+ < #Digit : ["\u0030" - "\u0039", "\u0660" - "\u0669", "\u06f0" - "\u06f9", "\u0966" - "\u096f", "\u09e6" - "\u09ef", "\u0a66" - "\u0a6f", "\u0ae6" - "\u0aef", "\u0b66" - "\u0b6f", "\u0be7" - "\u0bef", "\u0c66" - "\u0c6f", "\u0ce6" - "\u0cef", "\u0d66" - "\u0d6f", "\u0e50" - "\u0e59", "\u0ed0" - "\u0ed9", "\u0f20" - "\u0f29"] >
+}
+
+TOKEN :
+{
+ < #Extender : ["\u00b7", "\u02d0", "\u02d1", "\u0387", "\u0640", "\u0e46", "\u0ec6", "\u3005", "\u3031" - "\u3035", "\u309d" - "\u309e", "\u30fc" - "\u30fe"] >
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-control-cc/.classpath b/hyracks/hyracks-control-cc/.classpath
index 88cebb7..ebe2cde 100644
--- a/hyracks/hyracks-control-cc/.classpath
+++ b/hyracks/hyracks-control-cc/.classpath
@@ -1,6 +1,7 @@
<?xml version="1.0" encoding="UTF-8"?>
<classpath>
<classpathentry kind="src" path="src/main/java"/>
+ <classpathentry kind="src" path="src/main/resources"/>
<classpathentry kind="con" path="org.eclipse.jdt.launching.JRE_CONTAINER/org.eclipse.jdt.internal.debug.ui.launcher.StandardVMType/JavaSE-1.6"/>
<classpathentry kind="con" path="org.maven.ide.eclipse.MAVEN2_CLASSPATH_CONTAINER"/>
<classpathentry kind="output" path="target/classes"/>
diff --git a/hyracks/hyracks-control-cc/pom.xml b/hyracks/hyracks-control-cc/pom.xml
index 96992af..a975238 100644
--- a/hyracks/hyracks-control-cc/pom.xml
+++ b/hyracks/hyracks-control-cc/pom.xml
@@ -25,20 +25,6 @@
<scope>compile</scope>
</dependency>
<dependency>
- <groupId>org.eclipse.jetty</groupId>
- <artifactId>jetty-server</artifactId>
- <version>8.0.0.M0</version>
- <type>jar</type>
- <scope>compile</scope>
- </dependency>
- <dependency>
- <groupId>args4j</groupId>
- <artifactId>args4j</artifactId>
- <version>2.0.12</version>
- <type>jar</type>
- <scope>compile</scope>
- </dependency>
- <dependency>
<groupId>org.apache.wicket</groupId>
<artifactId>wicket</artifactId>
<version>1.4.7</version>
@@ -52,5 +38,12 @@
<type>jar</type>
<scope>compile</scope>
</dependency>
+ <dependency>
+ <groupId>org.eclipse.jetty</groupId>
+ <artifactId>jetty-server</artifactId>
+ <version>8.0.0.M1</version>
+ <type>jar</type>
+ <scope>compile</scope>
+ </dependency>
</dependencies>
</project>
diff --git a/hyracks/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/ClusterControllerService.java b/hyracks/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/ClusterControllerService.java
index 2597df7..1d8aca2 100644
--- a/hyracks/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/ClusterControllerService.java
+++ b/hyracks/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/ClusterControllerService.java
@@ -14,7 +14,12 @@
*/
package edu.uci.ics.hyracks.control.cc;
+import java.io.ByteArrayOutputStream;
+import java.io.File;
import java.io.IOException;
+import java.io.InputStream;
+import java.io.ObjectOutputStream;
+import java.io.OutputStream;
import java.io.PrintWriter;
import java.net.InetAddress;
import java.rmi.registry.LocateRegistry;
@@ -22,6 +27,7 @@
import java.util.EnumSet;
import java.util.HashMap;
import java.util.HashSet;
+import java.util.Hashtable;
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
@@ -43,29 +49,36 @@
import jol.core.Runtime;
import jol.core.Runtime.DebugLevel;
+import org.apache.commons.io.IOUtils;
+import org.eclipse.jetty.http.HttpMethods;
import org.eclipse.jetty.server.Handler;
import org.eclipse.jetty.server.Request;
import org.eclipse.jetty.server.handler.AbstractHandler;
import org.eclipse.jetty.server.handler.ContextHandler;
+import edu.uci.ics.hyracks.api.client.ClusterControllerInfo;
import edu.uci.ics.hyracks.api.client.IHyracksClientInterface;
+import edu.uci.ics.hyracks.api.comm.Endpoint;
+import edu.uci.ics.hyracks.api.control.IClusterController;
+import edu.uci.ics.hyracks.api.control.INodeController;
+import edu.uci.ics.hyracks.api.control.NodeParameters;
import edu.uci.ics.hyracks.api.dataflow.ActivityNodeId;
import edu.uci.ics.hyracks.api.dataflow.OperatorDescriptorId;
import edu.uci.ics.hyracks.api.dataflow.PortInstanceId;
+import edu.uci.ics.hyracks.api.exceptions.HyracksException;
import edu.uci.ics.hyracks.api.job.JobFlag;
+import edu.uci.ics.hyracks.api.job.JobPlan;
import edu.uci.ics.hyracks.api.job.JobSpecification;
import edu.uci.ics.hyracks.api.job.JobStatus;
import edu.uci.ics.hyracks.api.job.statistics.JobStatistics;
import edu.uci.ics.hyracks.api.job.statistics.StageletStatistics;
import edu.uci.ics.hyracks.control.cc.web.WebServer;
import edu.uci.ics.hyracks.control.common.AbstractRemoteService;
-import edu.uci.ics.hyracks.control.common.NodeParameters;
-import edu.uci.ics.hyracks.control.common.api.IClusterController;
-import edu.uci.ics.hyracks.control.common.api.INodeController;
-import edu.uci.ics.hyracks.control.common.comm.Endpoint;
-import edu.uci.ics.hyracks.control.common.job.JobPlan;
+import edu.uci.ics.hyracks.control.common.application.ApplicationContext;
+import edu.uci.ics.hyracks.control.common.context.ServerContext;
-public class ClusterControllerService extends AbstractRemoteService implements IClusterController {
+public class ClusterControllerService extends AbstractRemoteService implements IClusterController,
+ IHyracksClientInterface {
private static final long serialVersionUID = 1L;
private CCConfig ccConfig;
@@ -74,8 +87,14 @@
private final Map<String, NodeControllerState> nodeRegistry;
+ private final Map<String, ApplicationContext> applications;
+
+ private final ServerContext serverCtx;
+
private WebServer webServer;
+ private ClusterControllerInfo info;
+
private final IJobManager jobManager;
private final Executor taskExecutor;
@@ -87,11 +106,16 @@
public ClusterControllerService(CCConfig ccConfig) throws Exception {
this.ccConfig = ccConfig;
nodeRegistry = new LinkedHashMap<String, NodeControllerState>();
+ applications = new Hashtable<String, ApplicationContext>();
+ serverCtx = new ServerContext(ServerContext.ServerType.CLUSTER_CONTROLLER, new File(
+ ClusterControllerService.class.getName()));
Set<DebugLevel> jolDebugLevel = LOGGER.isLoggable(Level.FINE) ? Runtime.DEBUG_ALL : new HashSet<DebugLevel>();
jolRuntime = (Runtime) Runtime.create(jolDebugLevel, System.err);
jobManager = new JOLJobManagerImpl(this, jolRuntime);
taskExecutor = Executors.newCachedThreadPool();
- webServer = new WebServer(new Handler[] { getAdminConsoleHandler(), getApplicationDataHandler() });
+ webServer = new WebServer();
+ webServer.addHandler(getAdminConsoleHandler());
+ webServer.addHandler(getApplicationInstallationHandler());
this.timer = new Timer(true);
}
@@ -103,6 +127,8 @@
registry.rebind(IClusterController.class.getName(), this);
webServer.setPort(ccConfig.httpPort);
webServer.start();
+ info = new ClusterControllerInfo();
+ info.setWebPort(webServer.getListeningPort());
timer.schedule(new DeadNodeSweeper(), 0, ccConfig.heartbeatPeriod);
LOGGER.log(Level.INFO, "Started ClusterControllerService");
}
@@ -115,13 +141,12 @@
}
@Override
- public UUID createJob(JobSpecification jobSpec) throws Exception {
- return jobManager.createJob(jobSpec, EnumSet.noneOf(JobFlag.class));
- }
-
- @Override
- public UUID createJob(JobSpecification jobSpec, EnumSet<JobFlag> jobFlags) throws Exception {
- return jobManager.createJob(jobSpec, jobFlags);
+ public UUID createJob(String appName, byte[] jobSpec, EnumSet<JobFlag> jobFlags) throws Exception {
+ ApplicationContext appCtx = applications.get(appName);
+ if (appCtx == null) {
+ throw new HyracksException("No application with id " + appName + " found");
+ }
+ return jobManager.createJob(appName, (JobSpecification) appCtx.deserialize(jobSpec), jobFlags);
}
@Override
@@ -138,6 +163,7 @@
jobManager.registerNode(id);
LOGGER.log(Level.INFO, "Registered INodeController: id = " + id);
NodeParameters params = new NodeParameters();
+ params.setClusterControllerInfo(info);
params.setHeartbeatPeriod(ccConfig.heartbeatPeriod);
return params;
}
@@ -237,12 +263,54 @@
return handler;
}
- private Handler getApplicationDataHandler() {
+ private Handler getApplicationInstallationHandler() {
ContextHandler handler = new ContextHandler("/applications");
handler.setHandler(new AbstractHandler() {
@Override
public void handle(String target, Request baseRequest, HttpServletRequest request,
HttpServletResponse response) throws IOException, ServletException {
+ try {
+ while (target.startsWith("/")) {
+ target = target.substring(1);
+ }
+ while (target.endsWith("/")) {
+ target = target.substring(0, target.length() - 1);
+ }
+ String[] parts = target.split("/");
+ if (parts.length != 1) {
+ return;
+ }
+ String appName = parts[0];
+ ApplicationContext appCtx;
+ appCtx = applications.get(appName);
+ if (appCtx != null) {
+ if (HttpMethods.PUT.equals(request.getMethod())) {
+ OutputStream os = appCtx.getHarOutputStream();
+ try {
+ IOUtils.copyLarge(request.getInputStream(), os);
+ } finally {
+ os.close();
+ }
+ } else if (HttpMethods.GET.equals(request.getMethod())) {
+ if (!appCtx.containsHar()) {
+ response.setStatus(HttpServletResponse.SC_NOT_FOUND);
+ } else {
+ InputStream is = appCtx.getHarInputStream();
+ response.setContentType("application/octet-stream");
+ response.setStatus(HttpServletResponse.SC_OK);
+ try {
+ IOUtils.copyLarge(is, response.getOutputStream());
+ } finally {
+ is.close();
+ }
+ }
+ }
+ baseRequest.setHandled(true);
+ }
+ } catch (IOException e) {
+ e.printStackTrace();
+ throw e;
+ }
}
});
return handler;
@@ -274,6 +342,47 @@
jobManager.notifyNodeFailure(nodeId);
}
+ @Override
+ public void createApplication(String appName) throws Exception {
+ synchronized (applications) {
+ if (applications.containsKey(appName)) {
+ throw new HyracksException("Duplicate application with name: " + appName + " being created.");
+ }
+ ApplicationContext appCtx = new ApplicationContext(serverCtx, appName);
+ applications.put(appName, appCtx);
+ }
+ }
+
+ @Override
+ public void destroyApplication(String appName) throws Exception {
+ ApplicationContext appCtx = applications.remove(appName);
+ if (appCtx != null) {
+ synchronized (this) {
+ for (NodeControllerState ncs : nodeRegistry.values()) {
+ ncs.getNodeController().destroyApplication(appName);
+ }
+ }
+ appCtx.deinitialize();
+ }
+ }
+
+ @Override
+ public void startApplication(String appName) throws Exception {
+ ApplicationContext appCtx = applications.get(appName);
+ appCtx.initialize();
+ boolean deployHar = appCtx.containsHar();
+ synchronized (this) {
+ for (NodeControllerState ncs : nodeRegistry.values()) {
+ ncs.getNodeController().createApplication(appName, deployHar);
+ }
+ }
+ }
+
+ @Override
+ public ClusterControllerInfo getClusterControllerInfo() throws Exception {
+ return info;
+ }
+
private class DeadNodeSweeper extends TimerTask {
@Override
public void run() {
@@ -346,19 +455,28 @@
return accumulator == null ? null : accumulator.getResult();
}
+ private static byte[] serialize(Object o) throws IOException {
+ ByteArrayOutputStream baos = new ByteArrayOutputStream();
+ ObjectOutputStream oos = new ObjectOutputStream(baos);
+ oos.writeObject(o);
+ return baos.toByteArray();
+ }
+
static class Phase1Installer implements RemoteOp<Map<PortInstanceId, Endpoint>> {
private String nodeId;
private UUID jobId;
+ private String appName;
private JobPlan plan;
private UUID stageId;
private int attempt;
private Map<ActivityNodeId, Set<Integer>> tasks;
private Map<OperatorDescriptorId, Set<Integer>> opPartitions;
- public Phase1Installer(String nodeId, UUID jobId, JobPlan plan, UUID stageId, int attempt,
+ public Phase1Installer(String nodeId, UUID jobId, String appName, JobPlan plan, UUID stageId, int attempt,
Map<ActivityNodeId, Set<Integer>> tasks, Map<OperatorDescriptorId, Set<Integer>> opPartitions) {
this.nodeId = nodeId;
this.jobId = jobId;
+ this.appName = appName;
this.plan = plan;
this.stageId = stageId;
this.attempt = attempt;
@@ -368,7 +486,7 @@
@Override
public Map<PortInstanceId, Endpoint> execute(INodeController node) throws Exception {
- return node.initializeJobletPhase1(jobId, plan, stageId, attempt, tasks, opPartitions);
+ return node.initializeJobletPhase1(appName, jobId, serialize(plan), stageId, attempt, tasks, opPartitions);
}
@Override
@@ -385,17 +503,19 @@
static class Phase2Installer implements RemoteOp<Void> {
private String nodeId;
private UUID jobId;
+ private String appName;
private JobPlan plan;
private UUID stageId;
private Map<ActivityNodeId, Set<Integer>> tasks;
private Map<OperatorDescriptorId, Set<Integer>> opPartitions;
private Map<PortInstanceId, Endpoint> globalPortMap;
- public Phase2Installer(String nodeId, UUID jobId, JobPlan plan, UUID stageId,
+ public Phase2Installer(String nodeId, UUID jobId, String appName, JobPlan plan, UUID stageId,
Map<ActivityNodeId, Set<Integer>> tasks, Map<OperatorDescriptorId, Set<Integer>> opPartitions,
Map<PortInstanceId, Endpoint> globalPortMap) {
this.nodeId = nodeId;
this.jobId = jobId;
+ this.appName = appName;
this.plan = plan;
this.stageId = stageId;
this.tasks = tasks;
@@ -405,7 +525,7 @@
@Override
public Void execute(INodeController node) throws Exception {
- node.initializeJobletPhase2(jobId, plan, stageId, tasks, opPartitions, globalPortMap);
+ node.initializeJobletPhase2(appName, jobId, serialize(plan), stageId, tasks, opPartitions, globalPortMap);
return null;
}
@@ -544,19 +664,4 @@
return portMap;
}
}
-
- @Override
- public void createApplication(String appName) throws Exception {
-
- }
-
- @Override
- public void destroyApplication(String appName) throws Exception {
-
- }
-
- @Override
- public void startApplication(String appName) throws Exception {
-
- }
}
\ No newline at end of file
diff --git a/hyracks/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/IJobManager.java b/hyracks/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/IJobManager.java
index 9d500c0..bb90d0b 100644
--- a/hyracks/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/IJobManager.java
+++ b/hyracks/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/IJobManager.java
@@ -24,12 +24,12 @@
import edu.uci.ics.hyracks.api.job.statistics.StageletStatistics;
public interface IJobManager {
- public UUID createJob(JobSpecification jobSpec, EnumSet<JobFlag> jobFlags) throws Exception;
+ public UUID createJob(String appName, JobSpecification jobSpec, EnumSet<JobFlag> jobFlags) throws Exception;
public void start(UUID jobId) throws Exception;
public void notifyStageletComplete(UUID jobId, UUID stageId, int attempt, String nodeId,
- StageletStatistics statistics) throws Exception;
+ StageletStatistics statistics) throws Exception;
public void notifyStageletFailure(UUID jobId, UUID stageId, int attempt, String nodeId) throws Exception;
diff --git a/hyracks/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/JOLJobManagerImpl.java b/hyracks/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/JOLJobManagerImpl.java
index d7fd61d..22cce67 100644
--- a/hyracks/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/JOLJobManagerImpl.java
+++ b/hyracks/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/JOLJobManagerImpl.java
@@ -36,6 +36,7 @@
import jol.types.table.Function;
import jol.types.table.Key;
import jol.types.table.TableName;
+import edu.uci.ics.hyracks.api.comm.Endpoint;
import edu.uci.ics.hyracks.api.constraints.AbsoluteLocationConstraint;
import edu.uci.ics.hyracks.api.constraints.ChoiceLocationConstraint;
import edu.uci.ics.hyracks.api.constraints.ExplicitPartitionConstraint;
@@ -52,13 +53,12 @@
import edu.uci.ics.hyracks.api.dataflow.OperatorDescriptorId;
import edu.uci.ics.hyracks.api.dataflow.PortInstanceId;
import edu.uci.ics.hyracks.api.job.JobFlag;
+import edu.uci.ics.hyracks.api.job.JobPlan;
import edu.uci.ics.hyracks.api.job.JobSpecification;
import edu.uci.ics.hyracks.api.job.JobStatus;
import edu.uci.ics.hyracks.api.job.statistics.JobStatistics;
import edu.uci.ics.hyracks.api.job.statistics.StageStatistics;
import edu.uci.ics.hyracks.api.job.statistics.StageletStatistics;
-import edu.uci.ics.hyracks.control.common.comm.Endpoint;
-import edu.uci.ics.hyracks.control.common.job.JobPlan;
public class JOLJobManagerImpl implements IJobManager {
private static final Logger LOGGER = Logger.getLogger(JOLJobManagerImpl.class.getName());
@@ -194,8 +194,9 @@
UUID jobId = (UUID) data[0];
UUID stageId = (UUID) data[1];
Integer attempt = (Integer) data[2];
- JobPlan plan = (JobPlan) data[3];
- Set<List> ts = (Set<List>) data[4];
+ String appName = (String) data[3];
+ JobPlan plan = (JobPlan) data[4];
+ Set<List> ts = (Set<List>) data[5];
Map<OperatorDescriptorId, Set<Integer>> opPartitions = new HashMap<OperatorDescriptorId, Set<Integer>>();
for (List t2 : ts) {
Object[] t2Data = t2.toArray();
@@ -229,7 +230,7 @@
aParts.add((Integer) lData[1]);
}
p1is[i++] = new ClusterControllerService.Phase1Installer((String) t2Data[0], jobId,
- plan, stageId, attempt, tasks, opPartitions);
+ appName, plan, stageId, attempt, tasks, opPartitions);
}
LOGGER.info("Stage start - Phase 1");
Map<PortInstanceId, Endpoint> globalPortMap = ccs.runRemote(p1is,
@@ -257,7 +258,7 @@
aParts.add((Integer) lData[1]);
}
p2is[i] = new ClusterControllerService.Phase2Installer((String) t2Data[0], jobId,
- plan, stageId, tasks, opPartitions, globalPortMap);
+ appName, plan, stageId, tasks, opPartitions, globalPortMap);
p3is[i] = new ClusterControllerService.Phase3Installer((String) t2Data[0], jobId,
stageId);
ss[i] = new ClusterControllerService.StageStarter((String) t2Data[0], jobId,
@@ -368,7 +369,7 @@
}
@Override
- public UUID createJob(JobSpecification jobSpec, EnumSet<JobFlag> jobFlags) throws Exception {
+ public UUID createJob(String appName, JobSpecification jobSpec, EnumSet<JobFlag> jobFlags) throws Exception {
final UUID jobId = UUID.randomUUID();
final JobPlanBuilder builder = new JobPlanBuilder();
@@ -420,7 +421,8 @@
cdTuples.add(ConnectorDescriptorTable.createTuple(jobId, jobSpec, e.getValue()));
}
- BasicTupleSet jobTuples = new BasicTupleSet(JobTable.createInitialJobTuple(jobId, jobSpec, builder.getPlan()));
+ BasicTupleSet jobTuples = new BasicTupleSet(JobTable.createInitialJobTuple(jobId, appName, jobSpec,
+ builder.getPlan()));
jolRuntime.schedule(JOL_SCOPE, JobTable.TABLE_NAME, jobTuples, null);
jolRuntime.schedule(JOL_SCOPE, OperatorDescriptorTable.TABLE_NAME, odTuples, null);
@@ -481,7 +483,7 @@
if (jobTuple == null) {
return null;
}
- return (JobStatus) jobTuple.value(1);
+ return (JobStatus) jobTuple.value(JobTable.JOBSTATUS_FIELD_INDEX);
} catch (BadKeyException e) {
throw new RuntimeException(e);
}
@@ -583,7 +585,8 @@
public JobStatistics waitForCompletion(UUID jobId) throws Exception {
synchronized (jobTable) {
Tuple jobTuple = null;
- while ((jobTuple = jobTable.lookupJob(jobId)) != null && jobTuple.value(1) != JobStatus.TERMINATED) {
+ while ((jobTuple = jobTable.lookupJob(jobId)) != null
+ && jobTuple.value(JobTable.JOBSTATUS_FIELD_INDEX) != JobStatus.TERMINATED) {
jobTable.wait();
}
return jobTuple == null ? null : jobTable.buildJobStatistics(jobTuple);
@@ -599,21 +602,29 @@
private static Key PRIMARY_KEY = new Key(0);
@SuppressWarnings("unchecked")
- private static final Class[] SCHEMA = new Class[] { UUID.class, JobStatus.class, JobSpecification.class,
- JobPlan.class, Set.class };
+ private static final Class[] SCHEMA = new Class[] { UUID.class, String.class, JobStatus.class,
+ JobSpecification.class, JobPlan.class, Set.class };
+
+ public static final int JOBID_FIELD_INDEX = 0;
+ public static final int APPNAME_FIELD_INDEX = 1;
+ public static final int JOBSTATUS_FIELD_INDEX = 2;
+ public static final int JOBSPEC_FIELD_INDEX = 3;
+ public static final int JOBPLAN_FIELD_INDEX = 4;
+ public static final int STATISTICS_FIELD_INDEX = 5;
public JobTable(Runtime context) {
super(context, TABLE_NAME, PRIMARY_KEY, SCHEMA);
}
@SuppressWarnings("unchecked")
- static Tuple createInitialJobTuple(UUID jobId, JobSpecification jobSpec, JobPlan plan) {
- return new Tuple(jobId, JobStatus.INITIALIZED, jobSpec, plan, new HashSet());
+ static Tuple createInitialJobTuple(UUID jobId, String appName, JobSpecification jobSpec, JobPlan plan) {
+ return new Tuple(jobId, appName, JobStatus.INITIALIZED, jobSpec, plan, new HashSet());
}
@SuppressWarnings("unchecked")
JobStatistics buildJobStatistics(Tuple jobTuple) {
- Set<Set<StageletStatistics>> statsSet = (Set<Set<StageletStatistics>>) jobTuple.value(4);
+ Set<Set<StageletStatistics>> statsSet = (Set<Set<StageletStatistics>>) jobTuple
+ .value(JobTable.STATISTICS_FIELD_INDEX);
JobStatistics stats = new JobStatistics();
if (statsSet != null) {
for (Set<StageletStatistics> stageStatsSet : statsSet) {
@@ -822,8 +833,8 @@
private static Key PRIMARY_KEY = new Key(0, 1, 2);
@SuppressWarnings("unchecked")
- private static final Class[] SCHEMA = new Class[] { UUID.class, UUID.class, Integer.class, JobPlan.class,
- Set.class };
+ private static final Class[] SCHEMA = new Class[] { UUID.class, UUID.class, Integer.class, String.class,
+ JobPlan.class, Set.class };
public StartMessageTable(Runtime context) {
super(context, TABLE_NAME, PRIMARY_KEY, SCHEMA);
@@ -1010,6 +1021,11 @@
private static final Class[] SCHEMA = new Class[] { UUID.class, OperatorDescriptorId.class, Integer.class,
Integer.class };
+ public static final int JOBID_FIELD_INDEX = 0;
+ public static final int ODID_FIELD_INDEX = 1;
+ public static final int NPARTITIONS_FIELD_INDEX = 2;
+ public static final int ORDINAL_FIELD_INDEX = 3;
+
public ExpandPartitionCountConstraintTableFunction() {
super(TABLE_NAME, SCHEMA);
}
@@ -1019,9 +1035,9 @@
TupleSet result = new BasicTupleSet();
int counter = 0;
for (Tuple t : tuples) {
- int nPartitions = (Integer) t.value(2);
+ int nPartitions = (Integer) t.value(NPARTITIONS_FIELD_INDEX);
for (int i = 0; i < nPartitions; ++i) {
- result.add(new Tuple(t.value(0), t.value(1), i, counter++));
+ result.add(new Tuple(t.value(JOBID_FIELD_INDEX), t.value(ODID_FIELD_INDEX), i, counter++));
}
}
return result;
diff --git a/hyracks/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/JobPlanBuilder.java b/hyracks/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/JobPlanBuilder.java
index fcb796e..d5056d3 100644
--- a/hyracks/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/JobPlanBuilder.java
+++ b/hyracks/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/JobPlanBuilder.java
@@ -12,8 +12,8 @@
import edu.uci.ics.hyracks.api.dataflow.IActivityGraphBuilder;
import edu.uci.ics.hyracks.api.dataflow.IActivityNode;
import edu.uci.ics.hyracks.api.job.JobFlag;
+import edu.uci.ics.hyracks.api.job.JobPlan;
import edu.uci.ics.hyracks.api.job.JobSpecification;
-import edu.uci.ics.hyracks.control.common.job.JobPlan;
public class JobPlanBuilder implements IActivityGraphBuilder {
private static final Logger LOGGER = Logger.getLogger(JobPlanBuilder.class.getName());
diff --git a/hyracks/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/JobPlanner.java b/hyracks/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/JobPlanner.java
index 956c658..af3b77f 100644
--- a/hyracks/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/JobPlanner.java
+++ b/hyracks/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/JobPlanner.java
@@ -29,12 +29,12 @@
import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
import edu.uci.ics.hyracks.api.dataflow.OperatorDescriptorId;
import edu.uci.ics.hyracks.api.job.JobFlag;
+import edu.uci.ics.hyracks.api.job.JobPlan;
import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.api.job.JobStage;
import edu.uci.ics.hyracks.api.util.Pair;
import edu.uci.ics.hyracks.control.cc.job.IOperatorDescriptorVisitor;
import edu.uci.ics.hyracks.control.cc.job.PlanUtils;
-import edu.uci.ics.hyracks.control.common.job.JobPlan;
-import edu.uci.ics.hyracks.control.common.job.JobStage;
public class JobPlanner {
private static final Logger LOGGER = Logger.getLogger(JobPlanner.class.getName());
diff --git a/hyracks/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/NodeControllerState.java b/hyracks/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/NodeControllerState.java
index e4d611b..a638c09 100644
--- a/hyracks/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/NodeControllerState.java
+++ b/hyracks/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/NodeControllerState.java
@@ -1,6 +1,6 @@
package edu.uci.ics.hyracks.control.cc;
-import edu.uci.ics.hyracks.control.common.api.INodeController;
+import edu.uci.ics.hyracks.api.control.INodeController;
public class NodeControllerState {
private final INodeController nodeController;
diff --git a/hyracks/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/web/WebServer.java b/hyracks/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/web/WebServer.java
index dfffcb3..12dfe21 100644
--- a/hyracks/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/web/WebServer.java
+++ b/hyracks/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/web/WebServer.java
@@ -18,28 +18,33 @@
import org.eclipse.jetty.server.Handler;
import org.eclipse.jetty.server.Server;
import org.eclipse.jetty.server.handler.ContextHandlerCollection;
+import org.eclipse.jetty.server.handler.HandlerCollection;
import org.eclipse.jetty.server.nio.SelectChannelConnector;
public class WebServer {
- private Server server;
- private SelectChannelConnector connector;
+ private final Server server;
+ private final SelectChannelConnector connector;
+ private final HandlerCollection handlerCollection;
- public WebServer(Handler[] handlers) throws Exception {
+ public WebServer() throws Exception {
server = new Server();
connector = new SelectChannelConnector();
server.setConnectors(new Connector[] { connector });
- ContextHandlerCollection handler = new ContextHandlerCollection();
- handler.setHandlers(handlers);
- server.setHandler(handler);
+ handlerCollection = new ContextHandlerCollection();
+ server.setHandler(handlerCollection);
}
public void setPort(int port) {
connector.setPort(port);
}
+ public int getListeningPort() {
+ return connector.getLocalPort();
+ }
+
public void start() throws Exception {
server.start();
}
@@ -47,4 +52,8 @@
public void stop() throws Exception {
server.stop();
}
+
+ public void addHandler(Handler handler) {
+ handlerCollection.addHandler(handler);
+ }
}
\ No newline at end of file
diff --git a/hyracks/hyracks-control-nc/src/main/resources/edu/uci/ics/hyracks/control/cc/scheduler.olg b/hyracks/hyracks-control-cc/src/main/resources/edu/uci/ics/hyracks/control/cc/scheduler.olg
similarity index 86%
rename from hyracks/hyracks-control-nc/src/main/resources/edu/uci/ics/hyracks/control/cc/scheduler.olg
rename to hyracks/hyracks-control-cc/src/main/resources/edu/uci/ics/hyracks/control/cc/scheduler.olg
index b230e0e..0be0401 100644
--- a/hyracks/hyracks-control-nc/src/main/resources/edu/uci/ics/hyracks/control/cc/scheduler.olg
+++ b/hyracks/hyracks-control-cc/src/main/resources/edu/uci/ics/hyracks/control/cc/scheduler.olg
@@ -12,7 +12,11 @@
import edu.uci.ics.hyracks.api.dataflow.ConnectorDescriptorId;
import edu.uci.ics.hyracks.api.dataflow.IConnectorDescriptor;
import edu.uci.ics.hyracks.api.job.JobStatus;
-import edu.uci.ics.hyracks.control.common.job.JobPlan;
+import edu.uci.ics.hyracks.api.job.JobPlan;
+
+watch(job, a);
+watch(job, i);
+watch(job, d);
define(activitystage_temp, keys(), {UUID, OperatorDescriptorId, ActivityNodeId, Integer});
@@ -79,7 +83,7 @@
define(jobattempt, keys(), {UUID, Integer});
jobattempt(JobId, 0) :-
- job(JobId, edu.uci.ics.hyracks.api.job.JobStatus.INITIALIZED, _, _, _),
+ job(JobId, _, edu.uci.ics.hyracks.api.job.JobStatus.INITIALIZED, _, _, _),
jobstart(JobId, _);
jobattempt(JobId, NextAttempt) :-
@@ -98,8 +102,8 @@
stagestart_INITIAL stagestart(JobId, 0, Attempt) :-
jobattempt#insert(JobId, Attempt);
-update_job_status_RUNNING job(JobId, edu.uci.ics.hyracks.api.job.JobStatus.RUNNING, JobSpec, JobPlan, null) :-
- job(JobId, edu.uci.ics.hyracks.api.job.JobStatus.INITIALIZED, JobSpec, JobPlan, _),
+update_job_status_RUNNING job(JobId, AppName, edu.uci.ics.hyracks.api.job.JobStatus.RUNNING, JobSpec, JobPlan, null) :-
+ job(JobId, AppName, edu.uci.ics.hyracks.api.job.JobStatus.INITIALIZED, JobSpec, JobPlan, _),
jobstart(JobId, _);
stagestart_NEXT stagestart(JobId, NextStageNumber, Attempt) :-
@@ -195,11 +199,11 @@
watch(activitystart, a);
-define(stageletstart, keys(0, 1, 3, 4), {UUID, UUID, JobPlan, String, Integer, Set});
+define(stageletstart, keys(0, 1, 4, 5), {UUID, UUID, String, JobPlan, String, Integer, Set});
-stageletstart(JobId, StageId, JobPlan, NodeId, Attempt, set<ActivityInfo>) :-
+stageletstart(JobId, StageId, AppName, JobPlan, NodeId, Attempt, set<ActivityInfo>) :-
activitystart#insert(JobId, _, ActivityId, StageNumber, Attempt, StageId, NodeId, Partition),
- job(JobId, edu.uci.ics.hyracks.api.job.JobStatus.RUNNING, _, JobPlan, _)
+ job(JobId, AppName, edu.uci.ics.hyracks.api.job.JobStatus.RUNNING, _, JobPlan, _)
{
ActivityInfo := [ActivityId, Partition];
};
@@ -207,18 +211,18 @@
watch(stageletstart, a);
watch(stageletstart, i);
-define(startmessage_agg, keys(0, 1, 2), {UUID, UUID, Integer, JobPlan, Set});
+define(startmessage_agg, keys(0, 1, 2), {UUID, UUID, Integer, String, JobPlan, Set});
-startmessage_agg(JobId, StageId, Attempt, JobPlan, set<Tuple>) :-
- stageletstart#insert(JobId, StageId, JobPlan, NodeId, Attempt, ActivityInfoSet),
+startmessage_agg(JobId, StageId, Attempt, AppName, JobPlan, set<Tuple>) :-
+ stageletstart#insert(JobId, StageId, AppName, JobPlan, NodeId, Attempt, ActivityInfoSet),
availablenodes(NodeId),
ActivityInfoSet.size() != 0
{
Tuple := [NodeId, ActivityInfoSet];
};
-startmessage(JobId, StageId, Attempt, JobPlan, TSet) :-
- startmessage_agg(JobId, StageId, Attempt, JobPlan, TSet);
+startmessage(JobId, StageId, Attempt, AppName, JobPlan, TSet) :-
+ startmessage_agg(JobId, StageId, Attempt, AppName, JobPlan, TSet);
watch(startmessage, a);
watch(startmessage, i);
@@ -227,11 +231,11 @@
stageletabort(JobId, StageId, JobPlan, NodeId, Attempt, ActivityIdSet) :-
stageletfailure(JobId, StageId, NodeId, Attempt),
- stageletstart(JobId, StageId, JobPlan, NodeId, Attempt, ActivityIdSet);
+ stageletstart(JobId, StageId, _, JobPlan, NodeId, Attempt, ActivityIdSet);
stageletabort(JobId, StageId, JobPlan, NodeIdOther, Attempt, ActivityIdSet) :-
- stageletstart(JobId, StageId, JobPlan, NodeId, Attempt, _),
- stageletstart(JobId, StageId, _, NodeIdOther, Attempt, ActivityIdSet),
+ stageletstart(JobId, StageId, _, JobPlan, NodeId, Attempt, _),
+ stageletstart(JobId, StageId, _, _, NodeIdOther, Attempt, ActivityIdSet),
failednodes#insert(NodeId),
notin stageletcomplete(JobId, StageId, NodeId, Attempt, _);
@@ -287,13 +291,13 @@
stageletcomplete(JobId, StageId, NodeId, Attempt, Statistics);
stagefinish(JobId, StageNumber, Attempt, SSet) :-
- startmessage_agg(JobId, StageId, Attempt, _, TSet),
+ startmessage_agg(JobId, StageId, Attempt, _, _, TSet),
stageletcomplete_agg(JobId, StageId, Attempt, SSet),
jobstage(JobId, StageNumber, StageId),
TSet.size() == SSet.size();
-update_job_status_TERMINATED job(JobId, edu.uci.ics.hyracks.api.job.JobStatus.TERMINATED, JobSpec, JobPlan, null) :-
- job(JobId, edu.uci.ics.hyracks.api.job.JobStatus.RUNNING, JobSpec, JobPlan, _),
+update_job_status_TERMINATED job(JobId, AppName, edu.uci.ics.hyracks.api.job.JobStatus.TERMINATED, JobSpec, JobPlan, null) :-
+ job(JobId, AppName, edu.uci.ics.hyracks.api.job.JobStatus.RUNNING, JobSpec, JobPlan, _),
stagestart#insert(JobId, StageNumber, Attempt),
stagefinish(JobId, _, Attempt, SSet),
notin jobstage(JobId, StageNumber);
@@ -307,4 +311,4 @@
notin jobstage(JobId, StageNumber);
jobcleanup(JobId, NodeIdSet) :-
- jobcleanup_agg(JobId, NodeIdSet);
\ No newline at end of file
+ jobcleanup_agg(JobId, NodeIdSet);
diff --git a/hyracks/hyracks-control-common/pom.xml b/hyracks/hyracks-control-common/pom.xml
index 3573835..1fe14e8 100644
--- a/hyracks/hyracks-control-common/pom.xml
+++ b/hyracks/hyracks-control-common/pom.xml
@@ -25,9 +25,9 @@
<scope>compile</scope>
</dependency>
<dependency>
- <groupId>args4j</groupId>
- <artifactId>args4j</artifactId>
- <version>2.0.12</version>
+ <groupId>commons-io</groupId>
+ <artifactId>commons-io</artifactId>
+ <version>1.4</version>
<type>jar</type>
<scope>compile</scope>
</dependency>
diff --git a/hyracks/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/application/ApplicationContext.java b/hyracks/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/application/ApplicationContext.java
new file mode 100644
index 0000000..ab15b80
--- /dev/null
+++ b/hyracks/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/application/ApplicationContext.java
@@ -0,0 +1,181 @@
+/*
+ * 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.application;
+
+import java.io.ByteArrayInputStream;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.ObjectInputStream;
+import java.io.OutputStream;
+import java.net.MalformedURLException;
+import java.net.URL;
+import java.net.URLClassLoader;
+import java.util.ArrayList;
+import java.util.Enumeration;
+import java.util.List;
+import java.util.Properties;
+import java.util.zip.ZipEntry;
+import java.util.zip.ZipFile;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.io.IOUtils;
+
+import edu.uci.ics.hyracks.api.application.IApplicationContext;
+import edu.uci.ics.hyracks.api.application.IBootstrap;
+import edu.uci.ics.hyracks.control.common.context.ServerContext;
+
+public class ApplicationContext implements IApplicationContext {
+ private static final String APPLICATION_ROOT = "applications";
+ private static final String CLUSTER_CONTROLLER_BOOTSTRAP_CLASS_KEY = "cc.bootstrap.class";
+ private static final String NODE_CONTROLLER_BOOTSTRAP_CLASS_KEY = "nc.bootstrap.class";
+
+ private ServerContext serverCtx;
+ private final String appName;
+ private final File applicationRootDir;
+ private ClassLoader classLoader;
+ private ApplicationStatus status;
+ private Properties deploymentDescriptor;
+ private IBootstrap bootstrap;
+
+ public ApplicationContext(ServerContext serverCtx, String appName) throws IOException {
+ 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();
+ }
+
+ public String getApplicationName() {
+ return appName;
+ }
+
+ public void initialize() throws Exception {
+ if (status != ApplicationStatus.CREATED) {
+ throw new IllegalStateException();
+ }
+ if (expandArchive()) {
+ File expandedFolder = getExpandedFolder();
+ List<URL> urls = new ArrayList<URL>();
+ findJarFiles(expandedFolder, urls);
+ classLoader = new URLClassLoader(urls.toArray(new URL[urls.size()]));
+ deploymentDescriptor = parseDeploymentDescriptor();
+
+ String bootstrapClass = null;
+ switch (serverCtx.getServerType()) {
+ case CLUSTER_CONTROLLER: {
+ bootstrapClass = deploymentDescriptor.getProperty(CLUSTER_CONTROLLER_BOOTSTRAP_CLASS_KEY);
+ }
+ case NODE_CONTROLLER: {
+ bootstrapClass = deploymentDescriptor.getProperty(NODE_CONTROLLER_BOOTSTRAP_CLASS_KEY);
+ }
+ }
+ if (bootstrapClass != null) {
+ bootstrap = (IBootstrap) classLoader.loadClass(bootstrapClass).newInstance();
+ bootstrap.setApplicationContext(this);
+ bootstrap.start();
+ }
+ } else {
+ classLoader = getClass().getClassLoader();
+ }
+ status = ApplicationStatus.INITIALIZED;
+ }
+
+ private void findJarFiles(File dir, List<URL> urls) throws MalformedURLException {
+ for (File f : dir.listFiles()) {
+ if (f.isDirectory()) {
+ findJarFiles(f, urls);
+ } else if (f.getName().endsWith(".jar") || f.getName().endsWith(".zip")) {
+ urls.add(f.toURI().toURL());
+ }
+ }
+ }
+
+ private Properties parseDeploymentDescriptor() throws IOException {
+ InputStream in = classLoader.getResourceAsStream("/hyracks-deployment.properties");
+ Properties props = new Properties();
+ if (in != null) {
+ try {
+ props.load(in);
+ } finally {
+ in.close();
+ }
+ }
+ return props;
+ }
+
+ private boolean expandArchive() throws IOException {
+ File archiveFile = getArchiveFile();
+ if (archiveFile.exists()) {
+ File expandedFolder = getExpandedFolder();
+ FileUtils.deleteDirectory(expandedFolder);
+ ZipFile zf = new ZipFile(archiveFile);
+ for (Enumeration<? extends ZipEntry> i = zf.entries(); i.hasMoreElements();) {
+ ZipEntry ze = i.nextElement();
+ String name = ze.getName();
+ if (name.endsWith("/")) {
+ continue;
+ }
+ InputStream is = zf.getInputStream(ze);
+ OutputStream os = FileUtils.openOutputStream(new File(expandedFolder, name));
+ try {
+ IOUtils.copyLarge(is, os);
+ } finally {
+ os.close();
+ is.close();
+ }
+ }
+ return true;
+ }
+ return false;
+ }
+
+ private File getExpandedFolder() {
+ return new File(applicationRootDir, "expanded");
+ }
+
+ public void deinitialize() throws Exception {
+ status = ApplicationStatus.DEINITIALIZED;
+ if (bootstrap != null) {
+ bootstrap.stop();
+ }
+ File expandedFolder = getExpandedFolder();
+ FileUtils.deleteDirectory(expandedFolder);
+ }
+
+ public Object deserialize(byte[] bytes) throws IOException, ClassNotFoundException {
+ ObjectInputStream ois = new ClassLoaderObjectInputStream(new ByteArrayInputStream(bytes), classLoader);
+ return ois.readObject();
+ }
+
+ public OutputStream getHarOutputStream() throws IOException {
+ return new FileOutputStream(getArchiveFile());
+ }
+
+ private File getArchiveFile() {
+ return new File(applicationRootDir, "application.har");
+ }
+
+ public InputStream getHarInputStream() throws IOException {
+ return new FileInputStream(getArchiveFile());
+ }
+
+ public boolean containsHar() {
+ return getArchiveFile().exists();
+ }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/service/ILifeCycle.java b/hyracks/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/application/ApplicationStatus.java
similarity index 78%
copy from hyracks/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/service/ILifeCycle.java
copy to hyracks/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/application/ApplicationStatus.java
index c9f5abb..8a97ad6 100644
--- a/hyracks/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/service/ILifeCycle.java
+++ b/hyracks/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/application/ApplicationStatus.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.common.service;
+package edu.uci.ics.hyracks.control.common.application;
-public interface ILifeCycle {
- public void start() throws Exception;
-
- public void stop() throws Exception;
+public enum ApplicationStatus {
+ CREATED,
+ INITIALIZED,
+ DEINITIALIZED
}
\ No newline at end of file
diff --git a/hyracks/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/application/ClassLoaderObjectInputStream.java b/hyracks/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/application/ClassLoaderObjectInputStream.java
new file mode 100644
index 0000000..1ae29ba
--- /dev/null
+++ b/hyracks/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/application/ClassLoaderObjectInputStream.java
@@ -0,0 +1,20 @@
+package edu.uci.ics.hyracks.control.common.application;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.ObjectInputStream;
+import java.io.ObjectStreamClass;
+
+public class ClassLoaderObjectInputStream extends ObjectInputStream {
+ private ClassLoader classLoader;
+
+ protected ClassLoaderObjectInputStream(InputStream in, ClassLoader classLoader) throws IOException,
+ SecurityException {
+ super(in);
+ this.classLoader = classLoader;
+ }
+
+ protected Class<?> resolveClass(ObjectStreamClass desc) throws ClassNotFoundException {
+ return Class.forName(desc.getName(), false, classLoader);
+ }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/context/ServerContext.java b/hyracks/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/context/ServerContext.java
new file mode 100644
index 0000000..7eca794
--- /dev/null
+++ b/hyracks/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/context/ServerContext.java
@@ -0,0 +1,40 @@
+/*
+ * 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.context;
+
+import java.io.File;
+
+public class ServerContext {
+ public enum ServerType {
+ CLUSTER_CONTROLLER,
+ NODE_CONTROLLER,
+ }
+
+ private final ServerType type;
+ private final File baseDir;
+
+ public ServerContext(ServerType type, File baseDir) throws Exception {
+ this.type = type;
+ this.baseDir = baseDir;
+ }
+
+ public ServerType getServerType() {
+ return type;
+ }
+
+ public File getBaseDir() {
+ return baseDir;
+ }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/service/IService.java b/hyracks/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/service/IService.java
index 6bde315..4e81066 100644
--- a/hyracks/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/service/IService.java
+++ b/hyracks/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/service/IService.java
@@ -14,6 +14,8 @@
*/
package edu.uci.ics.hyracks.control.common.service;
+public interface IService {
+ public void start() throws Exception;
-public interface IService extends ILifeCycle {
+ public void stop() throws Exception;
}
\ No newline at end of file
diff --git a/hyracks/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Joblet.java b/hyracks/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Joblet.java
index eb7ac72..cb0128d 100644
--- a/hyracks/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Joblet.java
+++ b/hyracks/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Joblet.java
@@ -35,7 +35,7 @@
private final Map<OperatorDescriptorId, Map<Integer, IOperatorEnvironment>> envMap;
- public Joblet(NodeControllerService nodeController, UUID jobId) throws Exception {
+ public Joblet(NodeControllerService nodeController, UUID jobId) {
this.nodeController = nodeController;
this.jobId = jobId;
stageletMap = new HashMap<UUID, Stagelet>();
@@ -87,14 +87,15 @@
return nodeController.getExecutor();
}
- public synchronized void notifyStageletComplete(UUID stageId, int attempt, StageletStatistics stats) throws Exception {
+ public synchronized void notifyStageletComplete(UUID stageId, int attempt, StageletStatistics stats)
+ throws Exception {
stageletMap.remove(stageId);
nodeController.notifyStageComplete(jobId, stageId, attempt, stats);
}
public void notifyStageletFailed(UUID stageId, int attempt) throws Exception {
stageletMap.remove(stageId);
- nodeController.notifyStageFailed(jobId, stageId, attempt);
+ nodeController.notifyStageFailed(jobId, stageId, attempt);
}
public NodeControllerService getNodeController() {
diff --git a/hyracks/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/NCDriver.java b/hyracks/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/NCDriver.java
index af1f3dc..d8c978a 100644
--- a/hyracks/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/NCDriver.java
+++ b/hyracks/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/NCDriver.java
@@ -18,7 +18,7 @@
import edu.uci.ics.dcache.client.DCacheClient;
import edu.uci.ics.dcache.client.DCacheClientConfig;
-import edu.uci.ics.hyracks.control.common.api.NCConfig;
+import edu.uci.ics.hyracks.api.control.NCConfig;
public class NCDriver {
public static void main(String args[]) throws Exception {
diff --git a/hyracks/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/NodeControllerService.java b/hyracks/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/NodeControllerService.java
index 14c5339..b80d0f5 100644
--- a/hyracks/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/NodeControllerService.java
+++ b/hyracks/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/NodeControllerService.java
@@ -14,6 +14,9 @@
*/
package edu.uci.ics.hyracks.control.nc;
+import java.io.File;
+import java.io.InputStream;
+import java.io.OutputStream;
import java.net.InetAddress;
import java.nio.ByteBuffer;
import java.rmi.registry.LocateRegistry;
@@ -21,6 +24,7 @@
import java.text.MessageFormat;
import java.util.ArrayList;
import java.util.HashMap;
+import java.util.Hashtable;
import java.util.List;
import java.util.Map;
import java.util.Set;
@@ -34,10 +38,22 @@
import java.util.regex.Matcher;
import java.util.regex.Pattern;
+import org.apache.commons.io.IOUtils;
+import org.apache.http.HttpResponse;
+import org.apache.http.client.HttpClient;
+import org.apache.http.client.methods.HttpGet;
+import org.apache.http.impl.client.DefaultHttpClient;
+
+import edu.uci.ics.hyracks.api.comm.Endpoint;
import edu.uci.ics.hyracks.api.comm.IConnectionDemultiplexer;
import edu.uci.ics.hyracks.api.comm.IFrameReader;
import edu.uci.ics.hyracks.api.comm.IFrameWriter;
import edu.uci.ics.hyracks.api.context.IHyracksContext;
+import edu.uci.ics.hyracks.api.control.IClusterController;
+import edu.uci.ics.hyracks.api.control.INodeController;
+import edu.uci.ics.hyracks.api.control.NCConfig;
+import edu.uci.ics.hyracks.api.control.NodeCapability;
+import edu.uci.ics.hyracks.api.control.NodeParameters;
import edu.uci.ics.hyracks.api.dataflow.ActivityNodeId;
import edu.uci.ics.hyracks.api.dataflow.Direction;
import edu.uci.ics.hyracks.api.dataflow.IActivityNode;
@@ -51,17 +67,14 @@
import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.exceptions.HyracksException;
import edu.uci.ics.hyracks.api.job.JobFlag;
+import edu.uci.ics.hyracks.api.job.JobPlan;
import edu.uci.ics.hyracks.api.job.JobSpecification;
import edu.uci.ics.hyracks.api.job.statistics.StageletStatistics;
import edu.uci.ics.hyracks.control.common.AbstractRemoteService;
-import edu.uci.ics.hyracks.control.common.NodeCapability;
-import edu.uci.ics.hyracks.control.common.NodeParameters;
-import edu.uci.ics.hyracks.control.common.api.IClusterController;
-import edu.uci.ics.hyracks.control.common.api.INodeController;
-import edu.uci.ics.hyracks.control.common.api.NCConfig;
-import edu.uci.ics.hyracks.control.common.comm.Endpoint;
-import edu.uci.ics.hyracks.control.common.job.JobPlan;
+import edu.uci.ics.hyracks.control.common.application.ApplicationContext;
+import edu.uci.ics.hyracks.control.common.context.ServerContext;
import edu.uci.ics.hyracks.control.nc.comm.ConnectionManager;
import edu.uci.ics.hyracks.control.nc.comm.DemuxDataReceiveListenerFactory;
import edu.uci.ics.hyracks.control.nc.runtime.HyracksContext;
@@ -90,6 +103,10 @@
private NodeParameters nodeParameters;
+ private final ServerContext serverCtx;
+
+ private final Map<String, ApplicationContext> applications;
+
public NodeControllerService(NCConfig ncConfig) throws Exception {
this.ncConfig = ncConfig;
id = ncConfig.nodeId;
@@ -102,6 +119,9 @@
jobletMap = new HashMap<UUID, Joblet>();
executor = Executors.newCachedThreadPool();
timer = new Timer(true);
+ serverCtx = new ServerContext(ServerContext.ServerType.NODE_CONTROLLER, new File(new File(
+ NodeControllerService.class.getName()), id));
+ applications = new Hashtable<String, ApplicationContext>();
}
private static Logger LOGGER = Logger.getLogger(NodeControllerService.class.getName());
@@ -165,12 +185,15 @@
}
@Override
- public Map<PortInstanceId, Endpoint> initializeJobletPhase1(UUID jobId, final JobPlan plan, UUID stageId,
- int attempt, Map<ActivityNodeId, Set<Integer>> tasks, Map<OperatorDescriptorId, Set<Integer>> opPartitions)
- throws Exception {
+ public Map<PortInstanceId, Endpoint> initializeJobletPhase1(String appName, UUID jobId, byte[] planBytes,
+ UUID stageId, int attempt, Map<ActivityNodeId, Set<Integer>> tasks,
+ Map<OperatorDescriptorId, Set<Integer>> opPartitions) throws Exception {
try {
LOGGER.log(Level.INFO, String.valueOf(jobId) + "[" + id + ":" + stageId + "]: Initializing Joblet Phase 1");
+ ApplicationContext appCtx = applications.get(appName);
+ final JobPlan plan = (JobPlan) appCtx.deserialize(planBytes);
+
IRecordDescriptorProvider rdp = new IRecordDescriptorProvider() {
@Override
public RecordDescriptor getOutputRecordDescriptor(OperatorDescriptorId opId, int outputIndex) {
@@ -248,8 +271,7 @@
final int receiverIndex, JobPlan plan, final Stagelet stagelet, int nProducerCount, int nConsumerCount)
throws HyracksDataException {
final IFrameReader reader = conn.createReceiveSideReader(ctx, plan.getJobSpecification()
- .getConnectorRecordDescriptor(conn), demux, receiverIndex, nProducerCount,
- nConsumerCount);
+ .getConnectorRecordDescriptor(conn), demux, receiverIndex, nProducerCount, nConsumerCount);
return plan.getJobFlags().contains(JobFlag.COLLECT_FRAME_COUNTS) ? new IFrameReader() {
private int frameCount;
@@ -281,11 +303,14 @@
}
@Override
- public void initializeJobletPhase2(UUID jobId, final JobPlan plan, UUID stageId,
+ public void initializeJobletPhase2(String appName, UUID jobId, byte[] planBytes, UUID stageId,
Map<ActivityNodeId, Set<Integer>> tasks, Map<OperatorDescriptorId, Set<Integer>> opPartitions,
final Map<PortInstanceId, Endpoint> globalPortMap) throws Exception {
try {
LOGGER.log(Level.INFO, String.valueOf(jobId) + "[" + id + ":" + stageId + "]: Initializing Joblet Phase 2");
+ ApplicationContext appCtx = applications.get(appName);
+ final JobPlan plan = (JobPlan) appCtx.deserialize(planBytes);
+
final Joblet ji = getLocalJoblet(jobId);
Stagelet si = (Stagelet) ji.getStagelet(stageId);
final Map<OperatorInstanceId, OperatorRunnable> honMap = si.getOperatorMap();
@@ -326,8 +351,8 @@
}
};
or.setFrameWriter(j, conn.createSendSideWriter(ctx, plan.getJobSpecification()
- .getConnectorRecordDescriptor(conn), edwFactory, i,
- opPartitions.get(producerOpId).size(), opPartitions.get(consumerOpId).size()), spec
+ .getConnectorRecordDescriptor(conn), edwFactory, i, opPartitions.get(producerOpId)
+ .size(), opPartitions.get(consumerOpId).size()), spec
.getConnectorRecordDescriptor(conn));
}
}
@@ -412,11 +437,21 @@
}
public void notifyStageComplete(UUID jobId, UUID stageId, int attempt, StageletStatistics stats) throws Exception {
- ccs.notifyStageletComplete(jobId, stageId, attempt, id, stats);
+ try {
+ ccs.notifyStageletComplete(jobId, stageId, attempt, id, stats);
+ } catch (Exception e) {
+ e.printStackTrace();
+ throw e;
+ }
}
public void notifyStageFailed(UUID jobId, UUID stageId, int attempt) throws Exception {
- ccs.notifyStageletFailure(jobId, stageId, attempt, id);
+ try {
+ ccs.notifyStageletFailure(jobId, stageId, attempt, id);
+ } catch (Exception e) {
+ e.printStackTrace();
+ throw e;
+ }
}
@Override
@@ -457,4 +492,39 @@
}
}
}
+
+ @Override
+ public void createApplication(String appName, boolean deployHar) throws Exception {
+ ApplicationContext appCtx;
+ synchronized (applications) {
+ if (applications.containsKey(appName)) {
+ throw new HyracksException("Duplicate application with name: " + appName + " being created.");
+ }
+ appCtx = new ApplicationContext(serverCtx, appName);
+ applications.put(appName, appCtx);
+ }
+ if (deployHar) {
+ HttpClient hc = new DefaultHttpClient();
+ HttpGet get = new HttpGet("http://" + ncConfig.ccHost + ":"
+ + nodeParameters.getClusterControllerInfo().getWebPort() + "/applications/" + appName);
+ HttpResponse response = hc.execute(get);
+ InputStream is = response.getEntity().getContent();
+ OutputStream os = appCtx.getHarOutputStream();
+ try {
+ IOUtils.copyLarge(is, os);
+ } finally {
+ os.close();
+ is.close();
+ }
+ }
+ appCtx.initialize();
+ }
+
+ @Override
+ public void destroyApplication(String appName) throws Exception {
+ ApplicationContext appCtx = applications.remove(appName);
+ if (appCtx != null) {
+ appCtx.deinitialize();
+ }
+ }
}
\ No newline at end of file
diff --git a/hyracks/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Stagelet.java b/hyracks/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Stagelet.java
index 696ce29..d6c3ee2 100644
--- a/hyracks/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Stagelet.java
+++ b/hyracks/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Stagelet.java
@@ -25,10 +25,10 @@
import java.util.logging.Level;
import java.util.logging.Logger;
+import edu.uci.ics.hyracks.api.comm.Endpoint;
import edu.uci.ics.hyracks.api.dataflow.OperatorDescriptorId;
import edu.uci.ics.hyracks.api.dataflow.OperatorInstanceId;
import edu.uci.ics.hyracks.api.job.statistics.StageletStatistics;
-import edu.uci.ics.hyracks.control.common.comm.Endpoint;
import edu.uci.ics.hyracks.control.nc.runtime.OperatorRunnable;
public class Stagelet {
diff --git a/hyracks/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/comm/ConnectionManager.java b/hyracks/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/comm/ConnectionManager.java
index 452f25e..b4868c0 100644
--- a/hyracks/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/comm/ConnectionManager.java
+++ b/hyracks/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/comm/ConnectionManager.java
@@ -41,9 +41,9 @@
import edu.uci.ics.hyracks.api.comm.IDataReceiveListener;
import edu.uci.ics.hyracks.api.comm.IDataReceiveListenerFactory;
import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.comm.NetworkAddress;
import edu.uci.ics.hyracks.api.context.IHyracksContext;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.control.common.comm.NetworkAddress;
public class ConnectionManager {
private static final Logger LOGGER = Logger.getLogger(ConnectionManager.class.getName());
diff --git a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/ArrayTupleBuilder.java b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/ArrayTupleBuilder.java
index c087767..996d7c3 100644
--- a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/ArrayTupleBuilder.java
+++ b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/ArrayTupleBuilder.java
@@ -17,6 +17,7 @@
import java.io.DataOutputStream;
import java.io.IOException;
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
@@ -49,7 +50,7 @@
return baaos.size();
}
- public void addField(FrameTupleAccessor accessor, int tIndex, int fIndex) throws HyracksDataException {
+ public void addField(IFrameTupleAccessor accessor, int tIndex, int fIndex) throws HyracksDataException {
int startOffset = accessor.getTupleStartOffset(tIndex);
int fStartOffset = accessor.getFieldStartOffset(tIndex, fIndex);
int fLen = accessor.getFieldEndOffset(tIndex, fIndex) - fStartOffset;
diff --git a/hyracks/hyracks-examples/pom.xml b/hyracks/hyracks-examples/pom.xml
new file mode 100644
index 0000000..14e9a97
--- /dev/null
+++ b/hyracks/hyracks-examples/pom.xml
@@ -0,0 +1,11 @@
+<project>
+ <modelVersion>4.0.0</modelVersion>
+ <groupId>edu.uci.ics.hyracks</groupId>
+ <artifactId>hyracks-examples</artifactId>
+ <version>0.0.1</version>
+ <packaging>pom</packaging>
+
+ <modules>
+ <module>tpch-example</module>
+ </modules>
+</project>
diff --git a/hyracks/hyracks-examples/tpch-example/pom.xml b/hyracks/hyracks-examples/tpch-example/pom.xml
new file mode 100644
index 0000000..08c283e
--- /dev/null
+++ b/hyracks/hyracks-examples/tpch-example/pom.xml
@@ -0,0 +1,13 @@
+<project>
+ <modelVersion>4.0.0</modelVersion>
+ <groupId>edu.uci.ics.hyracks.examples</groupId>
+ <artifactId>tpch-example</artifactId>
+ <version>0.0.1</version>
+ <packaging>pom</packaging>
+
+ <modules>
+ <module>tpchhelper</module>
+ <module>tpchclient</module>
+ <module>tpchapp</module>
+ </modules>
+</project>
diff --git a/hyracks/hyracks-examples/tpch-example/tpchapp/pom.xml b/hyracks/hyracks-examples/tpch-example/tpchapp/pom.xml
new file mode 100644
index 0000000..9ad9724
--- /dev/null
+++ b/hyracks/hyracks-examples/tpch-example/tpchapp/pom.xml
@@ -0,0 +1,52 @@
+<project>
+ <modelVersion>4.0.0</modelVersion>
+ <groupId>edu.uci.ics.hyracks.examples.tpch</groupId>
+ <artifactId>tpchapp</artifactId>
+ <version>0.0.1</version>
+
+ <build>
+ <plugins>
+ <plugin>
+ <groupId>org.apache.maven.plugins</groupId>
+ <artifactId>maven-dependency-plugin</artifactId>
+ <executions>
+ <execution>
+ <id>copy-dependencies</id>
+ <phase>package</phase>
+ <goals>
+ <goal>copy-dependencies</goal>
+ </goals>
+ <configuration>
+ <outputDirectory>target/application/lib</outputDirectory>
+ </configuration>
+ </execution>
+ </executions>
+ </plugin>
+ <plugin>
+ <artifactId>maven-assembly-plugin</artifactId>
+ <version>2.2-beta-5</version>
+ <executions>
+ <execution>
+ <configuration>
+ <descriptors>
+ <descriptor>src/main/assembly/app-assembly.xml</descriptor>
+ </descriptors>
+ </configuration>
+ <phase>package</phase>
+ <goals>
+ <goal>attached</goal>
+ </goals>
+ </execution>
+ </executions>
+ </plugin>
+ </plugins>
+ </build>
+ <dependencies>
+ <dependency>
+ <groupId>edu.uci.ics.hyracks.examples.tpch</groupId>
+ <artifactId>tpchhelper</artifactId>
+ <version>0.0.1</version>
+ <scope>compile</scope>
+ </dependency>
+ </dependencies>
+</project>
diff --git a/hyracks/hyracks-examples/tpch-example/tpchapp/src/main/assembly/app-assembly.xml b/hyracks/hyracks-examples/tpch-example/tpchapp/src/main/assembly/app-assembly.xml
new file mode 100644
index 0000000..43ace6c
--- /dev/null
+++ b/hyracks/hyracks-examples/tpch-example/tpchapp/src/main/assembly/app-assembly.xml
@@ -0,0 +1,13 @@
+<assembly>
+ <id>app-assembly</id>
+ <formats>
+ <format>zip</format>
+ </formats>
+ <includeBaseDirectory>false</includeBaseDirectory>
+ <fileSets>
+ <fileSet>
+ <directory>target/application/lib</directory>
+ <outputDirectory>lib</outputDirectory>
+ </fileSet>
+ </fileSets>
+</assembly>
diff --git a/hyracks/hyracks-examples/tpch-example/tpchclient/.classpath b/hyracks/hyracks-examples/tpch-example/tpchclient/.classpath
new file mode 100644
index 0000000..1f3c1ff
--- /dev/null
+++ b/hyracks/hyracks-examples/tpch-example/tpchclient/.classpath
@@ -0,0 +1,7 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<classpath>
+ <classpathentry kind="src" output="target/classes" path="src/main/java"/>
+ <classpathentry kind="con" path="org.eclipse.jdt.launching.JRE_CONTAINER/org.eclipse.jdt.internal.debug.ui.launcher.StandardVMType/JavaSE-1.6"/>
+ <classpathentry kind="con" path="org.maven.ide.eclipse.MAVEN2_CLASSPATH_CONTAINER"/>
+ <classpathentry kind="output" path="target/classes"/>
+</classpath>
diff --git a/hyracks/hyracks-examples/tpch-example/tpchclient/.project b/hyracks/hyracks-examples/tpch-example/tpchclient/.project
new file mode 100644
index 0000000..b0effc6
--- /dev/null
+++ b/hyracks/hyracks-examples/tpch-example/tpchclient/.project
@@ -0,0 +1,23 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<projectDescription>
+ <name>tpchclient</name>
+ <comment></comment>
+ <projects>
+ </projects>
+ <buildSpec>
+ <buildCommand>
+ <name>org.eclipse.jdt.core.javabuilder</name>
+ <arguments>
+ </arguments>
+ </buildCommand>
+ <buildCommand>
+ <name>org.maven.ide.eclipse.maven2Builder</name>
+ <arguments>
+ </arguments>
+ </buildCommand>
+ </buildSpec>
+ <natures>
+ <nature>org.maven.ide.eclipse.maven2Nature</nature>
+ <nature>org.eclipse.jdt.core.javanature</nature>
+ </natures>
+</projectDescription>
diff --git a/hyracks/hyracks-examples/tpch-example/tpchclient/.settings/org.eclipse.jdt.core.prefs b/hyracks/hyracks-examples/tpch-example/tpchclient/.settings/org.eclipse.jdt.core.prefs
new file mode 100644
index 0000000..2f0f2e3
--- /dev/null
+++ b/hyracks/hyracks-examples/tpch-example/tpchclient/.settings/org.eclipse.jdt.core.prefs
@@ -0,0 +1,6 @@
+#Wed Aug 11 19:09:15 PDT 2010
+eclipse.preferences.version=1
+org.eclipse.jdt.core.compiler.codegen.targetPlatform=1.6
+org.eclipse.jdt.core.compiler.compliance=1.6
+org.eclipse.jdt.core.compiler.problem.forbiddenReference=warning
+org.eclipse.jdt.core.compiler.source=1.6
diff --git a/hyracks/hyracks-examples/tpch-example/tpchclient/.settings/org.maven.ide.eclipse.prefs b/hyracks/hyracks-examples/tpch-example/tpchclient/.settings/org.maven.ide.eclipse.prefs
new file mode 100644
index 0000000..1b13d8b
--- /dev/null
+++ b/hyracks/hyracks-examples/tpch-example/tpchclient/.settings/org.maven.ide.eclipse.prefs
@@ -0,0 +1,9 @@
+#Wed Aug 11 19:09:15 PDT 2010
+activeProfiles=
+eclipse.preferences.version=1
+fullBuildGoals=process-test-resources
+includeModules=false
+resolveWorkspaceProjects=true
+resourceFilterGoals=process-resources resources\:testResources
+skipCompilerPlugin=true
+version=1
diff --git a/hyracks/hyracks-examples/tpch-example/tpchclient/pom.xml b/hyracks/hyracks-examples/tpch-example/tpchclient/pom.xml
new file mode 100644
index 0000000..648bdcf
--- /dev/null
+++ b/hyracks/hyracks-examples/tpch-example/tpchclient/pom.xml
@@ -0,0 +1,56 @@
+<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.examples.tpch</groupId>
+ <artifactId>tpchclient</artifactId>
+ <version>0.0.1</version>
+ <dependencies>
+ <dependency>
+ <groupId>edu.uci.ics.hyracks</groupId>
+ <artifactId>hyracks-dataflow-std</artifactId>
+ <version>0.1.0</version>
+ <scope>compile</scope>
+ </dependency>
+ <dependency>
+ <groupId>edu.uci.ics.hyracks.examples.tpch</groupId>
+ <artifactId>tpchhelper</artifactId>
+ <version>0.0.1</version>
+ <type>jar</type>
+ <scope>compile</scope>
+ </dependency>
+ </dependencies>
+ <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>
+ <plugin>
+ <groupId>org.codehaus.mojo</groupId>
+ <artifactId>appassembler-maven-plugin</artifactId>
+ <executions>
+ <execution>
+ <configuration>
+ <programs>
+ <program>
+ <mainClass>edu.uci.ics.hyracks.examples.tpch.client.Main</mainClass>
+ <name>tpchclient</name>
+ </program>
+ </programs>
+ <repositoryLayout>flat</repositoryLayout>
+ <repositoryName>lib</repositoryName>
+ </configuration>
+ <phase>package</phase>
+ <goals>
+ <goal>assemble</goal>
+ </goals>
+ </execution>
+ </executions>
+ </plugin>
+ </plugins>
+ </build>
+</project>
diff --git a/hyracks/hyracks-examples/tpch-example/tpchclient/src/main/java/edu/uci/ics/hyracks/examples/tpch/client/Main.java b/hyracks/hyracks-examples/tpch-example/tpchclient/src/main/java/edu/uci/ics/hyracks/examples/tpch/client/Main.java
new file mode 100644
index 0000000..cefb997
--- /dev/null
+++ b/hyracks/hyracks-examples/tpch-example/tpchclient/src/main/java/edu/uci/ics/hyracks/examples/tpch/client/Main.java
@@ -0,0 +1,178 @@
+package edu.uci.ics.hyracks.examples.tpch.client;
+
+import java.io.File;
+import java.util.UUID;
+
+import edu.uci.ics.hyracks.api.client.HyracksRMIConnection;
+import edu.uci.ics.hyracks.api.client.IHyracksClientConnection;
+import edu.uci.ics.hyracks.api.constraints.AbsoluteLocationConstraint;
+import edu.uci.ics.hyracks.api.constraints.ChoiceLocationConstraint;
+import edu.uci.ics.hyracks.api.constraints.ExplicitPartitionConstraint;
+import edu.uci.ics.hyracks.api.constraints.LocationConstraint;
+import edu.uci.ics.hyracks.api.constraints.PartitionConstraint;
+import edu.uci.ics.hyracks.api.constraints.PartitionCountConstraint;
+import edu.uci.ics.hyracks.api.dataflow.IConnectorDescriptor;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.dataflow.common.data.comparators.StringBinaryComparatorFactory;
+import edu.uci.ics.hyracks.dataflow.common.data.hash.StringBinaryHashFunctionFactory;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.StringSerializerDeserializer;
+import edu.uci.ics.hyracks.dataflow.common.data.partition.FieldHashPartitionComputerFactory;
+import edu.uci.ics.hyracks.dataflow.std.connectors.MToNHashPartitioningConnectorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.file.CSVFileScanOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
+import edu.uci.ics.hyracks.dataflow.std.group.HashGroupOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.join.InMemoryHashJoinOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.misc.PrinterOperatorDescriptor;
+import edu.uci.ics.hyracks.examples.tpch.helper.CountAccumulatingAggregatorFactory;
+
+public class Main {
+ public static void main(String[] args) throws Exception {
+ String appName = args[0];
+ String host;
+ int port = 1099;
+ switch (args.length) {
+ case 3:
+ port = Integer.parseInt(args[2]);
+ case 2:
+ host = args[1];
+ break;
+ default:
+ System.err.println("One or Two arguments expected: <cchost> [<ccport>]");
+ return;
+ }
+ IHyracksClientConnection hcc = new HyracksRMIConnection(host, port);
+
+ JobSpecification job = createJob();
+
+ long start = System.currentTimeMillis();
+ UUID jobId = hcc.createJob(appName, job);
+ hcc.start(jobId);
+ hcc.waitForCompletion(jobId);
+ long end = System.currentTimeMillis();
+ System.err.println(start + " " + end + " " + (end - start));
+ }
+
+ private static JobSpecification createJob() {
+ JobSpecification spec = new JobSpecification();
+
+ FileSplit[] custSplits = createCustomerFileSplits();
+ RecordDescriptor custDesc = new RecordDescriptor(new ISerializerDeserializer[] {
+ StringSerializerDeserializer.INSTANCE, StringSerializerDeserializer.INSTANCE,
+ StringSerializerDeserializer.INSTANCE, StringSerializerDeserializer.INSTANCE,
+ StringSerializerDeserializer.INSTANCE, StringSerializerDeserializer.INSTANCE,
+ StringSerializerDeserializer.INSTANCE, StringSerializerDeserializer.INSTANCE });
+
+ FileSplit[] ordersSplits = createOrdersFileSplits();
+ RecordDescriptor ordersDesc = new RecordDescriptor(new ISerializerDeserializer[] {
+ StringSerializerDeserializer.INSTANCE, StringSerializerDeserializer.INSTANCE,
+ StringSerializerDeserializer.INSTANCE, StringSerializerDeserializer.INSTANCE,
+ StringSerializerDeserializer.INSTANCE, StringSerializerDeserializer.INSTANCE,
+ StringSerializerDeserializer.INSTANCE, StringSerializerDeserializer.INSTANCE,
+ StringSerializerDeserializer.INSTANCE });
+
+ RecordDescriptor custOrderJoinDesc = new RecordDescriptor(new ISerializerDeserializer[] {
+ StringSerializerDeserializer.INSTANCE, StringSerializerDeserializer.INSTANCE,
+ StringSerializerDeserializer.INSTANCE, StringSerializerDeserializer.INSTANCE,
+ StringSerializerDeserializer.INSTANCE, StringSerializerDeserializer.INSTANCE,
+ StringSerializerDeserializer.INSTANCE, StringSerializerDeserializer.INSTANCE,
+ StringSerializerDeserializer.INSTANCE, StringSerializerDeserializer.INSTANCE,
+ StringSerializerDeserializer.INSTANCE, StringSerializerDeserializer.INSTANCE,
+ StringSerializerDeserializer.INSTANCE, StringSerializerDeserializer.INSTANCE,
+ StringSerializerDeserializer.INSTANCE, StringSerializerDeserializer.INSTANCE,
+ StringSerializerDeserializer.INSTANCE });
+
+ CSVFileScanOperatorDescriptor ordScanner = new CSVFileScanOperatorDescriptor(spec, ordersSplits, ordersDesc,
+ '|', "'\"");
+ ordScanner.setPartitionConstraint(createRRPartitionConstraint(10));
+
+ CSVFileScanOperatorDescriptor custScanner = new CSVFileScanOperatorDescriptor(spec, custSplits, custDesc, '|',
+ "'\"");
+ custScanner.setPartitionConstraint(createRRPartitionConstraint(10));
+
+ InMemoryHashJoinOperatorDescriptor join = new InMemoryHashJoinOperatorDescriptor(spec, new int[] { 0 },
+ new int[] { 1 }, new IBinaryHashFunctionFactory[] { StringBinaryHashFunctionFactory.INSTANCE },
+ new IBinaryComparatorFactory[] { StringBinaryComparatorFactory.INSTANCE }, custOrderJoinDesc, 6000000);
+ join.setPartitionConstraint(new PartitionCountConstraint(40));
+
+ RecordDescriptor groupResultDesc = new RecordDescriptor(new ISerializerDeserializer[] {
+ StringSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE });
+
+ HashGroupOperatorDescriptor gby = new HashGroupOperatorDescriptor(spec, new int[] { 6 },
+ new FieldHashPartitionComputerFactory(new int[] { 6 },
+ new IBinaryHashFunctionFactory[] { StringBinaryHashFunctionFactory.INSTANCE }),
+ new IBinaryComparatorFactory[] { StringBinaryComparatorFactory.INSTANCE },
+ new CountAccumulatingAggregatorFactory(), groupResultDesc, 16);
+ gby.setPartitionConstraint(new PartitionCountConstraint(40));
+
+ PrinterOperatorDescriptor printer = new PrinterOperatorDescriptor(spec);
+ printer.setPartitionConstraint(new PartitionCountConstraint(40));
+
+ IConnectorDescriptor ordJoinConn = new MToNHashPartitioningConnectorDescriptor(spec,
+ new FieldHashPartitionComputerFactory(new int[] { 1 },
+ new IBinaryHashFunctionFactory[] { StringBinaryHashFunctionFactory.INSTANCE }));
+ spec.connect(ordJoinConn, ordScanner, 0, join, 1);
+
+ IConnectorDescriptor custJoinConn = new MToNHashPartitioningConnectorDescriptor(spec,
+ new FieldHashPartitionComputerFactory(new int[] { 0 },
+ new IBinaryHashFunctionFactory[] { StringBinaryHashFunctionFactory.INSTANCE }));
+ spec.connect(custJoinConn, custScanner, 0, join, 0);
+
+ IConnectorDescriptor joinGroupConn = new MToNHashPartitioningConnectorDescriptor(spec,
+ new FieldHashPartitionComputerFactory(new int[] { 6 },
+ new IBinaryHashFunctionFactory[] { StringBinaryHashFunctionFactory.INSTANCE }));
+ spec.connect(joinGroupConn, join, 0, gby, 0);
+
+ IConnectorDescriptor gbyPrinterConn = new OneToOneConnectorDescriptor(spec);
+ spec.connect(gbyPrinterConn, gby, 0, printer, 0);
+
+ spec.addRoot(printer);
+ return spec;
+ }
+
+ private static FileSplit[] createOrdersFileSplits() {
+ FileSplit fss[] = new FileSplit[40];
+ for (int i = 0; i < fss.length; ++i) {
+ fss[i] = new FileSplit("foo", new File("/home/ubuntu/vinayakb/data/tpch.40.splits/disk" + (i % 4)
+ + "/orders" + i + ".dat"));
+ }
+ return fss;
+ }
+
+ private static FileSplit[] createCustomerFileSplits() {
+ FileSplit fss[] = new FileSplit[40];
+ for (int i = 0; i < fss.length; ++i) {
+ fss[i] = new FileSplit("foo", new File("/home/ubuntu/vinayakb/data/tpch.40.splits/disk" + (i % 4)
+ + "/customer" + i + ".dat"));
+ }
+ return fss;
+ }
+
+ private static final LocationConstraint[] LCS = { new AbsoluteLocationConstraint("asterix-001"),
+ new AbsoluteLocationConstraint("asterix-002"), new AbsoluteLocationConstraint("asterix-003"),
+ new AbsoluteLocationConstraint("asterix-004"), new AbsoluteLocationConstraint("asterix-005"),
+ new AbsoluteLocationConstraint("asterix-006"), new AbsoluteLocationConstraint("asterix-007"),
+ new AbsoluteLocationConstraint("asterix-008"), new AbsoluteLocationConstraint("asterix-009"),
+ new AbsoluteLocationConstraint("asterix-010"), };
+
+ private static PartitionConstraint createRRPartitionConstraint(int k) {
+ LocationConstraint[] lcs = new LocationConstraint[40];
+ for (int i = 0; i < lcs.length; ++i) {
+ lcs[i] = createRRSteppedChoiceConstraint(i, k);
+ }
+ return new ExplicitPartitionConstraint(lcs);
+ }
+
+ private static LocationConstraint createRRSteppedChoiceConstraint(int index, int choices) {
+ LocationConstraint[] lcs = new LocationConstraint[choices];
+ for (int i = 0; i < choices; ++i) {
+ lcs[i] = LCS[(index + i) % LCS.length];
+ }
+ return new ChoiceLocationConstraint(lcs);
+ }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-examples/tpch-example/tpchhelper/.classpath b/hyracks/hyracks-examples/tpch-example/tpchhelper/.classpath
new file mode 100644
index 0000000..1f3c1ff
--- /dev/null
+++ b/hyracks/hyracks-examples/tpch-example/tpchhelper/.classpath
@@ -0,0 +1,7 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<classpath>
+ <classpathentry kind="src" output="target/classes" path="src/main/java"/>
+ <classpathentry kind="con" path="org.eclipse.jdt.launching.JRE_CONTAINER/org.eclipse.jdt.internal.debug.ui.launcher.StandardVMType/JavaSE-1.6"/>
+ <classpathentry kind="con" path="org.maven.ide.eclipse.MAVEN2_CLASSPATH_CONTAINER"/>
+ <classpathentry kind="output" path="target/classes"/>
+</classpath>
diff --git a/hyracks/hyracks-examples/tpch-example/tpchhelper/.project b/hyracks/hyracks-examples/tpch-example/tpchhelper/.project
new file mode 100644
index 0000000..41b2574
--- /dev/null
+++ b/hyracks/hyracks-examples/tpch-example/tpchhelper/.project
@@ -0,0 +1,23 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<projectDescription>
+ <name>tpchhelper</name>
+ <comment></comment>
+ <projects>
+ </projects>
+ <buildSpec>
+ <buildCommand>
+ <name>org.eclipse.jdt.core.javabuilder</name>
+ <arguments>
+ </arguments>
+ </buildCommand>
+ <buildCommand>
+ <name>org.maven.ide.eclipse.maven2Builder</name>
+ <arguments>
+ </arguments>
+ </buildCommand>
+ </buildSpec>
+ <natures>
+ <nature>org.maven.ide.eclipse.maven2Nature</nature>
+ <nature>org.eclipse.jdt.core.javanature</nature>
+ </natures>
+</projectDescription>
diff --git a/hyracks/hyracks-examples/tpch-example/tpchhelper/.settings/org.eclipse.jdt.core.prefs b/hyracks/hyracks-examples/tpch-example/tpchhelper/.settings/org.eclipse.jdt.core.prefs
new file mode 100644
index 0000000..4898439
--- /dev/null
+++ b/hyracks/hyracks-examples/tpch-example/tpchhelper/.settings/org.eclipse.jdt.core.prefs
@@ -0,0 +1,6 @@
+#Wed Aug 11 19:04:13 PDT 2010
+eclipse.preferences.version=1
+org.eclipse.jdt.core.compiler.codegen.targetPlatform=1.6
+org.eclipse.jdt.core.compiler.compliance=1.6
+org.eclipse.jdt.core.compiler.problem.forbiddenReference=warning
+org.eclipse.jdt.core.compiler.source=1.6
diff --git a/hyracks/hyracks-examples/tpch-example/tpchhelper/.settings/org.maven.ide.eclipse.prefs b/hyracks/hyracks-examples/tpch-example/tpchhelper/.settings/org.maven.ide.eclipse.prefs
new file mode 100644
index 0000000..032b327
--- /dev/null
+++ b/hyracks/hyracks-examples/tpch-example/tpchhelper/.settings/org.maven.ide.eclipse.prefs
@@ -0,0 +1,9 @@
+#Wed Aug 11 19:04:13 PDT 2010
+activeProfiles=
+eclipse.preferences.version=1
+fullBuildGoals=process-test-resources
+includeModules=false
+resolveWorkspaceProjects=true
+resourceFilterGoals=process-resources resources\:testResources
+skipCompilerPlugin=true
+version=1
diff --git a/hyracks/hyracks-examples/tpch-example/tpchhelper/pom.xml b/hyracks/hyracks-examples/tpch-example/tpchhelper/pom.xml
new file mode 100644
index 0000000..14b9ec7
--- /dev/null
+++ b/hyracks/hyracks-examples/tpch-example/tpchhelper/pom.xml
@@ -0,0 +1,33 @@
+<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.examples.tpch</groupId>
+ <artifactId>tpchhelper</artifactId>
+ <version>0.0.1</version>
+ <dependencies>
+ <dependency>
+ <groupId>edu.uci.ics.hyracks</groupId>
+ <artifactId>hyracks-dataflow-std</artifactId>
+ <version>0.1.0</version>
+ <scope>compile</scope>
+ </dependency>
+ <dependency>
+ <groupId>edu.uci.ics.hyracks</groupId>
+ <artifactId>hyracks-api</artifactId>
+ <version>0.1.0</version>
+ <scope>compile</scope>
+ </dependency>
+ </dependencies>
+ <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>
+</project>
diff --git a/hyracks/hyracks-examples/tpch-example/tpchhelper/src/main/java/edu/uci/ics/hyracks/examples/tpch/helper/CountAccumulatingAggregatorFactory.java b/hyracks/hyracks-examples/tpch-example/tpchhelper/src/main/java/edu/uci/ics/hyracks/examples/tpch/helper/CountAccumulatingAggregatorFactory.java
new file mode 100644
index 0000000..cbc8a57
--- /dev/null
+++ b/hyracks/hyracks-examples/tpch-example/tpchhelper/src/main/java/edu/uci/ics/hyracks/examples/tpch/helper/CountAccumulatingAggregatorFactory.java
@@ -0,0 +1,41 @@
+package edu.uci.ics.hyracks.examples.tpch.helper;
+
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
+import edu.uci.ics.hyracks.dataflow.std.group.IAccumulatingAggregator;
+import edu.uci.ics.hyracks.dataflow.std.group.IAccumulatingAggregatorFactory;
+
+public class CountAccumulatingAggregatorFactory implements IAccumulatingAggregatorFactory {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public IAccumulatingAggregator createAggregator(RecordDescriptor inRecordDesc, RecordDescriptor outRecordDescriptor) {
+ final ArrayTupleBuilder tb = new ArrayTupleBuilder(2);
+ return new IAccumulatingAggregator() {
+ private int count;
+
+ @Override
+ public boolean output(FrameTupleAppender appender, IFrameTupleAccessor accessor, int tIndex)
+ throws HyracksDataException {
+ tb.reset();
+ tb.addField(accessor, tIndex, 0);
+ tb.addField(IntegerSerializerDeserializer.INSTANCE, count);
+ return appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize());
+ }
+
+ @Override
+ public void init(IFrameTupleAccessor accessor, int tIndex) throws HyracksDataException {
+ count = 0;
+ }
+
+ @Override
+ public void accumulate(IFrameTupleAccessor accessor, int tIndex) throws HyracksDataException {
+ ++count;
+ }
+ };
+ }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-server/src/test/java/edu/uci/ics/hyracks/tests/integration/AbstractIntegrationTest.java b/hyracks/hyracks-server/src/test/java/edu/uci/ics/hyracks/tests/integration/AbstractIntegrationTest.java
index 6232e6c..c542f51 100644
--- a/hyracks/hyracks-server/src/test/java/edu/uci/ics/hyracks/tests/integration/AbstractIntegrationTest.java
+++ b/hyracks/hyracks-server/src/test/java/edu/uci/ics/hyracks/tests/integration/AbstractIntegrationTest.java
@@ -22,11 +22,11 @@
import edu.uci.ics.hyracks.api.client.HyracksLocalConnection;
import edu.uci.ics.hyracks.api.client.IHyracksClientConnection;
+import edu.uci.ics.hyracks.api.control.NCConfig;
import edu.uci.ics.hyracks.api.job.JobFlag;
import edu.uci.ics.hyracks.api.job.JobSpecification;
import edu.uci.ics.hyracks.control.cc.CCConfig;
import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
-import edu.uci.ics.hyracks.control.common.api.NCConfig;
import edu.uci.ics.hyracks.control.nc.NodeControllerService;
public abstract class AbstractIntegrationTest {
@@ -63,6 +63,7 @@
nc2.start();
hcc = new HyracksLocalConnection(cc);
+ hcc.createApplication("test", null);
}
@AfterClass
@@ -73,7 +74,7 @@
}
void runTest(JobSpecification spec) throws Exception {
- UUID jobId = hcc.createJob(spec, EnumSet.of(JobFlag.COLLECT_FRAME_COUNTS));
+ UUID jobId = hcc.createJob("test", spec, EnumSet.of(JobFlag.COLLECT_FRAME_COUNTS));
System.err.println(spec.toJSON());
hcc.start(jobId);
System.err.print(jobId);
diff --git a/hyracks/pom.xml b/hyracks/pom.xml
index f6742d1..7adacb9 100644
--- a/hyracks/pom.xml
+++ b/hyracks/pom.xml
@@ -13,5 +13,7 @@
<module>hyracks-control-cc</module>
<module>hyracks-control-nc</module>
<module>hyracks-server</module>
+ <module>hyracks-cli</module>
+ <module>hyracks-examples</module>
</modules>
</project>