You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@asterixdb.apache.org by mh...@apache.org on 2016/12/02 04:20:03 UTC

[2/2] asterixdb git commit: Enable Replication Tests on Multi-NC Setup

Enable Replication Tests on Multi-NC Setup

- Move replication IT from vagrant VMs to multi-NC setup.
- Add 'node' command to kill specific node process.
- Fix file path passing to replication manager.

Change-Id: I542b212e04469e2701690f464f821a5189b97f12
Reviewed-on: https://asterix-gerrit.ics.uci.edu/1364
Sonar-Qube: Jenkins <je...@fulliautomatix.ics.uci.edu>
Reviewed-by: abdullah alamoudi <ba...@gmail.com>
Tested-by: Jenkins <je...@fulliautomatix.ics.uci.edu>
Integration-Tests: Jenkins <je...@fulliautomatix.ics.uci.edu>


Project: http://git-wip-us.apache.org/repos/asf/asterixdb/repo
Commit: http://git-wip-us.apache.org/repos/asf/asterixdb/commit/2772f12a
Tree: http://git-wip-us.apache.org/repos/asf/asterixdb/tree/2772f12a
Diff: http://git-wip-us.apache.org/repos/asf/asterixdb/diff/2772f12a

Branch: refs/heads/master
Commit: 2772f12ac932b3d9401a1bb0d7292fd9714c6a53
Parents: 4c958b2
Author: Murtadha Hubail <mh...@uci.edu>
Authored: Thu Dec 1 22:40:53 2016 +0300
Committer: Murtadha Hubail <hu...@gmail.com>
Committed: Thu Dec 1 20:19:04 2016 -0800

----------------------------------------------------------------------
 .../common/AsterixHyracksIntegrationUtil.java   |  12 +-
 .../api/http/servlet/ClusterAPIServlet.java     |  11 +-
 .../apache/asterix/test/aql/TestExecutor.java   |  84 ++++---
 .../installer/command/ConfigureCommand.java     |  15 +-
 .../clusters/local/local_with_replication.xml   |  76 +++++++
 .../test/AsterixExternalLibraryIT.java          |   2 +-
 .../test/AsterixInstallerIntegrationUtil.java   |   9 +-
 .../installer/test/AsterixLifecycleIT.java      |   2 +-
 .../installer/test/AsterixRestartIT.java        |  11 +-
 .../asterix/installer/test/ReplicationIT.java   | 219 ++++---------------
 .../integrationts/replication/data/fbu.adm      |  10 +
 .../node_failback/node_failback.10.get.http     |   2 +-
 .../node_failback/node_failback.2.update.aql    |   2 +-
 .../node_failback/node_failback.3.node.aql      |  28 +++
 .../node_failback/node_failback.3.vscript.aql   |   1 -
 .../node_failback/node_failback.5.get.http      |   2 +-
 .../node_failback/node_failback.8.mgx.aql       |   1 +
 .../node_failback/node_failback.8.vmgx.aql      |   1 -
 .../node_failback/node_failback.9.sleep.aql     |   2 +-
 .../failover/bulkload/bulkload.2.update.aql     |   3 +-
 .../failover/bulkload/bulkload.3.node.aql       |  28 +++
 .../failover/bulkload/bulkload.3.txnqbc.aql     |  31 ---
 .../failover/bulkload/bulkload.4.sleep.aql      |   1 +
 .../failover/bulkload/bulkload.4.vscript.aql    |   1 -
 .../failover/bulkload/bulkload.5.query.aql      |  31 +++
 .../failover/bulkload/bulkload.5.sleep.aql      |   1 -
 .../failover/bulkload/bulkload.6.txnqar.aql     |  31 ---
 .../mem_component_recovery.2.update.aql         |   2 +-
 .../mem_component_recovery.3.node.aql           |  28 +++
 .../mem_component_recovery.3.txnqbc.aql         |  32 ---
 .../mem_component_recovery.4.sleep.aql          |   1 +
 .../mem_component_recovery.4.vscript.aql        |   1 -
 .../mem_component_recovery.5.query.aql          |  32 +++
 .../mem_component_recovery.5.sleep.aql          |   1 -
 .../mem_component_recovery.6.txnqar.aql         |  32 ---
 .../metadata_node/metadata_node.2.node.aql      |  28 +++
 .../metadata_node/metadata_node.2.txnqbc.aql    |  30 ---
 .../metadata_node/metadata_node.3.sleep.aql     |   1 +
 .../metadata_node/metadata_node.3.vscript.aql   |   1 -
 .../metadata_node/metadata_node.4.query.aql     |  36 +++
 .../metadata_node/metadata_node.4.sleep.aql     |   1 -
 .../metadata_node/metadata_node.5.txnqar.aql    |  32 ---
 .../node_failback.cluster_state.10.adm          | 137 +-----------
 .../node_failback.cluster_state.5.adm           | 137 +-----------
 .../results/failover/bulkload/bulkload.5.aql    |   1 +
 .../mem_component_recovery.5.aql                |   1 +
 .../failover/metadata_node/metadata_node.4.aql  |   1 +
 .../runtime/util/ClusterStateManager.java       |   8 +
 .../PersistentLocalResourceRepository.java      |  38 ++--
 asterixdb/pom.xml                               |   2 -
 50 files changed, 482 insertions(+), 718 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2772f12a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/AsterixHyracksIntegrationUtil.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/AsterixHyracksIntegrationUtil.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/AsterixHyracksIntegrationUtil.java
index a8095af..b0b2e36 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/AsterixHyracksIntegrationUtil.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/AsterixHyracksIntegrationUtil.java
@@ -78,9 +78,10 @@ public class AsterixHyracksIntegrationUtil {
         // Starts ncs.
         List<String> nodes = propertiesAccessor.getNodeNames();
         List<NodeControllerService> nodeControllers = new ArrayList<>();
+        List<Thread> startupThreads = new ArrayList<>();
         for (String ncName : nodes) {
-            NodeControllerService nodeControllerService =
-                    new NodeControllerService(fixupIODevices(createNCConfig(ncName)));
+            NodeControllerService nodeControllerService = new NodeControllerService(
+                    fixupIODevices(createNCConfig(ncName)));
             nodeControllers.add(nodeControllerService);
             Thread ncStartThread = new Thread("IntegrationUtil-" + ncName) {
                 @Override
@@ -93,7 +94,11 @@ public class AsterixHyracksIntegrationUtil {
                 }
             };
             ncStartThread.start();
-            ncStartThread.join();
+            startupThreads.add(ncStartThread);
+        }
+        //wait until all NCs complete their startup
+        for (Thread thread : startupThreads) {
+            thread.join();
         }
         hcc = new HyracksConnection(cc.getConfig().clientNetIpAddress, cc.getConfig().clientNetPort);
         ncs = nodeControllers.toArray(new NodeControllerService[nodeControllers.size()]);
@@ -157,7 +162,6 @@ public class AsterixHyracksIntegrationUtil {
         return ncConfig;
     }
 
-
     public String[] getNcNames() {
         return propertiesAccessor.getNodeNames().toArray(new String[propertiesAccessor.getNodeNames().size()]);
     }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2772f12a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/ClusterAPIServlet.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/ClusterAPIServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/ClusterAPIServlet.java
index 49730a0..158317b 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/ClusterAPIServlet.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/ClusterAPIServlet.java
@@ -77,9 +77,11 @@ public class ClusterAPIServlet extends HttpServlet {
                 case "/replication":
                     json = getReplicationJSON();
                     break;
+                case "/summary":
+                    json = getClusterStateSummaryJSON();
+                    break;
                 default:
                     throw new IllegalArgumentException();
-
             }
             response.setStatus(HttpServletResponse.SC_OK);
             responseWriter.write(json.toString(4));
@@ -92,6 +94,10 @@ public class ClusterAPIServlet extends HttpServlet {
         responseWriter.flush();
     }
 
+    protected JSONObject getClusterStateSummaryJSON() throws JSONException {
+        return ClusterStateManager.INSTANCE.getClusterStateSummary();
+    }
+
     protected JSONObject getReplicationJSON() throws JSONException {
         for (AbstractAsterixProperties props : getPropertiesInstances()) {
             if (props instanceof AsterixReplicationProperties) {
@@ -117,8 +123,7 @@ public class ClusterAPIServlet extends HttpServlet {
         return AbstractAsterixProperties.getImplementations();
     }
 
-    protected JSONObject getClusterStateJSON(HttpServletRequest request, String pathToNode)
-            throws JSONException {
+    protected JSONObject getClusterStateJSON(HttpServletRequest request, String pathToNode) throws JSONException {
         JSONObject json;
         json = ClusterStateManager.INSTANCE.getClusterStateDescription();
         Map<String, Object> allProperties = getAllClusterProperties();

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2772f12a/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/aql/TestExecutor.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/aql/TestExecutor.java b/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/aql/TestExecutor.java
index 4080325..2da57e3 100644
--- a/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/aql/TestExecutor.java
+++ b/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/aql/TestExecutor.java
@@ -80,11 +80,11 @@ public class TestExecutor {
     // see
     // https://stackoverflow.com/questions/417142/what-is-the-maximum-length-of-a-url-in-different-browsers/417184
     private static final long MAX_URL_LENGTH = 2000l;
-    private static final Pattern JAVA_BLOCK_COMMENT_PATTERN =
-            Pattern.compile("/\\*.*\\*/", Pattern.MULTILINE | Pattern.DOTALL);
+    private static final Pattern JAVA_BLOCK_COMMENT_PATTERN = Pattern.compile("/\\*.*\\*/",
+            Pattern.MULTILINE | Pattern.DOTALL);
     private static final Pattern REGEX_LINES_PATTERN = Pattern.compile("^(-)?/(.*)/([im]*)$");
-    private static final Pattern POLL_TIMEOUT_PATTERN =
-            Pattern.compile("polltimeoutsecs=(\\d+)(\\D|$)", Pattern.MULTILINE);
+    private static final Pattern POLL_TIMEOUT_PATTERN = Pattern.compile("polltimeoutsecs=(\\d+)(\\D|$)",
+            Pattern.MULTILINE);
     private static final Pattern POLL_DELAY_PATTERN = Pattern.compile("polldelaysecs=(\\d+)(\\D|$)", Pattern.MULTILINE);
     public static final int TRUNCATE_THRESHOLD = 16384;
 
@@ -199,9 +199,8 @@ public class TestExecutor {
 
     private void throwLineChanged(File scriptFile, String lineExpected, String lineActual, int num)
             throws ComparisonException {
-        throw new ComparisonException(
-                "Result for " + scriptFile + " changed at line " + num + ":\n< "
-                        + truncateIfLong(lineExpected) + "\n> " + truncateIfLong(lineActual));
+        throw new ComparisonException("Result for " + scriptFile + " changed at line " + num + ":\n< "
+                + truncateIfLong(lineExpected) + "\n> " + truncateIfLong(lineActual));
     }
 
     private String truncateIfLong(String string) {
@@ -211,8 +210,7 @@ public class TestExecutor {
         final StringBuilder truncatedString = new StringBuilder(string);
         truncatedString.setLength(TRUNCATE_THRESHOLD);
         truncatedString.append("\n<truncated ")
-                .append(StorageUtil.toHumanReadableSize(string.length() - TRUNCATE_THRESHOLD))
-                .append("...>");
+                .append(StorageUtil.toHumanReadableSize(string.length() - TRUNCATE_THRESHOLD)).append("...>");
         return truncatedString.toString();
     }
 
@@ -327,8 +325,8 @@ public class TestExecutor {
                 }
                 Matcher m = REGEX_LINES_PATTERN.matcher(lineExpected);
                 if (!m.matches()) {
-                    throw new IllegalArgumentException("Each line of regex file must conform to: [-]/regex/[flags]: "
-                            + expectedFile);
+                    throw new IllegalArgumentException(
+                            "Each line of regex file must conform to: [-]/regex/[flags]: " + expectedFile);
                 }
                 String negateStr = m.group(1);
                 String expression = m.group(2);
@@ -346,8 +344,8 @@ public class TestExecutor {
                 if (match && !negate || negate && !match) {
                     continue;
                 }
-                throw new Exception("Result for " + scriptFile + ": expected pattern '" + expression +
-                        "' not found in result.");
+                throw new Exception(
+                        "Result for " + scriptFile + ": expected pattern '" + expression + "' not found in result.");
             }
         } catch (Exception e) {
             System.err.println("Actual results file: " + actualFile.toString());
@@ -404,14 +402,12 @@ public class TestExecutor {
                 String[] errors = { result.getJSONArray("error-code").getString(0), result.getString("summary"),
                         result.getString("stacktrace") };
                 GlobalConfig.ASTERIX_LOGGER.log(Level.SEVERE, errors[2]);
-                exceptionMsg = "HTTP operation failed: " + errors[0]
-                        + "\nSTATUS LINE: " + httpResponse.getStatusLine()
+                exceptionMsg = "HTTP operation failed: " + errors[0] + "\nSTATUS LINE: " + httpResponse.getStatusLine()
                         + "\nSUMMARY: " + errors[1] + "\nSTACKTRACE: " + errors[2];
             } catch (Exception e) {
                 // whoops, not JSON (e.g. 404) - just include the body
                 GlobalConfig.ASTERIX_LOGGER.log(Level.SEVERE, errorBody);
-                exceptionMsg = "HTTP operation failed:"
-                        + "\nSTATUS LINE: " + httpResponse.getStatusLine()
+                exceptionMsg = "HTTP operation failed:" + "\nSTATUS LINE: " + httpResponse.getStatusLine()
                         + "\nERROR_BODY: " + errorBody;
             }
             throw new Exception(exceptionMsg);
@@ -457,7 +453,7 @@ public class TestExecutor {
     }
 
     private List<CompilationUnit.Parameter> injectStatement(String statement, String stmtParamName,
-                                                            List<CompilationUnit.Parameter> otherParams) {
+            List<CompilationUnit.Parameter> otherParams) {
         CompilationUnit.Parameter stmtParam = new CompilationUnit.Parameter();
         stmtParam.setName(stmtParamName);
         stmtParam.setValue(statement);
@@ -488,7 +484,7 @@ public class TestExecutor {
     }
 
     private HttpUriRequest constructGetMethod(String endpoint, OutputFormat fmt,
-                                              List<CompilationUnit.Parameter> params) {
+            List<CompilationUnit.Parameter> params) {
 
         HttpUriRequest method = constructGetMethod(endpoint, params);
         // Set accepted output response type
@@ -506,7 +502,7 @@ public class TestExecutor {
     }
 
     private HttpUriRequest constructPostMethod(String endpoint, OutputFormat fmt,
-                                              List<CompilationUnit.Parameter> params) {
+            List<CompilationUnit.Parameter> params) {
 
         HttpUriRequest method = constructPostMethod(endpoint, params);
         // Set accepted output response type
@@ -565,8 +561,7 @@ public class TestExecutor {
     // Insert and Delete statements are executed here
     public void executeUpdate(String str, String url) throws Exception {
         // Create a method instance.
-        HttpUriRequest request = RequestBuilder.post(url)
-                .setEntity(new StringEntity(str, StandardCharsets.UTF_8))
+        HttpUriRequest request = RequestBuilder.post(url).setEntity(new StringEntity(str, StandardCharsets.UTF_8))
                 .build();
 
         // Execute the method.
@@ -593,10 +588,8 @@ public class TestExecutor {
         final String url = getEndpoint(Servlets.QUERY_RESULT);
 
         // Create a method instance.
-        HttpUriRequest request = RequestBuilder.get(url)
-                .addParameter("handle", handle)
-                .setHeader("Accept", fmt.mimeType())
-                .build();
+        HttpUriRequest request = RequestBuilder.get(url).addParameter("handle", handle)
+                .setHeader("Accept", fmt.mimeType()).build();
 
         HttpResponse response = executeAndCheckHttpRequest(request);
         return response.getEntity().getContent();
@@ -610,8 +603,7 @@ public class TestExecutor {
     // create function statement
     public void executeDDL(String str, String url) throws Exception {
         // Create a method instance.
-        HttpUriRequest request = RequestBuilder.post(url)
-                .setEntity(new StringEntity(str, StandardCharsets.UTF_8))
+        HttpUriRequest request = RequestBuilder.post(url).setEntity(new StringEntity(str, StandardCharsets.UTF_8))
                 .build();
 
         // Execute the method.
@@ -622,8 +614,8 @@ public class TestExecutor {
     // and returns the contents as a string
     // This string is later passed to REST API for execution.
     public String readTestFile(File testFile) throws Exception {
-        BufferedReader reader =
-                new BufferedReader(new InputStreamReader(new FileInputStream(testFile), StandardCharsets.UTF_8));
+        BufferedReader reader = new BufferedReader(
+                new InputStreamReader(new FileInputStream(testFile), StandardCharsets.UTF_8));
         String line;
         StringBuilder stringBuilder = new StringBuilder();
         String ls = System.getProperty("line.separator");
@@ -746,7 +738,7 @@ public class TestExecutor {
                 long limitTime = startTime + TimeUnit.SECONDS.toMillis(timeoutSecs);
                 ctx.setType(ctx.getType().substring("poll".length()));
                 Exception finalException;
-                LOGGER.fine("polling for up to " + timeoutSecs + " seconds w/ " + retryDelaySecs  + " second(s) delay");
+                LOGGER.fine("polling for up to " + timeoutSecs + " seconds w/ " + retryDelaySecs + " second(s) delay");
                 while (true) {
                     try {
                         executeTest(testCaseCtx, ctx, statement, isDmlRecoveryTest, pb, cUnit, queryCount,
@@ -901,8 +893,8 @@ public class TestExecutor {
             case "get":
             case "post":
                 if (!"http".equals(ctx.extension())) {
-                    throw new IllegalArgumentException("Unexpected format for method " + ctx.getType() + ": "
-                            + ctx.extension());
+                    throw new IllegalArgumentException(
+                            "Unexpected format for method " + ctx.getType() + ": " + ctx.extension());
                 }
                 fmt = OutputFormat.forCompilationUnit(cUnit);
                 String endpoint = stripJavaComments(statement).trim();
@@ -924,7 +916,7 @@ public class TestExecutor {
                 queryCount.increment();
                 break;
             case "server": // (start <test server name> <port>
-                           // [<arg1>][<arg2>][<arg3>]...|stop (<port>|all))
+                               // [<arg1>][<arg2>][<arg3>]...|stop (<port>|all))
                 try {
                     lines = statement.trim().split("\n");
                     String[] command = lines[lines.length - 1].trim().split(" ");
@@ -1001,11 +993,35 @@ public class TestExecutor {
                         throw new Exception("invalid library format");
                 }
                 break;
+            case "node":
+                command = stripJavaComments(statement).trim().split(" ");
+                String commandType = command[0];
+                String nodeId = command[1];
+                if (commandType.equals("kill")) {
+                    killNC(nodeId, cUnit);
+                }
+                break;
             default:
                 throw new IllegalArgumentException("No statements of type " + ctx.getType());
         }
     }
 
+    private void killNC(String nodeId, CompilationUnit cUnit) throws Exception {
+        //get node process id
+        OutputFormat fmt = OutputFormat.forCompilationUnit(cUnit);
+        String endpoint = "/admin/cluster/node/" + nodeId + "/config";
+        InputStream executeJSONGet = executeJSONGet(fmt, "http://" + host + ":" + port + endpoint);
+        StringWriter actual = new StringWriter();
+        IOUtils.copy(executeJSONGet, actual, StandardCharsets.UTF_8);
+        String config = actual.toString();
+        String nodePid = StringUtils.substringBetween(config, "\"pid\": ", ",").trim();
+        if (nodePid == null) {
+            throw new IllegalArgumentException("Coud not find process for node id: " + nodeId);
+        }
+        ProcessBuilder pb = new ProcessBuilder("kill", "-9", nodePid);
+        pb.start().waitFor();
+    }
+
     public void executeTest(String actualPath, TestCaseContext testCaseCtx, ProcessBuilder pb,
             boolean isDmlRecoveryTest, TestGroup failedGroup) throws Exception {
         File testFile;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2772f12a/asterixdb/asterix-installer/src/main/java/org/apache/asterix/installer/command/ConfigureCommand.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/main/java/org/apache/asterix/installer/command/ConfigureCommand.java b/asterixdb/asterix-installer/src/main/java/org/apache/asterix/installer/command/ConfigureCommand.java
index aeb118f..6d2b4b9 100644
--- a/asterixdb/asterix-installer/src/main/java/org/apache/asterix/installer/command/ConfigureCommand.java
+++ b/asterixdb/asterix-installer/src/main/java/org/apache/asterix/installer/command/ConfigureCommand.java
@@ -21,6 +21,8 @@ package org.apache.asterix.installer.command;
 import java.io.File;
 import java.io.FileNotFoundException;
 import java.io.FileOutputStream;
+import java.nio.file.Files;
+import java.nio.file.Paths;
 
 import javax.xml.bind.JAXBContext;
 import javax.xml.bind.JAXBException;
@@ -42,6 +44,7 @@ public class ConfigureCommand extends AbstractCommand {
     @Override
     protected void execCommand() throws Exception {
         configureCluster("local", "local.xml");
+        configureCluster("local", "local_with_replication.xml");
         configureCluster("demo", "demo.xml");
 
         String installerConfPath = InstallerDriver.getManagixHome() + File.separator + InstallerDriver.MANAGIX_CONF_XML;
@@ -51,9 +54,8 @@ public class ConfigureCommand extends AbstractCommand {
 
         configuration.setConfigured(true);
         configuration.getBackup().setBackupDir(InstallerDriver.getManagixHome() + File.separator + "backup");
-        configuration.getZookeeper().setHomeDir(
-                InstallerDriver.getManagixHome() + File.separator + InstallerDriver.MANAGIX_INTERNAL_DIR
-                        + File.separator + "zookeeper_home");
+        configuration.getZookeeper().setHomeDir(InstallerDriver.getManagixHome() + File.separator
+                + InstallerDriver.MANAGIX_INTERNAL_DIR + File.separator + "zookeeper_home");
         configuration.getZookeeper().getServers().setJavaHome(System.getProperty("java.home"));
 
         Marshaller marshaller = ctx.createMarshaller();
@@ -61,11 +63,14 @@ public class ConfigureCommand extends AbstractCommand {
         marshaller.marshal(configuration, new FileOutputStream(installerConfPath));
     }
 
-    private void configureCluster(String dir, String file) throws JAXBException, PropertyException,
-            FileNotFoundException {
+    private void configureCluster(String dir, String file)
+            throws JAXBException, PropertyException, FileNotFoundException {
         String clusterDir = InstallerDriver.getManagixHome() + File.separator + "clusters" + File.separator + dir;
         String localClusterPath = clusterDir + File.separator + file;
 
+        if (!Files.exists(Paths.get(localClusterPath))) {
+            return;
+        }
         Cluster cluster = EventUtil.getCluster(localClusterPath);
         String workingDir = clusterDir + File.separator + "working_dir";
         cluster.setWorkingDir(new WorkingDir(workingDir, true));

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2772f12a/asterixdb/asterix-installer/src/main/resources/clusters/local/local_with_replication.xml
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/main/resources/clusters/local/local_with_replication.xml b/asterixdb/asterix-installer/src/main/resources/clusters/local/local_with_replication.xml
new file mode 100644
index 0000000..e6a3547
--- /dev/null
+++ b/asterixdb/asterix-installer/src/main/resources/clusters/local/local_with_replication.xml
@@ -0,0 +1,76 @@
+<?xml version="1.0" encoding="UTF-8" standalone="yes"?>
+<!--
+ ! Licensed to the Apache Software Foundation (ASF) under one
+ ! or more contributor license agreements.  See the NOTICE file
+ ! distributed with this work for additional information
+ ! regarding copyright ownership.  The ASF licenses this file
+ ! to you under the Apache License, Version 2.0 (the
+ ! "License"); you may not use this file except in compliance
+ ! with the License.  You may obtain a copy of the License at
+ !
+ !   http://www.apache.org/licenses/LICENSE-2.0
+ !
+ ! Unless required by applicable law or agreed to in writing,
+ ! software distributed under the License is distributed on an
+ ! "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ ! KIND, either express or implied.  See the License for the
+ ! specific language governing permissions and limitations
+ ! under the License.
+ !-->
+<cluster xmlns="cluster">
+  <java_home>/usr/lib/jvm/jdk-8-oracle-x64/jre</java_home>
+  <log_dir>/tmp/asterix/logs</log_dir>
+  <txn_log_dir>/tmp/asterix/txnLogs</txn_log_dir>
+  <store>storage</store>
+  <working_dir>
+    <dir>/tmp/asterix-installer</dir>
+    <NFS>true</NFS>
+  </working_dir>
+  <!-- Sets the time duration between two heartbeats from each node controller in milliseconds (default: 10000) -->
+  <heartbeat_period>1000</heartbeat_period>
+  <!-- Sets the maximum number of missed heartbeats before a node is marked as dead (default: 5) -->
+  <max_heartbeat_lapse_periods>5</max_heartbeat_lapse_periods>
+  <!-- Sets the time duration between two profile dumps from each node controller in milliseconds. 0 to disable. (default: 0) -->
+  <profile_dump_period>0</profile_dump_period>
+  <!-- Sets the default number of job attempts allowed if not specified in the job specification. (default: 5) -->
+  <default_max_job_attempts>5</default_max_job_attempts>
+  <!-- Limits the number of historical jobs remembered by the system to the specified value. (default: 10) -->
+  <job_history_size>10</job_history_size>
+  <!-- Limits the amount of time results for asynchronous jobs should be retained by the system in milliseconds. (default: 24 hours) -->
+  <result_time_to_live>86400000</result_time_to_live>
+  <!-- The duration within which an instance of the result cleanup should be invoked in milliseconds. (default: 1 minute) -->
+  <result_sweep_threshold>60000</result_sweep_threshold>
+
+  <metadata_node>nc1</metadata_node>
+
+  <data_replication>
+    <enabled>true</enabled>
+    <replication_port>2000</replication_port>
+    <replication_factor>2</replication_factor>
+    <auto_failover>true</auto_failover>
+    <replication_time_out>10</replication_time_out>
+  </data_replication>
+
+  <master_node>
+    <id>master</id>
+    <client_ip>127.0.0.1</client_ip>
+    <cluster_ip>127.0.0.1</cluster_ip>
+    <client_port>1098</client_port>
+    <cluster_port>1099</cluster_port>
+    <http_port>8888</http_port>
+  </master_node>
+  <node>
+    <id>nc1</id>
+    <cluster_ip>127.0.0.1</cluster_ip>
+    <txn_log_dir>/tmp/asterix/nc1/txnLogs</txn_log_dir>
+    <iodevices>/tmp/asterix/nc1/p1,/tmp/asterix/nc1/p2</iodevices>
+    <replication_port>2000</replication_port>
+  </node>
+  <node>
+    <id>nc2</id>
+    <cluster_ip>127.0.0.1</cluster_ip>
+    <txn_log_dir>/tmp/asterix/nc2/txnLogs</txn_log_dir>
+    <iodevices>/tmp/asterix/nc2/p1,/tmp/asterix/nc2/p2</iodevices>
+    <replication_port>2001</replication_port>
+  </node>
+</cluster>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2772f12a/asterixdb/asterix-installer/src/test/java/org/apache/asterix/installer/test/AsterixExternalLibraryIT.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/java/org/apache/asterix/installer/test/AsterixExternalLibraryIT.java b/asterixdb/asterix-installer/src/test/java/org/apache/asterix/installer/test/AsterixExternalLibraryIT.java
index ae98d19..5dde951 100644
--- a/asterixdb/asterix-installer/src/test/java/org/apache/asterix/installer/test/AsterixExternalLibraryIT.java
+++ b/asterixdb/asterix-installer/src/test/java/org/apache/asterix/installer/test/AsterixExternalLibraryIT.java
@@ -55,7 +55,7 @@ public class AsterixExternalLibraryIT {
     @BeforeClass
     public static void setUp() throws Exception {
         try {
-            AsterixInstallerIntegrationUtil.init();
+            AsterixInstallerIntegrationUtil.init(AsterixInstallerIntegrationUtil.LOCAL_CLUSTER_PATH);
             File asterixInstallerProjectDir = new File(System.getProperty("user.dir"));
             String asterixExternalLibraryPath = asterixInstallerProjectDir.getAbsolutePath() + File.separator
                     + LIBRARY_PATH;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2772f12a/asterixdb/asterix-installer/src/test/java/org/apache/asterix/installer/test/AsterixInstallerIntegrationUtil.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/java/org/apache/asterix/installer/test/AsterixInstallerIntegrationUtil.java b/asterixdb/asterix-installer/src/test/java/org/apache/asterix/installer/test/AsterixInstallerIntegrationUtil.java
index 34a8733..e840796 100644
--- a/asterixdb/asterix-installer/src/test/java/org/apache/asterix/installer/test/AsterixInstallerIntegrationUtil.java
+++ b/asterixdb/asterix-installer/src/test/java/org/apache/asterix/installer/test/AsterixInstallerIntegrationUtil.java
@@ -53,19 +53,22 @@ public class AsterixInstallerIntegrationUtil {
     private static final int zookeeperClientPort = 2900;
     private static final int zookeeperTestClientPort = 3945;
     private static IHyracksClientConnection hcc;
+    private static final String CLUSTERS_BASE_PATH = "clusters" + File.separator + "local" + File.separator;
+    public static final String LOCAL_CLUSTER_PATH = CLUSTERS_BASE_PATH + File.separator + "local.xml";
+    public static final String LOCAL_CLUSTER_WITH_REPLICATION_PATH = CLUSTERS_BASE_PATH + File.separator
+            + "local_with_replication.xml";
 
     public static void deinit() throws Exception {
         deleteInstance();
         stopZookeeper();
     }
 
-    public static void init() throws Exception {
+    public static void init(String clusterPath) throws Exception {
         managixHome = getManagixHome();
         System.setProperty("log4j.configuration",
                 managixHome + File.separator + "conf" + File.separator + "log4j.properties");
 
-        clusterConfigurationPath = managixHome + File.separator + "clusters" + File.separator + "local" + File.separator
-                + "local.xml";
+        clusterConfigurationPath = managixHome + File.separator + clusterPath;
 
         InstallerDriver.setManagixHome(managixHome);
 

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2772f12a/asterixdb/asterix-installer/src/test/java/org/apache/asterix/installer/test/AsterixLifecycleIT.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/java/org/apache/asterix/installer/test/AsterixLifecycleIT.java b/asterixdb/asterix-installer/src/test/java/org/apache/asterix/installer/test/AsterixLifecycleIT.java
index 04e73ed..4dfc5c5 100644
--- a/asterixdb/asterix-installer/src/test/java/org/apache/asterix/installer/test/AsterixLifecycleIT.java
+++ b/asterixdb/asterix-installer/src/test/java/org/apache/asterix/installer/test/AsterixLifecycleIT.java
@@ -53,7 +53,7 @@ public class AsterixLifecycleIT {
 
     @BeforeClass
     public static void setUp() throws Exception {
-        AsterixInstallerIntegrationUtil.init();
+        AsterixInstallerIntegrationUtil.init(AsterixInstallerIntegrationUtil.LOCAL_CLUSTER_PATH);
         TestCaseContext.Builder b = new TestCaseContext.Builder();
         testCaseCollection = b.build(new File(PATH_BASE));
         File outdir = new File(PATH_ACTUAL);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2772f12a/asterixdb/asterix-installer/src/test/java/org/apache/asterix/installer/test/AsterixRestartIT.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/java/org/apache/asterix/installer/test/AsterixRestartIT.java b/asterixdb/asterix-installer/src/test/java/org/apache/asterix/installer/test/AsterixRestartIT.java
index d567145..96b420b 100644
--- a/asterixdb/asterix-installer/src/test/java/org/apache/asterix/installer/test/AsterixRestartIT.java
+++ b/asterixdb/asterix-installer/src/test/java/org/apache/asterix/installer/test/AsterixRestartIT.java
@@ -21,17 +21,20 @@ package org.apache.asterix.installer.test;
 import java.io.File;
 import java.util.ArrayList;
 import java.util.Collection;
-import java.util.List;
 import java.util.Map;
 import java.util.logging.Logger;
 
-import org.apache.asterix.common.config.AsterixStorageProperties;
 import org.apache.asterix.event.model.AsterixInstance.State;
 import org.apache.asterix.test.aql.TestExecutor;
 import org.apache.asterix.test.base.RetainLogsRule;
 import org.apache.asterix.testframework.context.TestCaseContext;
 import org.apache.commons.lang3.StringUtils;
-import org.junit.*;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
 import org.junit.rules.TestRule;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
@@ -73,7 +76,7 @@ public class AsterixRestartIT {
                     + "resources" + File.separator + "integrationts" + File.separator + "restart"
                     + File.separator + "scripts";
             env.put("SCRIPT_HOME", scriptHomePath);
-            AsterixInstallerIntegrationUtil.init();
+            AsterixInstallerIntegrationUtil.init(AsterixInstallerIntegrationUtil.LOCAL_CLUSTER_PATH);
         } catch (Throwable th) {
             th.printStackTrace();
             throw th;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2772f12a/asterixdb/asterix-installer/src/test/java/org/apache/asterix/installer/test/ReplicationIT.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/java/org/apache/asterix/installer/test/ReplicationIT.java b/asterixdb/asterix-installer/src/test/java/org/apache/asterix/installer/test/ReplicationIT.java
index 37aa59d..5f47849 100644
--- a/asterixdb/asterix-installer/src/test/java/org/apache/asterix/installer/test/ReplicationIT.java
+++ b/asterixdb/asterix-installer/src/test/java/org/apache/asterix/installer/test/ReplicationIT.java
@@ -19,152 +19,78 @@
 package org.apache.asterix.installer.test;
 
 import java.io.File;
-import java.io.FilenameFilter;
-import java.io.IOException;
-import java.io.InputStream;
-import java.nio.charset.StandardCharsets;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Map;
 import java.util.logging.Logger;
 
+import org.apache.asterix.event.model.AsterixInstance.State;
 import org.apache.asterix.test.aql.TestExecutor;
+import org.apache.asterix.test.base.RetainLogsRule;
 import org.apache.asterix.testframework.context.TestCaseContext;
-import org.apache.commons.io.IOUtils;
 import org.apache.commons.lang3.StringUtils;
-import org.codehaus.plexus.util.FileUtils;
 import org.junit.After;
-import org.junit.AfterClass;
 import org.junit.Before;
 import org.junit.BeforeClass;
+import org.junit.Rule;
 import org.junit.Test;
+import org.junit.rules.TestRule;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
-import org.junit.runners.Parameterized.Parameters;
 
 @RunWith(Parameterized.class)
 public class ReplicationIT {
 
-    private static final String PATH_BASE = StringUtils
-            .join(new String[] { "src", "test", "resources", "integrationts", "replication" }, File.separator);
-    private static final String CLUSTER_BASE = StringUtils
-            .join(new String[] { "src", "test", "resources", "clusterts" }, File.separator);
-    private static final String PATH_ACTUAL = "target" + File.separator + "repliationtest" + File.separator;
-    private static String managixFolderName;
+    private static final String PATH_BASE = "src/test/resources/integrationts/replication/";
+    private static final String PATH_ACTUAL = "target" + File.separator + "ittest" + File.separator;
     private static final Logger LOGGER = Logger.getLogger(ReplicationIT.class.getName());
-    private static File asterixProjectDir = new File(System.getProperty("user.dir"));
-    private static final String CLUSTER_CC_ADDRESS = "10.10.0.2";
-    private static final int CLUSTER_CC_API_PORT = 19002;
+    private static String reportPath = new File(
+            StringUtils.join(new String[] { "target", "failsafe-reports" }, File.separator)).getAbsolutePath();
+
+    private final TestExecutor testExecutor = new TestExecutor();
+    private TestCaseContext tcCtx;
+    private static String scriptHomePath;
+    private static File asterixInstallerPath;
     private static ProcessBuilder pb;
     private static Map<String, String> env;
-    private final static TestExecutor testExecutor = new TestExecutor(CLUSTER_CC_ADDRESS, CLUSTER_CC_API_PORT);
-    private static String SCRIPT_HOME = "/vagrant/scripts/";
-    private static String MANAGIX_HOME = "/tmp/asterix/bin/managix ";
-    private static final String INSTANCE_NAME = "asterix";
-    protected TestCaseContext tcCtx;
 
     public ReplicationIT(TestCaseContext tcCtx) {
         this.tcCtx = tcCtx;
     }
 
+    @Rule
+    public TestRule retainLogs = new RetainLogsRule(AsterixInstallerIntegrationUtil.getManagixHome(), reportPath);
+
     @BeforeClass
     public static void setUp() throws Exception {
-        File outdir = new File(PATH_ACTUAL);
-        outdir.mkdirs();
-
-        // vagrant setup
-        File installerTargetDir = new File(asterixProjectDir, "target");
-        String[] installerFiles = installerTargetDir.list(new FilenameFilter() {
-            @Override
-            public boolean accept(File dir, String name) {
-                return new File(dir, name).isDirectory() && name.startsWith("asterix-installer")
-                        && name.endsWith("binary-assembly");
-            }
-        });
-
-        if (installerFiles == null || installerFiles.length == 0) {
-            throw new Exception("Couldn't find installer binaries");
-        }
-
-        managixFolderName = installerFiles[0];
-
-        //copy tests data
-        FileUtils.copyDirectoryStructure(
-                new File(StringUtils.join(
-                        new String[] { "..", "asterix-replication", "src", "test", "resources", "data" },
-                        File.separator)),
-                new File(StringUtils.join(new String[] { "src", "test", "resources", "clusterts", "data" },
-                        File.separator)));
-
-        //copy tests scripts
-        FileUtils.copyDirectoryStructure(
-                new File(StringUtils.join(
-                        new String[] { "..", "asterix-replication", "src", "test", "resources", "scripts" },
-                        File.separator)),
-                new File(StringUtils.join(new String[] { "src", "test", "resources", "clusterts", "scripts" },
-                        File.separator)));
-
-        invoke("cp", "-r", installerTargetDir.toString() + "/" + managixFolderName,
-                asterixProjectDir + "/" + CLUSTER_BASE);
-
-        remoteInvoke("cp -r /vagrant/" + managixFolderName + " /tmp/asterix");
-
-        pb = new ProcessBuilder();
-        env = pb.environment();
-        env.put("SCRIPT_HOME", SCRIPT_HOME);
-        env.put("MANAGIX_HOME", MANAGIX_HOME);
-        File cwd = new File(asterixProjectDir.toString() + "/" + CLUSTER_BASE);
-        pb.directory(cwd);
-        pb.redirectErrorStream(true);
-
-        //make scripts executable
-        String chmodScriptsCmd = "chmod -R +x " + SCRIPT_HOME;
-        remoteInvoke(chmodScriptsCmd, "cc");
-        remoteInvoke(chmodScriptsCmd, "nc1");
-        remoteInvoke(chmodScriptsCmd, "nc2");
-
-        //managix configure
-        logOutput(managixInvoke("configure").getInputStream());
-
-        //managix validate
-        String validateOutput = IOUtils.toString(managixInvoke("validate").getInputStream(),
-                StandardCharsets.UTF_8.name());
-        if (validateOutput.contains("ERROR")) {
-            throw new Exception("Managix validate error: " + validateOutput);
+        try {
+            pb = new ProcessBuilder();
+            env = pb.environment();
+            asterixInstallerPath = new File(System.getProperty("user.dir"));
+            scriptHomePath = asterixInstallerPath + File.separator + "src" + File.separator + "test" + File.separator
+                    + "resources" + File.separator + "integrationts" + File.separator + "replication" + File.separator
+                    + "scripts";
+            env.put("SCRIPT_HOME", scriptHomePath);
+        } catch (Throwable th) {
+            th.printStackTrace();
+            throw th;
         }
     }
 
-    @AfterClass
-    public static void tearDown() throws Exception {
-        //remove files
-        remoteInvoke("rm -rf /vagrant/asterix");
-    }
-
     @Before
-    public void beforeTest() throws Exception {
-        //create instance
-        managixInvoke("create -n " + INSTANCE_NAME + " -c /vagrant/cluster_with_replication.xml").getInputStream();
+    public void before() throws Exception {
+        LOGGER.info("Creating new instance...");
+        AsterixInstallerIntegrationUtil.init(AsterixInstallerIntegrationUtil.LOCAL_CLUSTER_WITH_REPLICATION_PATH);
+        LOGGER.info("Instacne created.");
+        AsterixInstallerIntegrationUtil.transformIntoRequiredState(State.ACTIVE);
+        LOGGER.info("Instance is in ACTIVE state.");
     }
 
     @After
-    public void afterTest() throws Exception {
-        //stop instance
-        managixInvoke("stop -n " + INSTANCE_NAME);
-
-        //verify that all processes have been stopped
-        String killProcesses = "kill_cc_and_nc.sh";
-        executeVagrantScript("cc", killProcesses);
-        executeVagrantScript("nc1", killProcesses);
-        executeVagrantScript("nc2", killProcesses);
-
-        //delete storage
-        String deleteStorage = "delete_storage.sh";
-        executeVagrantScript("cc", deleteStorage);
-        executeVagrantScript("nc1", deleteStorage);
-        executeVagrantScript("nc2", deleteStorage);
-
-        //delete instance
-        managixInvoke("delete -n " + INSTANCE_NAME);
+    public void after() throws Exception {
+        LOGGER.info("Destroying instance...");
+        AsterixInstallerIntegrationUtil.deinit();
+        LOGGER.info("Instance destroyed.");
     }
 
     @Test
@@ -172,7 +98,7 @@ public class ReplicationIT {
         testExecutor.executeTest(PATH_ACTUAL, tcCtx, pb, false);
     }
 
-    @Parameters(name = "ReplicationIT {index}: {0}")
+    @Parameterized.Parameters(name = "ReplicationIT {index}: {0}")
     public static Collection<Object[]> tests() throws Exception {
         Collection<Object[]> testArgs = buildTestsInXml(TestCaseContext.DEFAULT_TESTSUITE_XML_NAME);
         if (testArgs.size() == 0) {
@@ -182,78 +108,11 @@ public class ReplicationIT {
     }
 
     protected static Collection<Object[]> buildTestsInXml(String xmlfile) throws Exception {
-        Collection<Object[]> testArgs = new ArrayList<Object[]>();
+        Collection<Object[]> testArgs = new ArrayList<>();
         TestCaseContext.Builder b = new TestCaseContext.Builder();
         for (TestCaseContext ctx : b.build(new File(PATH_BASE), xmlfile)) {
             testArgs.add(new Object[] { ctx });
         }
         return testArgs;
     }
-
-    public static boolean checkOutput(InputStream input, String requiredSubString) {
-        String candidate;
-        try {
-            candidate = IOUtils.toString(input, StandardCharsets.UTF_8.name());
-        } catch (IOException e) {
-            LOGGER.warning("Could not check output of subprocess");
-            return false;
-        }
-        return candidate.contains(requiredSubString);
-    }
-
-    public static boolean checkOutput(String candidate, String requiredSubString) {
-        return candidate.contains(requiredSubString);
-    }
-
-    public static String processOut(Process p) throws IOException {
-        InputStream input = p.getInputStream();
-        return IOUtils.toString(input, StandardCharsets.UTF_8.name());
-    }
-
-    public static void logOutput(InputStream input) {
-        try {
-            LOGGER.info(IOUtils.toString(input, StandardCharsets.UTF_8.name()));
-        } catch (IOException e) {
-            LOGGER.warning("Could not print output of subprocess");
-        }
-    }
-
-    private static Process invoke(String... args) throws Exception {
-        ProcessBuilder pb = new ProcessBuilder(args);
-        pb.redirectErrorStream(true);
-        Process p = pb.start();
-        return p;
-    }
-
-    private static Process remoteInvoke(String cmd) throws Exception {
-        ProcessBuilder pb = new ProcessBuilder("vagrant", "ssh", "cc", "-c", "MANAGIX_HOME=/tmp/asterix/ " + cmd);
-        File cwd = new File(asterixProjectDir.toString() + "/" + CLUSTER_BASE);
-        pb.directory(cwd);
-        pb.redirectErrorStream(true);
-        Process p = pb.start();
-        p.waitFor();
-        return p;
-    }
-
-    private static Process remoteInvoke(String cmd, String node) throws Exception {
-        ProcessBuilder pb = new ProcessBuilder("vagrant", "ssh", node, "--", cmd);
-        File cwd = new File(asterixProjectDir.toString() + "/" + CLUSTER_BASE);
-        pb.directory(cwd);
-        pb.redirectErrorStream(true);
-        Process p = pb.start();
-        p.waitFor();
-        return p;
-    }
-
-    private static Process managixInvoke(String cmd) throws Exception {
-        return remoteInvoke(MANAGIX_HOME + cmd);
-    }
-
-    private static String executeVagrantScript(String node, String scriptName) throws Exception {
-        pb.command("vagrant", "ssh", node, "--", SCRIPT_HOME + scriptName);
-        Process p = pb.start();
-        p.waitFor();
-        InputStream input = p.getInputStream();
-        return IOUtils.toString(input, StandardCharsets.UTF_8.name());
-    }
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2772f12a/asterixdb/asterix-installer/src/test/resources/integrationts/replication/data/fbu.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/integrationts/replication/data/fbu.adm b/asterixdb/asterix-installer/src/test/resources/integrationts/replication/data/fbu.adm
new file mode 100644
index 0000000..7e99ea4
--- /dev/null
+++ b/asterixdb/asterix-installer/src/test/resources/integrationts/replication/data/fbu.adm
@@ -0,0 +1,10 @@
+{"id":1,"alias":"Margarita","name":"MargaritaStoddard","user-since":datetime("2012-08-20T10:10:00"),"friend-ids":{{2,3,6,10}},"employment":[{"organization-name":"Codetechno","start-date":date("2006-08-06")}]}
+{"id":2,"alias":"Isbel","name":"IsbelDull","user-since":datetime("2011-01-22T10:10:00"),"friend-ids":{{1,4}},"employment":[{"organization-name":"Hexviafind","start-date":date("2010-04-27")}]}
+{"id":3,"alias":"Emory","name":"EmoryUnk","user-since":datetime("2012-07-10T10:10:00"),"friend-ids":{{1,5,8,9}},"employment":[{"organization-name":"geomedia","start-date":date("2010-06-17"),"end-date":date("2010-01-26")}]}
+{"id":4,"alias":"Nicholas","name":"NicholasStroh","user-since":datetime("2010-12-27T10:10:00"),"friend-ids":{{2}},"employment":[{"organization-name":"Zamcorporation","start-date":date("2010-06-08")}]}
+{"id":5,"alias":"Von","name":"VonKemble","user-since":datetime("2010-01-05T10:10:00"),"friend-ids":{{3,6,10}},"employment":[{"organization-name":"Kongreen","start-date":date("2010-11-27")}]}
+{"id":6,"alias":"Willis","name":"WillisWynne","user-since":datetime("2005-01-17T10:10:00"),"friend-ids":{{1,3,7}},"employment":[{"organization-name":"jaydax","start-date":date("2009-05-15")}]}
+{"id":7,"alias":"Suzanna","name":"SuzannaTillson","user-since":datetime("2012-08-07T10:10:00"),"friend-ids":{{6}},"employment":[{"organization-name":"Labzatron","start-date":date("2011-04-19")}]}
+{"id":8,"alias":"Nila","name":"NilaMilliron","user-since":datetime("2008-01-01T10:10:00"),"friend-ids":{{3}},"employment":[{"organization-name":"Plexlane","start-date":date("2010-02-28")}]}
+{"id":9,"alias":"Woodrow","name":"WoodrowNehling","user-since":datetime("2005-09-20T10:10:00"),"friend-ids":{{3,10}},"employment":[{"organization-name":"Zuncan","start-date":date("2003-04-22"),"end-date":date("2009-12-13")}]}
+{"id":10,"alias":"Bram","name":"BramHatch","user-since":datetime("2010-10-16T10:10:00"),"friend-ids":{{1,5,9}},"employment":[{"organization-name":"physcane","start-date":date("2007-06-05"),"end-date":date("2011-11-05")}]}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2772f12a/asterixdb/asterix-installer/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.10.get.http
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.10.get.http b/asterixdb/asterix-installer/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.10.get.http
index 2bd5e45..3faa945 100644
--- a/asterixdb/asterix-installer/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.10.get.http
+++ b/asterixdb/asterix-installer/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.10.get.http
@@ -27,4 +27,4 @@
  * Expected Result : Success
  * Date            : February 3 2016
  */
-/admin/cluster
+/admin/cluster/summary
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2772f12a/asterixdb/asterix-installer/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.2.update.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.2.update.aql b/asterixdb/asterix-installer/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.2.update.aql
index c0b4919..e10896a 100644
--- a/asterixdb/asterix-installer/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.2.update.aql
+++ b/asterixdb/asterix-installer/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.2.update.aql
@@ -30,7 +30,7 @@
 use dataverse TinySocial;
 
 load dataset FacebookUsers using localfs
-(("path"="asterix_nc1:///vagrant/data/fbu.adm"),
+(("path"="asterix_nc1://../../../../../src/test/resources/integrationts/replication/data/fbu.adm"),
 ("format"="adm"));
 
 insert into dataset TinySocial.FacebookUsersInMemory(for $x in dataset TinySocial.FacebookUsers return $x);
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2772f12a/asterixdb/asterix-installer/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.3.node.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.3.node.aql b/asterixdb/asterix-installer/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.3.node.aql
new file mode 100644
index 0000000..9d5dc9b
--- /dev/null
+++ b/asterixdb/asterix-installer/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.3.node.aql
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Test case Name  : bulkload.aql
+ * Description     : Check that Bulkload LSM component are replicated correclty.
+                     The test goes as follows:
+                     start 2 nodes, bulkload a dataset, query data, kill one node
+                     and wait until the failover complete, query the data again.
+ * Expected Result : Success
+ * Date            : January 6 2016
+ */
+kill asterix_nc1
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2772f12a/asterixdb/asterix-installer/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.3.vscript.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.3.vscript.aql b/asterixdb/asterix-installer/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.3.vscript.aql
deleted file mode 100644
index 5eec164..0000000
--- a/asterixdb/asterix-installer/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.3.vscript.aql
+++ /dev/null
@@ -1 +0,0 @@
-nc1 kill_cc_and_nc.sh
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2772f12a/asterixdb/asterix-installer/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.5.get.http
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.5.get.http b/asterixdb/asterix-installer/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.5.get.http
index 9d19b69..3faa945 100644
--- a/asterixdb/asterix-installer/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.5.get.http
+++ b/asterixdb/asterix-installer/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.5.get.http
@@ -27,4 +27,4 @@
  * Expected Result : Success
  * Date            : February 3 2016
  */
-admin/cluster
+/admin/cluster/summary
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2772f12a/asterixdb/asterix-installer/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.8.mgx.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.8.mgx.aql b/asterixdb/asterix-installer/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.8.mgx.aql
new file mode 100644
index 0000000..67b492c
--- /dev/null
+++ b/asterixdb/asterix-installer/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.8.mgx.aql
@@ -0,0 +1 @@
+startnode -n asterix -nodes nc1
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2772f12a/asterixdb/asterix-installer/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.8.vmgx.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.8.vmgx.aql b/asterixdb/asterix-installer/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.8.vmgx.aql
deleted file mode 100644
index 67b492c..0000000
--- a/asterixdb/asterix-installer/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.8.vmgx.aql
+++ /dev/null
@@ -1 +0,0 @@
-startnode -n asterix -nodes nc1
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2772f12a/asterixdb/asterix-installer/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.9.sleep.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.9.sleep.aql b/asterixdb/asterix-installer/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.9.sleep.aql
index 1746da6..d0000aa 100644
--- a/asterixdb/asterix-installer/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.9.sleep.aql
+++ b/asterixdb/asterix-installer/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.9.sleep.aql
@@ -1 +1 @@
-10000
\ No newline at end of file
+20000
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2772f12a/asterixdb/asterix-installer/src/test/resources/integrationts/replication/queries/failover/bulkload/bulkload.2.update.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/integrationts/replication/queries/failover/bulkload/bulkload.2.update.aql b/asterixdb/asterix-installer/src/test/resources/integrationts/replication/queries/failover/bulkload/bulkload.2.update.aql
index 94ecc27..90ef6a7 100644
--- a/asterixdb/asterix-installer/src/test/resources/integrationts/replication/queries/failover/bulkload/bulkload.2.update.aql
+++ b/asterixdb/asterix-installer/src/test/resources/integrationts/replication/queries/failover/bulkload/bulkload.2.update.aql
@@ -29,4 +29,5 @@
 use dataverse TinySocial;
 
 load dataset FacebookUsers using localfs
-(("path"="asterix_nc1:///vagrant/data/fbu.adm"),("format"="adm"));
\ No newline at end of file
+(("path"="asterix_nc1://../../../../../src/test/resources/integrationts/replication/data/fbu.adm"),
+("format"="adm"));
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2772f12a/asterixdb/asterix-installer/src/test/resources/integrationts/replication/queries/failover/bulkload/bulkload.3.node.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/integrationts/replication/queries/failover/bulkload/bulkload.3.node.aql b/asterixdb/asterix-installer/src/test/resources/integrationts/replication/queries/failover/bulkload/bulkload.3.node.aql
new file mode 100644
index 0000000..118b7e8
--- /dev/null
+++ b/asterixdb/asterix-installer/src/test/resources/integrationts/replication/queries/failover/bulkload/bulkload.3.node.aql
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Test case Name  : bulkload.aql
+ * Description     : Check that Bulkload LSM component are replicated correclty.
+                     The test goes as follows:
+                     start 2 nodes, bulkload a dataset, query data, kill one node
+                     and wait until the failover complete, query the data again.
+ * Expected Result : Success
+ * Date            : January 6 2016
+ */
+kill asterix_nc2
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2772f12a/asterixdb/asterix-installer/src/test/resources/integrationts/replication/queries/failover/bulkload/bulkload.3.txnqbc.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/integrationts/replication/queries/failover/bulkload/bulkload.3.txnqbc.aql b/asterixdb/asterix-installer/src/test/resources/integrationts/replication/queries/failover/bulkload/bulkload.3.txnqbc.aql
deleted file mode 100644
index 9c8cb96..0000000
--- a/asterixdb/asterix-installer/src/test/resources/integrationts/replication/queries/failover/bulkload/bulkload.3.txnqbc.aql
+++ /dev/null
@@ -1,31 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Test case Name  : bulkload.aql
- * Description     : Check that Bulkload LSM component are replicated correclty.
-                     The test goes as follows:
-                     start 2 nodes, bulkload a dataset, query data, kill one node
-                     and wait until the failover complete, query the data again.
- * Expected Result : Success
- * Date            : January 6 2016
- */
-
-use dataverse TinySocial;
-
-count (for $x in dataset FacebookUsers return $x);
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2772f12a/asterixdb/asterix-installer/src/test/resources/integrationts/replication/queries/failover/bulkload/bulkload.4.sleep.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/integrationts/replication/queries/failover/bulkload/bulkload.4.sleep.aql b/asterixdb/asterix-installer/src/test/resources/integrationts/replication/queries/failover/bulkload/bulkload.4.sleep.aql
new file mode 100644
index 0000000..51b0e76
--- /dev/null
+++ b/asterixdb/asterix-installer/src/test/resources/integrationts/replication/queries/failover/bulkload/bulkload.4.sleep.aql
@@ -0,0 +1 @@
+60000
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2772f12a/asterixdb/asterix-installer/src/test/resources/integrationts/replication/queries/failover/bulkload/bulkload.4.vscript.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/integrationts/replication/queries/failover/bulkload/bulkload.4.vscript.aql b/asterixdb/asterix-installer/src/test/resources/integrationts/replication/queries/failover/bulkload/bulkload.4.vscript.aql
deleted file mode 100644
index 5695ed7..0000000
--- a/asterixdb/asterix-installer/src/test/resources/integrationts/replication/queries/failover/bulkload/bulkload.4.vscript.aql
+++ /dev/null
@@ -1 +0,0 @@
-nc2 kill_cc_and_nc.sh
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2772f12a/asterixdb/asterix-installer/src/test/resources/integrationts/replication/queries/failover/bulkload/bulkload.5.query.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/integrationts/replication/queries/failover/bulkload/bulkload.5.query.aql b/asterixdb/asterix-installer/src/test/resources/integrationts/replication/queries/failover/bulkload/bulkload.5.query.aql
new file mode 100644
index 0000000..9c8cb96
--- /dev/null
+++ b/asterixdb/asterix-installer/src/test/resources/integrationts/replication/queries/failover/bulkload/bulkload.5.query.aql
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Test case Name  : bulkload.aql
+ * Description     : Check that Bulkload LSM component are replicated correclty.
+                     The test goes as follows:
+                     start 2 nodes, bulkload a dataset, query data, kill one node
+                     and wait until the failover complete, query the data again.
+ * Expected Result : Success
+ * Date            : January 6 2016
+ */
+
+use dataverse TinySocial;
+
+count (for $x in dataset FacebookUsers return $x);
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2772f12a/asterixdb/asterix-installer/src/test/resources/integrationts/replication/queries/failover/bulkload/bulkload.5.sleep.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/integrationts/replication/queries/failover/bulkload/bulkload.5.sleep.aql b/asterixdb/asterix-installer/src/test/resources/integrationts/replication/queries/failover/bulkload/bulkload.5.sleep.aql
deleted file mode 100644
index 51b0e76..0000000
--- a/asterixdb/asterix-installer/src/test/resources/integrationts/replication/queries/failover/bulkload/bulkload.5.sleep.aql
+++ /dev/null
@@ -1 +0,0 @@
-60000
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2772f12a/asterixdb/asterix-installer/src/test/resources/integrationts/replication/queries/failover/bulkload/bulkload.6.txnqar.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/integrationts/replication/queries/failover/bulkload/bulkload.6.txnqar.aql b/asterixdb/asterix-installer/src/test/resources/integrationts/replication/queries/failover/bulkload/bulkload.6.txnqar.aql
deleted file mode 100644
index 9c8cb96..0000000
--- a/asterixdb/asterix-installer/src/test/resources/integrationts/replication/queries/failover/bulkload/bulkload.6.txnqar.aql
+++ /dev/null
@@ -1,31 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Test case Name  : bulkload.aql
- * Description     : Check that Bulkload LSM component are replicated correclty.
-                     The test goes as follows:
-                     start 2 nodes, bulkload a dataset, query data, kill one node
-                     and wait until the failover complete, query the data again.
- * Expected Result : Success
- * Date            : January 6 2016
- */
-
-use dataverse TinySocial;
-
-count (for $x in dataset FacebookUsers return $x);
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2772f12a/asterixdb/asterix-installer/src/test/resources/integrationts/replication/queries/failover/mem_component_recovery/mem_component_recovery.2.update.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/integrationts/replication/queries/failover/mem_component_recovery/mem_component_recovery.2.update.aql b/asterixdb/asterix-installer/src/test/resources/integrationts/replication/queries/failover/mem_component_recovery/mem_component_recovery.2.update.aql
index d97f786..18ce908 100644
--- a/asterixdb/asterix-installer/src/test/resources/integrationts/replication/queries/failover/mem_component_recovery/mem_component_recovery.2.update.aql
+++ b/asterixdb/asterix-installer/src/test/resources/integrationts/replication/queries/failover/mem_component_recovery/mem_component_recovery.2.update.aql
@@ -29,6 +29,6 @@
 use dataverse TinySocial;
 
 load dataset FacebookUsers using localfs
-(("path"="asterix_nc1:///vagrant/data/fbu.adm"),("format"="adm"));
+(("path"="asterix_nc1://../../../../../src/test/resources/integrationts/replication/data/fbu.adm"),("format"="adm"));
 
 insert into dataset TinySocial.FacebookUsersInMemory(for $x in dataset TinySocial.FacebookUsers return $x);
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2772f12a/asterixdb/asterix-installer/src/test/resources/integrationts/replication/queries/failover/mem_component_recovery/mem_component_recovery.3.node.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/integrationts/replication/queries/failover/mem_component_recovery/mem_component_recovery.3.node.aql b/asterixdb/asterix-installer/src/test/resources/integrationts/replication/queries/failover/mem_component_recovery/mem_component_recovery.3.node.aql
new file mode 100644
index 0000000..118b7e8
--- /dev/null
+++ b/asterixdb/asterix-installer/src/test/resources/integrationts/replication/queries/failover/mem_component_recovery/mem_component_recovery.3.node.aql
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Test case Name  : bulkload.aql
+ * Description     : Check that Bulkload LSM component are replicated correclty.
+                     The test goes as follows:
+                     start 2 nodes, bulkload a dataset, query data, kill one node
+                     and wait until the failover complete, query the data again.
+ * Expected Result : Success
+ * Date            : January 6 2016
+ */
+kill asterix_nc2
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2772f12a/asterixdb/asterix-installer/src/test/resources/integrationts/replication/queries/failover/mem_component_recovery/mem_component_recovery.3.txnqbc.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/integrationts/replication/queries/failover/mem_component_recovery/mem_component_recovery.3.txnqbc.aql b/asterixdb/asterix-installer/src/test/resources/integrationts/replication/queries/failover/mem_component_recovery/mem_component_recovery.3.txnqbc.aql
deleted file mode 100644
index e25e409..0000000
--- a/asterixdb/asterix-installer/src/test/resources/integrationts/replication/queries/failover/mem_component_recovery/mem_component_recovery.3.txnqbc.aql
+++ /dev/null
@@ -1,32 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Test case Name  : mem_component_recovery.aql
- * Description     : Check that Memory LSM component are replicated and recovered correclty.
-                     The test goes as follows:
-                     start 2 nodes, bulkload a dataset, copy it to in-memory dataset, query
-                     data from memory, kill one node and wait until the failover complete,
-                     query the data again.
- * Expected Result : Success
- * Date            : January 6 2016
- */
-
-use dataverse TinySocial;
-
-count (for $x in dataset FacebookUsersInMemory return $x);
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2772f12a/asterixdb/asterix-installer/src/test/resources/integrationts/replication/queries/failover/mem_component_recovery/mem_component_recovery.4.sleep.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/integrationts/replication/queries/failover/mem_component_recovery/mem_component_recovery.4.sleep.aql b/asterixdb/asterix-installer/src/test/resources/integrationts/replication/queries/failover/mem_component_recovery/mem_component_recovery.4.sleep.aql
new file mode 100644
index 0000000..51b0e76
--- /dev/null
+++ b/asterixdb/asterix-installer/src/test/resources/integrationts/replication/queries/failover/mem_component_recovery/mem_component_recovery.4.sleep.aql
@@ -0,0 +1 @@
+60000
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2772f12a/asterixdb/asterix-installer/src/test/resources/integrationts/replication/queries/failover/mem_component_recovery/mem_component_recovery.4.vscript.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/integrationts/replication/queries/failover/mem_component_recovery/mem_component_recovery.4.vscript.aql b/asterixdb/asterix-installer/src/test/resources/integrationts/replication/queries/failover/mem_component_recovery/mem_component_recovery.4.vscript.aql
deleted file mode 100644
index 5695ed7..0000000
--- a/asterixdb/asterix-installer/src/test/resources/integrationts/replication/queries/failover/mem_component_recovery/mem_component_recovery.4.vscript.aql
+++ /dev/null
@@ -1 +0,0 @@
-nc2 kill_cc_and_nc.sh
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2772f12a/asterixdb/asterix-installer/src/test/resources/integrationts/replication/queries/failover/mem_component_recovery/mem_component_recovery.5.query.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/integrationts/replication/queries/failover/mem_component_recovery/mem_component_recovery.5.query.aql b/asterixdb/asterix-installer/src/test/resources/integrationts/replication/queries/failover/mem_component_recovery/mem_component_recovery.5.query.aql
new file mode 100644
index 0000000..e25e409
--- /dev/null
+++ b/asterixdb/asterix-installer/src/test/resources/integrationts/replication/queries/failover/mem_component_recovery/mem_component_recovery.5.query.aql
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Test case Name  : mem_component_recovery.aql
+ * Description     : Check that Memory LSM component are replicated and recovered correclty.
+                     The test goes as follows:
+                     start 2 nodes, bulkload a dataset, copy it to in-memory dataset, query
+                     data from memory, kill one node and wait until the failover complete,
+                     query the data again.
+ * Expected Result : Success
+ * Date            : January 6 2016
+ */
+
+use dataverse TinySocial;
+
+count (for $x in dataset FacebookUsersInMemory return $x);
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2772f12a/asterixdb/asterix-installer/src/test/resources/integrationts/replication/queries/failover/mem_component_recovery/mem_component_recovery.5.sleep.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/integrationts/replication/queries/failover/mem_component_recovery/mem_component_recovery.5.sleep.aql b/asterixdb/asterix-installer/src/test/resources/integrationts/replication/queries/failover/mem_component_recovery/mem_component_recovery.5.sleep.aql
deleted file mode 100644
index 51b0e76..0000000
--- a/asterixdb/asterix-installer/src/test/resources/integrationts/replication/queries/failover/mem_component_recovery/mem_component_recovery.5.sleep.aql
+++ /dev/null
@@ -1 +0,0 @@
-60000
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2772f12a/asterixdb/asterix-installer/src/test/resources/integrationts/replication/queries/failover/mem_component_recovery/mem_component_recovery.6.txnqar.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/integrationts/replication/queries/failover/mem_component_recovery/mem_component_recovery.6.txnqar.aql b/asterixdb/asterix-installer/src/test/resources/integrationts/replication/queries/failover/mem_component_recovery/mem_component_recovery.6.txnqar.aql
deleted file mode 100644
index e25e409..0000000
--- a/asterixdb/asterix-installer/src/test/resources/integrationts/replication/queries/failover/mem_component_recovery/mem_component_recovery.6.txnqar.aql
+++ /dev/null
@@ -1,32 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Test case Name  : mem_component_recovery.aql
- * Description     : Check that Memory LSM component are replicated and recovered correclty.
-                     The test goes as follows:
-                     start 2 nodes, bulkload a dataset, copy it to in-memory dataset, query
-                     data from memory, kill one node and wait until the failover complete,
-                     query the data again.
- * Expected Result : Success
- * Date            : January 6 2016
- */
-
-use dataverse TinySocial;
-
-count (for $x in dataset FacebookUsersInMemory return $x);
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2772f12a/asterixdb/asterix-installer/src/test/resources/integrationts/replication/queries/failover/metadata_node/metadata_node.2.node.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/integrationts/replication/queries/failover/metadata_node/metadata_node.2.node.aql b/asterixdb/asterix-installer/src/test/resources/integrationts/replication/queries/failover/metadata_node/metadata_node.2.node.aql
new file mode 100644
index 0000000..9d5dc9b
--- /dev/null
+++ b/asterixdb/asterix-installer/src/test/resources/integrationts/replication/queries/failover/metadata_node/metadata_node.2.node.aql
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Test case Name  : bulkload.aql
+ * Description     : Check that Bulkload LSM component are replicated correclty.
+                     The test goes as follows:
+                     start 2 nodes, bulkload a dataset, query data, kill one node
+                     and wait until the failover complete, query the data again.
+ * Expected Result : Success
+ * Date            : January 6 2016
+ */
+kill asterix_nc1
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2772f12a/asterixdb/asterix-installer/src/test/resources/integrationts/replication/queries/failover/metadata_node/metadata_node.2.txnqbc.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/integrationts/replication/queries/failover/metadata_node/metadata_node.2.txnqbc.aql b/asterixdb/asterix-installer/src/test/resources/integrationts/replication/queries/failover/metadata_node/metadata_node.2.txnqbc.aql
deleted file mode 100644
index 76bdcfe..0000000
--- a/asterixdb/asterix-installer/src/test/resources/integrationts/replication/queries/failover/metadata_node/metadata_node.2.txnqbc.aql
+++ /dev/null
@@ -1,30 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Test case Name  : metadata_node_recovery.aql
- * Description     : Check that metadata node failover is done correctly.
-                     The test goes as follows:
-                     start 2 nodes, create a dataset, kill metadata node
-                     and wait until the failover complete, verify the
-                     dataset still exists.
- * Expected Result : Success
- * Date            : January 6 2016
- */
-
-for $x in dataset Metadata.Dataset where $x.DatasetName ='FacebookUsers' return $x.DatasetName;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2772f12a/asterixdb/asterix-installer/src/test/resources/integrationts/replication/queries/failover/metadata_node/metadata_node.3.sleep.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/integrationts/replication/queries/failover/metadata_node/metadata_node.3.sleep.aql b/asterixdb/asterix-installer/src/test/resources/integrationts/replication/queries/failover/metadata_node/metadata_node.3.sleep.aql
new file mode 100644
index 0000000..51b0e76
--- /dev/null
+++ b/asterixdb/asterix-installer/src/test/resources/integrationts/replication/queries/failover/metadata_node/metadata_node.3.sleep.aql
@@ -0,0 +1 @@
+60000
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2772f12a/asterixdb/asterix-installer/src/test/resources/integrationts/replication/queries/failover/metadata_node/metadata_node.3.vscript.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/integrationts/replication/queries/failover/metadata_node/metadata_node.3.vscript.aql b/asterixdb/asterix-installer/src/test/resources/integrationts/replication/queries/failover/metadata_node/metadata_node.3.vscript.aql
deleted file mode 100644
index 5eec164..0000000
--- a/asterixdb/asterix-installer/src/test/resources/integrationts/replication/queries/failover/metadata_node/metadata_node.3.vscript.aql
+++ /dev/null
@@ -1 +0,0 @@
-nc1 kill_cc_and_nc.sh
\ No newline at end of file