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 2018/01/05 16:10:31 UTC

[03/10] asterixdb git commit: [ASTERIXDB-2195][REPL] Replace Static Replication

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/0a5b641a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/sync/ReplicaSynchronizer.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/sync/ReplicaSynchronizer.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/sync/ReplicaSynchronizer.java
new file mode 100644
index 0000000..9f397d2
--- /dev/null
+++ b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/sync/ReplicaSynchronizer.java
@@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.replication.sync;
+
+import java.io.IOException;
+
+import org.apache.asterix.common.api.INcApplicationContext;
+import org.apache.asterix.common.replication.IReplicationStrategy;
+import org.apache.asterix.replication.messaging.ReplicationProtocol;
+import org.apache.asterix.replication.messaging.CheckpointPartitionIndexesTask;
+import org.apache.asterix.replication.api.PartitionReplica;
+
+/**
+ * Performs the steps required to ensure any newly added replica
+ * will be in-sync with master
+ */
+public class ReplicaSynchronizer {
+
+    private final INcApplicationContext appCtx;
+    private final PartitionReplica replica;
+
+    public ReplicaSynchronizer(INcApplicationContext appCtx, PartitionReplica replica) {
+        this.appCtx = appCtx;
+        this.replica = replica;
+    }
+
+    public void sync() throws IOException {
+        syncFiles();
+        checkpointReplicaIndexes();
+        appCtx.getReplicationManager().register(replica);
+    }
+
+    private void syncFiles() throws IOException {
+        final ReplicaFilesSynchronizer fileSync = new ReplicaFilesSynchronizer(appCtx, replica);
+        fileSync.sync();
+        // flush replicated dataset to generate disk component for any remaining in-memory components
+        final IReplicationStrategy replStrategy = appCtx.getReplicationManager().getReplicationStrategy();
+        appCtx.getDatasetLifecycleManager().flushDataset(replStrategy);
+        // sync any newly generated files
+        fileSync.sync();
+    }
+
+    private void checkpointReplicaIndexes() throws IOException {
+        CheckpointPartitionIndexesTask task =
+                new CheckpointPartitionIndexesTask(replica.getIdentifier().getPartition());
+        ReplicationProtocol.sendTo(replica, task);
+        ReplicationProtocol.waitForAck(replica);
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/0a5b641a/asterixdb/asterix-replication/src/test/resources/data/fbu.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-replication/src/test/resources/data/fbu.adm b/asterixdb/asterix-replication/src/test/resources/data/fbu.adm
deleted file mode 100644
index 7e99ea4..0000000
--- a/asterixdb/asterix-replication/src/test/resources/data/fbu.adm
+++ /dev/null
@@ -1,10 +0,0 @@
-{"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/0a5b641a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/message/ReplicaEventMessage.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/message/ReplicaEventMessage.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/message/ReplicaEventMessage.java
deleted file mode 100644
index 8ae1bd7..0000000
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/message/ReplicaEventMessage.java
+++ /dev/null
@@ -1,65 +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.
- */
-package org.apache.asterix.runtime.message;
-
-import org.apache.asterix.common.api.INcApplicationContext;
-import org.apache.asterix.common.messaging.api.INcAddressedMessage;
-import org.apache.asterix.common.replication.Replica;
-import org.apache.asterix.common.replication.ReplicaEvent;
-import org.apache.hyracks.api.application.IClusterLifecycleListener.ClusterEventType;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-
-public class ReplicaEventMessage implements INcAddressedMessage {
-
-    private static final long serialVersionUID = 1L;
-    private final String nodeId;
-    private final ClusterEventType event;
-    private final String nodeIPAddress;
-    private final int nodePort;
-
-    public ReplicaEventMessage(String nodeId, String nodeIPAddress, int nodePort, ClusterEventType event) {
-        this.nodeId = nodeId;
-        this.nodeIPAddress = nodeIPAddress;
-        this.nodePort = nodePort;
-        this.event = event;
-    }
-
-    public String getNodeId() {
-        return nodeId;
-    }
-
-    public ClusterEventType getEvent() {
-        return event;
-    }
-
-    public String getNodeIPAddress() {
-        return nodeIPAddress;
-    }
-
-    @Override
-    public void handle(INcApplicationContext appContext) throws HyracksDataException, InterruptedException {
-        Replica replica = new Replica(nodeId, nodeIPAddress, nodePort);
-        appContext.getReplicationManager().reportReplicaEvent(new ReplicaEvent(replica, event));
-    }
-
-    @Override
-    public String toString() {
-        return ReplicaEventMessage.class.getSimpleName();
-    }
-}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/0a5b641a/asterixdb/asterix-server/src/test/java/org/apache/asterix/server/test/MetadataReplicationIT.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-server/src/test/java/org/apache/asterix/server/test/MetadataReplicationIT.java b/asterixdb/asterix-server/src/test/java/org/apache/asterix/server/test/MetadataReplicationIT.java
deleted file mode 100644
index 5ae6843..0000000
--- a/asterixdb/asterix-server/src/test/java/org/apache/asterix/server/test/MetadataReplicationIT.java
+++ /dev/null
@@ -1,168 +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.
- */
-package org.apache.asterix.server.test;
-
-import java.io.File;
-import java.nio.file.Paths;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Map;
-import java.util.concurrent.TimeUnit;
-
-import org.apache.asterix.test.base.RetainLogsRule;
-import org.apache.asterix.test.common.TestExecutor;
-import org.apache.asterix.testframework.context.TestCaseContext;
-import org.apache.commons.io.FileUtils;
-import org.apache.commons.lang3.StringUtils;
-import org.apache.hyracks.server.process.HyracksCCProcess;
-import org.apache.hyracks.server.process.HyracksNCServiceProcess;
-import org.apache.hyracks.server.process.HyracksVirtualCluster;
-import org.apache.logging.log4j.LogManager;
-import org.apache.logging.log4j.Logger;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TestRule;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-
-@RunWith(Parameterized.class)
-public class MetadataReplicationIT {
-
-    // Important paths and files for this test.
-
-    // The "target" subdirectory of asterix-server. All outputs go here.
-    public static final String TARGET_DIR = StringUtils.join(new String[] { "../asterix-server/target" },
-            File.separator);
-
-    // Directory where the NCs create and store all data, as configured by
-    // src/test/resources/NCServiceExecutionIT/cc.conf.
-    public static final String INSTANCE_DIR = StringUtils.join(new String[] { TARGET_DIR, "tmp" }, File.separator);
-
-    // The log directory, where all CC, NCService, and NC logs are written. CC and
-    // NCService logs are configured on the HyracksVirtualCluster below. NC logs
-    // are configured in src/test/resources/NCServiceExecutionIT/ncservice*.conf.
-    public static final String LOG_DIR = StringUtils.join(new String[] { TARGET_DIR, "failsafe-reports" },
-            File.separator);
-
-    // Directory where *.conf files are located.
-    public static final String CONF_DIR = StringUtils
-            .join(new String[] { TARGET_DIR, "test-classes", "MetadataReplicationIT" }, File.separator);
-
-    // The app.home specified for HyracksVirtualCluster. The NCService expects
-    // to find the NC startup script in ${app.home}/bin.
-    public static final String APP_HOME = StringUtils.join(new String[] { TARGET_DIR, "appassembler" }, File.separator);
-
-    // Path to the asterix-app directory. This is used as the current working
-    // directory for the CC and NCService processes, which allows relative file
-    // paths in "load" statements in test queries to find the right data. It is
-    // also used for HDFSCluster.
-    public static final String ASTERIX_APP_DIR = StringUtils.join(new String[] { "..", "asterix-app" }, File.separator);
-
-    // Path to the actual AQL test files, which we borrow from asterix-app. This is
-    // passed to TestExecutor.
-    protected static final String TESTS_DIR = StringUtils
-            .join(new String[] { ASTERIX_APP_DIR, "src", "test", "resources", "runtimets" }, File.separator);
-
-    // Path that actual results are written to. We create and clean this directory
-    // here, and also pass it to TestExecutor which writes the test output there.
-    public static final String ACTUAL_RESULTS_DIR = StringUtils.join(new String[] { TARGET_DIR, "ittest" },
-            File.separator);
-    private static final String PATH_BASE = Paths
-            .get("src", "test", "resources", "integrationts", "metadata_only_replication").toString() + File.separator;
-    private static final String PATH_ACTUAL = "target" + File.separator + "ittest" + File.separator;
-
-    private static final Logger LOGGER = LogManager.getLogger();
-    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 static HyracksCCProcess cc;
-    private static HyracksNCServiceProcess nc1;
-    private static HyracksNCServiceProcess nc2;
-    private static HyracksVirtualCluster cluster;
-
-    public MetadataReplicationIT(TestCaseContext tcCtx) {
-        this.tcCtx = tcCtx;
-    }
-
-    @Rule
-    public TestRule retainLogs = new RetainLogsRule(NCServiceExecutionIT.ASTERIX_APP_DIR, reportPath, this);
-
-    @Before
-    public void before() throws Exception {
-        LOGGER.info("Creating new instance...");
-        File instanceDir = new File(INSTANCE_DIR);
-        if (instanceDir.isDirectory()) {
-            FileUtils.deleteDirectory(instanceDir);
-        }
-
-        // HDFSCluster requires the input directory to end with a file separator.
-
-        cluster = new HyracksVirtualCluster(new File(APP_HOME), new File(ASTERIX_APP_DIR));
-        nc1 = cluster.addNCService(new File(CONF_DIR, "ncservice1.conf"), new File(LOG_DIR, "ncservice1.log"));
-
-        nc2 = cluster.addNCService(new File(CONF_DIR, "ncservice2.conf"), new File(LOG_DIR, "ncservice2.log"));
-
-        // Start CC
-        cc = cluster.start(new File(CONF_DIR, "cc.conf"), new File(LOG_DIR, "cc.log"));
-
-        LOGGER.info("Instance created.");
-        testExecutor.waitForClusterActive(30, TimeUnit.SECONDS);
-        LOGGER.info("Instance is in ACTIVE state.");
-
-    }
-
-    @After
-    public void after() throws Exception {
-        LOGGER.info("Destroying instance...");
-        cluster.stop();
-        LOGGER.info("Instance destroyed.");
-    }
-
-    @Test
-    public void test() throws Exception {
-        testExecutor.executeTest(PATH_ACTUAL, tcCtx, null, false);
-    }
-
-    @Parameterized.Parameters(name = "MetadataReplicationIT {index}: {0}")
-    public static Collection<Object[]> tests() throws Exception {
-        Collection<Object[]> testArgs = buildTestsInXml(TestCaseContext.DEFAULT_TESTSUITE_XML_NAME);
-        if (testArgs.size() == 0) {
-            testArgs = buildTestsInXml(TestCaseContext.DEFAULT_TESTSUITE_XML_NAME);
-        }
-        return testArgs;
-    }
-
-    protected static Collection<Object[]> buildTestsInXml(String xmlfile) throws Exception {
-        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;
-    }
-}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/0a5b641a/asterixdb/asterix-server/src/test/java/org/apache/asterix/server/test/ReplicationIT.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-server/src/test/java/org/apache/asterix/server/test/ReplicationIT.java b/asterixdb/asterix-server/src/test/java/org/apache/asterix/server/test/ReplicationIT.java
index aa2d97d..289dbf2 100644
--- a/asterixdb/asterix-server/src/test/java/org/apache/asterix/server/test/ReplicationIT.java
+++ b/asterixdb/asterix-server/src/test/java/org/apache/asterix/server/test/ReplicationIT.java
@@ -18,9 +18,19 @@
  */
 package org.apache.asterix.server.test;
 
+import static org.apache.asterix.server.test.NCServiceExecutionIT.APP_HOME;
+import static org.apache.asterix.server.test.NCServiceExecutionIT.ASTERIX_APP_DIR;
+import static org.apache.asterix.server.test.NCServiceExecutionIT.INSTANCE_DIR;
+import static org.apache.asterix.server.test.NCServiceExecutionIT.LOG_DIR;
+import static org.apache.asterix.server.test.NCServiceExecutionIT.TARGET_DIR;
+
 import java.io.File;
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
 import java.util.ArrayList;
 import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
 import java.util.concurrent.TimeUnit;
 
 import org.apache.asterix.test.base.RetainLogsRule;
@@ -28,8 +38,6 @@ import org.apache.asterix.test.common.TestExecutor;
 import org.apache.asterix.testframework.context.TestCaseContext;
 import org.apache.commons.io.FileUtils;
 import org.apache.commons.lang3.StringUtils;
-import org.apache.hyracks.server.process.HyracksCCProcess;
-import org.apache.hyracks.server.process.HyracksNCServiceProcess;
 import org.apache.hyracks.server.process.HyracksVirtualCluster;
 import org.apache.hyracks.util.file.FileUtil;
 import org.apache.logging.log4j.LogManager;
@@ -42,38 +50,38 @@ import org.junit.rules.TestRule;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
 
-import static org.apache.asterix.server.test.NCServiceExecutionIT.APP_HOME;
-import static org.apache.asterix.server.test.NCServiceExecutionIT.INSTANCE_DIR;
-import static org.apache.asterix.server.test.NCServiceExecutionIT.TARGET_DIR;
-import static org.apache.asterix.server.test.NCServiceExecutionIT.ASTERIX_APP_DIR;
-import static org.apache.asterix.server.test.NCServiceExecutionIT.LOG_DIR;
-
 @RunWith(Parameterized.class)
 public class ReplicationIT {
 
-    private static final String PATH_BASE = FileUtil.joinPath("src", "test", "resources", "integrationts",
-            "replication");
-    public static final String CONF_DIR = StringUtils.join(new String[] { TARGET_DIR, "test-classes", "ReplicationIT" },
-            File.separator);
+    private static final String PATH_BASE =
+            FileUtil.joinPath("src", "test", "resources", "integrationts", "replication");
+    private static final String CONF_DIR =
+            StringUtils.join(new String[] { TARGET_DIR, "test-classes", "ReplicationIT" }, File.separator);
     private static final String PATH_ACTUAL = FileUtil.joinPath("target", "ittest");
     private static final Logger LOGGER = LogManager.getLogger();
     private static String reportPath = new File(FileUtil.joinPath("target", "failsafe-reports")).getAbsolutePath();
+    private static final TestExecutor testExecutor = new TestExecutor();
+    private static HyracksVirtualCluster cluster;
 
-    private final TestExecutor testExecutor = new TestExecutor();
-    private TestCaseContext tcCtx;
-    private static ProcessBuilder pb;
+    static {
+        final Map<String, InetSocketAddress> ncEndPoints = new HashMap<>();
+        final Map<String, InetSocketAddress> replicationAddress = new HashMap<>();
+        final String ip = InetAddress.getLoopbackAddress().getHostAddress();
+        ncEndPoints.put("asterix_nc1", InetSocketAddress.createUnresolved(ip, 19004));
+        ncEndPoints.put("asterix_nc2", InetSocketAddress.createUnresolved(ip, 19005));
+        replicationAddress.put("asterix_nc1", InetSocketAddress.createUnresolved(ip, 2001));
+        replicationAddress.put("asterix_nc2", InetSocketAddress.createUnresolved(ip, 2002));
+        testExecutor.setNcEndPoints(ncEndPoints);
+        testExecutor.setNcReplicationAddress(replicationAddress);
+    }
 
-    private static HyracksCCProcess cc;
-    private static HyracksNCServiceProcess nc1;
-    private static HyracksNCServiceProcess nc2;
-    private static HyracksVirtualCluster cluster;
+    private TestCaseContext tcCtx;
 
     public ReplicationIT(TestCaseContext tcCtx) {
         this.tcCtx = tcCtx;
     }
 
-    @Rule
-    public TestRule retainLogs = new RetainLogsRule(NCServiceExecutionIT.ASTERIX_APP_DIR, reportPath, this);
+    @Rule public TestRule retainLogs = new RetainLogsRule(NCServiceExecutionIT.ASTERIX_APP_DIR, reportPath, this);
 
     @Before
     public void before() throws Exception {
@@ -83,24 +91,19 @@ public class ReplicationIT {
             FileUtils.deleteDirectory(instanceDir);
         }
 
-        // HDFSCluster requires the input directory to end with a file separator.
-
         cluster = new HyracksVirtualCluster(new File(APP_HOME), new File(ASTERIX_APP_DIR));
-        nc1 = cluster.addNCService(new File(CONF_DIR, "ncservice1.conf"), new File(LOG_DIR, "ncservice1.log"));
-
-        nc2 = cluster.addNCService(new File(CONF_DIR, "ncservice2.conf"), new File(LOG_DIR, "ncservice2.log"));
+        cluster.addNCService(new File(CONF_DIR, "ncservice1.conf"), new File(LOG_DIR, "ncservice1.log"));
+        cluster.addNCService(new File(CONF_DIR, "ncservice2.conf"), new File(LOG_DIR, "ncservice2.log"));
 
         // Start CC
-        cc = cluster.start(new File(CONF_DIR, "cc.conf"), new File(LOG_DIR, "cc.log"));
-
+        cluster.start(new File(CONF_DIR, "cc.conf"), new File(LOG_DIR, "cc.log"));
         LOGGER.info("Instance created.");
         testExecutor.waitForClusterActive(30, TimeUnit.SECONDS);
         LOGGER.info("Instance is in ACTIVE state.");
-
     }
 
     @After
-    public void after() throws Exception {
+    public void after() {
         LOGGER.info("Destroying instance...");
         cluster.stop();
         LOGGER.info("Instance destroyed.");
@@ -120,7 +123,7 @@ public class ReplicationIT {
         return testArgs;
     }
 
-    protected static Collection<Object[]> buildTestsInXml(String xmlfile) throws Exception {
+    private static Collection<Object[]> buildTestsInXml(String xmlfile) throws Exception {
         Collection<Object[]> testArgs = new ArrayList<>();
         TestCaseContext.Builder b = new TestCaseContext.Builder();
         for (TestCaseContext ctx : b.build(new File(PATH_BASE), xmlfile)) {

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/0a5b641a/asterixdb/asterix-server/src/test/resources/MetadataReplicationIT/cc.conf
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-server/src/test/resources/MetadataReplicationIT/cc.conf b/asterixdb/asterix-server/src/test/resources/MetadataReplicationIT/cc.conf
deleted file mode 100644
index 32e38ae..0000000
--- a/asterixdb/asterix-server/src/test/resources/MetadataReplicationIT/cc.conf
+++ /dev/null
@@ -1,52 +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.
-
-[nc/asterix_nc1]
-txn.log.dir=../asterix-server/target/tmp/asterix_nc1/txnlog
-core.dump.dir=../asterix-server/target/tmp/asterix_nc1/coredump
-iodevices=../asterix-server/target/tmp/asterix_nc1/iodevice1,../asterix-server/target/tmp/asterix_nc1/iodevice2
-jvm.args=-agentlib:jdwp=transport=dt_socket,server=y,suspend=n,address=5006
-replication.listen.port=2001
-nc.api.port=19004
-
-[nc/asterix_nc2]
-ncservice.port=9091
-txn.log.dir=../asterix-server/target/tmp/asterix_nc2/txnlog
-core.dump.dir=../asterix-server/target/tmp/asterix_nc2/coredump
-iodevices=../asterix-server/target/tmp/asterix_nc2/iodevice1,../asterix-server/target/tmp/asterix_nc2/iodevice2
-jvm.args=-agentlib:jdwp=transport=dt_socket,server=y,suspend=n,address=5007
-replication.listen.port=2002
-nc.api.port=19005
-
-[nc]
-address=127.0.0.1
-command=asterixnc
-app.class=org.apache.asterix.hyracks.bootstrap.NCApplication
-jvm.args=-Xmx4096m -Dnode.Resolver="org.apache.asterix.external.util.IdentitiyResolverFactory"
-storage.subdir=test_storage
-storage.memorycomponent.globalbudget = 1073741824
-
-[cc]
-address = 127.0.0.1
-app.class=org.apache.asterix.hyracks.bootstrap.CCApplication
-heartbeat.period=2000
-
-[common]
-log.level = INFO
-replication.enabled=true
-replication.factor=2
-replication.strategy=metadata_only

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/0a5b641a/asterixdb/asterix-server/src/test/resources/MetadataReplicationIT/ncservice1.conf
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-server/src/test/resources/MetadataReplicationIT/ncservice1.conf b/asterixdb/asterix-server/src/test/resources/MetadataReplicationIT/ncservice1.conf
deleted file mode 100644
index ba10142..0000000
--- a/asterixdb/asterix-server/src/test/resources/MetadataReplicationIT/ncservice1.conf
+++ /dev/null
@@ -1,20 +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.
-
-[ncservice]
-logdir=../asterix-server/target/failsafe-reports
-

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/0a5b641a/asterixdb/asterix-server/src/test/resources/MetadataReplicationIT/ncservice2.conf
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-server/src/test/resources/MetadataReplicationIT/ncservice2.conf b/asterixdb/asterix-server/src/test/resources/MetadataReplicationIT/ncservice2.conf
deleted file mode 100644
index 2036584..0000000
--- a/asterixdb/asterix-server/src/test/resources/MetadataReplicationIT/ncservice2.conf
+++ /dev/null
@@ -1,21 +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.
-
-[ncservice]
-logdir=../asterix-server/target/failsafe-reports
-port=9091
-

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/0a5b641a/asterixdb/asterix-server/src/test/resources/ReplicationIT/cc.conf
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-server/src/test/resources/ReplicationIT/cc.conf b/asterixdb/asterix-server/src/test/resources/ReplicationIT/cc.conf
index 776a89a..2b68dd2 100644
--- a/asterixdb/asterix-server/src/test/resources/ReplicationIT/cc.conf
+++ b/asterixdb/asterix-server/src/test/resources/ReplicationIT/cc.conf
@@ -48,4 +48,5 @@ heartbeat.period=2000
 [common]
 log.level = INFO
 replication.enabled=true
+replication.strategy=all
 replication.factor=2

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/0a5b641a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.1.ddl.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.1.ddl.aql b/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.1.ddl.aql
deleted file mode 100644
index 725ed61..0000000
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.1.ddl.aql
+++ /dev/null
@@ -1,50 +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     : Make sure metadata only replication completes as expected.
-                     The test goes as follows:
-                     start metadata node and a replica, update metadata, kill metadata node,
-                     start metadata node again, validate metadata, update metadata, kill replica,
-                     start replica, update metadata, kill metadata node, start metadata node,
-                     validate metadata.
- * Expected Result : Success
- * Date            : January 10 2017
- */
-drop dataverse Social if exists;
-create dataverse Social;
-use dataverse Social;
-
-create type EmploymentType as open {
-    organization-name: string,
-    start-date: date,
-    end-date: date?
-}
-
-create type UserType as closed {
-    id: int,
-    alias: string,
-    name: string,
-    user-since: datetime,
-    friend-ids: {{ int32 }},
-    employment: [EmploymentType]
-}
-
-create dataset Users(UserType)
-primary key id;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/0a5b641a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.10.node.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.10.node.aql b/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.10.node.aql
deleted file mode 100644
index f4685df..0000000
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.10.node.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     : Make sure metadata only replication completes as expected.
-                     The test goes as follows:
-                     start metadata node and a replica, update metadata, kill metadata node,
-                     start metadata node again, validate metadata, update metadata, kill replica,
-                     start replica, update metadata, kill metadata node, start metadata node,
-                     validate metadata.
- * Expected Result : Success
- * Date            : January 10 2017
- */
-kill asterix_nc2
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/0a5b641a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.11.sleep.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.11.sleep.aql b/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.11.sleep.aql
deleted file mode 100644
index 9c50bf1..0000000
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.11.sleep.aql
+++ /dev/null
@@ -1,19 +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.
-#
-60000
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/0a5b641a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.12.node.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.12.node.aql b/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.12.node.aql
deleted file mode 100644
index 26a6503..0000000
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.12.node.aql
+++ /dev/null
@@ -1,19 +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.
- */
-start asterix_nc2
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/0a5b641a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.13.sleep.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.13.sleep.aql b/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.13.sleep.aql
deleted file mode 100644
index dd83b4b..0000000
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.13.sleep.aql
+++ /dev/null
@@ -1,19 +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.
-#
-20000
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/0a5b641a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.14.ddl.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.14.ddl.aql b/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.14.ddl.aql
deleted file mode 100644
index eee7423..0000000
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.14.ddl.aql
+++ /dev/null
@@ -1,33 +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     : Make sure metadata only replication completes as expected.
-                     The test goes as follows:
-                     start metadata node and a replica, update metadata, kill metadata node,
-                     start metadata node again, validate metadata, update metadata, kill replica,
-                     start replica, update metadata, kill metadata node, start metadata node,
-                     validate metadata.
- * Expected Result : Success
- * Date            : January 10 2017
- */
-use dataverse Social;
-
-create dataset UsersAfterReplicaRecovery(UserType)
-primary key id;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/0a5b641a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.15.node.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.15.node.aql b/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.15.node.aql
deleted file mode 100644
index 720f33e..0000000
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.15.node.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     : Make sure metadata only replication completes as expected.
-                     The test goes as follows:
-                     start metadata node and a replica, update metadata, kill metadata node,
-                     start metadata node again, validate metadata, update metadata, kill replica,
-                     start replica, update metadata, kill metadata node, start metadata node,
-                     validate metadata.
- * Expected Result : Success
- * Date            : January 10 2017
- */
-kill asterix_nc1
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/0a5b641a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.16.sleep.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.16.sleep.aql b/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.16.sleep.aql
deleted file mode 100644
index 9c50bf1..0000000
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.16.sleep.aql
+++ /dev/null
@@ -1,19 +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.
-#
-60000
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/0a5b641a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql b/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
deleted file mode 100644
index b750c64..0000000
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
+++ /dev/null
@@ -1,19 +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.
- */
-start asterix_nc1
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/0a5b641a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.18.sleep.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.18.sleep.aql b/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.18.sleep.aql
deleted file mode 100644
index dd83b4b..0000000
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.18.sleep.aql
+++ /dev/null
@@ -1,19 +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.
-#
-20000
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/0a5b641a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.19.query.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.19.query.aql b/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.19.query.aql
deleted file mode 100644
index 5da6c59..0000000
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.19.query.aql
+++ /dev/null
@@ -1,35 +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     : Make sure metadata only replication completes as expected.
-                     The test goes as follows:
-                     start metadata node and a replica, update metadata, kill metadata node,
-                     start metadata node again, validate metadata, update metadata, kill replica,
-                     start replica, update metadata, kill metadata node, start metadata node,
-                     validate metadata.
- * Expected Result : Success
- * Date            : January 10 2017
- */
-count(
-for $x in dataset Metadata.Dataset
-where $x.DatasetName ='UsersAfterRecovery'
-or $x.DatasetName ='UsersAfterReplicaRecovery'
-return $x.DatasetName
-);
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/0a5b641a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.2.node.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.2.node.aql b/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.2.node.aql
deleted file mode 100644
index 720f33e..0000000
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.2.node.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     : Make sure metadata only replication completes as expected.
-                     The test goes as follows:
-                     start metadata node and a replica, update metadata, kill metadata node,
-                     start metadata node again, validate metadata, update metadata, kill replica,
-                     start replica, update metadata, kill metadata node, start metadata node,
-                     validate metadata.
- * Expected Result : Success
- * Date            : January 10 2017
- */
-kill asterix_nc1
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/0a5b641a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.3.sleep.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.3.sleep.aql b/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.3.sleep.aql
deleted file mode 100644
index 9c50bf1..0000000
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.3.sleep.aql
+++ /dev/null
@@ -1,19 +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.
-#
-60000
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/0a5b641a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.4.get.http
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.4.get.http b/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.4.get.http
deleted file mode 100644
index fdacfd0..0000000
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.4.get.http
+++ /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     : Make sure metadata only replication completes as expected.
-                     The test goes as follows:
-                     start metadata node and a replica, update metadata, kill metadata node,
-                     start metadata node again, validate metadata, update metadata, kill replica,
-                     start replica, update metadata, kill metadata node, start metadata node,
-                     validate metadata.
- * Expected Result : Success
- * Date            : January 10 2017
- */
-/admin/cluster/summary
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/0a5b641a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.5.node.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.5.node.aql b/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.5.node.aql
deleted file mode 100644
index b750c64..0000000
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.5.node.aql
+++ /dev/null
@@ -1,19 +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.
- */
-start asterix_nc1
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/0a5b641a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.6.sleep.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.6.sleep.aql b/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.6.sleep.aql
deleted file mode 100644
index dd83b4b..0000000
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.6.sleep.aql
+++ /dev/null
@@ -1,19 +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.
-#
-20000
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/0a5b641a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.7.get.http
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.7.get.http b/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.7.get.http
deleted file mode 100644
index fdacfd0..0000000
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.7.get.http
+++ /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     : Make sure metadata only replication completes as expected.
-                     The test goes as follows:
-                     start metadata node and a replica, update metadata, kill metadata node,
-                     start metadata node again, validate metadata, update metadata, kill replica,
-                     start replica, update metadata, kill metadata node, start metadata node,
-                     validate metadata.
- * Expected Result : Success
- * Date            : January 10 2017
- */
-/admin/cluster/summary
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/0a5b641a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.8.query.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.8.query.aql b/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.8.query.aql
deleted file mode 100644
index 778e9f3..0000000
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.8.query.aql
+++ /dev/null
@@ -1,34 +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     : Make sure metadata only replication completes as expected.
-                     The test goes as follows:
-                     start metadata node and a replica, update metadata, kill metadata node,
-                     start metadata node again, validate metadata, update metadata, kill replica,
-                     start replica, update metadata, kill metadata node, start metadata node,
-                     validate metadata.
- * Expected Result : Success
- * Date            : January 10 2017
- */
-count(
-for $x in dataset Metadata.Dataset
-where $x.DatasetName ='Users'
-return $x.DatasetName
-);
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/0a5b641a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.9.ddl.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.9.ddl.aql b/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.9.ddl.aql
deleted file mode 100644
index 376df70..0000000
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.9.ddl.aql
+++ /dev/null
@@ -1,33 +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     : Make sure metadata only replication completes as expected.
-                     The test goes as follows:
-                     start metadata node and a replica, update metadata, kill metadata node,
-                     start metadata node again, validate metadata, update metadata, kill replica,
-                     start replica, update metadata, kill metadata node, start metadata node,
-                     validate metadata.
- * Expected Result : Success
- * Date            : January 10 2017
- */
-use dataverse Social;
-
-create dataset UsersAfterRecovery(UserType)
-primary key id;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/0a5b641a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/results/metadata_recovery/metadata_node_recovery/metadata_node_recovery.cluster_state.4.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/results/metadata_recovery/metadata_node_recovery/metadata_node_recovery.cluster_state.4.adm b/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/results/metadata_recovery/metadata_node_recovery/metadata_node_recovery.cluster_state.4.adm
deleted file mode 100644
index c327ecf..0000000
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/results/metadata_recovery/metadata_node_recovery/metadata_node_recovery.cluster_state.4.adm
+++ /dev/null
@@ -1,38 +0,0 @@
-{
-  "metadata_node" : "asterix_nc1",
-  "partitions" : {
-    "0" : {
-      "active" : false,
-      "activeNodeId" : "asterix_nc1",
-      "iodeviceNum" : 0,
-      "nodeId" : "asterix_nc1",
-      "partitionId" : 0,
-      "pendingActivation" : false
-    },
-    "1" : {
-      "active" : false,
-      "activeNodeId" : "asterix_nc1",
-      "iodeviceNum" : 1,
-      "nodeId" : "asterix_nc1",
-      "partitionId" : 1,
-      "pendingActivation" : false
-    },
-    "2" : {
-      "active" : true,
-      "activeNodeId" : "asterix_nc2",
-      "iodeviceNum" : 0,
-      "nodeId" : "asterix_nc2",
-      "partitionId" : 2,
-      "pendingActivation" : false
-    },
-    "3" : {
-      "active" : true,
-      "activeNodeId" : "asterix_nc2",
-      "iodeviceNum" : 1,
-      "nodeId" : "asterix_nc2",
-      "partitionId" : 3,
-      "pendingActivation" : false
-    }
-  },
-  "state" : "UNUSABLE"
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/0a5b641a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/results/metadata_recovery/metadata_node_recovery/metadata_node_recovery.cluster_state.7.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/results/metadata_recovery/metadata_node_recovery/metadata_node_recovery.cluster_state.7.adm b/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/results/metadata_recovery/metadata_node_recovery/metadata_node_recovery.cluster_state.7.adm
deleted file mode 100644
index c0697b7..0000000
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/results/metadata_recovery/metadata_node_recovery/metadata_node_recovery.cluster_state.7.adm
+++ /dev/null
@@ -1,38 +0,0 @@
-{
-  "metadata_node" : "asterix_nc1",
-  "partitions" : {
-    "0" : {
-      "active" : true,
-      "activeNodeId" : "asterix_nc1",
-      "iodeviceNum" : 0,
-      "nodeId" : "asterix_nc1",
-      "partitionId" : 0,
-      "pendingActivation" : false
-    },
-    "1" : {
-      "active" : true,
-      "activeNodeId" : "asterix_nc1",
-      "iodeviceNum" : 1,
-      "nodeId" : "asterix_nc1",
-      "partitionId" : 1,
-      "pendingActivation" : false
-    },
-    "2" : {
-      "active" : true,
-      "activeNodeId" : "asterix_nc2",
-      "iodeviceNum" : 0,
-      "nodeId" : "asterix_nc2",
-      "partitionId" : 2,
-      "pendingActivation" : false
-    },
-    "3" : {
-      "active" : true,
-      "activeNodeId" : "asterix_nc2",
-      "iodeviceNum" : 1,
-      "nodeId" : "asterix_nc2",
-      "partitionId" : 3,
-      "pendingActivation" : false
-    }
-  },
-  "state" : "ACTIVE"
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/0a5b641a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/results/metadata_recovery/metadata_node_recovery/metadata_node_recovery.query.19.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/results/metadata_recovery/metadata_node_recovery/metadata_node_recovery.query.19.adm b/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/results/metadata_recovery/metadata_node_recovery/metadata_node_recovery.query.19.adm
deleted file mode 100644
index d8263ee..0000000
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/results/metadata_recovery/metadata_node_recovery/metadata_node_recovery.query.19.adm
+++ /dev/null
@@ -1 +0,0 @@
-2
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/0a5b641a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/results/metadata_recovery/metadata_node_recovery/metadata_node_recovery.query.8.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/results/metadata_recovery/metadata_node_recovery/metadata_node_recovery.query.8.adm b/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/results/metadata_recovery/metadata_node_recovery/metadata_node_recovery.query.8.adm
deleted file mode 100644
index 56a6051..0000000
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/results/metadata_recovery/metadata_node_recovery/metadata_node_recovery.query.8.adm
+++ /dev/null
@@ -1 +0,0 @@
-1
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/0a5b641a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/testsuite.xml
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/testsuite.xml b/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/testsuite.xml
deleted file mode 100644
index d0ac325..0000000
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/testsuite.xml
+++ /dev/null
@@ -1,27 +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-suite xmlns="urn:xml.testframework.asterix.apache.org" ResultOffsetPath="results" QueryOffsetPath="queries" QueryFileExtension=".aql">
-  <test-group name="metadata_recovery">
-    <test-case FilePath="metadata_recovery">
-      <compilation-unit name="metadata_node_recovery">
-        <output-dir compare="Text">metadata_node_recovery</output-dir>
-      </compilation-unit>
-    </test-case>
-  </test-group>
-</test-suite>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/0a5b641a/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.1.ddl.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.1.ddl.aql b/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.1.ddl.aql
deleted file mode 100644
index 2c49a01..0000000
--- a/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.1.ddl.aql
+++ /dev/null
@@ -1,59 +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  : node_failback.aql
- * Description     : Make sure node failback completes as expected.
-                     The test goes as follows:
-                     start 2 nodes, bulkload a dataset, copy it to in-memory dataset,
-                     kill one node and wait until the failover complete, query cluster state,
-                     query data, insert new data, start the killed node and wait for failback,
-                     query cluster state, query data.
- * Expected Result : Success
- * Date            : February 3 2016
- */
-
-drop dataverse TinySocial if exists;
-create dataverse TinySocial;
-use dataverse TinySocial;
-
-create type EmploymentType as open {
-    organization-name: string,
-    start-date: date,
-    end-date: date?
-}
-
-create type FacebookUserType as closed {
-    id: int,
-    alias: string,
-    name: string,
-    user-since: datetime,
-    friend-ids: {{ int32 }},
-    employment: [EmploymentType]
-}
-
-/********* 2. Create Datasets  ***********/
-use dataverse TinySocial;
-
-drop dataset FacebookUsers if exists;
-
-create dataset FacebookUsers(FacebookUserType)
-primary key id;
-
-create dataset FacebookUsersInMemory(FacebookUserType)
-primary key id;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/0a5b641a/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.10.get.http
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.10.get.http b/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.10.get.http
deleted file mode 100644
index 3faa945..0000000
--- a/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.10.get.http
+++ /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  : node_failback.aql
- * Description     : Make sure node failback completes as expected.
-                     The test goes as follows:
-                     start 2 nodes, bulkload a dataset, copy it to in-memory dataset,
-                     kill one node and wait until the failover complete, query cluster state,
-                     query data, insert new data, start the killed node and wait for failback,
-                     query cluster state, query data.
- * Expected Result : Success
- * Date            : February 3 2016
- */
-/admin/cluster/summary
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/0a5b641a/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.11.query.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.11.query.aql b/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.11.query.aql
deleted file mode 100644
index b09c3d3..0000000
--- a/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.11.query.aql
+++ /dev/null
@@ -1,33 +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  : node_failback.aql
- * Description     : Make sure node failback completes as expected.
-                     The test goes as follows:
-                     start 2 nodes, bulkload a dataset, copy it to in-memory dataset,
-                     kill one node and wait until the failover complete, query cluster state,
-                     query data, insert new data, start the killed node and wait for failback,
-                     query cluster state, query data.
- * Expected Result : Success
- * Date            : February 3 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/0a5b641a/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.2.update.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.2.update.aql b/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.2.update.aql
deleted file mode 100644
index 377e097..0000000
--- a/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.2.update.aql
+++ /dev/null
@@ -1,36 +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  : node_failback.aql
- * Description     : Make sure node failback completes as expected.
-                     The test goes as follows:
-                     start 2 nodes, bulkload a dataset, copy it to in-memory dataset,
-                     kill one node and wait until the failover complete, query cluster state,
-                     query data, insert new data, start the killed node and wait for failback,
-                     query cluster state, query data.
- * Expected Result : Success
- * Date            : February 3 2016
- */
-use dataverse TinySocial;
-
-load dataset FacebookUsers using localfs
-(("path"="asterix_nc1://../asterix-server/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/0a5b641a/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.3.node.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.3.node.aql b/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.3.node.aql
deleted file mode 100644
index 9d5dc9b..0000000
--- a/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.3.node.aql
+++ /dev/null
@@ -1,28 +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
- */
-kill asterix_nc1
\ No newline at end of file