Eliminated Application deployment code. Eliminated hyracks-cli. Fixed hyracks examples to work with the new application model
git-svn-id: https://hyracks.googlecode.com/svn/branches/fullstack_hyracks_ioc@2433 123451ca-8445-de46-9d55-352943316053
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/application/IApplicationContext.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/application/IApplicationContext.java
index 81cf511..1b079d6 100644
--- a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/application/IApplicationContext.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/application/IApplicationContext.java
@@ -19,22 +19,12 @@
import edu.uci.ics.hyracks.api.messages.IMessageBroker;
/**
- * Base class of the {@link ICCApplicationContext} and the
- * {@link INCApplicationContext}.
+ * Base class of the {@link ICCApplicationContext} and the {@link INCApplicationContext}.
*
* @author vinayakb
- *
*/
public interface IApplicationContext {
/**
- * Provides the Class Loader that loads classes for this Hyracks Application
- * at the CC.
- *
- * @return the application {@link ClassLoader}.
- */
- public ClassLoader getClassLoader();
-
- /**
* Gets the distributed state that is made available to all the Application
* Contexts of this application in the cluster.
*
@@ -45,6 +35,4 @@
public void setMessageBroker(IMessageBroker messageBroker);
public IMessageBroker getMessageBroker();
-
- public String getApplicationName();
}
\ No newline at end of file
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/application/ICCApplicationEntryPoint.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/application/ICCApplicationEntryPoint.java
new file mode 100644
index 0000000..cc1d93e
--- /dev/null
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/application/ICCApplicationEntryPoint.java
@@ -0,0 +1,5 @@
+package edu.uci.ics.hyracks.api.application;
+
+public interface ICCApplicationEntryPoint {
+ public void appMain(ICCApplicationContext ccAppCtx, String[] args) throws Exception;
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/application/INCApplicationEntryPoint.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/application/INCApplicationEntryPoint.java
new file mode 100644
index 0000000..afa824d
--- /dev/null
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/application/INCApplicationEntryPoint.java
@@ -0,0 +1,5 @@
+package edu.uci.ics.hyracks.api.application;
+
+public interface INCApplicationEntryPoint {
+ public void appMain(INCApplicationContext ncAppCtx, String[] args) throws Exception;
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/HyracksClientInterfaceFunctions.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/HyracksClientInterfaceFunctions.java
index e34e60d..910eb92 100644
--- a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/HyracksClientInterfaceFunctions.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/HyracksClientInterfaceFunctions.java
@@ -24,9 +24,6 @@
public enum FunctionId {
GET_CLUSTER_CONTROLLER_INFO,
GET_CLUSTER_TOPOLOGY,
- CREATE_APPLICATION,
- START_APPLICATION,
- DESTROY_APPLICATION,
CREATE_JOB,
GET_JOB_STATUS,
START_JOB,
@@ -49,63 +46,6 @@
}
}
- public static class CreateApplicationFunction extends Function {
- private static final long serialVersionUID = 1L;
-
- private final String appName;
-
- public CreateApplicationFunction(String appName) {
- this.appName = appName;
- }
-
- @Override
- public FunctionId getFunctionId() {
- return FunctionId.CREATE_APPLICATION;
- }
-
- public String getAppName() {
- return appName;
- }
- }
-
- public static class StartApplicationFunction extends Function {
- private static final long serialVersionUID = 1L;
-
- private final String appName;
-
- public StartApplicationFunction(String appName) {
- this.appName = appName;
- }
-
- @Override
- public FunctionId getFunctionId() {
- return FunctionId.START_APPLICATION;
- }
-
- public String getAppName() {
- return appName;
- }
- }
-
- public static class DestroyApplicationFunction extends Function {
- private static final long serialVersionUID = 1L;
-
- private final String appName;
-
- public DestroyApplicationFunction(String appName) {
- this.appName = appName;
- }
-
- @Override
- public FunctionId getFunctionId() {
- return FunctionId.DESTROY_APPLICATION;
- }
-
- public String getAppName() {
- return appName;
- }
- }
-
public static class GetJobStatusFunction extends Function {
private static final long serialVersionUID = 1L;
@@ -128,12 +68,10 @@
public static class StartJobFunction extends Function {
private static final long serialVersionUID = 1L;
- private final String appName;
private final byte[] acggfBytes;
private final EnumSet<JobFlag> jobFlags;
- public StartJobFunction(String appName, byte[] acggfBytes, EnumSet<JobFlag> jobFlags) {
- this.appName = appName;
+ public StartJobFunction(byte[] acggfBytes, EnumSet<JobFlag> jobFlags) {
this.acggfBytes = acggfBytes;
this.jobFlags = jobFlags;
}
@@ -143,10 +81,6 @@
return FunctionId.START_JOB;
}
- public String getAppName() {
- return appName;
- }
-
public byte[] getACGGFBytes() {
return acggfBytes;
}
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/HyracksClientInterfaceRemoteProxy.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/HyracksClientInterfaceRemoteProxy.java
index 4c06d42..b85998a 100644
--- a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/HyracksClientInterfaceRemoteProxy.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/HyracksClientInterfaceRemoteProxy.java
@@ -41,27 +41,6 @@
}
@Override
- public void createApplication(String appName) throws Exception {
- HyracksClientInterfaceFunctions.CreateApplicationFunction caf = new HyracksClientInterfaceFunctions.CreateApplicationFunction(
- appName);
- rpci.call(ipcHandle, caf);
- }
-
- @Override
- public void startApplication(String appName) throws Exception {
- HyracksClientInterfaceFunctions.StartApplicationFunction saf = new HyracksClientInterfaceFunctions.StartApplicationFunction(
- appName);
- rpci.call(ipcHandle, saf);
- }
-
- @Override
- public void destroyApplication(String appName) throws Exception {
- HyracksClientInterfaceFunctions.DestroyApplicationFunction daf = new HyracksClientInterfaceFunctions.DestroyApplicationFunction(
- appName);
- rpci.call(ipcHandle, daf);
- }
-
- @Override
public JobStatus getJobStatus(JobId jobId) throws Exception {
HyracksClientInterfaceFunctions.GetJobStatusFunction gjsf = new HyracksClientInterfaceFunctions.GetJobStatusFunction(
jobId);
@@ -69,9 +48,9 @@
}
@Override
- public JobId startJob(String appName, byte[] acggfBytes, EnumSet<JobFlag> jobFlags) throws Exception {
+ public JobId startJob(byte[] acggfBytes, EnumSet<JobFlag> jobFlags) throws Exception {
HyracksClientInterfaceFunctions.StartJobFunction sjf = new HyracksClientInterfaceFunctions.StartJobFunction(
- appName, acggfBytes, jobFlags);
+ acggfBytes, jobFlags);
return (JobId) rpci.call(ipcHandle, sjf);
}
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/HyracksConnection.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/HyracksConnection.java
index 227524c..2defa45 100644
--- a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/HyracksConnection.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/HyracksConnection.java
@@ -14,19 +14,11 @@
*/
package edu.uci.ics.hyracks.api.client;
-import java.io.File;
import java.net.InetSocketAddress;
import java.util.EnumSet;
import java.util.Map;
-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.client.impl.JobSpecificationActivityClusterGraphGeneratorFactory;
-import edu.uci.ics.hyracks.api.exceptions.HyracksException;
import edu.uci.ics.hyracks.api.job.IActivityClusterGraphGeneratorFactory;
import edu.uci.ics.hyracks.api.job.JobFlag;
import edu.uci.ics.hyracks.api.job.JobId;
@@ -76,46 +68,24 @@
}
@Override
- public void createApplication(String appName, File harFile) throws Exception {
- hci.createApplication(appName);
- 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);
- }
-
- @Override
- public void destroyApplication(String appName) throws Exception {
- hci.destroyApplication(appName);
- }
-
- @Override
public JobStatus getJobStatus(JobId jobId) throws Exception {
return hci.getJobStatus(jobId);
}
@Override
- public JobId startJob(String appName, JobSpecification jobSpec) throws Exception {
- return startJob(appName, jobSpec, EnumSet.noneOf(JobFlag.class));
+ public JobId startJob(JobSpecification jobSpec) throws Exception {
+ return startJob(jobSpec, EnumSet.noneOf(JobFlag.class));
}
@Override
- public JobId startJob(String appName, JobSpecification jobSpec, EnumSet<JobFlag> jobFlags) throws Exception {
+ public JobId startJob(JobSpecification jobSpec, EnumSet<JobFlag> jobFlags) throws Exception {
JobSpecificationActivityClusterGraphGeneratorFactory jsacggf = new JobSpecificationActivityClusterGraphGeneratorFactory(
jobSpec);
- return startJob(appName, jsacggf, jobFlags);
+ return startJob(jsacggf, jobFlags);
}
- public JobId startJob(String appName, IActivityClusterGraphGeneratorFactory acggf, EnumSet<JobFlag> jobFlags)
- throws Exception {
- return hci.startJob(appName, JavaSerializationUtils.serialize(acggf), jobFlags);
+ public JobId startJob(IActivityClusterGraphGeneratorFactory acggf, EnumSet<JobFlag> jobFlags) throws Exception {
+ return hci.startJob(JavaSerializationUtils.serialize(acggf), jobFlags);
}
@Override
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 bdbb544..fcfa722 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,7 +14,6 @@
*/
package edu.uci.ics.hyracks.api.client;
-import java.io.File;
import java.util.EnumSet;
import java.util.Map;
@@ -32,26 +31,6 @@
*/
public interface IHyracksClientConnection {
/**
- * Create a Hyracks Application
- *
- * @param appName
- * Name of the application
- * @param harFile
- * Archive that contains deployable code for the application
- * @throws Exception
- */
- public void createApplication(String appName, File harFile) throws Exception;
-
- /**
- * Destroy an already-deployed Hyracks application
- *
- * @param appName
- * Name of the application
- * @throws Exception
- */
- public void destroyApplication(String appName) throws Exception;
-
- /**
* Gets the status of the specified Job.
*
* @param jobId
@@ -70,7 +49,7 @@
* Job Specification
* @throws Exception
*/
- public JobId startJob(String appName, JobSpecification jobSpec) throws Exception;
+ public JobId startJob(JobSpecification jobSpec) throws Exception;
/**
* Start the specified Job.
@@ -83,7 +62,7 @@
* Flags
* @throws Exception
*/
- public JobId startJob(String appName, JobSpecification jobSpec, EnumSet<JobFlag> jobFlags) throws Exception;
+ public JobId startJob(JobSpecification jobSpec, EnumSet<JobFlag> jobFlags) throws Exception;
/**
* Start the specified Job.
@@ -96,8 +75,7 @@
* Flags
* @throws Exception
*/
- public JobId startJob(String appName, IActivityClusterGraphGeneratorFactory acggf, EnumSet<JobFlag> jobFlags)
- throws Exception;
+ public JobId startJob(IActivityClusterGraphGeneratorFactory acggf, EnumSet<JobFlag> jobFlags) throws Exception;
/**
* Waits until the specified job has completed, either successfully or has
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 ef5906e..33ce29a 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
@@ -25,15 +25,9 @@
public interface IHyracksClientInterface {
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 JobStatus getJobStatus(JobId jobId) throws Exception;
- public JobId startJob(String appName, byte[] acggfBytes, EnumSet<JobFlag> jobFlags) throws Exception;
+ public JobId startJob(byte[] acggfBytes, EnumSet<JobFlag> jobFlags) throws Exception;
public void waitForCompletion(JobId jobId) throws Exception;
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/impl/JobSpecificationActivityClusterGraphGeneratorFactory.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/impl/JobSpecificationActivityClusterGraphGeneratorFactory.java
index f36b7b3..dd45825 100644
--- a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/impl/JobSpecificationActivityClusterGraphGeneratorFactory.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/impl/JobSpecificationActivityClusterGraphGeneratorFactory.java
@@ -28,7 +28,7 @@
}
@Override
- public IActivityClusterGraphGenerator createActivityClusterGraphGenerator(String appName, JobId jobId,
+ public IActivityClusterGraphGenerator createActivityClusterGraphGenerator(JobId jobId,
final ICCApplicationContext ccAppCtx, EnumSet<JobFlag> jobFlags) throws HyracksException {
final JobActivityGraphBuilder builder = new JobActivityGraphBuilder(spec, jobFlags);
PlanUtils.visit(spec, new IConnectorDescriptorVisitor() {
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/IActivityClusterGraphGeneratorFactory.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/IActivityClusterGraphGeneratorFactory.java
index d801dd1..ae76455 100644
--- a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/IActivityClusterGraphGeneratorFactory.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/IActivityClusterGraphGeneratorFactory.java
@@ -21,6 +21,6 @@
import edu.uci.ics.hyracks.api.exceptions.HyracksException;
public interface IActivityClusterGraphGeneratorFactory extends Serializable {
- public IActivityClusterGraphGenerator createActivityClusterGraphGenerator(String appName, JobId jobId,
+ public IActivityClusterGraphGenerator createActivityClusterGraphGenerator(JobId jobId,
ICCApplicationContext ccAppCtx, EnumSet<JobFlag> jobFlags) throws HyracksException;
}
\ No newline at end of file
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/util/JavaSerializationUtils.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/util/JavaSerializationUtils.java
index 1185b5e..764348f 100644
--- a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/util/JavaSerializationUtils.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/util/JavaSerializationUtils.java
@@ -33,6 +33,14 @@
return baos.toByteArray();
}
+ public static Object deserialize(byte[] bytes) throws IOException, ClassNotFoundException {
+ if (bytes == null) {
+ return null;
+ }
+ ObjectInputStream ois = new ObjectInputStream(new ByteArrayInputStream(bytes));
+ return ois.readObject();
+ }
+
public static Object deserialize(byte[] bytes, ClassLoader classLoader) throws IOException, ClassNotFoundException {
if (bytes == null) {
return null;
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
deleted file mode 100644
index f01c724..0000000
--- a/hyracks/hyracks-cli/src/main/java/edu/uci/ics/hyracks/cli/CLI.java
+++ /dev/null
@@ -1,63 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.cli;
-
-import java.io.IOException;
-
-import jline.ConsoleReader;
-
-public class CLI {
- private static final String HYRACKS_PROMPT = "hyracks> ";
- private static final String HYRAX_CONTINUE_PROMPT = "> ";
- private final ConsoleReader reader;
- private final Session session;
-
- public CLI(String[] args) throws IOException {
- reader = new ConsoleReader();
- session = new Session();
- }
-
- public void run() throws IOException {
- boolean eof = false;
- while (true) {
- String prompt = HYRACKS_PROMPT;
- StringBuilder command = new StringBuilder();
- 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 {
- CommandExecutor.execute(session, command.toString());
- } catch (Exception e) {
- e.printStackTrace();
- }
- }
- }
-}
\ No newline at end of file
diff --git a/hyracks/hyracks-cli/src/main/java/edu/uci/ics/hyracks/cli/CommandExecutor.java b/hyracks/hyracks-cli/src/main/java/edu/uci/ics/hyracks/cli/CommandExecutor.java
deleted file mode 100644
index 6bdd136..0000000
--- a/hyracks/hyracks-cli/src/main/java/edu/uci/ics/hyracks/cli/CommandExecutor.java
+++ /dev/null
@@ -1,30 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.cli;
-
-import java.io.StringReader;
-import java.util.List;
-
-import edu.uci.ics.hyracks.cli.commands.Command;
-
-public class CommandExecutor {
- public static void execute(Session session, String command) throws Exception {
- CLIParser parser = new CLIParser(new StringReader(command));
- List<Command> cmds = parser.Commands();
- for (Command cmd : cmds) {
- cmd.run(session);
- }
- }
-}
\ No newline at end of file
diff --git a/hyracks/hyracks-cli/src/main/java/edu/uci/ics/hyracks/cli/Main.java b/hyracks/hyracks-cli/src/main/java/edu/uci/ics/hyracks/cli/Main.java
deleted file mode 100644
index 5f7ef0b..0000000
--- a/hyracks/hyracks-cli/src/main/java/edu/uci/ics/hyracks/cli/Main.java
+++ /dev/null
@@ -1,22 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.cli;
-
-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/Session.java b/hyracks/hyracks-cli/src/main/java/edu/uci/ics/hyracks/cli/Session.java
deleted file mode 100644
index f15b1bb..0000000
--- a/hyracks/hyracks-cli/src/main/java/edu/uci/ics/hyracks/cli/Session.java
+++ /dev/null
@@ -1,29 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.cli;
-
-import edu.uci.ics.hyracks.api.client.IHyracksClientConnection;
-
-public class Session {
- private IHyracksClientConnection connection;
-
- public void setConnection(IHyracksClientConnection connection) {
- this.connection = connection;
- }
-
- public IHyracksClientConnection getConnection() {
- return connection;
- }
-}
\ 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
deleted file mode 100644
index ee5849b..0000000
--- a/hyracks/hyracks-cli/src/main/java/edu/uci/ics/hyracks/cli/commands/Command.java
+++ /dev/null
@@ -1,7 +0,0 @@
-package edu.uci.ics.hyracks.cli.commands;
-
-import edu.uci.ics.hyracks.cli.Session;
-
-public abstract class Command {
- public abstract void run(Session session) 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
deleted file mode 100644
index 168280b..0000000
--- a/hyracks/hyracks-cli/src/main/java/edu/uci/ics/hyracks/cli/commands/ConnectCommand.java
+++ /dev/null
@@ -1,28 +0,0 @@
-package edu.uci.ics.hyracks.cli.commands;
-
-import edu.uci.ics.hyracks.api.client.HyracksConnection;
-import edu.uci.ics.hyracks.api.client.IHyracksClientConnection;
-import edu.uci.ics.hyracks.cli.Session;
-
-public class ConnectCommand extends Command {
- private String host;
-
- private int port;
-
- public ConnectCommand(String hostPortStr) {
- int idx = hostPortStr.indexOf(':');
- host = hostPortStr;
- port = 1098;
- if (idx != -1) {
- host = hostPortStr.substring(0, idx);
- port = Integer.valueOf(hostPortStr.substring(idx + 1));
- }
- }
-
- @Override
- public void run(Session session) throws Exception {
- System.err.println("Connecting to host: " + host + ", port: " + port);
- IHyracksClientConnection conn = new HyracksConnection(host, port);
- session.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
deleted file mode 100644
index 7b35ee5..0000000
--- a/hyracks/hyracks-cli/src/main/java/edu/uci/ics/hyracks/cli/commands/CreateApplicationCommand.java
+++ /dev/null
@@ -1,27 +0,0 @@
-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.Session;
-
-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(Session session) throws Exception {
- IHyracksClientConnection hcc = session.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
deleted file mode 100644
index 03aed1e..0000000
--- a/hyracks/hyracks-cli/src/main/java/edu/uci/ics/hyracks/cli/commands/DestroyApplicationCommand.java
+++ /dev/null
@@ -1,22 +0,0 @@
-package edu.uci.ics.hyracks.cli.commands;
-
-import edu.uci.ics.hyracks.api.client.IHyracksClientConnection;
-import edu.uci.ics.hyracks.cli.Session;
-
-public class DestroyApplicationCommand extends Command {
- private String appName;
-
- public DestroyApplicationCommand(String appName) {
- this.appName = appName;
- }
-
- @Override
- public void run(Session session) throws Exception {
- IHyracksClientConnection hcc = session.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
deleted file mode 100644
index facf0f1..0000000
--- a/hyracks/hyracks-cli/src/main/java/edu/uci/ics/hyracks/cli/commands/DisconnectCommand.java
+++ /dev/null
@@ -1,11 +0,0 @@
-package edu.uci.ics.hyracks.cli.commands;
-
-import edu.uci.ics.hyracks.cli.Session;
-
-public class DisconnectCommand extends Command {
- @Override
- public void run(Session session) throws Exception {
- System.err.println("Disconnecting...");
- session.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
deleted file mode 100644
index 0f386cb..0000000
--- a/hyracks/hyracks-cli/src/main/javacc/cli.jj
+++ /dev/null
@@ -1,131 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-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/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/ClusterControllerService.java b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/ClusterControllerService.java
index 5a33891..677cd18 100644
--- a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/ClusterControllerService.java
+++ b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/ClusterControllerService.java
@@ -18,7 +18,6 @@
import java.io.FileReader;
import java.net.InetSocketAddress;
import java.util.HashMap;
-import java.util.Hashtable;
import java.util.LinkedHashMap;
import java.util.Map;
import java.util.Set;
@@ -32,6 +31,7 @@
import org.xml.sax.InputSource;
+import edu.uci.ics.hyracks.api.application.ICCApplicationEntryPoint;
import edu.uci.ics.hyracks.api.client.ClusterControllerInfo;
import edu.uci.ics.hyracks.api.client.HyracksClientInterfaceFunctions;
import edu.uci.ics.hyracks.api.client.NodeControllerInfo;
@@ -43,11 +43,7 @@
import edu.uci.ics.hyracks.control.cc.application.CCApplicationContext;
import edu.uci.ics.hyracks.control.cc.job.JobRun;
import edu.uci.ics.hyracks.control.cc.web.WebServer;
-import edu.uci.ics.hyracks.control.cc.work.ApplicationCreateWork;
-import edu.uci.ics.hyracks.control.cc.work.ApplicationDestroyWork;
import edu.uci.ics.hyracks.control.cc.work.ApplicationMessageWork;
-import edu.uci.ics.hyracks.control.cc.work.ApplicationStartWork;
-import edu.uci.ics.hyracks.control.cc.work.ApplicationStateChangeWork;
import edu.uci.ics.hyracks.control.cc.work.GetIpAddressNodeNameMapWork;
import edu.uci.ics.hyracks.control.cc.work.GetJobStatusWork;
import edu.uci.ics.hyracks.control.cc.work.GetNodeControllersInfoWork;
@@ -93,14 +89,14 @@
private final Map<String, Set<String>> ipAddressNodeNameMap;
- private final Map<String, CCApplicationContext> applications;
-
private final ServerContext serverCtx;
private final WebServer webServer;
private ClusterControllerInfo info;
+ private CCApplicationContext appCtx;
+
private final Map<JobId, JobRun> activeRunMap;
private final Map<JobId, JobRun> runMapArchive;
@@ -123,7 +119,6 @@
jobLog = new LogFile(jobLogFolder);
nodeRegistry = new LinkedHashMap<String, NodeControllerState>();
ipAddressNodeNameMap = new HashMap<String, Set<String>>();
- applications = new Hashtable<String, CCApplicationContext>();
serverCtx = new ServerContext(ServerContext.ServerType.CLUSTER_CONTROLLER, new File(ccConfig.ccRoot));
executor = Executors.newCachedThreadPool();
IIPCI ccIPCI = new ClusterControllerIPCI();
@@ -190,9 +185,22 @@
webServer.getListeningPort());
timer.schedule(sweeper, 0, ccConfig.heartbeatPeriod);
jobLog.open();
+ startApplication();
LOGGER.log(Level.INFO, "Started ClusterControllerService");
}
+ private void startApplication() throws Exception {
+ appCtx = new CCApplicationContext(serverCtx, ccContext);
+ String className = ccConfig.appCCMainClass;
+ if (className != null) {
+ Class<?> c = Class.forName(className);
+ ICCApplicationEntryPoint aep = (ICCApplicationEntryPoint) c.newInstance();
+ String[] args = ccConfig.appArgs == null ? null : ccConfig.appArgs.toArray(new String[ccConfig.appArgs
+ .size()]);
+ aep.appMain(appCtx, args);
+ }
+ }
+
@Override
public void stop() throws Exception {
LOGGER.log(Level.INFO, "Stopping ClusterControllerService");
@@ -212,10 +220,6 @@
return ccContext;
}
- public Map<String, CCApplicationContext> getApplicationMap() {
- return applications;
- }
-
public Map<JobId, JobRun> getActiveRunMap() {
return activeRunMap;
}
@@ -248,6 +252,10 @@
return ccConfig;
}
+ public CCApplicationContext getApplicationContext() {
+ return appCtx;
+ }
+
private JobId createJobId() {
return new JobId(jobCounter++);
}
@@ -285,27 +293,6 @@
return;
}
- case CREATE_APPLICATION: {
- HyracksClientInterfaceFunctions.CreateApplicationFunction caf = (HyracksClientInterfaceFunctions.CreateApplicationFunction) fn;
- workQueue.schedule(new ApplicationCreateWork(ClusterControllerService.this, caf.getAppName(),
- new IPCResponder<Object>(handle, mid)));
- return;
- }
-
- case START_APPLICATION: {
- HyracksClientInterfaceFunctions.StartApplicationFunction saf = (HyracksClientInterfaceFunctions.StartApplicationFunction) fn;
- workQueue.schedule(new ApplicationStartWork(ClusterControllerService.this, saf.getAppName(),
- new IPCResponder<Object>(handle, mid)));
- return;
- }
-
- case DESTROY_APPLICATION: {
- HyracksClientInterfaceFunctions.DestroyApplicationFunction daf = (HyracksClientInterfaceFunctions.DestroyApplicationFunction) fn;
- workQueue.schedule(new ApplicationDestroyWork(ClusterControllerService.this, daf.getAppName(),
- new IPCResponder<Object>(handle, mid)));
- return;
- }
-
case GET_JOB_STATUS: {
HyracksClientInterfaceFunctions.GetJobStatusFunction gjsf = (HyracksClientInterfaceFunctions.GetJobStatusFunction) fn;
workQueue.schedule(new GetJobStatusWork(ClusterControllerService.this, gjsf.getJobId(),
@@ -316,8 +303,8 @@
case START_JOB: {
HyracksClientInterfaceFunctions.StartJobFunction sjf = (HyracksClientInterfaceFunctions.StartJobFunction) fn;
JobId jobId = createJobId();
- workQueue.schedule(new JobStartWork(ClusterControllerService.this, sjf.getAppName(), sjf
- .getACGGFBytes(), sjf.getJobFlags(), jobId, new IPCResponder<JobId>(handle, mid)));
+ workQueue.schedule(new JobStartWork(ClusterControllerService.this, sjf.getACGGFBytes(), sjf
+ .getJobFlags(), jobId, new IPCResponder<JobId>(handle, mid)));
return;
}
@@ -416,15 +403,10 @@
return;
}
- case APPLICATION_STATE_CHANGE_RESPONSE: {
- CCNCFunctions.ApplicationStateChangeResponseFunction astrf = (CCNCFunctions.ApplicationStateChangeResponseFunction) fn;
- workQueue.schedule(new ApplicationStateChangeWork(ClusterControllerService.this, astrf));
- return;
- }
case SEND_APPLICATION_MESSAGE: {
CCNCFunctions.SendApplicationMessageFunction rsf = (CCNCFunctions.SendApplicationMessageFunction) fn;
workQueue.schedule(new ApplicationMessageWork(ClusterControllerService.this, rsf.getMessage(), rsf
- .getAppName(), rsf.getNodeId()));
+ .getNodeId()));
return;
}
diff --git a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/application/CCApplicationContext.java b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/application/CCApplicationContext.java
index 24dfa7c..7e1581a 100644
--- a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/application/CCApplicationContext.java
+++ b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/application/CCApplicationContext.java
@@ -22,13 +22,11 @@
import java.util.Set;
import edu.uci.ics.hyracks.api.application.ICCApplicationContext;
-import edu.uci.ics.hyracks.api.application.ICCBootstrap;
import edu.uci.ics.hyracks.api.context.ICCContext;
import edu.uci.ics.hyracks.api.exceptions.HyracksException;
import edu.uci.ics.hyracks.api.job.IActivityClusterGraphGeneratorFactory;
import edu.uci.ics.hyracks.api.job.IJobLifecycleListener;
import edu.uci.ics.hyracks.api.job.JobId;
-import edu.uci.ics.hyracks.api.util.JavaSerializationUtils;
import edu.uci.ics.hyracks.control.common.application.ApplicationContext;
import edu.uci.ics.hyracks.control.common.context.ServerContext;
import edu.uci.ics.hyracks.control.common.work.IResultCallback;
@@ -44,42 +42,18 @@
private List<IJobLifecycleListener> jobLifecycleListeners;
- public CCApplicationContext(ServerContext serverCtx, ICCContext ccContext, String appName) throws IOException {
- super(serverCtx, appName);
+ public CCApplicationContext(ServerContext serverCtx, ICCContext ccContext) throws IOException {
+ super(serverCtx);
this.ccContext = ccContext;
initPendingNodeIds = new HashSet<String>();
deinitPendingNodeIds = new HashSet<String>();
jobLifecycleListeners = new ArrayList<IJobLifecycleListener>();
}
- @Override
- protected void start() throws Exception {
- ((ICCBootstrap) bootstrap).setApplicationContext(this);
- bootstrap.start();
- }
-
public ICCContext getCCContext() {
return ccContext;
}
- public IActivityClusterGraphGeneratorFactory createActivityClusterGraphGeneratorFactory(byte[] bytes)
- throws HyracksException {
- try {
- return (IActivityClusterGraphGeneratorFactory) JavaSerializationUtils.deserialize(bytes, getClassLoader());
- } catch (IOException e) {
- throw new HyracksException(e);
- } catch (ClassNotFoundException e) {
- throw new HyracksException(e);
- }
- }
-
- @Override
- protected void stop() throws Exception {
- if (bootstrap != null) {
- bootstrap.stop();
- }
- }
-
@Override
public void setDistributedState(Serializable state) {
this.distributedState = state;
@@ -108,28 +82,4 @@
l.notifyJobCreation(jobId, acggf);
}
}
-
- public Set<String> getInitializationPendingNodeIds() {
- return initPendingNodeIds;
- }
-
- public Set<String> getDeinitializationPendingNodeIds() {
- return deinitPendingNodeIds;
- }
-
- public IResultCallback<Object> getInitializationCallback() {
- return initializationCallback;
- }
-
- public void setInitializationCallback(IResultCallback<Object> initializationCallback) {
- this.initializationCallback = initializationCallback;
- }
-
- public IResultCallback<Object> getDeinitializationCallback() {
- return deinitializationCallback;
- }
-
- public void setDeinitializationCallback(IResultCallback<Object> deinitializationCallback) {
- this.deinitializationCallback = deinitializationCallback;
- }
}
\ No newline at end of file
diff --git a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/JobRun.java b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/JobRun.java
index 6f26de2..33e1ff6 100644
--- a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/JobRun.java
+++ b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/JobRun.java
@@ -47,8 +47,6 @@
public class JobRun implements IJobStatusConditionVariable {
private final JobId jobId;
- private final String applicationName;
-
private final IActivityClusterGraphGenerator acgg;
private final ActivityClusterGraph acg;
@@ -83,10 +81,9 @@
private Exception pendingException;
- public JobRun(ClusterControllerService ccs, JobId jobId, String applicationName,
- IActivityClusterGraphGenerator acgg, EnumSet<JobFlag> jobFlags) {
+ public JobRun(ClusterControllerService ccs, JobId jobId, IActivityClusterGraphGenerator acgg,
+ EnumSet<JobFlag> jobFlags) {
this.jobId = jobId;
- this.applicationName = applicationName;
this.acgg = acgg;
this.acg = acgg.initialize();
this.scheduler = new JobScheduler(ccs, this, acgg.getConstraints());
@@ -103,10 +100,6 @@
return jobId;
}
- public String getApplicationName() {
- return applicationName;
- }
-
public ActivityClusterGraph getActivityClusterGraph() {
return acg;
}
@@ -208,7 +201,6 @@
JSONObject result = new JSONObject();
result.put("job-id", jobId.toString());
- result.put("application-name", applicationName);
result.put("status", getStatus());
result.put("create-time", getCreateTime());
result.put("start-time", getCreateTime());
diff --git a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/JobScheduler.java b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/JobScheduler.java
index b163db5..34b7dc7 100644
--- a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/JobScheduler.java
+++ b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/JobScheduler.java
@@ -429,7 +429,6 @@
private void startTasks(Map<String, List<TaskAttemptDescriptor>> taskAttemptMap) throws HyracksException {
final JobId jobId = jobRun.getJobId();
final ActivityClusterGraph acg = jobRun.getActivityClusterGraph();
- final String appName = jobRun.getApplicationName();
final Map<ConnectorDescriptorId, IConnectorPolicy> connectorPolicies = new HashMap<ConnectorDescriptorId, IConnectorPolicy>(
jobRun.getConnectorPolicyMap());
for (Map.Entry<String, List<TaskAttemptDescriptor>> entry : taskAttemptMap.entrySet()) {
@@ -444,7 +443,7 @@
}
try {
byte[] jagBytes = changed ? JavaSerializationUtils.serialize(acg) : null;
- node.getNodeController().startTasks(appName, jobId, jagBytes, taskDescriptors, connectorPolicies,
+ node.getNodeController().startTasks(jobId, jagBytes, taskDescriptors, connectorPolicies,
jobRun.getFlags());
} catch (Exception e) {
e.printStackTrace();
diff --git a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/web/ApplicationInstallationHandler.java b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/web/ApplicationInstallationHandler.java
deleted file mode 100644
index 73f3a1a0..0000000
--- a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/web/ApplicationInstallationHandler.java
+++ /dev/null
@@ -1,117 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.control.cc.web;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-
-import javax.servlet.ServletException;
-import javax.servlet.http.HttpServletRequest;
-import javax.servlet.http.HttpServletResponse;
-
-import org.apache.commons.io.IOUtils;
-import org.eclipse.jetty.http.HttpMethods;
-import org.eclipse.jetty.server.Request;
-import org.eclipse.jetty.server.handler.AbstractHandler;
-
-import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
-import edu.uci.ics.hyracks.control.common.application.ApplicationContext;
-import edu.uci.ics.hyracks.control.common.work.SynchronizableWork;
-
-public class ApplicationInstallationHandler extends AbstractHandler {
- private ClusterControllerService ccs;
-
- public ApplicationInstallationHandler(ClusterControllerService ccs) {
- this.ccs = ccs;
- }
-
- @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;
- }
- final String appName = parts[0];
- if (HttpMethods.PUT.equals(request.getMethod())) {
- class OutputStreamGetter extends SynchronizableWork {
- private OutputStream os;
-
- @Override
- protected void doRun() throws Exception {
- ApplicationContext appCtx;
- appCtx = ccs.getApplicationMap().get(appName);
- if (appCtx != null) {
- os = appCtx.getHarOutputStream();
- }
- }
- }
- OutputStreamGetter r = new OutputStreamGetter();
- try {
- ccs.getWorkQueue().scheduleAndSync(r);
- } catch (Exception e) {
- throw new IOException(e);
- }
- try {
- IOUtils.copyLarge(request.getInputStream(), r.os);
- } finally {
- r.os.close();
- }
- } else if (HttpMethods.GET.equals(request.getMethod())) {
- class InputStreamGetter extends SynchronizableWork {
- private InputStream is;
-
- @Override
- protected void doRun() throws Exception {
- ApplicationContext appCtx;
- appCtx = ccs.getApplicationMap().get(appName);
- if (appCtx != null && appCtx.containsHar()) {
- is = appCtx.getHarInputStream();
- }
- }
- }
- InputStreamGetter r = new InputStreamGetter();
- try {
- ccs.getWorkQueue().scheduleAndSync(r);
- } catch (Exception e) {
- throw new IOException(e);
- }
- if (r.is == null) {
- response.setStatus(HttpServletResponse.SC_NOT_FOUND);
- } else {
- response.setContentType("application/octet-stream");
- response.setStatus(HttpServletResponse.SC_OK);
- try {
- IOUtils.copyLarge(r.is, response.getOutputStream());
- } finally {
- r.is.close();
- }
- }
- }
- baseRequest.setHandled(true);
- } catch (IOException e) {
- e.printStackTrace();
- throw e;
- }
- }
-}
\ No newline at end of file
diff --git a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/web/WebServer.java b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/web/WebServer.java
index c32cb97..e39e766 100644
--- a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/web/WebServer.java
+++ b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/web/WebServer.java
@@ -70,10 +70,6 @@
addHandler(createAdminConsoleHandler());
addHandler(createStaticResourcesHandler());
-
- handler = new ContextHandler("/applications");
- handler.setHandler(new ApplicationInstallationHandler(ccs));
- addHandler(handler);
}
private Handler createAdminConsoleHandler() {
diff --git a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/ApplicationCreateWork.java b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/ApplicationCreateWork.java
deleted file mode 100644
index 15d6d1f..0000000
--- a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/ApplicationCreateWork.java
+++ /dev/null
@@ -1,61 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.control.cc.work;
-
-import java.io.IOException;
-import java.util.Map;
-
-import edu.uci.ics.hyracks.api.exceptions.HyracksException;
-import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
-import edu.uci.ics.hyracks.control.cc.application.CCApplicationContext;
-import edu.uci.ics.hyracks.control.common.application.ApplicationStatus;
-import edu.uci.ics.hyracks.control.common.work.AbstractWork;
-import edu.uci.ics.hyracks.control.common.work.IResultCallback;
-
-public class ApplicationCreateWork extends AbstractWork {
- private final ClusterControllerService ccs;
- private final String appName;
- private IResultCallback<Object> callback;
-
- public ApplicationCreateWork(ClusterControllerService ccs, String appName, IResultCallback<Object> callback) {
- this.ccs = ccs;
- this.appName = appName;
- this.callback = callback;
- }
-
- @Override
- public void run() {
- try {
- Map<String, CCApplicationContext> applications = ccs.getApplicationMap();
- if (applications.containsKey(appName)) {
- callback.setException(new HyracksException("Duplicate application with name: " + appName
- + " being created."));
- return;
- }
- CCApplicationContext appCtx;
- try {
- appCtx = new CCApplicationContext(ccs.getServerContext(), ccs.getCCContext(), appName);
- } catch (IOException e) {
- callback.setException(e);
- return;
- }
- appCtx.setStatus(ApplicationStatus.CREATED);
- applications.put(appName, appCtx);
- callback.setValue(null);
- } catch (Exception e) {
- callback.setException(e);
- }
- }
-}
\ No newline at end of file
diff --git a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/ApplicationDestroyWork.java b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/ApplicationDestroyWork.java
deleted file mode 100644
index fc86c4c..0000000
--- a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/ApplicationDestroyWork.java
+++ /dev/null
@@ -1,64 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.control.cc.work;
-
-import java.util.Map;
-
-import edu.uci.ics.hyracks.api.exceptions.HyracksException;
-import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
-import edu.uci.ics.hyracks.control.cc.NodeControllerState;
-import edu.uci.ics.hyracks.control.cc.application.CCApplicationContext;
-import edu.uci.ics.hyracks.control.common.application.ApplicationStatus;
-import edu.uci.ics.hyracks.control.common.base.INodeController;
-import edu.uci.ics.hyracks.control.common.work.AbstractWork;
-import edu.uci.ics.hyracks.control.common.work.IResultCallback;
-
-public class ApplicationDestroyWork extends AbstractWork {
- private final ClusterControllerService ccs;
- private final String appName;
- private IResultCallback<Object> callback;
-
- public ApplicationDestroyWork(ClusterControllerService ccs, String appName, IResultCallback<Object> callback) {
- this.ccs = ccs;
- this.appName = appName;
- this.callback = callback;
- }
-
- @Override
- public void run() {
- try {
- final CCApplicationContext appCtx = ccs.getApplicationMap().get(appName);
- if (appCtx == null) {
- callback.setException(new HyracksException("No application with name: " + appName));
- return;
- }
- if (appCtx.getStatus() == ApplicationStatus.IN_DEINITIALIZATION
- || appCtx.getStatus() == ApplicationStatus.DEINITIALIZED) {
- return;
- }
- Map<String, NodeControllerState> nodeMap = ccs.getNodeMap();
- appCtx.getDeinitializationPendingNodeIds().addAll(nodeMap.keySet());
- appCtx.setStatus(ApplicationStatus.IN_DEINITIALIZATION);
- appCtx.setDeinitializationCallback(callback);
- for (String nodeId : ccs.getNodeMap().keySet()) {
- NodeControllerState nodeState = nodeMap.get(nodeId);
- final INodeController node = nodeState.getNodeController();
- node.destroyApplication(appName);
- }
- } catch (Exception e) {
- callback.setException(e);
- }
- }
-}
\ No newline at end of file
diff --git a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/ApplicationMessageWork.java b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/ApplicationMessageWork.java
index 22ff84d..b880c8a 100644
--- a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/ApplicationMessageWork.java
+++ b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/ApplicationMessageWork.java
@@ -18,14 +18,14 @@
import java.util.logging.Level;
import java.util.logging.Logger;
+import edu.uci.ics.hyracks.api.application.ICCApplicationContext;
import edu.uci.ics.hyracks.api.messages.IMessage;
+import edu.uci.ics.hyracks.api.util.JavaSerializationUtils;
import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
-import edu.uci.ics.hyracks.control.common.application.ApplicationContext;
import edu.uci.ics.hyracks.control.common.work.AbstractWork;
/**
* @author rico
- *
*/
public class ApplicationMessageWork extends AbstractWork {
@@ -33,26 +33,24 @@
private byte[] message;
private String nodeId;
private ClusterControllerService ccs;
- private String appName;
- public ApplicationMessageWork(ClusterControllerService ccs, byte[] message, String appName, String nodeId) {
+ public ApplicationMessageWork(ClusterControllerService ccs, byte[] message, String nodeId) {
this.ccs = ccs;
this.nodeId = nodeId;
this.message = message;
- this.appName = appName;
}
@Override
public void run() {
-
- final ApplicationContext ctx = ccs.getApplicationMap().get(appName);
+ final ICCApplicationContext ctx = ccs.getApplicationContext();
try {
- final IMessage data = (IMessage) ctx.deserialize(message);
- (new Thread() {
+ final IMessage data = (IMessage) JavaSerializationUtils.deserialize(message);
+ ccs.getExecutor().execute(new Runnable() {
+ @Override
public void run() {
ctx.getMessageBroker().receivedMessage(data, nodeId);
}
- }).start();
+ });
} catch (IOException e) {
LOGGER.log(Level.WARNING, "Error in stats reporting", e);
} catch (ClassNotFoundException e) {
@@ -64,5 +62,4 @@
public String toString() {
return "nodeID: " + nodeId;
}
-
-}
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/ApplicationStartWork.java b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/ApplicationStartWork.java
deleted file mode 100644
index e4ad56c..0000000
--- a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/ApplicationStartWork.java
+++ /dev/null
@@ -1,80 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.control.cc.work;
-
-import java.util.HashMap;
-import java.util.Map;
-
-import edu.uci.ics.hyracks.api.exceptions.HyracksException;
-import edu.uci.ics.hyracks.api.util.JavaSerializationUtils;
-import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
-import edu.uci.ics.hyracks.control.cc.NodeControllerState;
-import edu.uci.ics.hyracks.control.cc.application.CCApplicationContext;
-import edu.uci.ics.hyracks.control.common.application.ApplicationStatus;
-import edu.uci.ics.hyracks.control.common.base.INodeController;
-import edu.uci.ics.hyracks.control.common.work.AbstractWork;
-import edu.uci.ics.hyracks.control.common.work.IResultCallback;
-
-public class ApplicationStartWork extends AbstractWork {
- private final ClusterControllerService ccs;
- private final String appName;
- private final IResultCallback<Object> callback;
-
- public ApplicationStartWork(ClusterControllerService ccs, String appName, IResultCallback<Object> callback) {
- this.ccs = ccs;
- this.appName = appName;
- this.callback = callback;
- }
-
- @Override
- public void run() {
- try {
- final CCApplicationContext appCtx = ccs.getApplicationMap().get(appName);
- if (appCtx == null) {
- callback.setException(new HyracksException("No application with name: " + appName));
- return;
- }
- if (appCtx.getStatus() != ApplicationStatus.CREATED) {
- callback.setException(new HyracksException("Application in incorrect state for starting: "
- + appCtx.getStatus()));
- }
- final Map<String, NodeControllerState> nodeMapCopy = new HashMap<String, NodeControllerState>(
- ccs.getNodeMap());
- appCtx.getInitializationPendingNodeIds().addAll(nodeMapCopy.keySet());
- appCtx.setStatus(ApplicationStatus.IN_INITIALIZATION);
- appCtx.setInitializationCallback(callback);
- ccs.getExecutor().execute(new Runnable() {
- @Override
- public void run() {
- try {
- appCtx.initializeClassPath();
- appCtx.initialize();
- final byte[] distributedState = JavaSerializationUtils.serialize(appCtx.getDistributedState());
- final boolean deployHar = appCtx.containsHar();
- for (final String nodeId : nodeMapCopy.keySet()) {
- NodeControllerState nodeState = nodeMapCopy.get(nodeId);
- final INodeController node = nodeState.getNodeController();
- node.createApplication(appName, deployHar, distributedState);
- }
- } catch (Exception e) {
- callback.setException(e);
- }
- }
- });
- } catch (Exception e) {
- callback.setException(e);
- }
- }
-}
\ No newline at end of file
diff --git a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/ApplicationStateChangeWork.java b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/ApplicationStateChangeWork.java
deleted file mode 100644
index f6271fe..0000000
--- a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/ApplicationStateChangeWork.java
+++ /dev/null
@@ -1,85 +0,0 @@
-package edu.uci.ics.hyracks.control.cc.work;
-
-import java.util.Set;
-import java.util.logging.Logger;
-
-import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
-import edu.uci.ics.hyracks.control.cc.application.CCApplicationContext;
-import edu.uci.ics.hyracks.control.common.application.ApplicationStatus;
-import edu.uci.ics.hyracks.control.common.ipc.CCNCFunctions;
-import edu.uci.ics.hyracks.control.common.work.AbstractWork;
-import edu.uci.ics.hyracks.control.common.work.IResultCallback;
-
-public class ApplicationStateChangeWork extends AbstractWork {
- private static final Logger LOGGER = Logger.getLogger(ApplicationStateChangeWork.class.getName());
-
- private final ClusterControllerService ccs;
- private final CCNCFunctions.ApplicationStateChangeResponseFunction ascrf;
-
- public ApplicationStateChangeWork(ClusterControllerService ccs,
- CCNCFunctions.ApplicationStateChangeResponseFunction ascrf) {
- this.ccs = ccs;
- this.ascrf = ascrf;
- }
-
- @Override
- public void run() {
- final CCApplicationContext appCtx = ccs.getApplicationMap().get(ascrf.getApplicationName());
- if (appCtx == null) {
- LOGGER.warning("Got ApplicationStateChangeResponse for application " + ascrf.getApplicationName()
- + " that does not exist");
- return;
- }
- switch (ascrf.getStatus()) {
- case INITIALIZED: {
- Set<String> pendingNodeIds = appCtx.getInitializationPendingNodeIds();
- boolean changed = pendingNodeIds.remove(ascrf.getNodeId());
- if (!changed) {
- LOGGER.warning("Got ApplicationStateChangeResponse for application " + ascrf.getApplicationName()
- + " from unexpected node " + ascrf.getNodeId() + " to state " + ascrf.getStatus());
- return;
- }
- if (pendingNodeIds.isEmpty()) {
- appCtx.setStatus(ApplicationStatus.INITIALIZED);
- IResultCallback<Object> callback = appCtx.getInitializationCallback();
- appCtx.setInitializationCallback(null);
- callback.setValue(null);
- }
- return;
- }
-
- case DEINITIALIZED: {
- Set<String> pendingNodeIds = appCtx.getDeinitializationPendingNodeIds();
- boolean changed = pendingNodeIds.remove(ascrf.getNodeId());
- if (!changed) {
- LOGGER.warning("Got ApplicationStateChangeResponse for application " + ascrf.getApplicationName()
- + " from unexpected node " + ascrf.getNodeId() + " to state " + ascrf.getStatus());
- return;
- }
- if (pendingNodeIds.isEmpty()) {
- appCtx.setStatus(ApplicationStatus.DEINITIALIZED);
- ccs.getExecutor().execute(new Runnable() {
- @Override
- public void run() {
- try {
- appCtx.deinitialize();
- } catch (Exception e) {
- e.printStackTrace();
- }
- ccs.getWorkQueue().schedule(new AbstractWork() {
- @Override
- public void run() {
- ccs.getApplicationMap().remove(ascrf.getApplicationName());
- IResultCallback<Object> callback = appCtx.getDeinitializationCallback();
- appCtx.setDeinitializationCallback(null);
- callback.setValue(null);
- }
- });
- }
- });
- }
- return;
- }
- }
- }
-}
\ No newline at end of file
diff --git a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/GetJobSummariesJSONWork.java b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/GetJobSummariesJSONWork.java
index a0afd61..20035a4 100644
--- a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/GetJobSummariesJSONWork.java
+++ b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/GetJobSummariesJSONWork.java
@@ -44,7 +44,6 @@
JSONObject jo = new JSONObject();
jo.put("type", "job-summary");
jo.put("job-id", run.getJobId().toString());
- jo.put("application-name", run.getApplicationName());
jo.put("create-time", run.getCreateTime());
jo.put("start-time", run.getCreateTime());
jo.put("end-time", run.getCreateTime());
diff --git a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/JobCleanupWork.java b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/JobCleanupWork.java
index 6b5ff03..b304b21 100644
--- a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/JobCleanupWork.java
+++ b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/JobCleanupWork.java
@@ -69,7 +69,7 @@
}
}
} else {
- CCApplicationContext appCtx = ccs.getApplicationMap().get(run.getApplicationName());
+ CCApplicationContext appCtx = ccs.getApplicationContext();
if (appCtx != null) {
try {
appCtx.notifyJobFinish(jobId);
diff --git a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/JobStartWork.java b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/JobStartWork.java
index b6a33cd..f7d6c4c 100644
--- a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/JobStartWork.java
+++ b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/JobStartWork.java
@@ -16,12 +16,12 @@
import java.util.EnumSet;
-import edu.uci.ics.hyracks.api.exceptions.HyracksException;
import edu.uci.ics.hyracks.api.job.IActivityClusterGraphGenerator;
import edu.uci.ics.hyracks.api.job.IActivityClusterGraphGeneratorFactory;
import edu.uci.ics.hyracks.api.job.JobFlag;
import edu.uci.ics.hyracks.api.job.JobId;
import edu.uci.ics.hyracks.api.job.JobStatus;
+import edu.uci.ics.hyracks.api.util.JavaSerializationUtils;
import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
import edu.uci.ics.hyracks.control.cc.application.CCApplicationContext;
import edu.uci.ics.hyracks.control.cc.job.JobRun;
@@ -33,30 +33,25 @@
private final byte[] acggfBytes;
private final EnumSet<JobFlag> jobFlags;
private final JobId jobId;
- private final String appName;
private final IResultCallback<JobId> callback;
- public JobStartWork(ClusterControllerService ccs, String appName, byte[] acggfBytes, EnumSet<JobFlag> jobFlags,
- JobId jobId, IResultCallback<JobId> callback) {
+ public JobStartWork(ClusterControllerService ccs, byte[] acggfBytes, EnumSet<JobFlag> jobFlags, JobId jobId,
+ IResultCallback<JobId> callback) {
this.jobId = jobId;
this.ccs = ccs;
this.acggfBytes = acggfBytes;
this.jobFlags = jobFlags;
- this.appName = appName;
this.callback = callback;
}
@Override
protected void doRun() throws Exception {
try {
- final CCApplicationContext appCtx = ccs.getApplicationMap().get(appName);
- if (appCtx == null) {
- throw new HyracksException("No application with id " + appName + " found");
- }
- IActivityClusterGraphGeneratorFactory acggf = appCtx.createActivityClusterGraphGeneratorFactory(acggfBytes);
- IActivityClusterGraphGenerator acgg = acggf.createActivityClusterGraphGenerator(appName, jobId, appCtx,
- jobFlags);
- JobRun run = new JobRun(ccs, jobId, appName, acgg, jobFlags);
+ final CCApplicationContext appCtx = ccs.getApplicationContext();
+ IActivityClusterGraphGeneratorFactory acggf = (IActivityClusterGraphGeneratorFactory) JavaSerializationUtils
+ .deserialize(acggfBytes);
+ IActivityClusterGraphGenerator acgg = acggf.createActivityClusterGraphGenerator(jobId, appCtx, jobFlags);
+ JobRun run = new JobRun(ccs, jobId, acgg, jobFlags);
run.setStatus(JobStatus.INITIALIZED, null);
ccs.getActiveRunMap().put(jobId, run);
appCtx.notifyJobCreation(jobId, acggf);
diff --git a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/JobletCleanupNotificationWork.java b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/JobletCleanupNotificationWork.java
index 9927289..ed58c43 100644
--- a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/JobletCleanupNotificationWork.java
+++ b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/JobletCleanupNotificationWork.java
@@ -59,7 +59,7 @@
ncs.getActiveJobIds().remove(jobId);
}
if (cleanupPendingNodes.isEmpty()) {
- CCApplicationContext appCtx = ccs.getApplicationMap().get(run.getApplicationName());
+ CCApplicationContext appCtx = ccs.getApplicationContext();
if (appCtx != null) {
try {
appCtx.notifyJobFinish(jobId);
diff --git a/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/application/ApplicationContext.java b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/application/ApplicationContext.java
index fc755a5..5251584 100644
--- a/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/application/ApplicationContext.java
+++ b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/application/ApplicationContext.java
@@ -14,180 +14,20 @@
*/
package edu.uci.ics.hyracks.control.common.application;
-import java.io.File;
-import java.io.FileInputStream;
-import java.io.FileOutputStream;
import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
import java.io.Serializable;
-import java.net.MalformedURLException;
-import java.net.URL;
-import java.net.URLClassLoader;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.Comparator;
-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.api.messages.IMessageBroker;
-import edu.uci.ics.hyracks.api.util.JavaSerializationUtils;
import edu.uci.ics.hyracks.control.common.context.ServerContext;
public abstract 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";
-
protected ServerContext serverCtx;
- protected final String appName;
- protected final File applicationRootDir;
- protected ClassLoader classLoader;
- protected ApplicationStatus status;
- protected Properties deploymentDescriptor;
- protected IBootstrap bootstrap;
protected Serializable distributedState;
protected IMessageBroker messageBroker;
- public ApplicationContext(ServerContext serverCtx, String appName) throws IOException {
+ public ApplicationContext(ServerContext serverCtx) throws IOException {
this.serverCtx = serverCtx;
- this.appName = appName;
- this.applicationRootDir = new File(new File(serverCtx.getBaseDir(), APPLICATION_ROOT), appName);
- FileUtils.deleteDirectory(applicationRootDir);
- applicationRootDir.mkdirs();
- }
-
- public String getApplicationName() {
- return appName;
- }
-
- public void initializeClassPath() throws Exception {
- if (expandArchive()) {
- File expandedFolder = getExpandedFolder();
- List<URL> urls = new ArrayList<URL>();
- findJarFiles(expandedFolder, urls);
- Collections.sort(urls, new Comparator<URL>() {
- @Override
- public int compare(URL o1, URL o2) {
- return o1.getFile().compareTo(o2.getFile());
- }
- });
- classLoader = new URLClassLoader(urls.toArray(new URL[urls.size()]));
- } else {
- classLoader = getClass().getClassLoader();
- }
- deploymentDescriptor = parseDeploymentDescriptor();
- }
-
- public void initialize() throws Exception {
- if (deploymentDescriptor != null) {
- String bootstrapClass = null;
- switch (serverCtx.getServerType()) {
- case CLUSTER_CONTROLLER: {
- bootstrapClass = deploymentDescriptor.getProperty(CLUSTER_CONTROLLER_BOOTSTRAP_CLASS_KEY);
- break;
- }
- case NODE_CONTROLLER: {
- bootstrapClass = deploymentDescriptor.getProperty(NODE_CONTROLLER_BOOTSTRAP_CLASS_KEY);
- break;
- }
- }
- if (bootstrapClass != null) {
- bootstrap = (IBootstrap) classLoader.loadClass(bootstrapClass).newInstance();
- start();
- }
- }
- }
-
- protected abstract void start() throws Exception;
-
- protected abstract void stop() throws Exception;
-
- 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 {
- stop();
- File expandedFolder = getExpandedFolder();
- FileUtils.deleteDirectory(expandedFolder);
- }
-
- public Object deserialize(byte[] bytes) throws IOException, ClassNotFoundException {
- return JavaSerializationUtils.deserialize(bytes, classLoader);
- }
-
- 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();
}
@Override
@@ -196,19 +36,6 @@
}
@Override
- public ClassLoader getClassLoader() {
- return classLoader;
- }
-
- public void setStatus(ApplicationStatus status) {
- this.status = status;
- }
-
- public ApplicationStatus getStatus() {
- return status;
- }
-
- @Override
public void setMessageBroker(IMessageBroker messageBroker) {
this.messageBroker = messageBroker;
}
diff --git a/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/base/IClusterController.java b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/base/IClusterController.java
index 0c5bb2f..3099bbb 100644
--- a/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/base/IClusterController.java
+++ b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/base/IClusterController.java
@@ -18,7 +18,6 @@
import edu.uci.ics.hyracks.api.dataflow.TaskAttemptId;
import edu.uci.ics.hyracks.api.job.JobId;
-import edu.uci.ics.hyracks.control.common.application.ApplicationStatus;
import edu.uci.ics.hyracks.control.common.controllers.NodeRegistration;
import edu.uci.ics.hyracks.control.common.heartbeat.HeartbeatData;
import edu.uci.ics.hyracks.control.common.job.PartitionDescriptor;
@@ -46,9 +45,7 @@
public void registerPartitionRequest(PartitionRequest partitionRequest) throws Exception;
- public void notifyApplicationStateChange(String nodeId, String appName, ApplicationStatus status) throws Exception;
-
- public void sendApplicationMessageToCC(byte[] data, String appName, String nodeId) throws Exception;
+ public void sendApplicationMessageToCC(byte[] data, String nodeId) throws Exception;
public void getNodeControllerInfos() throws Exception;
}
\ No newline at end of file
diff --git a/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/base/INodeController.java b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/base/INodeController.java
index 049adf8..c589c97 100644
--- a/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/base/INodeController.java
+++ b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/base/INodeController.java
@@ -29,17 +29,12 @@
import edu.uci.ics.hyracks.control.common.job.TaskAttemptDescriptor;
public interface INodeController {
- public void startTasks(String appName, JobId jobId, byte[] planBytes, List<TaskAttemptDescriptor> taskDescriptors,
+ public void startTasks(JobId jobId, byte[] planBytes, List<TaskAttemptDescriptor> taskDescriptors,
Map<ConnectorDescriptorId, IConnectorPolicy> connectorPolicies, EnumSet<JobFlag> flags) throws Exception;
public void abortTasks(JobId jobId, List<TaskAttemptId> tasks) throws Exception;
public void cleanUpJoblet(JobId jobId, JobStatus status) throws Exception;
- public void createApplication(String appName, boolean deployHar, byte[] serializedDistributedState)
- throws Exception;
-
- public void destroyApplication(String appName) throws Exception;
-
public void reportPartitionAvailability(PartitionId pid, NetworkAddress networkAddress) throws Exception;
}
\ No newline at end of file
diff --git a/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/controllers/CCConfig.java b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/controllers/CCConfig.java
index 6c208fe..6a4b325 100644
--- a/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/controllers/CCConfig.java
+++ b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/controllers/CCConfig.java
@@ -17,7 +17,9 @@
import java.io.File;
import java.util.List;
+import org.kohsuke.args4j.Argument;
import org.kohsuke.args4j.Option;
+import org.kohsuke.args4j.spi.StopOptionHandler;
public class CCConfig {
@Option(name = "-client-net-ip-address", usage = "Sets the IP Address to listen for connections from clients", required = true)
@@ -26,7 +28,7 @@
@Option(name = "-client-net-port", usage = "Sets the port to listen for connections from clients (default 1098)")
public int clientNetPort = 1098;
- @Option(name = "-cluster-net-ip-address", usage = "Sets the IP Address to listen for connections from ", required = true)
+ @Option(name = "-cluster-net-ip-address", usage = "Sets the IP Address to listen for connections from", required = true)
public String clusterNetIpAddress;
@Option(name = "-cluster-net-port", usage = "Sets the port to listen for connections from node controllers (default 1099)")
@@ -56,6 +58,13 @@
@Option(name = "-cluster-topology", usage = "Sets the XML file that defines the cluster topology. (default: null)")
public File clusterTopologyDefinition = null;
+ @Option(name = "-app-cc-main-class", usage = "Application CC Main Class")
+ public String appCCMainClass;
+
+ @Argument
+ @Option(name = "--", handler = StopOptionHandler.class)
+ public List<String> appArgs;
+
public void toCommandLine(List<String> cList) {
cList.add("-client-net-ip-address");
cList.add(clientNetIpAddress);
@@ -83,5 +92,15 @@
cList.add("-cluster-topology");
cList.add(clusterTopologyDefinition.getAbsolutePath());
}
+ if (appCCMainClass != null) {
+ cList.add("-app-cc-main-class");
+ cList.add(appCCMainClass);
+ }
+ if (appArgs != null && !appArgs.isEmpty()) {
+ cList.add("--");
+ for (String appArg : appArgs) {
+ cList.add(appArg);
+ }
+ }
}
}
\ No newline at end of file
diff --git a/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/controllers/NCConfig.java b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/controllers/NCConfig.java
index 167eb4b..970e8aa 100644
--- a/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/controllers/NCConfig.java
+++ b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/controllers/NCConfig.java
@@ -17,7 +17,9 @@
import java.io.Serializable;
import java.util.List;
+import org.kohsuke.args4j.Argument;
import org.kohsuke.args4j.Option;
+import org.kohsuke.args4j.spi.StopOptionHandler;
public class NCConfig implements Serializable {
private static final long serialVersionUID = 1L;
@@ -55,6 +57,13 @@
@Option(name = "-max-memory", usage = "Maximum memory usable at this Node Controller in bytes (default: -1 auto)")
public int maxMemory = -1;
+ @Option(name = "-app-nc-main-class", usage = "Application NC Main Class")
+ public String appNCMainClass;
+
+ @Argument
+ @Option(name = "--", handler = StopOptionHandler.class)
+ public List<String> appArgs;
+
public void toCommandLine(List<String> cList) {
cList.add("-cc-host");
cList.add(ccHost);
@@ -80,5 +89,15 @@
cList.add(String.valueOf(nNetThreads));
cList.add("-max-memory");
cList.add(String.valueOf(maxMemory));
+ if (appNCMainClass != null) {
+ cList.add("-app-nc-main-class");
+ cList.add(appNCMainClass);
+ }
+ if (appArgs != null && !appArgs.isEmpty()) {
+ cList.add("--");
+ for (String appArg : appArgs) {
+ cList.add(appArg);
+ }
+ }
}
}
\ No newline at end of file
diff --git a/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/ipc/CCNCFunctions.java b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/ipc/CCNCFunctions.java
index 557a8cb..8cc4a54 100644
--- a/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/ipc/CCNCFunctions.java
+++ b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/ipc/CCNCFunctions.java
@@ -40,7 +40,6 @@
import edu.uci.ics.hyracks.api.job.JobId;
import edu.uci.ics.hyracks.api.job.JobStatus;
import edu.uci.ics.hyracks.api.partitions.PartitionId;
-import edu.uci.ics.hyracks.control.common.application.ApplicationStatus;
import edu.uci.ics.hyracks.control.common.controllers.NodeParameters;
import edu.uci.ics.hyracks.control.common.controllers.NodeRegistration;
import edu.uci.ics.hyracks.control.common.heartbeat.HeartbeatData;
@@ -68,14 +67,11 @@
REPORT_PROFILE,
REGISTER_PARTITION_PROVIDER,
REGISTER_PARTITION_REQUEST,
- APPLICATION_STATE_CHANGE_RESPONSE,
NODE_REGISTRATION_RESULT,
START_TASKS,
ABORT_TASKS,
CLEANUP_JOBLET,
- CREATE_APPLICATION,
- DESTROY_APPLICATION,
REPORT_PARTITION_AVAILABILITY,
SEND_APPLICATION_MESSAGE,
GET_NODE_CONTROLLERS_INFO,
@@ -88,7 +84,6 @@
private static final long serialVersionUID = 1L;
private byte[] serializedMessage;
private String nodeId;
- private String appName;
public String getNodeId() {
return nodeId;
@@ -102,11 +97,9 @@
return serializedMessage;
}
- public SendApplicationMessageFunction(byte[] data, String appName, String nodeId) {
- super();
+ public SendApplicationMessageFunction(byte[] data, String nodeId) {
this.serializedMessage = data;
this.nodeId = nodeId;
- this.appName = appName;
}
@Override
@@ -114,10 +107,6 @@
return FunctionId.SEND_APPLICATION_MESSAGE;
}
- public String getAppName() {
- return appName;
- }
-
}
public static abstract class Function implements Serializable {
@@ -438,37 +427,6 @@
}
}
- public static class ApplicationStateChangeResponseFunction extends Function {
- private static final long serialVersionUID = 1L;
-
- private final String nodeId;
- private final String appName;
- private final ApplicationStatus status;
-
- public ApplicationStateChangeResponseFunction(String nodeId, String appName, ApplicationStatus status) {
- this.nodeId = nodeId;
- this.appName = appName;
- this.status = status;
- }
-
- @Override
- public FunctionId getFunctionId() {
- return FunctionId.APPLICATION_STATE_CHANGE_RESPONSE;
- }
-
- public String getNodeId() {
- return nodeId;
- }
-
- public String getApplicationName() {
- return appName;
- }
-
- public ApplicationStatus getStatus() {
- return status;
- }
- }
-
public static class NodeRegistrationResult extends Function {
private static final long serialVersionUID = 1L;
@@ -498,17 +456,14 @@
public static class StartTasksFunction extends Function {
private static final long serialVersionUID = 1L;
- private final String appName;
private final JobId jobId;
private final byte[] planBytes;
private final List<TaskAttemptDescriptor> taskDescriptors;
private final Map<ConnectorDescriptorId, IConnectorPolicy> connectorPolicies;
private final EnumSet<JobFlag> flags;
- public StartTasksFunction(String appName, JobId jobId, byte[] planBytes,
- List<TaskAttemptDescriptor> taskDescriptors,
+ public StartTasksFunction(JobId jobId, byte[] planBytes, List<TaskAttemptDescriptor> taskDescriptors,
Map<ConnectorDescriptorId, IConnectorPolicy> connectorPolicies, EnumSet<JobFlag> flags) {
- this.appName = appName;
this.jobId = jobId;
this.planBytes = planBytes;
this.taskDescriptors = taskDescriptors;
@@ -521,10 +476,6 @@
return FunctionId.START_TASKS;
}
- public String getAppName() {
- return appName;
- }
-
public JobId getJobId() {
return jobId;
}
@@ -596,56 +547,6 @@
}
}
- public static class CreateApplicationFunction extends Function {
- private static final long serialVersionUID = 1L;
-
- private final String appName;
- private final boolean deployHar;
- private final byte[] serializedDistributedState;
-
- public CreateApplicationFunction(String appName, boolean deployHar, byte[] serializedDistributedState) {
- this.appName = appName;
- this.deployHar = deployHar;
- this.serializedDistributedState = serializedDistributedState;
- }
-
- @Override
- public FunctionId getFunctionId() {
- return FunctionId.CREATE_APPLICATION;
- }
-
- public String getAppName() {
- return appName;
- }
-
- public boolean isDeployHar() {
- return deployHar;
- }
-
- public byte[] getSerializedDistributedState() {
- return serializedDistributedState;
- }
- }
-
- public static class DestroyApplicationFunction extends Function {
- private static final long serialVersionUID = 1L;
-
- private final String appName;
-
- public DestroyApplicationFunction(String appName) {
- this.appName = appName;
- }
-
- @Override
- public FunctionId getFunctionId() {
- return FunctionId.DESTROY_APPLICATION;
- }
-
- public String getAppName() {
- return appName;
- }
- }
-
public static class GetNodeControllersInfoFunction extends Function {
private static final long serialVersionUID = 1L;
diff --git a/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/ipc/ClusterControllerRemoteProxy.java b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/ipc/ClusterControllerRemoteProxy.java
index bbaab4e..5fd2302 100644
--- a/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/ipc/ClusterControllerRemoteProxy.java
+++ b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/ipc/ClusterControllerRemoteProxy.java
@@ -18,7 +18,6 @@
import edu.uci.ics.hyracks.api.dataflow.TaskAttemptId;
import edu.uci.ics.hyracks.api.job.JobId;
-import edu.uci.ics.hyracks.control.common.application.ApplicationStatus;
import edu.uci.ics.hyracks.control.common.base.IClusterController;
import edu.uci.ics.hyracks.control.common.controllers.NodeRegistration;
import edu.uci.ics.hyracks.control.common.heartbeat.HeartbeatData;
@@ -95,16 +94,8 @@
}
@Override
- public void notifyApplicationStateChange(String nodeId, String appName, ApplicationStatus status) throws Exception {
- CCNCFunctions.ApplicationStateChangeResponseFunction fn = new CCNCFunctions.ApplicationStateChangeResponseFunction(
- nodeId, appName, status);
- ipcHandle.send(-1, fn, null);
- }
-
- @Override
- public void sendApplicationMessageToCC(byte[] data, String appName, String nodeId) throws Exception {
- CCNCFunctions.SendApplicationMessageFunction fn = new CCNCFunctions.SendApplicationMessageFunction(data,
- appName, nodeId);
+ public void sendApplicationMessageToCC(byte[] data, String nodeId) throws Exception {
+ CCNCFunctions.SendApplicationMessageFunction fn = new CCNCFunctions.SendApplicationMessageFunction(data, nodeId);
ipcHandle.send(-1, fn, null);
}
diff --git a/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/ipc/NodeControllerRemoteProxy.java b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/ipc/NodeControllerRemoteProxy.java
index 10c0a7c..e4355aa 100644
--- a/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/ipc/NodeControllerRemoteProxy.java
+++ b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/ipc/NodeControllerRemoteProxy.java
@@ -38,10 +38,10 @@
}
@Override
- public void startTasks(String appName, JobId jobId, byte[] planBytes, List<TaskAttemptDescriptor> taskDescriptors,
+ public void startTasks(JobId jobId, byte[] planBytes, List<TaskAttemptDescriptor> taskDescriptors,
Map<ConnectorDescriptorId, IConnectorPolicy> connectorPolicies, EnumSet<JobFlag> flags) throws Exception {
- CCNCFunctions.StartTasksFunction stf = new CCNCFunctions.StartTasksFunction(appName, jobId, planBytes,
- taskDescriptors, connectorPolicies, flags);
+ CCNCFunctions.StartTasksFunction stf = new CCNCFunctions.StartTasksFunction(jobId, planBytes, taskDescriptors,
+ connectorPolicies, flags);
ipcHandle.send(-1, stf, null);
}
@@ -58,20 +58,6 @@
}
@Override
- public void createApplication(String appName, boolean deployHar, byte[] serializedDistributedState)
- throws Exception {
- CCNCFunctions.CreateApplicationFunction caf = new CCNCFunctions.CreateApplicationFunction(appName, deployHar,
- serializedDistributedState);
- ipcHandle.send(-1, caf, null);
- }
-
- @Override
- public void destroyApplication(String appName) throws Exception {
- CCNCFunctions.DestroyApplicationFunction daf = new CCNCFunctions.DestroyApplicationFunction(appName);
- ipcHandle.send(-1, daf, null);
- }
-
- @Override
public void reportPartitionAvailability(PartitionId pid, NetworkAddress networkAddress) throws Exception {
CCNCFunctions.ReportPartitionAvailabilityFunction rpaf = new CCNCFunctions.ReportPartitionAvailabilityFunction(
pid, networkAddress);
diff --git a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/NodeControllerService.java b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/NodeControllerService.java
index 0195143..fb1e57c 100644
--- a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/NodeControllerService.java
+++ b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/NodeControllerService.java
@@ -43,6 +43,7 @@
import org.apache.commons.lang3.mutable.Mutable;
import org.apache.commons.lang3.mutable.MutableObject;
+import edu.uci.ics.hyracks.api.application.INCApplicationEntryPoint;
import edu.uci.ics.hyracks.api.client.NodeControllerInfo;
import edu.uci.ics.hyracks.api.context.IHyracksRootContext;
import edu.uci.ics.hyracks.api.io.IODeviceHandle;
@@ -69,8 +70,6 @@
import edu.uci.ics.hyracks.control.nc.work.ApplicationMessageWork;
import edu.uci.ics.hyracks.control.nc.work.BuildJobProfilesWork;
import edu.uci.ics.hyracks.control.nc.work.CleanupJobletWork;
-import edu.uci.ics.hyracks.control.nc.work.CreateApplicationWork;
-import edu.uci.ics.hyracks.control.nc.work.DestroyApplicationWork;
import edu.uci.ics.hyracks.control.nc.work.ReportPartitionAvailabilityWork;
import edu.uci.ics.hyracks.control.nc.work.StartTasksWork;
import edu.uci.ics.hyracks.ipc.api.IIPCHandle;
@@ -114,7 +113,7 @@
private final ServerContext serverCtx;
- private final Map<String, NCApplicationContext> applications;
+ private NCApplicationContext appCtx;
private final MemoryMXBean memoryMXBean;
@@ -147,7 +146,6 @@
timer = new Timer(true);
serverCtx = new ServerContext(ServerContext.ServerType.NODE_CONTROLLER, new File(new File(
NodeControllerService.class.getName()), id));
- applications = new Hashtable<String, NCApplicationContext>();
memoryMXBean = ManagementFactory.getMemoryMXBean();
gcMXBeans = ManagementFactory.getGarbageCollectorMXBeans();
threadMXBean = ManagementFactory.getThreadMXBean();
@@ -161,6 +159,10 @@
return ctx;
}
+ public NCApplicationContext getApplicationContext() {
+ return appCtx;
+ }
+
private static List<IODeviceHandle> getDevices(String ioDevices) {
List<IODeviceHandle> devices = new ArrayList<IODeviceHandle>();
StringTokenizer tok = new StringTokenizer(ioDevices, ",");
@@ -205,6 +207,9 @@
LOGGER.log(Level.INFO, "Starting NodeControllerService");
ipc.start();
netManager.start();
+
+ startApplication();
+
IIPCHandle ccIPCHandle = ipc.getHandle(new InetSocketAddress(ncConfig.ccHost, ncConfig.ccPort));
this.ccs = new ClusterControllerRemoteProxy(ccIPCHandle);
HeartbeatSchema.GarbageCollectorInfo[] gcInfos = new HeartbeatSchema.GarbageCollectorInfo[gcMXBeans.size()];
@@ -242,6 +247,18 @@
LOGGER.log(Level.INFO, "Started NodeControllerService");
}
+ private void startApplication() throws Exception {
+ appCtx = new NCApplicationContext(serverCtx, ctx, id);
+ String className = ncConfig.appNCMainClass;
+ if (className != null) {
+ Class<?> c = Class.forName(className);
+ INCApplicationEntryPoint aep = (INCApplicationEntryPoint) c.newInstance();
+ String[] args = ncConfig.appArgs == null ? new String[0] : ncConfig.appArgs
+ .toArray(new String[ncConfig.appArgs.size()]);
+ aep.appMain(appCtx, args);
+ }
+ }
+
@Override
public void stop() throws Exception {
LOGGER.log(Level.INFO, "Stopping NodeControllerService");
@@ -261,10 +278,6 @@
return serverCtx;
}
- public Map<String, NCApplicationContext> getApplications() {
- return applications;
- }
-
public Map<JobId, Joblet> getJobletMap() {
return jobletMap;
}
@@ -400,13 +413,13 @@
case SEND_APPLICATION_MESSAGE: {
CCNCFunctions.SendApplicationMessageFunction amf = (CCNCFunctions.SendApplicationMessageFunction) fn;
queue.schedule(new ApplicationMessageWork(NodeControllerService.this, amf.getMessage(), amf
- .getAppName(), amf.getNodeId()));
+ .getNodeId()));
return;
}
case START_TASKS: {
CCNCFunctions.StartTasksFunction stf = (CCNCFunctions.StartTasksFunction) fn;
- queue.schedule(new StartTasksWork(NodeControllerService.this, stf.getAppName(), stf.getJobId(), stf
- .getPlanBytes(), stf.getTaskDescriptors(), stf.getConnectorPolicies(), stf.getFlags()));
+ queue.schedule(new StartTasksWork(NodeControllerService.this, stf.getJobId(), stf.getPlanBytes(),
+ stf.getTaskDescriptors(), stf.getConnectorPolicies(), stf.getFlags()));
return;
}
@@ -422,19 +435,6 @@
return;
}
- case CREATE_APPLICATION: {
- CCNCFunctions.CreateApplicationFunction caf = (CCNCFunctions.CreateApplicationFunction) fn;
- queue.schedule(new CreateApplicationWork(NodeControllerService.this, caf.getAppName(), caf
- .isDeployHar(), caf.getSerializedDistributedState()));
- return;
- }
-
- case DESTROY_APPLICATION: {
- CCNCFunctions.DestroyApplicationFunction daf = (CCNCFunctions.DestroyApplicationFunction) fn;
- queue.schedule(new DestroyApplicationWork(NodeControllerService.this, daf.getAppName()));
- return;
- }
-
case REPORT_PARTITION_AVAILABILITY: {
CCNCFunctions.ReportPartitionAvailabilityFunction rpaf = (CCNCFunctions.ReportPartitionAvailabilityFunction) fn;
queue.schedule(new ReportPartitionAvailabilityWork(NodeControllerService.this, rpaf
@@ -459,7 +459,7 @@
}
}
- public void sendApplicationMessageToCC(byte[] data, String appName, String nodeId) throws Exception {
- ccs.sendApplicationMessageToCC(data, appName, nodeId);
+ public void sendApplicationMessageToCC(byte[] data, String nodeId) throws Exception {
+ ccs.sendApplicationMessageToCC(data, nodeId);
}
}
\ No newline at end of file
diff --git a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Task.java b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Task.java
index eba3ec9..310878f 100644
--- a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Task.java
+++ b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Task.java
@@ -349,7 +349,6 @@
@Override
public void sendApplicationMessageToCC(byte[] message, String nodeId) throws Exception {
- this.ncs.sendApplicationMessageToCC(message, this.getJobletContext().getApplicationContext()
- .getApplicationName(), nodeId);
+ this.ncs.sendApplicationMessageToCC(message, nodeId);
}
}
\ No newline at end of file
diff --git a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/application/NCApplicationContext.java b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/application/NCApplicationContext.java
index 1121c6c..3e07fb5 100644
--- a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/application/NCApplicationContext.java
+++ b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/application/NCApplicationContext.java
@@ -4,7 +4,6 @@
import java.io.Serializable;
import edu.uci.ics.hyracks.api.application.INCApplicationContext;
-import edu.uci.ics.hyracks.api.application.INCBootstrap;
import edu.uci.ics.hyracks.api.context.IHyracksRootContext;
import edu.uci.ics.hyracks.control.common.application.ApplicationContext;
import edu.uci.ics.hyracks.control.common.context.ServerContext;
@@ -14,9 +13,8 @@
private final IHyracksRootContext rootCtx;
private Object appObject;
- public NCApplicationContext(ServerContext serverCtx, IHyracksRootContext rootCtx, String appName, String nodeId)
- throws IOException {
- super(serverCtx, appName);
+ public NCApplicationContext(ServerContext serverCtx, IHyracksRootContext rootCtx, String nodeId) throws IOException {
+ super(serverCtx);
this.nodeId = nodeId;
this.rootCtx = rootCtx;
}
@@ -31,19 +29,6 @@
}
@Override
- protected void start() throws Exception {
- ((INCBootstrap) bootstrap).setApplicationContext(this);
- bootstrap.start();
- }
-
- @Override
- protected void stop() throws Exception {
- if (bootstrap != null) {
- bootstrap.stop();
- }
- }
-
- @Override
public IHyracksRootContext getRootContext() {
return rootCtx;
}
diff --git a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/ApplicationMessageWork.java b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/ApplicationMessageWork.java
index deb1b75..a1499b8 100644
--- a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/ApplicationMessageWork.java
+++ b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/ApplicationMessageWork.java
@@ -19,35 +19,31 @@
import java.util.logging.Logger;
import edu.uci.ics.hyracks.api.messages.IMessage;
+import edu.uci.ics.hyracks.api.util.JavaSerializationUtils;
import edu.uci.ics.hyracks.control.common.work.AbstractWork;
import edu.uci.ics.hyracks.control.nc.NodeControllerService;
import edu.uci.ics.hyracks.control.nc.application.NCApplicationContext;
/**
* @author rico
- *
*/
public class ApplicationMessageWork extends AbstractWork {
-
private static final Logger LOGGER = Logger.getLogger(ApplicationMessageWork.class.getName());
private byte[] message;
private String nodeId;
private NodeControllerService ncs;
- private String appName;
- public ApplicationMessageWork(NodeControllerService ncs, byte[] message, String appName, String nodeId) {
+ public ApplicationMessageWork(NodeControllerService ncs, byte[] message, String nodeId) {
this.ncs = ncs;
this.nodeId = nodeId;
this.message = message;
- this.appName = appName;
}
@Override
public void run() {
-
- NCApplicationContext ctx = ncs.getApplications().get(appName);
+ NCApplicationContext ctx = ncs.getApplicationContext();
try {
- IMessage data = (IMessage) ctx.deserialize(message);
+ IMessage data = (IMessage) JavaSerializationUtils.deserialize(message);
if (ctx.getMessageBroker() != null) {
ctx.getMessageBroker().receivedMessage(data, nodeId);
} else {
@@ -64,5 +60,4 @@
public String toString() {
return "nodeID: " + nodeId;
}
-
-}
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/CreateApplicationWork.java b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/CreateApplicationWork.java
deleted file mode 100644
index 6eb1a95..0000000
--- a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/CreateApplicationWork.java
+++ /dev/null
@@ -1,88 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.control.nc.work;
-
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.io.Serializable;
-import java.util.Map;
-
-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.exceptions.HyracksException;
-import edu.uci.ics.hyracks.control.common.application.ApplicationStatus;
-import edu.uci.ics.hyracks.control.common.controllers.NCConfig;
-import edu.uci.ics.hyracks.control.common.controllers.NodeParameters;
-import edu.uci.ics.hyracks.control.common.work.AbstractWork;
-import edu.uci.ics.hyracks.control.nc.NodeControllerService;
-import edu.uci.ics.hyracks.control.nc.application.NCApplicationContext;
-
-public class CreateApplicationWork extends AbstractWork {
- private final NodeControllerService ncs;
-
- private final String appName;
-
- private final boolean deployHar;
-
- private final byte[] serializedDistributedState;
-
- public CreateApplicationWork(NodeControllerService ncs, String appName, boolean deployHar,
- byte[] serializedDistributedState) {
- this.ncs = ncs;
- this.appName = appName;
- this.deployHar = deployHar;
- this.serializedDistributedState = serializedDistributedState;
- }
-
- @Override
- public void run() {
- try {
- NCApplicationContext appCtx;
- Map<String, NCApplicationContext> applications = ncs.getApplications();
- if (applications.containsKey(appName)) {
- throw new HyracksException("Duplicate application with name: " + appName + " being created.");
- }
- appCtx = new NCApplicationContext(ncs.getServerContext(), ncs.getRootContext(), appName, ncs.getId());
- applications.put(appName, appCtx);
- if (deployHar) {
- NCConfig ncConfig = ncs.getConfiguration();
- NodeParameters nodeParameters = ncs.getNodeParameters();
- 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.initializeClassPath();
- appCtx.setDistributedState((Serializable) appCtx.deserialize(serializedDistributedState));
- appCtx.initialize();
- ncs.getClusterController()
- .notifyApplicationStateChange(ncs.getId(), appName, ApplicationStatus.INITIALIZED);
- } catch (Exception e) {
- throw new RuntimeException(e);
- }
- }
-}
\ No newline at end of file
diff --git a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/DestroyApplicationWork.java b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/DestroyApplicationWork.java
deleted file mode 100644
index b104ce8..0000000
--- a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/DestroyApplicationWork.java
+++ /dev/null
@@ -1,49 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.control.nc.work;
-
-import java.util.Map;
-
-import edu.uci.ics.hyracks.control.common.application.ApplicationContext;
-import edu.uci.ics.hyracks.control.common.application.ApplicationStatus;
-import edu.uci.ics.hyracks.control.common.work.AbstractWork;
-import edu.uci.ics.hyracks.control.nc.NodeControllerService;
-import edu.uci.ics.hyracks.control.nc.application.NCApplicationContext;
-
-public class DestroyApplicationWork extends AbstractWork {
- private final NodeControllerService ncs;
-
- private final String appName;
-
- public DestroyApplicationWork(NodeControllerService ncs, String appName) {
- this.ncs = ncs;
- this.appName = appName;
- }
-
- @Override
- public void run() {
- try {
- Map<String, NCApplicationContext> applications = ncs.getApplications();
- ApplicationContext appCtx = applications.remove(appName);
- if (appCtx != null) {
- appCtx.deinitialize();
- }
- ncs.getClusterController().notifyApplicationStateChange(ncs.getId(), appName,
- ApplicationStatus.DEINITIALIZED);
- } catch (Exception e) {
- throw new RuntimeException(e);
- }
- }
-}
\ No newline at end of file
diff --git a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/StartTasksWork.java b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/StartTasksWork.java
index 0c0fa3d..fffecc2 100644
--- a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/StartTasksWork.java
+++ b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/StartTasksWork.java
@@ -42,6 +42,7 @@
import edu.uci.ics.hyracks.api.job.JobFlag;
import edu.uci.ics.hyracks.api.job.JobId;
import edu.uci.ics.hyracks.api.partitions.PartitionId;
+import edu.uci.ics.hyracks.api.util.JavaSerializationUtils;
import edu.uci.ics.hyracks.control.common.job.TaskAttemptDescriptor;
import edu.uci.ics.hyracks.control.common.work.AbstractWork;
import edu.uci.ics.hyracks.control.nc.Joblet;
@@ -59,8 +60,6 @@
private final NodeControllerService ncs;
- private final String appName;
-
private final JobId jobId;
private final byte[] acgBytes;
@@ -71,11 +70,10 @@
private final EnumSet<JobFlag> flags;
- public StartTasksWork(NodeControllerService ncs, String appName, JobId jobId, byte[] acgBytes,
+ public StartTasksWork(NodeControllerService ncs, JobId jobId, byte[] acgBytes,
List<TaskAttemptDescriptor> taskDescriptors,
Map<ConnectorDescriptorId, IConnectorPolicy> connectorPoliciesMap, EnumSet<JobFlag> flags) {
this.ncs = ncs;
- this.appName = appName;
this.jobId = jobId;
this.acgBytes = acgBytes;
this.taskDescriptors = taskDescriptors;
@@ -86,10 +84,9 @@
@Override
public void run() {
try {
- Map<String, NCApplicationContext> applications = ncs.getApplications();
- NCApplicationContext appCtx = applications.get(appName);
+ NCApplicationContext appCtx = ncs.getApplicationContext();
final Joblet joblet = getOrCreateLocalJoblet(jobId, appCtx, acgBytes == null ? null
- : (ActivityClusterGraph) appCtx.deserialize(acgBytes));
+ : (ActivityClusterGraph) JavaSerializationUtils.deserialize(acgBytes));
final ActivityClusterGraph acg = joblet.getActivityClusterGraph();
IRecordDescriptorProvider rdp = new IRecordDescriptorProvider() {
diff --git a/hyracks/hyracks-examples/btree-example/btreeapp/pom.xml b/hyracks/hyracks-examples/btree-example/btreeapp/pom.xml
deleted file mode 100644
index 6350054..0000000
--- a/hyracks/hyracks-examples/btree-example/btreeapp/pom.xml
+++ /dev/null
@@ -1,88 +0,0 @@
-<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.btree</groupId>
- <artifactId>btreeapp</artifactId>
- <version>0.2.3-SNAPSHOT</version>
- <name>btreeapp</name>
-
- <parent>
- <groupId>edu.uci.ics.hyracks.examples</groupId>
- <artifactId>btree-example</artifactId>
- <version>0.2.3-SNAPSHOT</version>
- </parent>
-
- <build>
- <pluginManagement>
- <plugins>
- <plugin>
- <groupId>org.eclipse.m2e</groupId>
- <artifactId>lifecycle-mapping</artifactId>
- <version>1.0.0</version>
- <configuration>
- <lifecycleMappingMetadata>
- <pluginExecutions>
- <pluginExecution>
- <pluginExecutionFilter>
- <groupId>org.apache.maven.plugins</groupId>
- <artifactId>maven-dependency-plugin</artifactId>
- <versionRange>[1.0.0,)</versionRange>
- <goals>
- <goal>copy-dependencies</goal>
- </goals>
- </pluginExecutionFilter>
- <action>
- <ignore />
- </action>
- </pluginExecution>
- </pluginExecutions>
- </lifecycleMappingMetadata>
- </configuration>
- </plugin>
- </plugins>
- </pluginManagement>
-
- <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.btree</groupId>
- <artifactId>btreehelper</artifactId>
- <version>0.2.3-SNAPSHOT</version>
- <scope>compile</scope>
- </dependency>
- </dependencies>
-</project>
diff --git a/hyracks/hyracks-examples/btree-example/btreeapp/src/main/assembly/app-assembly.xml b/hyracks/hyracks-examples/btree-example/btreeapp/src/main/assembly/app-assembly.xml
deleted file mode 100644
index 43ace6c..0000000
--- a/hyracks/hyracks-examples/btree-example/btreeapp/src/main/assembly/app-assembly.xml
+++ /dev/null
@@ -1,13 +0,0 @@
-<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/btree-example/btreeclient/pom.xml b/hyracks/hyracks-examples/btree-example/btreeclient/pom.xml
index 82919ca..ba296ac 100644
--- a/hyracks/hyracks-examples/btree-example/btreeclient/pom.xml
+++ b/hyracks/hyracks-examples/btree-example/btreeclient/pom.xml
@@ -44,6 +44,7 @@
<plugin>
<groupId>org.codehaus.mojo</groupId>
<artifactId>appassembler-maven-plugin</artifactId>
+ <version>1.3</version>
<executions>
<execution>
<configuration>
diff --git a/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/InsertPipelineExample.java b/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/InsertPipelineExample.java
index b6e8c72..8785f16 100644
--- a/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/InsertPipelineExample.java
+++ b/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/InsertPipelineExample.java
@@ -62,9 +62,6 @@
@Option(name = "-port", usage = "Hyracks Cluster Controller Port (default: 1098)")
public int port = 1098;
- @Option(name = "-app", usage = "Hyracks Application name", required = true)
- public String app;
-
@Option(name = "-target-ncs", usage = "Comma separated list of node-controller names to use", required = true)
public String ncs;
@@ -88,7 +85,7 @@
JobSpecification job = createJob(options);
long start = System.currentTimeMillis();
- JobId jobId = hcc.startJob(options.app, job);
+ JobId jobId = hcc.startJob(job);
hcc.waitForCompletion(jobId);
long end = System.currentTimeMillis();
System.err.println(start + " " + end + " " + (end - start));
diff --git a/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/PrimaryIndexBulkLoadExample.java b/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/PrimaryIndexBulkLoadExample.java
index a6c7ea6..e9c3b1c 100644
--- a/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/PrimaryIndexBulkLoadExample.java
+++ b/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/PrimaryIndexBulkLoadExample.java
@@ -60,9 +60,6 @@
@Option(name = "-port", usage = "Hyracks Cluster Controller Port (default: 1098)")
public int port = 1098;
- @Option(name = "-app", usage = "Hyracks Application name", required = true)
- public String app;
-
@Option(name = "-target-ncs", usage = "Comma separated list of node-controller names to use", required = true)
public String ncs;
@@ -86,7 +83,7 @@
JobSpecification job = createJob(options);
long start = System.currentTimeMillis();
- JobId jobId = hcc.startJob(options.app, job);
+ JobId jobId = hcc.startJob(job);
hcc.waitForCompletion(jobId);
long end = System.currentTimeMillis();
System.err.println(start + " " + end + " " + (end - start));
diff --git a/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/PrimaryIndexSearchExample.java b/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/PrimaryIndexSearchExample.java
index d24ba33..f9e567c 100644
--- a/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/PrimaryIndexSearchExample.java
+++ b/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/PrimaryIndexSearchExample.java
@@ -59,9 +59,6 @@
@Option(name = "-port", usage = "Hyracks Cluster Controller Port (default: 1098)")
public int port = 1098;
- @Option(name = "-app", usage = "Hyracks Application name", required = true)
- public String app;
-
@Option(name = "-target-ncs", usage = "Comma separated list of node-controller names to use", required = true)
public String ncs;
@@ -79,7 +76,7 @@
JobSpecification job = createJob(options);
long start = System.currentTimeMillis();
- JobId jobId = hcc.startJob(options.app, job);
+ JobId jobId = hcc.startJob(job);
hcc.waitForCompletion(jobId);
long end = System.currentTimeMillis();
System.err.println(start + " " + end + " " + (end - start));
diff --git a/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/SecondaryIndexBulkLoadExample.java b/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/SecondaryIndexBulkLoadExample.java
index 5aa338a..bb3e174 100644
--- a/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/SecondaryIndexBulkLoadExample.java
+++ b/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/SecondaryIndexBulkLoadExample.java
@@ -55,9 +55,6 @@
@Option(name = "-port", usage = "Hyracks Cluster Controller Port (default: 1098)")
public int port = 1098;
- @Option(name = "-app", usage = "Hyracks Application name", required = true)
- public String app;
-
@Option(name = "-target-ncs", usage = "Comma separated list of node-controller names to use", required = true)
public String ncs;
@@ -81,7 +78,7 @@
JobSpecification job = createJob(options);
long start = System.currentTimeMillis();
- JobId jobId = hcc.startJob(options.app, job);
+ JobId jobId = hcc.startJob(job);
hcc.waitForCompletion(jobId);
long end = System.currentTimeMillis();
System.err.println(start + " " + end + " " + (end - start));
diff --git a/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/SecondaryIndexSearchExample.java b/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/SecondaryIndexSearchExample.java
index 277668b..1b4bb40 100644
--- a/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/SecondaryIndexSearchExample.java
+++ b/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/SecondaryIndexSearchExample.java
@@ -59,9 +59,6 @@
@Option(name = "-port", usage = "Hyracks Cluster Controller Port (default: 1098)")
public int port = 1098;
- @Option(name = "-app", usage = "Hyracks Application name", required = true)
- public String app;
-
@Option(name = "-target-ncs", usage = "Comma separated list of node-controller names to use", required = true)
public String ncs;
@@ -82,7 +79,7 @@
JobSpecification job = createJob(options);
long start = System.currentTimeMillis();
- JobId jobId = hcc.startJob(options.app, job);
+ JobId jobId = hcc.startJob(job);
hcc.waitForCompletion(jobId);
long end = System.currentTimeMillis();
System.err.println(start + " " + end + " " + (end - start));
diff --git a/hyracks/hyracks-cli/pom.xml b/hyracks/hyracks-examples/btree-example/btreeserver/pom.xml
similarity index 60%
copy from hyracks/hyracks-cli/pom.xml
copy to hyracks/hyracks-examples/btree-example/btreeserver/pom.xml
index 3933be0..ec6747d 100644
--- a/hyracks/hyracks-cli/pom.xml
+++ b/hyracks/hyracks-examples/btree-example/btreeserver/pom.xml
@@ -1,53 +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</groupId>
- <artifactId>hyracks-cli</artifactId>
+ <groupId>edu.uci.ics.hyracks.examples.btree</groupId>
+ <artifactId>btreeserver</artifactId>
<version>0.2.3-SNAPSHOT</version>
- <name>hyracks-cli</name>
+ <name>btreeserver</name>
<parent>
- <groupId>edu.uci.ics.hyracks</groupId>
- <artifactId>hyracks</artifactId>
+ <groupId>edu.uci.ics.hyracks.examples</groupId>
+ <artifactId>btree-example</artifactId>
<version>0.2.3-SNAPSHOT</version>
</parent>
<build>
<plugins>
<plugin>
- <groupId>org.apache.maven.plugins</groupId>
- <artifactId>maven-compiler-plugin</artifactId>
- <version>2.0.2</version>
- <configuration>
- <source>1.6</source>
- <target>1.6</target>
- </configuration>
- </plugin>
- <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>
+ <version>1.3</version>
<executions>
<execution>
<configuration>
<programs>
<program>
- <mainClass>edu.uci.ics.hyracks.cli.Main</mainClass>
- <name>hyrackscli</name>
+ <mainClass>edu.uci.ics.hyracks.control.cc.CCDriver</mainClass>
+ <name>hyrackscc</name>
+ </program>
+ <program>
+ <mainClass>edu.uci.ics.hyracks.control.nc.NCDriver</mainClass>
+ <name>hyracksnc</name>
</program>
</programs>
<repositoryLayout>flat</repositoryLayout>
@@ -81,16 +61,23 @@
</build>
<dependencies>
<dependency>
- <groupId>jline</groupId>
- <artifactId>jline</artifactId>
- <version>0.9.94</version>
+ <groupId>edu.uci.ics.hyracks.examples.btree</groupId>
+ <artifactId>btreehelper</artifactId>
+ <version>0.2.3-SNAPSHOT</version>
+ <scope>compile</scope>
+ </dependency>
+ <dependency>
+ <groupId>edu.uci.ics.hyracks</groupId>
+ <artifactId>hyracks-control-cc</artifactId>
+ <version>0.2.3-SNAPSHOT</version>
<type>jar</type>
<scope>compile</scope>
</dependency>
<dependency>
<groupId>edu.uci.ics.hyracks</groupId>
- <artifactId>hyracks-api</artifactId>
+ <artifactId>hyracks-control-nc</artifactId>
<version>0.2.3-SNAPSHOT</version>
+ <type>jar</type>
<scope>compile</scope>
</dependency>
</dependencies>
diff --git a/hyracks/hyracks-cli/src/main/assembly/binary-assembly.xml b/hyracks/hyracks-examples/btree-example/btreeserver/src/main/assembly/binary-assembly.xml
similarity index 81%
rename from hyracks/hyracks-cli/src/main/assembly/binary-assembly.xml
rename to hyracks/hyracks-examples/btree-example/btreeserver/src/main/assembly/binary-assembly.xml
index 0500499..cd598d9 100644
--- a/hyracks/hyracks-cli/src/main/assembly/binary-assembly.xml
+++ b/hyracks/hyracks-examples/btree-example/btreeserver/src/main/assembly/binary-assembly.xml
@@ -15,5 +15,9 @@
<directory>target/appassembler/lib</directory>
<outputDirectory>lib</outputDirectory>
</fileSet>
+ <fileSet>
+ <directory>docs</directory>
+ <outputDirectory>docs</outputDirectory>
+ </fileSet>
</fileSets>
</assembly>
diff --git a/hyracks/hyracks-examples/btree-example/pom.xml b/hyracks/hyracks-examples/btree-example/pom.xml
index 0ff4fc3..d8a4367 100644
--- a/hyracks/hyracks-examples/btree-example/pom.xml
+++ b/hyracks/hyracks-examples/btree-example/pom.xml
@@ -15,6 +15,6 @@
<modules>
<module>btreehelper</module>
<module>btreeclient</module>
- <module>btreeapp</module>
+ <module>btreeserver</module>
</modules>
</project>
diff --git a/hyracks/hyracks-examples/hadoop-compat-example/hadoopcompatapp/src/main/assembly/app-assembly.xml b/hyracks/hyracks-examples/hadoop-compat-example/hadoopcompatapp/src/main/assembly/app-assembly.xml
deleted file mode 100644
index 43ace6c..0000000
--- a/hyracks/hyracks-examples/hadoop-compat-example/hadoopcompatapp/src/main/assembly/app-assembly.xml
+++ /dev/null
@@ -1,13 +0,0 @@
-<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/hadoop-compat-example/hadoopcompatclient/pom.xml b/hyracks/hyracks-examples/hadoop-compat-example/hadoopcompatclient/pom.xml
index 2f0b00f..dd96db8 100644
--- a/hyracks/hyracks-examples/hadoop-compat-example/hadoopcompatclient/pom.xml
+++ b/hyracks/hyracks-examples/hadoop-compat-example/hadoopcompatclient/pom.xml
@@ -40,6 +40,7 @@
<plugin>
<groupId>org.codehaus.mojo</groupId>
<artifactId>appassembler-maven-plugin</artifactId>
+ <version>1.3</version>
<executions>
<execution>
<configuration>
diff --git a/hyracks/hyracks-examples/hadoop-compat-example/hadoopcompatapp/conf/local_cluster.conf b/hyracks/hyracks-examples/hadoop-compat-example/hadoopcompatserver/conf/local_cluster.conf
similarity index 100%
rename from hyracks/hyracks-examples/hadoop-compat-example/hadoopcompatapp/conf/local_cluster.conf
rename to hyracks/hyracks-examples/hadoop-compat-example/hadoopcompatserver/conf/local_cluster.conf
diff --git a/hyracks/hyracks-examples/hadoop-compat-example/hadoopcompatapp/data/file1.txt b/hyracks/hyracks-examples/hadoop-compat-example/hadoopcompatserver/data/file1.txt
similarity index 100%
rename from hyracks/hyracks-examples/hadoop-compat-example/hadoopcompatapp/data/file1.txt
rename to hyracks/hyracks-examples/hadoop-compat-example/hadoopcompatserver/data/file1.txt
diff --git a/hyracks/hyracks-examples/hadoop-compat-example/hadoopcompatapp/data/file2.txt b/hyracks/hyracks-examples/hadoop-compat-example/hadoopcompatserver/data/file2.txt
similarity index 100%
rename from hyracks/hyracks-examples/hadoop-compat-example/hadoopcompatapp/data/file2.txt
rename to hyracks/hyracks-examples/hadoop-compat-example/hadoopcompatserver/data/file2.txt
diff --git a/hyracks/hyracks-examples/hadoop-compat-example/hadoopcompatapp/job/wordcount/wordcount.job b/hyracks/hyracks-examples/hadoop-compat-example/hadoopcompatserver/job/wordcount/wordcount.job
similarity index 100%
rename from hyracks/hyracks-examples/hadoop-compat-example/hadoopcompatapp/job/wordcount/wordcount.job
rename to hyracks/hyracks-examples/hadoop-compat-example/hadoopcompatserver/job/wordcount/wordcount.job
diff --git a/hyracks/hyracks-examples/hadoop-compat-example/hadoopcompatapp/pom.xml b/hyracks/hyracks-examples/hadoop-compat-example/hadoopcompatserver/pom.xml
similarity index 61%
rename from hyracks/hyracks-examples/hadoop-compat-example/hadoopcompatapp/pom.xml
rename to hyracks/hyracks-examples/hadoop-compat-example/hadoopcompatserver/pom.xml
index 70c663d..b538db5 100644
--- a/hyracks/hyracks-examples/hadoop-compat-example/hadoopcompatapp/pom.xml
+++ b/hyracks/hyracks-examples/hadoop-compat-example/hadoopcompatserver/pom.xml
@@ -1,9 +1,9 @@
<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.compat</groupId>
- <artifactId>hadoopcompatapp</artifactId>
+ <artifactId>hadoopcompatserver</artifactId>
<version>0.2.3-SNAPSHOT</version>
- <name>hadoopcompatapp</name>
+ <name>hadoopcompatserver</name>
<parent>
<groupId>edu.uci.ics.hyracks.examples</groupId>
@@ -12,49 +12,31 @@
</parent>
<build>
- <pluginManagement>
- <plugins>
- <plugin>
- <groupId>org.eclipse.m2e</groupId>
- <artifactId>lifecycle-mapping</artifactId>
- <version>1.0.0</version>
- <configuration>
- <lifecycleMappingMetadata>
- <pluginExecutions>
- <pluginExecution>
- <pluginExecutionFilter>
- <groupId>org.apache.maven.plugins</groupId>
- <artifactId>maven-dependency-plugin</artifactId>
- <versionRange>[1.0.0,)</versionRange>
- <goals>
- <goal>copy-dependencies</goal>
- </goals>
- </pluginExecutionFilter>
- <action>
- <ignore />
- </action>
- </pluginExecution>
- </pluginExecutions>
- </lifecycleMappingMetadata>
- </configuration>
- </plugin>
- </plugins>
- </pluginManagement>
-
<plugins>
<plugin>
- <groupId>org.apache.maven.plugins</groupId>
- <artifactId>maven-dependency-plugin</artifactId>
+ <groupId>org.codehaus.mojo</groupId>
+ <artifactId>appassembler-maven-plugin</artifactId>
+ <version>1.3</version>
<executions>
<execution>
- <id>copy-dependencies</id>
+ <configuration>
+ <programs>
+ <program>
+ <mainClass>edu.uci.ics.hyracks.control.cc.CCDriver</mainClass>
+ <name>hyrackscc</name>
+ </program>
+ <program>
+ <mainClass>edu.uci.ics.hyracks.control.nc.NCDriver</mainClass>
+ <name>hyracksnc</name>
+ </program>
+ </programs>
+ <repositoryLayout>flat</repositoryLayout>
+ <repositoryName>lib</repositoryName>
+ </configuration>
<phase>package</phase>
<goals>
- <goal>copy-dependencies</goal>
+ <goal>assemble</goal>
</goals>
- <configuration>
- <outputDirectory>target/application/lib</outputDirectory>
- </configuration>
</execution>
</executions>
</plugin>
@@ -65,7 +47,7 @@
<execution>
<configuration>
<descriptors>
- <descriptor>src/main/assembly/app-assembly.xml</descriptor>
+ <descriptor>src/main/assembly/binary-assembly.xml</descriptor>
</descriptors>
</configuration>
<phase>package</phase>
@@ -80,8 +62,7 @@
<artifactId>hyracks-virtualcluster-maven-plugin</artifactId>
<version>0.2.3-SNAPSHOT</version>
<configuration>
- <hyracksServerHome>${basedir}/../../../hyracks-server/target/hyracks-server-${project.version}-binary-assembly</hyracksServerHome>
- <hyracksCLIHome>${basedir}/../../../hyracks-cli/target/hyracks-cli-${project.version}-binary-assembly</hyracksCLIHome>
+ <hyracksServerHome>${basedir}/target/hadoopcompatserver-${project.version}-binary-assembly</hyracksServerHome>
<jvmOptions>${jvm.extraargs}</jvmOptions>
</configuration>
<executions>
@@ -91,9 +72,6 @@
<goals>
<goal>start-cc</goal>
</goals>
- <configuration>
- <workingDir>${project.build.directory}</workingDir>
- </configuration>
</execution>
<execution>
<id>hyracks-nc1-start</id>
@@ -105,7 +83,6 @@
<nodeId>NC1</nodeId>
<dataIpAddress>127.0.0.1</dataIpAddress>
<ccHost>localhost</ccHost>
- <workingDir>${project.build.directory}</workingDir>
</configuration>
</execution>
<execution>
@@ -118,29 +95,16 @@
<nodeId>NC2</nodeId>
<dataIpAddress>127.0.0.1</dataIpAddress>
<ccHost>localhost</ccHost>
- <workingDir>${project.build.directory}</workingDir>
</configuration>
</execution>
<execution>
- <id>deploy-app</id>
- <phase>pre-integration-test</phase>
+ <id>stop-services</id>
+ <phase>post-integration-test</phase>
<goals>
- <goal>deploy-app</goal>
+ <goal>stop-services</goal>
</goals>
- <configuration>
- <ccHost>localhost</ccHost>
- <appName>compat</appName>
- <harFile>${project.build.directory}/hadoopcompatapp-${project.version}-app-assembly.zip</harFile>
- </configuration>
</execution>
- <execution>
- <id>stop-services</id>
- <phase>post-integration-test</phase>
- <goals>
- <goal>stop-services</goal>
- </goals>
- </execution>
- </executions>
+ </executions>
</plugin>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
@@ -174,6 +138,20 @@
<version>0.2.3-SNAPSHOT</version>
<scope>compile</scope>
</dependency>
+ <dependency>
+ <groupId>edu.uci.ics.hyracks</groupId>
+ <artifactId>hyracks-control-cc</artifactId>
+ <version>0.2.3-SNAPSHOT</version>
+ <type>jar</type>
+ <scope>compile</scope>
+ </dependency>
+ <dependency>
+ <groupId>edu.uci.ics.hyracks</groupId>
+ <artifactId>hyracks-control-nc</artifactId>
+ <version>0.2.3-SNAPSHOT</version>
+ <type>jar</type>
+ <scope>compile</scope>
+ </dependency>
<dependency>
<groupId>edu.uci.ics.hyracks.examples.compat</groupId>
<artifactId>hadoopcompatclient</artifactId>
diff --git a/hyracks/hyracks-cli/src/main/assembly/binary-assembly.xml b/hyracks/hyracks-examples/hadoop-compat-example/hadoopcompatserver/src/main/assembly/binary-assembly.xml
similarity index 81%
copy from hyracks/hyracks-cli/src/main/assembly/binary-assembly.xml
copy to hyracks/hyracks-examples/hadoop-compat-example/hadoopcompatserver/src/main/assembly/binary-assembly.xml
index 0500499..cd598d9 100644
--- a/hyracks/hyracks-cli/src/main/assembly/binary-assembly.xml
+++ b/hyracks/hyracks-examples/hadoop-compat-example/hadoopcompatserver/src/main/assembly/binary-assembly.xml
@@ -15,5 +15,9 @@
<directory>target/appassembler/lib</directory>
<outputDirectory>lib</outputDirectory>
</fileSet>
+ <fileSet>
+ <directory>docs</directory>
+ <outputDirectory>docs</outputDirectory>
+ </fileSet>
</fileSets>
</assembly>
diff --git a/hyracks/hyracks-examples/hadoop-compat-example/hadoopcompatapp/src/test/java/edu/uci/ics/hyracks/examples/compat/test/WordCountCompatibilityIT.java b/hyracks/hyracks-examples/hadoop-compat-example/hadoopcompatserver/src/test/java/edu/uci/ics/hyracks/examples/compat/test/WordCountCompatibilityIT.java
similarity index 100%
rename from hyracks/hyracks-examples/hadoop-compat-example/hadoopcompatapp/src/test/java/edu/uci/ics/hyracks/examples/compat/test/WordCountCompatibilityIT.java
rename to hyracks/hyracks-examples/hadoop-compat-example/hadoopcompatserver/src/test/java/edu/uci/ics/hyracks/examples/compat/test/WordCountCompatibilityIT.java
diff --git a/hyracks/hyracks-examples/hadoop-compat-example/pom.xml b/hyracks/hyracks-examples/hadoop-compat-example/pom.xml
index a2cf5ae..0614034 100644
--- a/hyracks/hyracks-examples/hadoop-compat-example/pom.xml
+++ b/hyracks/hyracks-examples/hadoop-compat-example/pom.xml
@@ -15,7 +15,7 @@
<modules>
<module>hadoopcompathelper</module>
<module>hadoopcompatclient</module>
- <module>hadoopcompatapp</module>
+ <module>hadoopcompatserver</module>
</modules>
<dependencies>
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AbstractIntegrationTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AbstractIntegrationTest.java
index 023bdd9..ce2eb83 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AbstractIntegrationTest.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AbstractIntegrationTest.java
@@ -94,7 +94,6 @@
nc2.start();
hcc = new HyracksConnection(ccConfig.clientNetIpAddress, ccConfig.clientNetPort);
- hcc.createApplication("test", null);
if (LOGGER.isLoggable(Level.INFO)) {
LOGGER.info("Starting CC in " + ccRoot.getAbsolutePath());
}
@@ -111,7 +110,7 @@
if (LOGGER.isLoggable(Level.INFO)) {
LOGGER.info(spec.toJSON().toString(2));
}
- JobId jobId = hcc.startJob("test", spec, EnumSet.of(JobFlag.PROFILE_RUNTIME));
+ JobId jobId = hcc.startJob(spec, EnumSet.of(JobFlag.PROFILE_RUNTIME));
if (LOGGER.isLoggable(Level.INFO)) {
LOGGER.info(jobId.toString());
}
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AbstractMultiNCIntegrationTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AbstractMultiNCIntegrationTest.java
index e0b8c73..279e5f9 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AbstractMultiNCIntegrationTest.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AbstractMultiNCIntegrationTest.java
@@ -90,7 +90,6 @@
}
hcc = new HyracksConnection(ccConfig.clientNetIpAddress, ccConfig.clientNetPort);
- hcc.createApplication("test", null);
if (LOGGER.isLoggable(Level.INFO)) {
LOGGER.info("Starting CC in " + ccRoot.getAbsolutePath());
}
@@ -108,7 +107,7 @@
if (LOGGER.isLoggable(Level.INFO)) {
LOGGER.info(spec.toJSON().toString(2));
}
- JobId jobId = hcc.startJob("test", spec, EnumSet.of(JobFlag.PROFILE_RUNTIME));
+ JobId jobId = hcc.startJob(spec, EnumSet.of(JobFlag.PROFILE_RUNTIME));
if (LOGGER.isLoggable(Level.INFO)) {
LOGGER.info(jobId.toString());
}
diff --git a/hyracks/hyracks-examples/text-example/pom.xml b/hyracks/hyracks-examples/text-example/pom.xml
index ba8649e..367e0a5 100644
--- a/hyracks/hyracks-examples/text-example/pom.xml
+++ b/hyracks/hyracks-examples/text-example/pom.xml
@@ -15,6 +15,6 @@
<modules>
<module>texthelper</module>
<module>textclient</module>
- <module>textapp</module>
+ <module>textserver</module>
</modules>
</project>
diff --git a/hyracks/hyracks-examples/text-example/textapp/src/main/assembly/app-assembly.xml b/hyracks/hyracks-examples/text-example/textapp/src/main/assembly/app-assembly.xml
deleted file mode 100644
index 43ace6c..0000000
--- a/hyracks/hyracks-examples/text-example/textapp/src/main/assembly/app-assembly.xml
+++ /dev/null
@@ -1,13 +0,0 @@
-<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/text-example/textclient/pom.xml b/hyracks/hyracks-examples/text-example/textclient/pom.xml
index d593ef9..901f1fb2 100644
--- a/hyracks/hyracks-examples/text-example/textclient/pom.xml
+++ b/hyracks/hyracks-examples/text-example/textclient/pom.xml
@@ -40,6 +40,7 @@
<plugin>
<groupId>org.codehaus.mojo</groupId>
<artifactId>appassembler-maven-plugin</artifactId>
+ <version>1.3</version>
<executions>
<execution>
<id>textclient</id>
diff --git a/hyracks/hyracks-examples/text-example/textclient/src/main/java/edu/uci/ics/hyracks/examples/text/client/ExternalGroupClient.java b/hyracks/hyracks-examples/text-example/textclient/src/main/java/edu/uci/ics/hyracks/examples/text/client/ExternalGroupClient.java
index 943f232..18813a7 100644
--- a/hyracks/hyracks-examples/text-example/textclient/src/main/java/edu/uci/ics/hyracks/examples/text/client/ExternalGroupClient.java
+++ b/hyracks/hyracks-examples/text-example/textclient/src/main/java/edu/uci/ics/hyracks/examples/text/client/ExternalGroupClient.java
@@ -74,9 +74,6 @@
@Option(name = "-port", usage = "Hyracks Cluster Controller Port (default: 1098)")
public int port = 1098;
- @Option(name = "-app", usage = "Hyracks Application name", required = true)
- public String app;
-
@Option(name = "-infile-splits", usage = "Comma separated list of file-splits for the input. A file-split is <node-name>:<path>", required = true)
public String inFileSplits;
@@ -122,7 +119,7 @@
System.out.print(i + "\t" + (System.currentTimeMillis() - start));
start = System.currentTimeMillis();
- JobId jobId = hcc.startJob(options.app, job);
+ JobId jobId = hcc.startJob(job);
hcc.waitForCompletion(jobId);
System.out.println("\t" + (System.currentTimeMillis() - start));
}
diff --git a/hyracks/hyracks-examples/text-example/textclient/src/main/java/edu/uci/ics/hyracks/examples/text/client/WordCountMain.java b/hyracks/hyracks-examples/text-example/textclient/src/main/java/edu/uci/ics/hyracks/examples/text/client/WordCountMain.java
index 31019ab..5b6fad9 100644
--- a/hyracks/hyracks-examples/text-example/textclient/src/main/java/edu/uci/ics/hyracks/examples/text/client/WordCountMain.java
+++ b/hyracks/hyracks-examples/text-example/textclient/src/main/java/edu/uci/ics/hyracks/examples/text/client/WordCountMain.java
@@ -65,9 +65,6 @@
@Option(name = "-port", usage = "Hyracks Cluster Controller Port (default: 1098)")
public int port = 1098;
- @Option(name = "-app", usage = "Hyracks Application name", required = true)
- public String app;
-
@Option(name = "-infile-splits", usage = "Comma separated list of file-splits for the input. A file-split is <node-name>:<path>", required = true)
public String inFileSplits;
@@ -101,8 +98,8 @@
options.algo, options.htSize, options.sbSize, options.format);
long start = System.currentTimeMillis();
- JobId jobId = hcc.startJob(options.app, job, options.runtimeProfiling ? EnumSet.of(JobFlag.PROFILE_RUNTIME)
- : EnumSet.noneOf(JobFlag.class));
+ JobId jobId = hcc.startJob(job,
+ options.runtimeProfiling ? EnumSet.of(JobFlag.PROFILE_RUNTIME) : EnumSet.noneOf(JobFlag.class));
hcc.waitForCompletion(jobId);
long end = System.currentTimeMillis();
System.err.println(start + " " + end + " " + (end - start));
diff --git a/hyracks/hyracks-examples/text-example/textapp/data/file1.txt b/hyracks/hyracks-examples/text-example/textserver/data/file1.txt
similarity index 100%
rename from hyracks/hyracks-examples/text-example/textapp/data/file1.txt
rename to hyracks/hyracks-examples/text-example/textserver/data/file1.txt
diff --git a/hyracks/hyracks-examples/text-example/textapp/data/file2.txt b/hyracks/hyracks-examples/text-example/textserver/data/file2.txt
similarity index 100%
rename from hyracks/hyracks-examples/text-example/textapp/data/file2.txt
rename to hyracks/hyracks-examples/text-example/textserver/data/file2.txt
diff --git a/hyracks/hyracks-examples/text-example/textapp/pom.xml b/hyracks/hyracks-examples/text-example/textserver/pom.xml
similarity index 66%
rename from hyracks/hyracks-examples/text-example/textapp/pom.xml
rename to hyracks/hyracks-examples/text-example/textserver/pom.xml
index 8ac69d8..1b1f2d4 100644
--- a/hyracks/hyracks-examples/text-example/textapp/pom.xml
+++ b/hyracks/hyracks-examples/text-example/textserver/pom.xml
@@ -1,9 +1,9 @@
<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.text</groupId>
- <artifactId>textapp</artifactId>
+ <artifactId>textserver</artifactId>
<version>0.2.3-SNAPSHOT</version>
- <name>textapp</name>
+ <name>textserver</name>
<parent>
<groupId>edu.uci.ics.hyracks.examples</groupId>
@@ -12,49 +12,31 @@
</parent>
<build>
- <pluginManagement>
- <plugins>
- <plugin>
- <groupId>org.eclipse.m2e</groupId>
- <artifactId>lifecycle-mapping</artifactId>
- <version>1.0.0</version>
- <configuration>
- <lifecycleMappingMetadata>
- <pluginExecutions>
- <pluginExecution>
- <pluginExecutionFilter>
- <groupId>org.apache.maven.plugins</groupId>
- <artifactId>maven-dependency-plugin</artifactId>
- <versionRange>[1.0.0,)</versionRange>
- <goals>
- <goal>copy-dependencies</goal>
- </goals>
- </pluginExecutionFilter>
- <action>
- <ignore />
- </action>
- </pluginExecution>
- </pluginExecutions>
- </lifecycleMappingMetadata>
- </configuration>
- </plugin>
- </plugins>
- </pluginManagement>
-
<plugins>
<plugin>
- <groupId>org.apache.maven.plugins</groupId>
- <artifactId>maven-dependency-plugin</artifactId>
+ <groupId>org.codehaus.mojo</groupId>
+ <artifactId>appassembler-maven-plugin</artifactId>
+ <version>1.3</version>
<executions>
<execution>
- <id>copy-dependencies</id>
+ <configuration>
+ <programs>
+ <program>
+ <mainClass>edu.uci.ics.hyracks.control.cc.CCDriver</mainClass>
+ <name>hyrackscc</name>
+ </program>
+ <program>
+ <mainClass>edu.uci.ics.hyracks.control.nc.NCDriver</mainClass>
+ <name>hyracksnc</name>
+ </program>
+ </programs>
+ <repositoryLayout>flat</repositoryLayout>
+ <repositoryName>lib</repositoryName>
+ </configuration>
<phase>package</phase>
<goals>
- <goal>copy-dependencies</goal>
+ <goal>assemble</goal>
</goals>
- <configuration>
- <outputDirectory>target/application/lib</outputDirectory>
- </configuration>
</execution>
</executions>
</plugin>
@@ -65,7 +47,7 @@
<execution>
<configuration>
<descriptors>
- <descriptor>src/main/assembly/app-assembly.xml</descriptor>
+ <descriptor>src/main/assembly/binary-assembly.xml</descriptor>
</descriptors>
</configuration>
<phase>package</phase>
@@ -80,8 +62,7 @@
<artifactId>hyracks-virtualcluster-maven-plugin</artifactId>
<version>0.2.3-SNAPSHOT</version>
<configuration>
- <hyracksServerHome>${basedir}/../../../hyracks-server/target/hyracks-server-${project.version}-binary-assembly</hyracksServerHome>
- <hyracksCLIHome>${basedir}/../../../hyracks-cli/target/hyracks-cli-${project.version}-binary-assembly</hyracksCLIHome>
+ <hyracksServerHome>${basedir}/target/textserver-${project.version}-binary-assembly</hyracksServerHome>
<jvmOptions>${jvm.extraargs}</jvmOptions>
</configuration>
<executions>
@@ -117,18 +98,6 @@
</configuration>
</execution>
<execution>
- <id>deploy-app</id>
- <phase>pre-integration-test</phase>
- <goals>
- <goal>deploy-app</goal>
- </goals>
- <configuration>
- <ccHost>localhost</ccHost>
- <appName>text</appName>
- <harFile>${project.build.directory}/textapp-${project.version}-app-assembly.zip</harFile>
- </configuration>
- </execution>
- <execution>
<id>stop-services</id>
<phase>post-integration-test</phase>
<goals>
@@ -170,6 +139,20 @@
<scope>compile</scope>
</dependency>
<dependency>
+ <groupId>edu.uci.ics.hyracks</groupId>
+ <artifactId>hyracks-control-cc</artifactId>
+ <version>0.2.3-SNAPSHOT</version>
+ <type>jar</type>
+ <scope>compile</scope>
+ </dependency>
+ <dependency>
+ <groupId>edu.uci.ics.hyracks</groupId>
+ <artifactId>hyracks-control-nc</artifactId>
+ <version>0.2.3-SNAPSHOT</version>
+ <type>jar</type>
+ <scope>compile</scope>
+ </dependency>
+ <dependency>
<groupId>edu.uci.ics.hyracks.examples.text</groupId>
<artifactId>textclient</artifactId>
<version>0.2.3-SNAPSHOT</version>
diff --git a/hyracks/hyracks-cli/src/main/assembly/binary-assembly.xml b/hyracks/hyracks-examples/text-example/textserver/src/main/assembly/binary-assembly.xml
similarity index 81%
copy from hyracks/hyracks-cli/src/main/assembly/binary-assembly.xml
copy to hyracks/hyracks-examples/text-example/textserver/src/main/assembly/binary-assembly.xml
index 0500499..cd598d9 100644
--- a/hyracks/hyracks-cli/src/main/assembly/binary-assembly.xml
+++ b/hyracks/hyracks-examples/text-example/textserver/src/main/assembly/binary-assembly.xml
@@ -15,5 +15,9 @@
<directory>target/appassembler/lib</directory>
<outputDirectory>lib</outputDirectory>
</fileSet>
+ <fileSet>
+ <directory>docs</directory>
+ <outputDirectory>docs</outputDirectory>
+ </fileSet>
</fileSets>
</assembly>
diff --git a/hyracks/hyracks-examples/text-example/textapp/src/test/java/edu/uci/ics/hyracks/examples/text/test/WordCountIT.java b/hyracks/hyracks-examples/text-example/textserver/src/test/java/edu/uci/ics/hyracks/examples/text/test/WordCountIT.java
similarity index 90%
rename from hyracks/hyracks-examples/text-example/textapp/src/test/java/edu/uci/ics/hyracks/examples/text/test/WordCountIT.java
rename to hyracks/hyracks-examples/text-example/textserver/src/test/java/edu/uci/ics/hyracks/examples/text/test/WordCountIT.java
index 9659288..76af72d 100644
--- a/hyracks/hyracks-examples/text-example/textapp/src/test/java/edu/uci/ics/hyracks/examples/text/test/WordCountIT.java
+++ b/hyracks/hyracks-examples/text-example/textserver/src/test/java/edu/uci/ics/hyracks/examples/text/test/WordCountIT.java
@@ -10,7 +10,7 @@
@Test
public void runWordCount() throws Exception {
WordCountMain.main(new String[] { "-host", "localhost", "-infile-splits", getInfileSplits(), "-outfile-splits",
- getOutfileSplits(), "-algo", "-hash", "-app", "text" });
+ getOutfileSplits(), "-algo", "-hash" });
}
private String getInfileSplits() {
@@ -22,4 +22,4 @@
return "NC1:" + new File("target/wc1.txt").getAbsolutePath() + ",NC2:"
+ new File("target/wc2.txt").getAbsolutePath();
}
-}
\ No newline at end of file
+}
diff --git a/hyracks/hyracks-examples/tpch-example/pom.xml b/hyracks/hyracks-examples/tpch-example/pom.xml
index b237c9b..93514de 100644
--- a/hyracks/hyracks-examples/tpch-example/pom.xml
+++ b/hyracks/hyracks-examples/tpch-example/pom.xml
@@ -14,6 +14,6 @@
<modules>
<module>tpchclient</module>
- <module>tpchapp</module>
+ <module>tpchserver</module>
</modules>
</project>
diff --git a/hyracks/hyracks-examples/tpch-example/tpchapp/pom.xml b/hyracks/hyracks-examples/tpch-example/tpchapp/pom.xml
deleted file mode 100644
index 76eb257..0000000
--- a/hyracks/hyracks-examples/tpch-example/tpchapp/pom.xml
+++ /dev/null
@@ -1,91 +0,0 @@
-<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>tpchapp</artifactId>
- <name>tpchapp</name>
- <parent>
- <groupId>edu.uci.ics.hyracks.examples</groupId>
- <artifactId>tpch-example</artifactId>
- <version>0.2.3-SNAPSHOT</version>
- </parent>
-
- <build>
- <pluginManagement>
- <plugins>
- <plugin>
- <groupId>org.eclipse.m2e</groupId>
- <artifactId>lifecycle-mapping</artifactId>
- <version>1.0.0</version>
- <configuration>
- <lifecycleMappingMetadata>
- <pluginExecutions>
- <pluginExecution>
- <pluginExecutionFilter>
- <groupId>org.apache.maven.plugins</groupId>
- <artifactId>maven-dependency-plugin</artifactId>
- <versionRange>[1.0.0,)</versionRange>
- <goals>
- <goal>copy-dependencies</goal>
- </goals>
- </pluginExecutionFilter>
- <action>
- <ignore />
- </action>
- </pluginExecution>
- </pluginExecutions>
- </lifecycleMappingMetadata>
- </configuration>
- </plugin>
- </plugins>
- </pluginManagement>
-
- <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</groupId>
- <artifactId>hyracks-dataflow-std</artifactId>
- <version>0.2.3-SNAPSHOT</version>
- <scope>compile</scope>
- </dependency>
- <dependency>
- <groupId>edu.uci.ics.hyracks</groupId>
- <artifactId>hyracks-data-std</artifactId>
- <version>0.2.3-SNAPSHOT</version>
- </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
deleted file mode 100644
index 43ace6c..0000000
--- a/hyracks/hyracks-examples/tpch-example/tpchapp/src/main/assembly/app-assembly.xml
+++ /dev/null
@@ -1,13 +0,0 @@
-<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/pom.xml b/hyracks/hyracks-examples/tpch-example/tpchclient/pom.xml
index 3170306..6b3f603 100644
--- a/hyracks/hyracks-examples/tpch-example/tpchclient/pom.xml
+++ b/hyracks/hyracks-examples/tpch-example/tpchclient/pom.xml
@@ -36,6 +36,7 @@
<plugin>
<groupId>org.codehaus.mojo</groupId>
<artifactId>appassembler-maven-plugin</artifactId>
+ <version>1.3</version>
<executions>
<execution>
<configuration>
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
index 01ccdef..efccba4 100644
--- 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
@@ -72,9 +72,6 @@
@Option(name = "-port", usage = "Hyracks Cluster Controller Port (default: 1098)", required = false)
public int port = 1098;
- @Option(name = "-app", usage = "Hyracks Application name", required = true)
- public String app;
-
@Option(name = "-infile-customer-splits", usage = "Comma separated list of file-splits for the CUSTOMER input. A file-split is <node-name>:<path>", required = true)
public String inFileCustomerSplits;
@@ -127,7 +124,7 @@
options.graceFactor, options.memSize, options.tableSize, options.hasGroupBy);
long start = System.currentTimeMillis();
- JobId jobId = hcc.startJob(options.app, job,
+ JobId jobId = hcc.startJob(job,
options.profile ? EnumSet.of(JobFlag.PROFILE_RUNTIME) : EnumSet.noneOf(JobFlag.class));
hcc.waitForCompletion(jobId);
long end = System.currentTimeMillis();
diff --git a/hyracks/hyracks-cli/pom.xml b/hyracks/hyracks-examples/tpch-example/tpchserver/pom.xml
similarity index 60%
rename from hyracks/hyracks-cli/pom.xml
rename to hyracks/hyracks-examples/tpch-example/tpchserver/pom.xml
index 3933be0..94d897a 100644
--- a/hyracks/hyracks-cli/pom.xml
+++ b/hyracks/hyracks-examples/tpch-example/tpchserver/pom.xml
@@ -1,53 +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</groupId>
- <artifactId>hyracks-cli</artifactId>
+ <groupId>edu.uci.ics.hyracks.examples.tpch</groupId>
+ <artifactId>tpchserver</artifactId>
<version>0.2.3-SNAPSHOT</version>
- <name>hyracks-cli</name>
+ <name>tpchserver</name>
<parent>
- <groupId>edu.uci.ics.hyracks</groupId>
- <artifactId>hyracks</artifactId>
+ <groupId>edu.uci.ics.hyracks.examples</groupId>
+ <artifactId>tpch-example</artifactId>
<version>0.2.3-SNAPSHOT</version>
</parent>
<build>
<plugins>
<plugin>
- <groupId>org.apache.maven.plugins</groupId>
- <artifactId>maven-compiler-plugin</artifactId>
- <version>2.0.2</version>
- <configuration>
- <source>1.6</source>
- <target>1.6</target>
- </configuration>
- </plugin>
- <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>
+ <version>1.3</version>
<executions>
<execution>
<configuration>
<programs>
<program>
- <mainClass>edu.uci.ics.hyracks.cli.Main</mainClass>
- <name>hyrackscli</name>
+ <mainClass>edu.uci.ics.hyracks.control.cc.CCDriver</mainClass>
+ <name>hyrackscc</name>
+ </program>
+ <program>
+ <mainClass>edu.uci.ics.hyracks.control.nc.NCDriver</mainClass>
+ <name>hyracksnc</name>
</program>
</programs>
<repositoryLayout>flat</repositoryLayout>
@@ -80,17 +60,29 @@
</plugins>
</build>
<dependencies>
+ <dependency>
+ <groupId>edu.uci.ics.hyracks</groupId>
+ <artifactId>hyracks-dataflow-std</artifactId>
+ <version>0.2.3-SNAPSHOT</version>
+ <scope>compile</scope>
+ </dependency>
+ <dependency>
+ <groupId>edu.uci.ics.hyracks</groupId>
+ <artifactId>hyracks-data-std</artifactId>
+ <version>0.2.3-SNAPSHOT</version>
+ </dependency>
<dependency>
- <groupId>jline</groupId>
- <artifactId>jline</artifactId>
- <version>0.9.94</version>
+ <groupId>edu.uci.ics.hyracks</groupId>
+ <artifactId>hyracks-control-cc</artifactId>
+ <version>0.2.3-SNAPSHOT</version>
<type>jar</type>
<scope>compile</scope>
</dependency>
<dependency>
<groupId>edu.uci.ics.hyracks</groupId>
- <artifactId>hyracks-api</artifactId>
+ <artifactId>hyracks-control-nc</artifactId>
<version>0.2.3-SNAPSHOT</version>
+ <type>jar</type>
<scope>compile</scope>
</dependency>
</dependencies>
diff --git a/hyracks/hyracks-cli/src/main/assembly/binary-assembly.xml b/hyracks/hyracks-examples/tpch-example/tpchserver/src/main/assembly/binary-assembly.xml
similarity index 81%
copy from hyracks/hyracks-cli/src/main/assembly/binary-assembly.xml
copy to hyracks/hyracks-examples/tpch-example/tpchserver/src/main/assembly/binary-assembly.xml
index 0500499..cd598d9 100644
--- a/hyracks/hyracks-cli/src/main/assembly/binary-assembly.xml
+++ b/hyracks/hyracks-examples/tpch-example/tpchserver/src/main/assembly/binary-assembly.xml
@@ -15,5 +15,9 @@
<directory>target/appassembler/lib</directory>
<outputDirectory>lib</outputDirectory>
</fileSet>
+ <fileSet>
+ <directory>docs</directory>
+ <outputDirectory>docs</outputDirectory>
+ </fileSet>
</fileSets>
</assembly>
diff --git a/hyracks/hyracks-hadoop-compat/pom.xml b/hyracks/hyracks-hadoop-compat/pom.xml
index 849efe3..3426293 100644
--- a/hyracks/hyracks-hadoop-compat/pom.xml
+++ b/hyracks/hyracks-hadoop-compat/pom.xml
@@ -25,6 +25,7 @@
<plugin>
<groupId>org.codehaus.mojo</groupId>
<artifactId>appassembler-maven-plugin</artifactId>
+ <version>1.3</version>
<executions>
<execution>
<configuration>
diff --git a/hyracks/hyracks-hadoop-compat/src/main/java/edu/uci/ics/hyracks/hadoop/compat/client/HyracksClient.java b/hyracks/hyracks-hadoop-compat/src/main/java/edu/uci/ics/hyracks/hadoop/compat/client/HyracksClient.java
index dfd229a..89cc1e5 100644
--- a/hyracks/hyracks-hadoop-compat/src/main/java/edu/uci/ics/hyracks/hadoop/compat/client/HyracksClient.java
+++ b/hyracks/hyracks-hadoop-compat/src/main/java/edu/uci/ics/hyracks/hadoop/compat/client/HyracksClient.java
@@ -40,35 +40,19 @@
initialize(properties);
}
- private Set<String> getRequiredLibs(Set<String> userLibs) {
- Set<String> requiredLibs = new HashSet<String>();
- for (String systemLib : systemLibs) {
- requiredLibs.add(systemLib);
- }
- for (String userLib : userLibs) {
- requiredLibs.add(userLib);
- }
- return requiredLibs;
- }
-
public JobStatus getJobStatus(JobId jobId) throws Exception {
return connection.getJobStatus(jobId);
}
- private void createApplication(String applicationName, Set<String> userLibs) throws Exception {
- connection.createApplication(applicationName,
- Utilities.getHyracksArchive(applicationName, getRequiredLibs(userLibs)));
- }
-
public HyracksRunningJob submitJob(String applicationName, JobSpecification spec) throws Exception {
String jobProfilingVal = System.getenv(jobProfilingKey);
boolean doProfiling = ("true".equalsIgnoreCase(jobProfilingVal));
JobId jobId;
if (doProfiling) {
System.out.println("PROFILING");
- jobId = connection.startJob(applicationName, spec, EnumSet.of(JobFlag.PROFILE_RUNTIME));
+ jobId = connection.startJob(spec, EnumSet.of(JobFlag.PROFILE_RUNTIME));
} else {
- jobId = connection.startJob(applicationName, spec);
+ jobId = connection.startJob(spec);
}
HyracksRunningJob runningJob = new HyracksRunningJob(jobId, spec, this);
return runningJob;
@@ -76,7 +60,6 @@
public HyracksRunningJob submitJob(String applicationName, JobSpecification spec, Set<String> userLibs)
throws Exception {
- createApplication(applicationName, userLibs);
return submitJob(applicationName, spec);
}
diff --git a/hyracks/hyracks-maven-plugins/hyracks-virtualcluster-maven-plugin/src/main/java/edu/uci/ics/hyracks/maven/plugin/AbstractHyracksCLIMojo.java b/hyracks/hyracks-maven-plugins/hyracks-virtualcluster-maven-plugin/src/main/java/edu/uci/ics/hyracks/maven/plugin/AbstractHyracksCLIMojo.java
deleted file mode 100644
index 7f3faef..0000000
--- a/hyracks/hyracks-maven-plugins/hyracks-virtualcluster-maven-plugin/src/main/java/edu/uci/ics/hyracks/maven/plugin/AbstractHyracksCLIMojo.java
+++ /dev/null
@@ -1,65 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.maven.plugin;
-
-import java.io.File;
-import java.io.PrintWriter;
-
-import org.apache.maven.plugin.MojoExecutionException;
-import org.apache.maven.plugin.MojoFailureException;
-
-public abstract class AbstractHyracksCLIMojo extends AbstractHyracksMojo {
- private static final String HYRACKS_CLI_SCRIPT = "bin" + File.separator + "hyrackscli";
-
- /**
- * @parameter
- * @required
- */
- protected File hyracksCLIHome;
-
- /**
- * @parameter
- * @required
- */
- private String ccHost;
-
- /**
- * @parameter
- */
- private int ccPort;
-
- @Override
- public void execute() throws MojoExecutionException, MojoFailureException {
- StringBuilder buffer = new StringBuilder();
- buffer.append(createConnectCommand());
- buffer.append('\n');
- buffer.append(getCommands());
- final Process proc = launch(new File(hyracksCLIHome, makeScriptName(HYRACKS_CLI_SCRIPT)), null, null);
- try {
- PrintWriter out = new PrintWriter(proc.getOutputStream());
- out.println(buffer.toString());
- out.close();
- proc.waitFor();
- } catch (Exception e) {
- throw new MojoExecutionException(e.getMessage());
- }
- }
-
- private String createConnectCommand() {
- return "connect to \"" + ccHost + (ccPort == 0 ? "" : (":" + ccPort)) + "\";";
- }
-
- protected abstract String getCommands();
-}
\ No newline at end of file
diff --git a/hyracks/hyracks-maven-plugins/hyracks-virtualcluster-maven-plugin/src/main/java/edu/uci/ics/hyracks/maven/plugin/HyracksAppDeploymentMojo.java b/hyracks/hyracks-maven-plugins/hyracks-virtualcluster-maven-plugin/src/main/java/edu/uci/ics/hyracks/maven/plugin/HyracksAppDeploymentMojo.java
deleted file mode 100644
index 76bbb5a..0000000
--- a/hyracks/hyracks-maven-plugins/hyracks-virtualcluster-maven-plugin/src/main/java/edu/uci/ics/hyracks/maven/plugin/HyracksAppDeploymentMojo.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.maven.plugin;
-
-import java.io.File;
-
-/**
- * @goal deploy-app
- */
-public class HyracksAppDeploymentMojo extends AbstractHyracksCLIMojo {
- /**
- * @parameter
- * @required
- */
- private String appName;
-
- /**
- * @parameter
- * @required
- */
- private File harFile;
-
- @Override
- protected String getCommands() {
- return "create application " + appName + " \"" + harFile.getAbsolutePath() + "\";";
- }
-}
\ No newline at end of file
diff --git a/hyracks/hyracks-server/pom.xml b/hyracks/hyracks-server/pom.xml
index 2699c27..6c6640e 100644
--- a/hyracks/hyracks-server/pom.xml
+++ b/hyracks/hyracks-server/pom.xml
@@ -22,6 +22,7 @@
<plugin>
<groupId>org.codehaus.mojo</groupId>
<artifactId>appassembler-maven-plugin</artifactId>
+ <version>1.3</version>
<executions>
<execution>
<configuration>
diff --git a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestNCApplicationContext.java b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestNCApplicationContext.java
index 0bd872c..0c64d7e 100644
--- a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestNCApplicationContext.java
+++ b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestNCApplicationContext.java
@@ -38,11 +38,6 @@
}
@Override
- public ClassLoader getClassLoader() {
- return getClass().getClassLoader();
- }
-
- @Override
public Serializable getDistributedState() {
return distributedState;
}
@@ -63,15 +58,9 @@
}
@Override
- public String getApplicationName() {
- // TODO Auto-generated method stub
- return null;
- }
-
- @Override
public void setMessageBroker(IMessageBroker staticticsConnector) {
// TODO Auto-generated method stub
-
+
}
@Override
diff --git a/hyracks/pom.xml b/hyracks/pom.xml
index 570421e..bcf06d1 100644
--- a/hyracks/pom.xml
+++ b/hyracks/pom.xml
@@ -44,6 +44,7 @@
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-surefire-plugin</artifactId>
+ <version>2.12</version>
<configuration>
<forkMode>pertest</forkMode>
<argLine>-enableassertions -Djava.util.logging.config.file=${user.home}/logging.properties -Xdebug -Xrunjdwp:transport=dt_socket,server=y,address=8000,suspend=n ${jvm.extraargs}</argLine>
@@ -52,15 +53,6 @@
</plugins>
</build>
- <reporting>
- <plugins>
- <plugin>
- <groupId>org.apache.maven.plugins</groupId>
- <artifactId>maven-changelog-plugin</artifactId>
- </plugin>
- </plugins>
- </reporting>
-
<distributionManagement>
<repository>
<id>hyracks-releases</id>
@@ -102,7 +94,6 @@
<module>hyracks-control</module>
<module>hyracks-net</module>
<module>hyracks-data</module>
- <module>hyracks-cli</module>
<module>hyracks-storage-common</module>
<module>hyracks-storage-am-common</module>
<module>hyracks-storage-am-btree</module>