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 2017/11/21 22:20:45 UTC

[01/12] asterixdb git commit: [ASTERIXDB-2171][STO] Remove Temporary Datasets

Repository: asterixdb
Updated Branches:
  refs/heads/master e5a65429d -> 2c04ae075


http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/runtime/CommitRuntime.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/runtime/CommitRuntime.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/runtime/CommitRuntime.java
index 672e881..d298bef 100644
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/runtime/CommitRuntime.java
+++ b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/runtime/CommitRuntime.java
@@ -27,9 +27,9 @@ import org.apache.asterix.common.transactions.ILogManager;
 import org.apache.asterix.common.transactions.ILogMarkerCallback;
 import org.apache.asterix.common.transactions.ITransactionContext;
 import org.apache.asterix.common.transactions.ITransactionManager;
-import org.apache.asterix.common.transactions.TxnId;
 import org.apache.asterix.common.transactions.LogRecord;
 import org.apache.asterix.common.transactions.LogType;
+import org.apache.asterix.common.transactions.TxnId;
 import org.apache.asterix.common.utils.TransactionUtil;
 import org.apache.hyracks.algebricks.runtime.operators.base.AbstractOneInputOneOutputOneFramePushRuntime;
 import org.apache.hyracks.api.comm.IFrame;
@@ -53,7 +53,6 @@ public class CommitRuntime extends AbstractOneInputOneOutputOneFramePushRuntime
     protected final TxnId txnId;
     protected final int datasetId;
     protected final int[] primaryKeyFields;
-    protected final boolean isTemporaryDatasetWriteJob;
     protected final boolean isWriteTransaction;
     protected final long[] longHashes;
     protected final IHyracksTaskContext ctx;
@@ -63,7 +62,7 @@ public class CommitRuntime extends AbstractOneInputOneOutputOneFramePushRuntime
     protected final boolean isSink;
 
     public CommitRuntime(IHyracksTaskContext ctx, TxnId txnId, int datasetId, int[] primaryKeyFields,
-            boolean isTemporaryDatasetWriteJob, boolean isWriteTransaction, int resourcePartition, boolean isSink) {
+            boolean isWriteTransaction, int resourcePartition, boolean isSink) {
         this.ctx = ctx;
         INcApplicationContext appCtx =
                 (INcApplicationContext) ctx.getJobletContext().getServiceContext().getApplicationContext();
@@ -73,7 +72,6 @@ public class CommitRuntime extends AbstractOneInputOneOutputOneFramePushRuntime
         this.datasetId = datasetId;
         this.primaryKeyFields = primaryKeyFields;
         this.tRef = new FrameTupleReference();
-        this.isTemporaryDatasetWriteJob = isTemporaryDatasetWriteJob;
         this.isWriteTransaction = isWriteTransaction;
         this.resourcePartition = resourcePartition;
         this.isSink = isSink;
@@ -102,27 +100,15 @@ public class CommitRuntime extends AbstractOneInputOneOutputOneFramePushRuntime
         tAccess.reset(buffer);
         int nTuple = tAccess.getTupleCount();
         for (int t = 0; t < nTuple; t++) {
-            if (isTemporaryDatasetWriteJob) {
-                /**
-                 * This "if branch" is for writes over temporary datasets. A temporary dataset does not require any lock
-                 * and does not generate any write-ahead update and commit log but generates flush log and job commit
-                 * log. However, a temporary dataset still MUST guarantee no-steal policy so that this notification call
-                 * should be delivered to PrimaryIndexOptracker and used correctly in order to decrement number of
-                 * active operation count of PrimaryIndexOptracker. By maintaining the count correctly and only allowing
-                 * flushing when the count is 0, it can guarantee the no-steal policy for temporary datasets, too.
-                 */
-                transactionContext.notifyEntityCommitted();
-            } else {
-                tRef.reset(tAccess, t);
-                try {
-                    formLogRecord(buffer, t);
-                    logMgr.log(logRecord);
-                    if (!isSink) {
-                        appendTupleToFrame(t);
-                    }
-                } catch (ACIDException e) {
-                    throw new HyracksDataException(e);
+            tRef.reset(tAccess, t);
+            try {
+                formLogRecord(buffer, t);
+                logMgr.log(logRecord);
+                if (!isSink) {
+                    appendTupleToFrame(t);
                 }
+            } catch (ACIDException e) {
+                throw new HyracksDataException(e);
             }
         }
         IFrame message = TaskUtil.get(HyracksConstants.KEY_MESSAGE, ctx);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/runtime/CommitRuntimeFactory.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/runtime/CommitRuntimeFactory.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/runtime/CommitRuntimeFactory.java
index 58f7e69..91db197 100644
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/runtime/CommitRuntimeFactory.java
+++ b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/runtime/CommitRuntimeFactory.java
@@ -34,17 +34,15 @@ public class CommitRuntimeFactory implements IPushRuntimeFactory {
     protected final TxnId txnId;
     protected final int datasetId;
     protected final int[] primaryKeyFields;
-    protected final boolean isTemporaryDatasetWriteJob;
     protected final boolean isWriteTransaction;
     protected int[] datasetPartitions;
     protected final boolean isSink;
 
-    public CommitRuntimeFactory(TxnId txnId, int datasetId, int[] primaryKeyFields, boolean isTemporaryDatasetWriteJob,
-            boolean isWriteTransaction, int[] datasetPartitions, boolean isSink) {
+    public CommitRuntimeFactory(TxnId txnId, int datasetId, int[] primaryKeyFields, boolean isWriteTransaction,
+            int[] datasetPartitions, boolean isSink) {
         this.txnId = txnId;
         this.datasetId = datasetId;
         this.primaryKeyFields = primaryKeyFields;
-        this.isTemporaryDatasetWriteJob = isTemporaryDatasetWriteJob;
         this.isWriteTransaction = isWriteTransaction;
         this.datasetPartitions = datasetPartitions;
         this.isSink = isSink;
@@ -58,8 +56,7 @@ public class CommitRuntimeFactory implements IPushRuntimeFactory {
     @Override
     public IPushRuntime createPushRuntime(IHyracksTaskContext ctx) throws HyracksDataException {
         IJobletEventListenerFactory fact = ctx.getJobletContext().getJobletEventListenerFactory();
-        return new CommitRuntime(ctx, ((IJobEventListenerFactory) fact).getTxnId(txnId), datasetId,
-                primaryKeyFields, isTemporaryDatasetWriteJob, isWriteTransaction,
-                datasetPartitions[ctx.getTaskAttemptId().getTaskId().getPartition()], isSink);
+        return new CommitRuntime(ctx, ((IJobEventListenerFactory) fact).getTxnId(txnId), datasetId, primaryKeyFields,
+                isWriteTransaction, datasetPartitions[ctx.getTaskAttemptId().getTaskId().getPartition()], isSink);
     }
 }


[12/12] asterixdb git commit: [ASTERIXDB-2171][STO] Remove Temporary Datasets

Posted by mh...@apache.org.
[ASTERIXDB-2171][STO] Remove Temporary Datasets

- user model changes: no
- storage format changes: no
- interface changes: yes
  - Removed all reference to temporary datasets.

Details:
- Remove temporary datasets grammar but reserve
  <TEMPORARY> token.
- Remove special callbacks for temp datasets.
- Remove metadata garbage collector thread.
- Remove isTemp flag from datasets.
- Remove special storage path for temp datasets.
- Remove temp datasets test cases.

Change-Id: Ib50ea27e2ec9c114508cd4a109cf646e2f20dcb0
Reviewed-on: https://asterix-gerrit.ics.uci.edu/2172
Sonar-Qube: Jenkins <je...@fulliautomatix.ics.uci.edu>
Integration-Tests: Jenkins <je...@fulliautomatix.ics.uci.edu>
Tested-by: Jenkins <je...@fulliautomatix.ics.uci.edu>
Reviewed-by: Michael Blow <mb...@apache.org>


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

Branch: refs/heads/master
Commit: 2c04ae0751acb521dedbc61c4cf30d2825d67ec9
Parents: e5a6542
Author: Murtadha Hubail <mh...@apache.org>
Authored: Tue Nov 21 15:58:01 2017 +0300
Committer: Murtadha Hubail <mh...@apache.org>
Committed: Tue Nov 21 14:20:03 2017 -0800

----------------------------------------------------------------------
 .../api/http/server/ConnectorApiServlet.java    |   7 +-
 .../asterix/app/translator/QueryTranslator.java |   6 +-
 .../hyracks/bootstrap/NCApplication.java        |  13 -
 .../http/servlet/ConnectorApiServletTest.java   |  10 +-
 .../app/bootstrap/TestNodeController.java       |   6 +-
 .../test/dataflow/ComponentRollbackTest.java    |   2 +-
 .../asterix/test/dataflow/LogMarkerTest.java    |   2 +-
 .../dataflow/MultiPartitionLSMIndexTest.java    |   2 +-
 .../SearchCursorComponentSwitchTest.java        |   2 +-
 .../TestLsmBTreeResourceFactoryProvider.java    |   3 +-
 .../asterix/test/logging/CheckpointingTest.java |   2 +-
 .../asterix/test/storage/DiskIsFullTest.java    |   2 +-
 .../IndexDropOperatorNodePushableTest.java      |   2 +-
 .../basic/temp_dataset/temp_dataset.1.ddl.aql   |  52 ----
 .../temp_dataset/temp_dataset.2.update.aql      |  24 --
 .../basic/temp_dataset/temp_dataset.3.query.aql |  29 ---
 .../basic/temp_dataset/temp_dataset.1.adm       |   1 -
 .../src/test/resources/metadata/testsuite.xml   |   5 -
 ...ete-from-loaded-dataset-with-index.1.ddl.aql |  52 ----
 ...-from-loaded-dataset-with-index.2.update.aql |  31 ---
 ...ete-from-loaded-dataset-with-index.3.ddl.aql |  30 ---
 ...-from-loaded-dataset-with-index.4.update.aql |  28 ---
 ...e-from-loaded-dataset-with-index.5.query.aql |  31 ---
 .../delete-from-loaded-dataset.1.ddl.aql        |  52 ----
 .../delete-from-loaded-dataset.2.update.aql     |  33 ---
 .../delete-from-loaded-dataset.3.query.aql      |  30 ---
 .../drop-empty-secondary-indexes.1.ddl.aql      |  75 ------
 .../drop-empty-secondary-indexes.2.update.aql   |  25 --
 .../drop-empty-secondary-indexes.3.query.aql    |  28 ---
 .../drop-index/drop-index.1.ddl.aql             |  51 ----
 .../drop-index/drop-index.2.update.aql          |  32 ---
 .../drop-index/drop-index.3.ddl.aql             |  35 ---
 .../drop-index/drop-index.4.query.aql           |  30 ---
 .../empty-load-with-index.1.ddl.aql             |  52 ----
 .../empty-load-with-index.2.update.aql          |  33 ---
 .../empty-load-with-index.3.query.aql           |  30 ---
 ...scan-dataset-with-correlated-index.1.ddl.aql |  39 ---
 ...n-dataset-with-correlated-index.2.update.aql |  42 ----
 ...scan-dataset-with-correlated-index.3.ddl.aql |  28 ---
 ...an-dataset-with-correlated-index.4.query.aql |  30 ---
 ...insert-and-scan-dataset-with-index.1.ddl.aql |  39 ---
 ...ert-and-scan-dataset-with-index.2.update.aql |  42 ----
 ...sert-and-scan-dataset-with-index.3.query.aql |  31 ---
 .../insert-and-scan-dataset.1.ddl.aql           |  36 ---
 .../insert-and-scan-dataset.2.update.aql        |  39 ---
 .../insert-and-scan-dataset.3.query.aql         |  32 ---
 .../q01_pricing_summary_report_nt.1.ddl.aql     |  51 ----
 .../q01_pricing_summary_report_nt.2.update.aql  |  30 ---
 .../q01_pricing_summary_report_nt.3.query.aql   |  46 ----
 .../scan-delete-rtree-secondary-index.1.ddl.aql |  44 ----
 ...an-delete-rtree-secondary-index.2.update.aql |  30 ---
 .../scan-delete-rtree-secondary-index.3.ddl.aql |  28 ---
 ...an-delete-rtree-secondary-index.4.update.aql |  28 ---
 ...can-delete-rtree-secondary-index.5.query.aql |  30 ---
 .../scan-insert-persistent-to-temp.1.ddl.aql    |  53 ----
 .../scan-insert-persistent-to-temp.2.update.aql |  35 ---
 .../scan-insert-persistent-to-temp.3.ddl.aql    |  30 ---
 .../scan-insert-persistent-to-temp.4.update.aql |  36 ---
 .../scan-insert-persistent-to-temp.5.query.aql  |  31 ---
 .../scan-insert-rtree-secondary-index.1.ddl.aql |  53 ----
 ...an-insert-rtree-secondary-index.2.update.aql |  35 ---
 .../scan-insert-rtree-secondary-index.3.ddl.aql |  30 ---
 ...an-insert-rtree-secondary-index.4.update.aql |  36 ---
 ...can-insert-rtree-secondary-index.5.query.aql |  31 ---
 .../scan-insert-temp-to-persistent.1.ddl.aql    |  53 ----
 .../scan-insert-temp-to-persistent.2.update.aql |  35 ---
 .../scan-insert-temp-to-persistent.3.ddl.aql    |  30 ---
 .../scan-insert-temp-to-persistent.4.update.aql |  36 ---
 .../scan-insert-temp-to-persistent.5.query.aql  |  31 ---
 .../temp_primary_plus_ngram_flush.1.ddl.aql     |  67 -----
 .../temp_primary_plus_ngram_flush.2.update.aql  |  46 ----
 .../temp_primary_plus_ngram_flush.3.query.aql   |  29 ---
 ...e-from-loaded-dataset-with-index.1.ddl.sqlpp |  53 ----
 ...rom-loaded-dataset-with-index.2.update.sqlpp |  30 ---
 ...e-from-loaded-dataset-with-index.3.ddl.sqlpp |  33 ---
 ...rom-loaded-dataset-with-index.4.update.sqlpp |  30 ---
 ...from-loaded-dataset-with-index.5.query.sqlpp |  33 ---
 .../delete-from-loaded-dataset.1.ddl.sqlpp      |  53 ----
 .../delete-from-loaded-dataset.2.update.sqlpp   |  32 ---
 .../delete-from-loaded-dataset.3.query.sqlpp    |  32 ---
 .../drop-empty-secondary-indexes.1.ddl.sqlpp    |  77 ------
 .../drop-empty-secondary-indexes.2.update.sqlpp |  25 --
 .../drop-empty-secondary-indexes.3.query.sqlpp  |  29 ---
 .../drop-index/drop-index.1.ddl.sqlpp           |  53 ----
 .../drop-index/drop-index.2.update.sqlpp        |  30 ---
 .../drop-index/drop-index.3.ddl.sqlpp           |  37 ---
 .../drop-index/drop-index.4.query.sqlpp         |  32 ---
 .../empty-load-with-index.1.ddl.sqlpp           |  55 ----
 .../empty-load-with-index.2.update.sqlpp        |  31 ---
 .../empty-load-with-index.3.query.sqlpp         |  32 ---
 ...an-dataset-with-correlated-index.1.ddl.sqlpp |  42 ----
 ...dataset-with-correlated-index.2.update.sqlpp |  35 ---
 ...an-dataset-with-correlated-index.3.ddl.sqlpp |  32 ---
 ...-dataset-with-correlated-index.4.query.sqlpp |  33 ---
 ...sert-and-scan-dataset-with-index.1.ddl.sqlpp |  42 ----
 ...t-and-scan-dataset-with-index.2.update.sqlpp |  34 ---
 ...rt-and-scan-dataset-with-index.3.query.sqlpp |  32 ---
 .../insert-and-scan-dataset.1.ddl.sqlpp         |  38 ---
 .../insert-and-scan-dataset.2.update.sqlpp      |  35 ---
 .../insert-and-scan-dataset.3.query.sqlpp       |  34 ---
 .../q01_pricing_summary_report_nt.1.ddl.sqlpp   |  52 ----
 ...q01_pricing_summary_report_nt.2.update.sqlpp |  29 ---
 .../q01_pricing_summary_report_nt.3.query.sqlpp |  44 ----
 ...can-delete-rtree-secondary-index.1.ddl.sqlpp |  45 ----
 ...-delete-rtree-secondary-index.2.update.sqlpp |  29 ---
 ...can-delete-rtree-secondary-index.3.ddl.sqlpp |  29 ---
 ...-delete-rtree-secondary-index.4.update.sqlpp |  29 ---
 ...n-delete-rtree-secondary-index.5.query.sqlpp |  32 ---
 .../scan-insert-persistent-to-temp.1.ddl.sqlpp  |  54 ----
 ...can-insert-persistent-to-temp.2.update.sqlpp |  32 ---
 .../scan-insert-persistent-to-temp.3.ddl.sqlpp  |  32 ---
 ...can-insert-persistent-to-temp.4.update.sqlpp |  32 ---
 ...scan-insert-persistent-to-temp.5.query.sqlpp |  33 ---
 ...can-insert-rtree-secondary-index.1.ddl.sqlpp |  54 ----
 ...-insert-rtree-secondary-index.2.update.sqlpp |  32 ---
 ...can-insert-rtree-secondary-index.3.ddl.sqlpp |  32 ---
 ...-insert-rtree-secondary-index.4.update.sqlpp |  32 ---
 ...n-insert-rtree-secondary-index.5.query.sqlpp |  33 ---
 .../scan-insert-temp-to-persistent.1.ddl.sqlpp  |  54 ----
 ...can-insert-temp-to-persistent.2.update.sqlpp |  32 ---
 .../scan-insert-temp-to-persistent.3.ddl.sqlpp  |  32 ---
 ...can-insert-temp-to-persistent.4.update.sqlpp |  32 ---
 ...scan-insert-temp-to-persistent.5.query.sqlpp |  33 ---
 .../rebalance/all_datasets/all_datasets.11.adm  |   2 +-
 .../rebalance/all_datasets/all_datasets.12.adm  |   2 +-
 .../rebalance/all_datasets/all_datasets.5.adm   |   2 +-
 .../rebalance/all_datasets/all_datasets.6.adm   |   2 +-
 .../single_dataset/single_dataset.5.adm         |   2 +-
 .../single_dataset/single_dataset.9.adm         |   2 +-
 .../single_dataset_with_index.10.adm            |   2 +-
 .../single_dataset_with_index.5.adm             |   2 +-
 .../single_dataverse/single_dataverse.11.adm    |   2 +-
 .../single_dataverse/single_dataverse.12.adm    |   2 +-
 .../single_dataverse/single_dataverse.5.adm     |   2 +-
 .../single_dataverse/single_dataverse.6.adm     |   2 +-
 .../compact-dataset-and-its-indexes.1.adm       | 167 -------------
 .../delete-from-loaded-dataset-with-index.1.adm | 167 -------------
 .../delete-from-loaded-dataset.1.adm            |  25 --
 .../drop-empty-secondary-indexes.1.adm          |   0
 .../temp-dataset/drop-index/drop-index.1.adm    |   1 -
 .../empty-load-with-index.1.adm                 |   1 -
 .../insert-and-scan-dataset-with-index.1.adm    | 240 ------------------
 .../insert-and-scan-dataset.1.adm               |  20 --
 .../q01_pricing_summary_report_nt.1.adm         |   4 -
 .../scan-delete-rtree-secondary-index.1.adm     |   1 -
 .../scan-insert-persistent-to-temp.1.adm        |   3 -
 .../scan-insert-rtree-secondary-index.1.adm     |   3 -
 .../scan-insert-persistent-to-temp.1.adm        |   3 -
 .../temp_primary_plus_ngram_flush.3.adm         |   1 -
 .../delete-from-loaded-dataset-with-index.1.ast |  22 --
 .../delete-from-loaded-dataset-with-index.2.ast |   1 -
 .../delete-from-loaded-dataset-with-index.3.ast |   1 -
 .../delete-from-loaded-dataset-with-index.4.ast |   1 -
 .../delete-from-loaded-dataset-with-index.5.ast |  31 ---
 .../delete-from-loaded-dataset.1.ast            |  22 --
 .../delete-from-loaded-dataset.2.ast            |   1 -
 .../delete-from-loaded-dataset.3.ast            |  22 --
 .../drop-empty-secondary-indexes.1.ast          |  22 --
 .../drop-empty-secondary-indexes.2.ast          |   0
 .../drop-empty-secondary-indexes.3.ast          |  29 ---
 .../temp-dataset/drop-index/drop-index.1.ast    |  22 --
 .../temp-dataset/drop-index/drop-index.2.ast    |   1 -
 .../temp-dataset/drop-index/drop-index.3.ast    |   1 -
 .../temp-dataset/drop-index/drop-index.4.ast    |  30 ---
 .../empty-load-with-index.1.ast                 |  22 --
 .../empty-load-with-index.2.ast                 |   1 -
 .../empty-load-with-index.3.ast                 |  24 --
 .../insert-and-scan-dataset-with-index.1.ast    |  10 -
 .../insert-and-scan-dataset-with-index.2.ast    |   1 -
 .../insert-and-scan-dataset-with-index.3.ast    |  17 --
 .../insert-and-scan-dataset.1.ast               |   7 -
 .../insert-and-scan-dataset.2.ast               |   1 -
 .../insert-and-scan-dataset.3.ast               |  17 --
 .../q01_pricing_summary_report_nt.1.ast         |  22 --
 .../q01_pricing_summary_report_nt.2.ast         |   1 -
 .../q01_pricing_summary_report_nt.3.ast         | 249 -------------------
 .../scan-delete-rtree-secondary-index.1.ast     |  15 --
 .../scan-delete-rtree-secondary-index.2.ast     |   1 -
 .../scan-delete-rtree-secondary-index.3.ast     |   1 -
 .../scan-delete-rtree-secondary-index.4.ast     |   1 -
 .../scan-delete-rtree-secondary-index.5.ast     |  45 ----
 .../scan-insert-persistent-to-temp.1.ast        |  22 --
 .../scan-insert-persistent-to-temp.2.ast        |   1 -
 .../scan-insert-persistent-to-temp.3.ast        |   1 -
 .../scan-insert-persistent-to-temp.4.ast        |   1 -
 .../scan-insert-persistent-to-temp.5.ast        |  45 ----
 .../scan-insert-rtree-secondary-index.1.ast     |  22 --
 .../scan-insert-rtree-secondary-index.2.ast     |   1 -
 .../scan-insert-rtree-secondary-index.3.ast     |   1 -
 .../scan-insert-rtree-secondary-index.4.ast     |   1 -
 .../scan-insert-rtree-secondary-index.5.ast     |  45 ----
 .../scan-insert-temp-to-persistent.1.ast        |  22 --
 .../scan-insert-temp-to-persistent.2.ast        |   1 -
 .../scan-insert-temp-to-persistent.3.ast        |   1 -
 .../scan-insert-temp-to-persistent.4.ast        |   1 -
 .../scan-insert-temp-to-persistent.5.ast        |  45 ----
 .../src/test/resources/runtimets/testsuite.xml  |  72 ------
 .../resources/runtimets/testsuite_sqlpp.xml     |  67 +----
 .../runtimets/testsuite_sqlpp_parser.xml        |  62 -----
 .../asterix/common/utils/StoragePathUtil.java   |   1 -
 .../primary_index_only.1.script.aql             |  19 --
 .../primary_index_only.2.ddl.aql                |  59 -----
 .../primary_index_only.3.update.aql             |  29 ---
 .../primary_index_only.4.txneu.aql              |  40 ---
 .../primary_index_only.5.txnqbc.aql             |  28 ---
 .../primary_index_only.6.script.aql             |  19 --
 .../primary_index_only.7.script.aql             |  19 --
 .../primary_index_only.8.txnqar.aql             |  28 ---
 .../primary_index_only.9.script.aql             |  19 --
 ...ry_plus_default_secondary_index.1.script.aql |  19 --
 ...imary_plus_default_secondary_index.2.ddl.aql |  71 ------
 ...ry_plus_default_secondary_index.3.update.aql |  29 ---
 ...ary_plus_default_secondary_index.4.txneu.aql |  43 ----
 ...ry_plus_default_secondary_index.5.txnqbc.aql |  28 ---
 ...ry_plus_default_secondary_index.6.script.aql |  19 --
 ...ry_plus_default_secondary_index.7.script.aql |  19 --
 ...ry_plus_default_secondary_index.8.txnqar.aql |  28 ---
 ...ry_plus_default_secondary_index.9.script.aql |  19 --
 ...ry_plus_keyword_secondary_index.1.script.aql |  19 --
 ...imary_plus_keyword_secondary_index.2.ddl.aql |  72 ------
 ...ry_plus_keyword_secondary_index.3.update.aql |  30 ---
 ...ary_plus_keyword_secondary_index.4.txneu.aql |  43 ----
 ...ry_plus_keyword_secondary_index.5.txnqbc.aql |  23 --
 ...ry_plus_keyword_secondary_index.6.script.aql |  19 --
 ...ry_plus_keyword_secondary_index.7.script.aql |  19 --
 ...ry_plus_keyword_secondary_index.8.txnqar.aql |  23 --
 ...ry_plus_keyword_secondary_index.9.script.aql |  19 --
 ...plus_multiple_secondary_indices.1.script.aql |  19 --
 ...ry_plus_multiple_secondary_indices.2.ddl.aql |  78 ------
 ...plus_multiple_secondary_indices.3.update.aql |  30 ---
 ..._plus_multiple_secondary_indices.4.txneu.aql |  43 ----
 ...plus_multiple_secondary_indices.5.txnqbc.aql |  30 ---
 ...plus_multiple_secondary_indices.6.script.aql |  19 --
 ...plus_multiple_secondary_indices.7.script.aql |  19 --
 ...plus_multiple_secondary_indices.8.txnqar.aql |  23 --
 ...plus_multiple_secondary_indices.9.script.aql |  19 --
 .../primary_plus_ngram_index.1.script.aql       |  19 --
 .../primary_plus_ngram_index.2.ddl.aql          |  71 ------
 .../primary_plus_ngram_index.3.update.aql       |  29 ---
 .../primary_plus_ngram_index.4.txneu.aql        |  43 ----
 .../primary_plus_ngram_index.5.txnqbc.aql       |  29 ---
 .../primary_plus_ngram_index.6.script.aql       |  19 --
 .../primary_plus_ngram_index.7.script.aql       |  19 --
 .../primary_plus_ngram_index.8.txnqar.aql       |  29 ---
 .../primary_plus_ngram_index.9.script.aql       |  19 --
 .../primary_plus_rtree_index.1.script.aql       |  19 --
 .../primary_plus_rtree_index.2.ddl.aql          |  72 ------
 .../primary_plus_rtree_index.3.update.aql       |  30 ---
 .../primary_plus_rtree_index.4.txneu.aql        |  43 ----
 .../primary_plus_rtree_index.5.txnqbc.aql       |  23 --
 .../primary_plus_rtree_index.6.script.aql       |  19 --
 .../primary_plus_rtree_index.7.script.aql       |  19 --
 .../primary_plus_rtree_index.8.txnqar.aql       |  23 --
 .../primary_plus_rtree_index.9.script.aql       |  19 --
 .../dataset_recovery.1.script.aql               |  19 --
 .../dataset_recovery.2.ddl.aql                  |  37 ---
 .../dataset_recovery.3.script.aql               |  19 --
 .../dataset_recovery.4.script.aql               |  19 --
 .../dataset_recovery.5.errddl.aql               |  20 --
 .../dataset_recovery.6.script.aql               |  19 --
 .../delete_after_recovery.1.script.aql          |  19 --
 .../delete_after_recovery.2.ddl.aql             |  59 -----
 .../delete_after_recovery.3.update.aql          |  29 ---
 .../delete_after_recovery.4.script.aql          |  19 --
 .../delete_after_recovery.5.script.aql          |  19 --
 .../delete_after_recovery.6.update.aql          |  23 --
 .../delete_after_recovery.7.query.aql           |  29 ---
 .../delete_after_recovery.8.script.aql          |  19 --
 .../insert_after_recovery.1.script.aql          |  19 --
 .../insert_after_recovery.2.ddl.aql             |  59 -----
 .../insert_after_recovery.3.update.aql          |  29 ---
 .../insert_after_recovery.4.script.aql          |  19 --
 .../insert_after_recovery.5.script.aql          |  19 --
 .../insert_after_recovery.6.update.aql          |  33 ---
 .../insert_after_recovery.7.query.aql           |  29 ---
 .../insert_after_recovery.8.script.aql          |  19 --
 .../load_after_recovery.1.script.aql            |  19 --
 .../load_after_recovery.2.ddl.aql               |  59 -----
 .../load_after_recovery.3.script.aql            |  19 --
 .../load_after_recovery.4.script.aql            |  19 --
 .../load_after_recovery.5.update.aql            |  29 ---
 .../load_after_recovery.6.query.aql             |  28 ---
 .../load_after_recovery.7.script.aql            |  19 --
 .../secondary_index_recovery.1.script.aql       |  19 --
 .../secondary_index_recovery.2.ddl.aql          |  39 ---
 .../secondary_index_recovery.3.script.aql       |  19 --
 .../secondary_index_recovery.4.script.aql       |  19 --
 .../secondary_index_recovery.5.errddl.aql       |  20 --
 .../secondary_index_recovery.6.script.aql       |  19 --
 .../delete_after_recovery.1.adm                 |   1 -
 .../insert_after_recovery.1.adm                 |   1 -
 .../load_after_recovery.1.adm                   |   1 -
 .../temp_primary_index_only/create_and_start.sh |  18 --
 .../temp_primary_index_only/kill_cc_and_nc.sh   |  18 --
 .../temp_primary_index_only/stop_and_delete.sh  |  20 --
 .../temp_primary_index_only/stop_and_start.sh   |  20 --
 .../create_and_start.sh                         |  18 --
 .../kill_cc_and_nc.sh                           |  18 --
 .../stop_and_delete.sh                          |  20 --
 .../stop_and_start.sh                           |  20 --
 .../create_and_start.sh                         |  18 --
 .../kill_cc_and_nc.sh                           |  18 --
 .../stop_and_delete.sh                          |  20 --
 .../stop_and_start.sh                           |  20 --
 .../create_and_start.sh                         |  18 --
 .../kill_cc_and_nc.sh                           |  18 --
 .../stop_and_delete.sh                          |  20 --
 .../stop_and_start.sh                           |  20 --
 .../create_and_start.sh                         |  18 --
 .../kill_cc_and_nc.sh                           |  18 --
 .../stop_and_delete.sh                          |  20 --
 .../stop_and_start.sh                           |  20 --
 .../create_and_start.sh                         |  18 --
 .../kill_cc_and_nc.sh                           |  18 --
 .../stop_and_delete.sh                          |  20 --
 .../stop_and_start.sh                           |  20 --
 .../temp_dataset_recovery/create_and_start.sh   |  18 --
 .../temp_dataset_recovery/kill_cc_and_nc.sh     |  18 --
 .../temp_dataset_recovery/stop_and_delete.sh    |  20 --
 .../temp_dataset_recovery/stop_and_start.sh     |  20 --
 .../create_and_start.sh                         |  18 --
 .../kill_cc_and_nc.sh                           |  18 --
 .../stop_and_delete.sh                          |  20 --
 .../stop_and_start.sh                           |  20 --
 .../create_and_start.sh                         |  18 --
 .../kill_cc_and_nc.sh                           |  18 --
 .../stop_and_delete.sh                          |  20 --
 .../stop_and_start.sh                           |  20 --
 .../create_and_start.sh                         |  18 --
 .../temp_load_after_recovery/kill_cc_and_nc.sh  |  18 --
 .../temp_load_after_recovery/stop_and_delete.sh |  20 --
 .../temp_load_after_recovery/stop_and_start.sh  |  20 --
 .../create_and_start.sh                         |  18 --
 .../kill_cc_and_nc.sh                           |  18 --
 .../stop_and_delete.sh                          |  20 --
 .../stop_and_start.sh                           |  20 --
 .../test/resources/transactionts/testsuite.xml  |  74 ------
 .../asterix-lang-aql/src/main/javacc/AQL.jj     |  11 +-
 .../common/statement/ExternalDetailsDecl.java   |   5 -
 .../common/statement/IDatasetDetailsDecl.java   |   2 -
 .../common/statement/InternalDetailsDecl.java   |   9 +-
 .../lang/common/visitor/FormatPrintVisitor.java |   3 +-
 .../asterix-lang-sqlpp/src/main/javacc/SQLPP.jj |   8 +-
 .../asterix/metadata/GarbageCollector.java      |  67 -----
 .../asterix/metadata/IDatasetDetails.java       |  25 +-
 .../apache/asterix/metadata/MetadataCache.java  |  24 --
 .../asterix/metadata/MetadataManager.java       |  85 ++-----
 .../asterix/metadata/api/IMetadataManager.java  |   7 -
 .../metadata/bootstrap/MetadataBootstrap.java   |   2 +-
 .../declared/BTreeResourceFactoryProvider.java  |   7 +-
 .../metadata/declared/MetadataProvider.java     |  22 --
 .../asterix/metadata/entities/Dataset.java      |  59 ++---
 .../entities/ExternalDatasetDetails.java        |  12 -
 .../entities/InternalDatasetDetails.java        |  17 +-
 .../DatasetTupleTranslator.java                 |   7 +-
 .../asterix/metadata/utils/DatasetUtil.java     |   8 +-
 .../utils/ExternalIndexingOperations.java       |   2 +-
 .../InvertedIndexResourceFactoryProvider.java   |   3 +-
 .../utils/RTreeResourceFactoryProvider.java     |   5 +-
 ...daryCorrelatedTreeIndexOperationsHelper.java |   6 +-
 .../SecondaryTreeIndexOperationsHelper.java     |   2 +-
 .../utils/SplitsAndConstraintsUtil.java         |   4 +-
 .../DatasetTupleTranslatorTest.java             |   2 +-
 .../IndexTupleTranslatorTest.java               |   2 +-
 ...tasetIndexModificationOperationCallback.java |  54 ----
 ...dexModificationOperationCallbackFactory.java |  80 ------
 ...dexModificationOperationCallbackFactory.java |  81 ------
 .../PersistentLocalResourceRepository.java      |  27 +-
 .../management/runtime/CommitRuntime.java       |  34 +--
 .../runtime/CommitRuntimeFactory.java           |  11 +-
 370 files changed, 123 insertions(+), 9806 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ConnectorApiServlet.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ConnectorApiServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ConnectorApiServlet.java
index b51187c..ba7ee12 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ConnectorApiServlet.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ConnectorApiServlet.java
@@ -102,7 +102,6 @@ public class ConnectorApiServlet extends AbstractServlet {
                     out.flush();
                     return;
                 }
-                boolean temp = dataset.getDatasetDetails().isTemp();
                 FileSplit[] fileSplits = metadataProvider.splitsForIndex(mdTxnCtx, dataset, datasetName);
                 ARecordType recordType = (ARecordType) metadataProvider.findType(dataset.getItemTypeDataverseName(),
                         dataset.getItemTypeName());
@@ -115,7 +114,7 @@ public class ConnectorApiServlet extends AbstractServlet {
                 }
                 pkStrBuf.delete(pkStrBuf.length() - 1, pkStrBuf.length());
                 // Constructs the returned json object.
-                formResponseObject(jsonResponse, fileSplits, recordType, pkStrBuf.toString(), temp,
+                formResponseObject(jsonResponse, fileSplits, recordType, pkStrBuf.toString(),
                         hcc.getNodeControllerInfos());
 
                 // Flush the cached contents of the dataset to file system.
@@ -138,10 +137,8 @@ public class ConnectorApiServlet extends AbstractServlet {
     }
 
     private void formResponseObject(ObjectNode jsonResponse, FileSplit[] fileSplits, ARecordType recordType,
-            String primaryKeys, boolean temp, Map<String, NodeControllerInfo> nodeMap) {
+            String primaryKeys, Map<String, NodeControllerInfo> nodeMap) {
         ArrayNode partititons = OBJECT_MAPPER.createArrayNode();
-        // Whether the dataset is temp or not
-        jsonResponse.put("temp", temp);
         // Adds a primary key.
         jsonResponse.put("keys", primaryKeys);
         // Adds record type.

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java
index 05debaa..2fc3258 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java
@@ -529,8 +529,6 @@ public class QueryTranslator extends AbstractLangTranslator implements IStatemen
         String compactionPolicy = dd.getCompactionPolicy();
         Map<String, String> compactionPolicyProperties = dd.getCompactionPolicyProperties();
         boolean defaultCompactionPolicy = compactionPolicy == null;
-        boolean temp = dd.getDatasetDetailsDecl().isTemp();
-
         MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
         boolean bActiveTxn = true;
         metadataProvider.setMetadataTxnContext(mdTxnCtx);
@@ -602,7 +600,7 @@ public class QueryTranslator extends AbstractLangTranslator implements IStatemen
                     }
                     datasetDetails = new InternalDatasetDetails(InternalDatasetDetails.FileStructure.BTREE,
                             InternalDatasetDetails.PartitioningStrategy.HASH, partitioningExprs, partitioningExprs,
-                            keySourceIndicators, partitioningTypes, autogenerated, filterField, temp);
+                            keySourceIndicators, partitioningTypes, autogenerated, filterField);
                     break;
                 case EXTERNAL:
                     String adapter = ((ExternalDetailsDecl) dd.getDatasetDetailsDecl()).getAdapter();
@@ -2874,7 +2872,7 @@ public class QueryTranslator extends AbstractLangTranslator implements IStatemen
                     new DropDatasetStatement(new Identifier(dataverseNameTo), pregelixStmt.getDatasetNameTo(), true);
             this.handleDatasetDropStatement(metadataProvider, dropStmt, hcc, null);
             IDatasetDetailsDecl idd = new InternalDetailsDecl(toIndex.getKeyFieldNames(),
-                    toIndex.getKeyFieldSourceIndicators(), false, null, toDataset.getDatasetDetails().isTemp());
+                    toIndex.getKeyFieldSourceIndicators(), false, null);
             DatasetDecl createToDataset = new DatasetDecl(new Identifier(dataverseNameTo),
                     pregelixStmt.getDatasetNameTo(), new Identifier(toDataset.getItemTypeDataverseName()),
                     new Identifier(toDataset.getItemTypeName()),

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/NCApplication.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/NCApplication.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/NCApplication.java
index a18535d..63f5bfc 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/NCApplication.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/NCApplication.java
@@ -226,19 +226,6 @@ public class NCApplication extends BaseNCApplication {
         //Delete working area files from failed jobs
         runtimeContext.getIoManager().deleteWorkspaceFiles();
 
-        //Reclaim storage for temporary datasets.
-        String storageDirName = ClusterProperties.INSTANCE.getStorageDirectoryName();
-        String[] ioDevices = ((PersistentLocalResourceRepository) runtimeContext.getLocalResourceRepository())
-                .getStorageMountingPoints();
-        for (String ioDevice : ioDevices) {
-            String tempDatasetsDir =
-                    ioDevice + storageDirName + File.separator + StoragePathUtil.TEMP_DATASETS_STORAGE_FOLDER;
-            File tmpDsDir = new File(tempDatasetsDir);
-            if (tmpDsDir.exists()) {
-                IoUtil.delete(tmpDsDir);
-            }
-        }
-
         //TODO
         //Reclaim storage for orphaned index artifacts in NCs.
         //Note: currently LSM indexes invalid components are deleted when an index is activated.

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/http/servlet/ConnectorApiServletTest.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/http/servlet/ConnectorApiServletTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/http/servlet/ConnectorApiServletTest.java
index cd58d8f..33f3a42 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/http/servlet/ConnectorApiServletTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/http/servlet/ConnectorApiServletTest.java
@@ -113,9 +113,7 @@ public class ConnectorApiServletTest {
         ObjectMapper om = new ObjectMapper();
         ObjectNode actualResponse = (ObjectNode) om.readTree(outputStream.toString());
 
-        // Checks the temp-or-not, primary key, data type of the dataset.
-        boolean temp = actualResponse.get("temp").asBoolean();
-        Assert.assertFalse(temp);
+        // Checks the primary key, data type of the dataset.
         String primaryKey = actualResponse.get("keys").asText();
         Assert.assertEquals("DataverseName,DatasetName", primaryKey);
         ARecordType recordType = (ARecordType) JSONDeserializerForTypes.convertFromJSON(actualResponse.get("type"));
@@ -154,12 +152,10 @@ public class ConnectorApiServletTest {
         nodeMap.put("asterix_nc2", mockInfo2);
         PA.invokeMethod(let,
                 "formResponseObject(" + ObjectNode.class.getName() + ", " + FileSplit.class.getName() + "[], "
-                        + ARecordType.class.getName() + ", " + String.class.getName() + ", boolean, "
-                        + Map.class.getName() + ")",
-                actualResponse, splits, recordType, primaryKey, true, nodeMap);
+                        + ARecordType.class.getName() + ", " + String.class.getName() + ", " + Map.class.getName() + ")",
+                actualResponse, splits, recordType, primaryKey, nodeMap);
         // Constructs expected response.
         ObjectNode expectedResponse = om.createObjectNode();
-        expectedResponse.put("temp", true);
         expectedResponse.put("keys", primaryKey);
         expectedResponse.set("type", recordType.toJSON());
         ArrayNode splitsArray = om.createArrayNode();

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/bootstrap/TestNodeController.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/bootstrap/TestNodeController.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/bootstrap/TestNodeController.java
index 505ef8f..72a9b44 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/bootstrap/TestNodeController.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/bootstrap/TestNodeController.java
@@ -189,7 +189,7 @@ public class TestNodeController {
                         op, true, indexHelperFactory, modOpCallbackFactory, null);
         CommitRuntime commitOp =
                 new CommitRuntime(ctx, getTxnJobId(ctx), dataset.getDatasetId(), primaryIndexInfo.primaryKeyIndexes,
-                        false, true, ctx.getTaskAttemptId().getTaskId().getPartition(), true);
+                        true, ctx.getTaskAttemptId().getTaskId().getPartition(), true);
         insertOp.setOutputFrameWriter(0, commitOp, primaryIndexInfo.rDesc);
         commitOp.setInputRecordDescriptor(0, primaryIndexInfo.rDesc);
         return Pair.of(insertOp, commitOp);
@@ -261,7 +261,7 @@ public class TestNodeController {
                     recordType, metaType, mergePolicy.first, mergePolicy.second);
             IndexBuilderFactory indexBuilderFactory =
                     new IndexBuilderFactory(storageComponentProvider.getStorageManager(),
-                            primaryIndexInfo.getFileSplitProvider(), resourceFactory, !dataset.isTemp());
+                            primaryIndexInfo.getFileSplitProvider(), resourceFactory, true);
             IHyracksTaskContext ctx = createTestContext(newJobId(), partition, false);
             IIndexBuilder indexBuilder = indexBuilderFactory.create(ctx, partition);
             indexBuilder.build();
@@ -475,7 +475,7 @@ public class TestNodeController {
             pkFieldsInCommitOp[i] = diff + i;
         }
         CommitRuntime commitOp = new CommitRuntime(ctx, getTxnJobId(ctx), dataset.getDatasetId(), pkFieldsInCommitOp,
-                false, true, ctx.getTaskAttemptId().getTaskId().getPartition(), true);
+                true, ctx.getTaskAttemptId().getTaskId().getPartition(), true);
         insertOp.setOutputFrameWriter(0, commitOp, upsertOutRecDesc);
         commitOp.setInputRecordDescriptor(0, upsertOutRecDesc);
         return Pair.of(insertOp, commitOp);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/ComponentRollbackTest.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/ComponentRollbackTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/ComponentRollbackTest.java
index 1ac1aa6..70436b5 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/ComponentRollbackTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/ComponentRollbackTest.java
@@ -137,7 +137,7 @@ public class ComponentRollbackTest {
         int partition = 0;
         dataset = new TestDataset(DATAVERSE_NAME, DATASET_NAME, DATAVERSE_NAME, DATA_TYPE_NAME, NODE_GROUP_NAME,
                 NoMergePolicyFactory.NAME, null, new InternalDatasetDetails(null, PartitioningStrategy.HASH,
-                        partitioningKeys, null, null, null, false, null, false),
+                        partitioningKeys, null, null, null, false, null),
                 null, DatasetType.INTERNAL, DATASET_ID, 0);
         PrimaryIndexInfo primaryIndexInfo = nc.createPrimaryIndex(dataset, KEY_TYPES, RECORD_TYPE, META_TYPE, null,
                 storageManager, KEY_INDEXES, KEY_INDICATORS_LIST, partition);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/LogMarkerTest.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/LogMarkerTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/LogMarkerTest.java
index aa08c2d..963cded 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/LogMarkerTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/LogMarkerTest.java
@@ -114,7 +114,7 @@ public class LogMarkerTest {
             partitioningKeys.add(Collections.singletonList("key"));
             Dataset dataset = new Dataset(DATAVERSE_NAME, DATASET_NAME, DATAVERSE_NAME, DATA_TYPE_NAME, NODE_GROUP_NAME,
                     NoMergePolicyFactory.NAME, null, new InternalDatasetDetails(null, PartitioningStrategy.HASH,
-                            partitioningKeys, null, null, null, false, null, false),
+                            partitioningKeys, null, null, null, false, null),
                     null, DatasetType.INTERNAL, DATASET_ID, 0);
             try {
                 PrimaryIndexInfo indexInfo = nc.createPrimaryIndex(dataset, KEY_TYPES, RECORD_TYPE, META_TYPE, null,

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/MultiPartitionLSMIndexTest.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/MultiPartitionLSMIndexTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/MultiPartitionLSMIndexTest.java
index ec9c2f6..2c8141ce 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/MultiPartitionLSMIndexTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/MultiPartitionLSMIndexTest.java
@@ -115,7 +115,7 @@ public class MultiPartitionLSMIndexTest {
         partitioningKeys.add(Collections.singletonList("key"));
         dataset = new TestDataset(DATAVERSE_NAME, DATASET_NAME, DATAVERSE_NAME, DATA_TYPE_NAME, NODE_GROUP_NAME,
                 NoMergePolicyFactory.NAME, null, new InternalDatasetDetails(null, PartitioningStrategy.HASH,
-                        partitioningKeys, null, null, null, false, null, false),
+                        partitioningKeys, null, null, null, false, null),
                 null, DatasetType.INTERNAL, DATASET_ID, 0);
         taskCtxs = new IHyracksTaskContext[NUM_PARTITIONS];
         indexDataflowHelpers = new IIndexDataflowHelper[NUM_PARTITIONS];

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/SearchCursorComponentSwitchTest.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/SearchCursorComponentSwitchTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/SearchCursorComponentSwitchTest.java
index 77d3795..b65ba03 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/SearchCursorComponentSwitchTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/SearchCursorComponentSwitchTest.java
@@ -123,7 +123,7 @@ public class SearchCursorComponentSwitchTest {
         partitioningKeys.add(Collections.singletonList("key"));
         dataset = new TestDataset(DATAVERSE_NAME, DATASET_NAME, DATAVERSE_NAME, DATA_TYPE_NAME, NODE_GROUP_NAME,
                 NoMergePolicyFactory.NAME, null, new InternalDatasetDetails(null, PartitioningStrategy.HASH,
-                        partitioningKeys, null, null, null, false, null, false),
+                        partitioningKeys, null, null, null, false, null),
                 null, DatasetType.INTERNAL, DATASET_ID, 0);
         PrimaryIndexInfo primaryIndexInfo = nc.createPrimaryIndex(dataset, KEY_TYPES, RECORD_TYPE, META_TYPE, null,
                 storageManager, KEY_INDEXES, KEY_INDICATORS_LIST, 0);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TestLsmBTreeResourceFactoryProvider.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TestLsmBTreeResourceFactoryProvider.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TestLsmBTreeResourceFactoryProvider.java
index 4511f42..7268296 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TestLsmBTreeResourceFactoryProvider.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TestLsmBTreeResourceFactoryProvider.java
@@ -66,7 +66,6 @@ public class TestLsmBTreeResourceFactoryProvider implements IResourceFactoryProv
         ITypeTraits[] typeTraits = getTypeTraits(mdProvider, dataset, index, recordType, metaType);
         IBinaryComparatorFactory[] cmpFactories = getCmpFactories(mdProvider, dataset, index, recordType, metaType);
         int[] bloomFilterFields = getBloomFilterFields(dataset, index);
-        boolean durable = !dataset.isTemp();
         double bloomFilterFalsePositiveRate = mdProvider.getStorageProperties().getBloomFilterFalsePositiveRate();
         ILSMOperationTrackerFactory opTrackerFactory = dataset.getIndexOperationTrackerFactory(index);
         ILSMIOOperationCallbackFactory ioOpCallbackFactory = dataset.getIoOperationCallbackFactory(index);
@@ -78,7 +77,7 @@ public class TestLsmBTreeResourceFactoryProvider implements IResourceFactoryProv
         AsterixVirtualBufferCacheProvider vbcProvider = new AsterixVirtualBufferCacheProvider(dataset.getDatasetId());
         return new TestLsmBtreeLocalResourceFactory(storageManager, typeTraits, cmpFactories, filterTypeTraits,
                 filterCmpFactories, filterFields, opTrackerFactory, ioOpCallbackFactory, metadataPageManagerFactory,
-                vbcProvider, ioSchedulerProvider, mergePolicyFactory, mergePolicyProperties, durable, bloomFilterFields,
+                vbcProvider, ioSchedulerProvider, mergePolicyFactory, mergePolicyProperties, true, bloomFilterFields,
                 bloomFilterFalsePositiveRate, index.isPrimaryIndex(), btreeFields);
     }
 

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/logging/CheckpointingTest.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/logging/CheckpointingTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/logging/CheckpointingTest.java
index 1be1d26..30cfb4f 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/logging/CheckpointingTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/logging/CheckpointingTest.java
@@ -123,7 +123,7 @@ public class CheckpointingTest {
             partitioningKeys.add(Collections.singletonList("key"));
             Dataset dataset = new Dataset(DATAVERSE_NAME, DATASET_NAME, DATAVERSE_NAME, DATA_TYPE_NAME, NODE_GROUP_NAME,
                     NoMergePolicyFactory.NAME, null, new InternalDatasetDetails(null, PartitioningStrategy.HASH,
-                            partitioningKeys, null, null, null, false, null, false),
+                            partitioningKeys, null, null, null, false, null),
                     null, DatasetType.INTERNAL, DATASET_ID, 0);
             try {
                 nc.createPrimaryIndex(dataset, KEY_TYPES, RECORD_TYPE, META_TYPE, null, storageManager, KEY_INDEXES,

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/storage/DiskIsFullTest.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/storage/DiskIsFullTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/storage/DiskIsFullTest.java
index 8e8b3e9..dabf9d3 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/storage/DiskIsFullTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/storage/DiskIsFullTest.java
@@ -121,7 +121,7 @@ public class DiskIsFullTest {
             partitioningKeys.add(Collections.singletonList("key"));
             Dataset dataset = new Dataset(DATAVERSE_NAME, DATASET_NAME, DATAVERSE_NAME, DATA_TYPE_NAME, NODE_GROUP_NAME,
                     NoMergePolicyFactory.NAME, null, new InternalDatasetDetails(null, PartitioningStrategy.HASH,
-                            partitioningKeys, null, null, null, false, null, false),
+                            partitioningKeys, null, null, null, false, null),
                     null, DatasetType.INTERNAL, DATASET_ID, 0);
             try {
                 nc.createPrimaryIndex(dataset, KEY_TYPES, RECORD_TYPE, META_TYPE, null, storageManager, KEY_INDEXES,

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/storage/IndexDropOperatorNodePushableTest.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/storage/IndexDropOperatorNodePushableTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/storage/IndexDropOperatorNodePushableTest.java
index cc6c0f7..402da6c 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/storage/IndexDropOperatorNodePushableTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/storage/IndexDropOperatorNodePushableTest.java
@@ -106,7 +106,7 @@ public class IndexDropOperatorNodePushableTest {
             Dataset dataset = new Dataset(DATAVERSE_NAME, DATASET_NAME, DATAVERSE_NAME, DATA_TYPE_NAME, NODE_GROUP_NAME,
                     NoMergePolicyFactory.NAME, null,
                     new InternalDatasetDetails(null, InternalDatasetDetails.PartitioningStrategy.HASH, partitioningKeys,
-                            null, null, null, false, null, false),
+                            null, null, null, false, null),
                     null, DatasetConfig.DatasetType.INTERNAL, DATASET_ID, 0);
             // create dataset
             TestNodeController.PrimaryIndexInfo indexInfo = nc.createPrimaryIndex(dataset, KEY_TYPES, RECORD_TYPE,

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/metadata/queries/basic/temp_dataset/temp_dataset.1.ddl.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/metadata/queries/basic/temp_dataset/temp_dataset.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/metadata/queries/basic/temp_dataset/temp_dataset.1.ddl.aql
deleted file mode 100644
index 6674622..0000000
--- a/asterixdb/asterix-app/src/test/resources/metadata/queries/basic/temp_dataset/temp_dataset.1.ddl.aql
+++ /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.
- */
-/*
- * Description  : Create a temporary dataset and verify the dataset does not exist in the metadata store
- * Expected Res : Success
- * Date         : March 12 2015
- */
-
-drop dataverse test if exists;
-create dataverse test;
-
-use dataverse test;
-
-create type LineItemType as closed {
-  l_orderkey: int64,
-  l_partkey: int64,
-  l_suppkey: int64,
-  l_linenumber: int64,
-  l_quantity: double,
-  l_extendedprice: double,
-  l_discount: double,
-  l_tax: double,
-  l_returnflag: string,
-  l_linestatus: string,
-  l_shipdate: string,
-  l_commitdate: string,
-  l_receiptdate: string,
-  l_shipinstruct: string,
-  l_shipmode: string,
-  l_comment: string
-}
-
-create temporary dataset LineItem(LineItemType)
-  primary key l_orderkey, l_linenumber;
-
-

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/metadata/queries/basic/temp_dataset/temp_dataset.2.update.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/metadata/queries/basic/temp_dataset/temp_dataset.2.update.aql b/asterixdb/asterix-app/src/test/resources/metadata/queries/basic/temp_dataset/temp_dataset.2.update.aql
deleted file mode 100644
index 3e06e46..0000000
--- a/asterixdb/asterix-app/src/test/resources/metadata/queries/basic/temp_dataset/temp_dataset.2.update.aql
+++ /dev/null
@@ -1,24 +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.
- */
-/*
- * Description  : Create dataverse and drop that dataverse and verify dataverse entries in metadata
- * Expected Res : Success
- * Date         : Sep 17 2012
- */
-

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/metadata/queries/basic/temp_dataset/temp_dataset.3.query.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/metadata/queries/basic/temp_dataset/temp_dataset.3.query.aql b/asterixdb/asterix-app/src/test/resources/metadata/queries/basic/temp_dataset/temp_dataset.3.query.aql
deleted file mode 100644
index 9d77309..0000000
--- a/asterixdb/asterix-app/src/test/resources/metadata/queries/basic/temp_dataset/temp_dataset.3.query.aql
+++ /dev/null
@@ -1,29 +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.
- */
-/*
- * Description  : Create dataverse and drop that dataverse and verify dataverse entries in metadata
- * Expected Res : Success
- * Date         : Sep 17 2012
- */
-
-
-for $l in dataset('Metadata.Dataset')
-where $l.DataverseName = 'test'
-return $l;
-

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/metadata/results/basic/temp_dataset/temp_dataset.1.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/metadata/results/basic/temp_dataset/temp_dataset.1.adm b/asterixdb/asterix-app/src/test/resources/metadata/results/basic/temp_dataset/temp_dataset.1.adm
deleted file mode 100644
index 8b13789..0000000
--- a/asterixdb/asterix-app/src/test/resources/metadata/results/basic/temp_dataset/temp_dataset.1.adm
+++ /dev/null
@@ -1 +0,0 @@
-

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/metadata/testsuite.xml
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/metadata/testsuite.xml b/asterixdb/asterix-app/src/test/resources/metadata/testsuite.xml
index 02efe62..036a4bc 100644
--- a/asterixdb/asterix-app/src/test/resources/metadata/testsuite.xml
+++ b/asterixdb/asterix-app/src/test/resources/metadata/testsuite.xml
@@ -324,11 +324,6 @@
       </compilation-unit>
     </test-case>
     <test-case FilePath="basic">
-      <compilation-unit name="temp_dataset">
-        <output-dir compare="Text">temp_dataset</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="basic">
       <compilation-unit name="issue_1419_drop_type_with_collection_1">
         <output-dir compare="Text">issue_1419_drop_type_with_collection</output-dir>
       </compilation-unit>

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/delete-from-loaded-dataset-with-index/delete-from-loaded-dataset-with-index.1.ddl.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/delete-from-loaded-dataset-with-index/delete-from-loaded-dataset-with-index.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/delete-from-loaded-dataset-with-index/delete-from-loaded-dataset-with-index.1.ddl.aql
deleted file mode 100644
index 3d529fb..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/delete-from-loaded-dataset-with-index/delete-from-loaded-dataset-with-index.1.ddl.aql
+++ /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.
- */
-/*
- * Description     : Delete a portion of data from a loaded temporary dataset with indexes.
- * Expected Result : Success
- * Date            : 27th March 2015
- *
- */
-
-drop dataverse test if exists;
-create dataverse test;
-
-use dataverse test;
-
-create type LineItemType as closed {
-  l_orderkey: int64,
-  l_partkey: int64,
-  l_suppkey: int64,
-  l_linenumber: int64,
-  l_quantity: int64,
-  l_extendedprice: double,
-  l_discount: double,
-  l_tax: double,
-  l_returnflag: string,
-  l_linestatus: string,
-  l_shipdate: string,
-  l_commitdate: string,
-  l_receiptdate: string,
-  l_shipinstruct: string,
-  l_shipmode: string,
-  l_comment: string
-}
-
-create temporary dataset LineItem(LineItemType)
-  primary key l_orderkey, l_linenumber;
-

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/delete-from-loaded-dataset-with-index/delete-from-loaded-dataset-with-index.2.update.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/delete-from-loaded-dataset-with-index/delete-from-loaded-dataset-with-index.2.update.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/delete-from-loaded-dataset-with-index/delete-from-loaded-dataset-with-index.2.update.aql
deleted file mode 100644
index bc2b6ee..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/delete-from-loaded-dataset-with-index/delete-from-loaded-dataset-with-index.2.update.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.
- */
-/*
- * Description     : Delete a portion of data from a loaded temporary dataset with indexes.
- * Expected Result : Success
- * Date            : 27th March 2015
- *
- */
-
-use dataverse test;
-
-load dataset LineItem 
-using localfs
-(("path"="asterix_nc1://data/tpch0.001/lineitem.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
-

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/delete-from-loaded-dataset-with-index/delete-from-loaded-dataset-with-index.3.ddl.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/delete-from-loaded-dataset-with-index/delete-from-loaded-dataset-with-index.3.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/delete-from-loaded-dataset-with-index/delete-from-loaded-dataset-with-index.3.ddl.aql
deleted file mode 100644
index 496ce13..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/delete-from-loaded-dataset-with-index/delete-from-loaded-dataset-with-index.3.ddl.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.
- */
-/*
- * Description     : Delete a portion of data from a loaded temporary dataset with indexes.
- * Expected Result : Success
- * Date            : 27th March 2015
- *
- */
-
-use dataverse test;
-
-create index idx_LineItem_partkey on LineItem(l_linenumber);
-create index idx_LineItem_suppkey on LineItem(l_suppkey);
-

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/delete-from-loaded-dataset-with-index/delete-from-loaded-dataset-with-index.4.update.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/delete-from-loaded-dataset-with-index/delete-from-loaded-dataset-with-index.4.update.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/delete-from-loaded-dataset-with-index/delete-from-loaded-dataset-with-index.4.update.aql
deleted file mode 100644
index edc4af6..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/delete-from-loaded-dataset-with-index/delete-from-loaded-dataset-with-index.4.update.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.
- */
-/*
- * Description     : Delete a portion of data from a loaded temporary dataset with indexes.
- * Expected Result : Success
- * Date            : 27th March 2015
- *
- */
-
-use dataverse test;
-
-delete $l from dataset LineItem where $l.l_suppkey>=2 or $l.l_linenumber>1;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/delete-from-loaded-dataset-with-index/delete-from-loaded-dataset-with-index.5.query.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/delete-from-loaded-dataset-with-index/delete-from-loaded-dataset-with-index.5.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/delete-from-loaded-dataset-with-index/delete-from-loaded-dataset-with-index.5.query.aql
deleted file mode 100644
index 88d4a86..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/delete-from-loaded-dataset-with-index/delete-from-loaded-dataset-with-index.5.query.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.
- */
-/*
- * Description     : Delete a portion of data from a loaded temporary dataset with indexes.
- * Expected Result : Success
- * Date            : 27th March 2015
- *
- */
-
-use dataverse test;
-     
-for $c in dataset('LineItem')
-where $c.l_suppkey<150
-order by $c.l_orderkey, $c.l_linenumber
-return $c

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/delete-from-loaded-dataset/delete-from-loaded-dataset.1.ddl.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/delete-from-loaded-dataset/delete-from-loaded-dataset.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/delete-from-loaded-dataset/delete-from-loaded-dataset.1.ddl.aql
deleted file mode 100644
index 7cf6796..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/delete-from-loaded-dataset/delete-from-loaded-dataset.1.ddl.aql
+++ /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.
- */
-/*
- * Description     : Delete a portion of data from a loaded temporary dataset.
- * Expected Result : Success
- * Date            : 27th March 2015
- *
- */
-
-drop dataverse test if exists;
-create dataverse test;
-
-use dataverse test;
-
-create type LineItemType as closed {
-  l_orderkey: int64,
-  l_partkey: int64,
-  l_suppkey: int64,
-  l_linenumber: int64,
-  l_quantity: int64,
-  l_extendedprice: double,
-  l_discount: double,
-  l_tax: double,
-  l_returnflag: string,
-  l_linestatus: string,
-  l_shipdate: string,
-  l_commitdate: string,
-  l_receiptdate: string,
-  l_shipinstruct: string,
-  l_shipmode: string,
-  l_comment: string
-}
-
-create temporary dataset LineItem(LineItemType)
-  primary key l_orderkey, l_linenumber;
-

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/delete-from-loaded-dataset/delete-from-loaded-dataset.2.update.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/delete-from-loaded-dataset/delete-from-loaded-dataset.2.update.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/delete-from-loaded-dataset/delete-from-loaded-dataset.2.update.aql
deleted file mode 100644
index e6b98c7..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/delete-from-loaded-dataset/delete-from-loaded-dataset.2.update.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.
- */
-/*
- * Description     : Delete a portion of data from a loaded temporary dataset.
- * Expected Result : Success
- * Date            : 27th March 2015
- *
- */
-
-use dataverse test;
-
-load dataset LineItem 
-using localfs
-(("path"="asterix_nc1://data/tpch0.001/lineitem.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
-
-delete $l from dataset LineItem where $l.l_orderkey>=10;
-

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/delete-from-loaded-dataset/delete-from-loaded-dataset.3.query.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/delete-from-loaded-dataset/delete-from-loaded-dataset.3.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/delete-from-loaded-dataset/delete-from-loaded-dataset.3.query.aql
deleted file mode 100644
index c88cf8c..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/delete-from-loaded-dataset/delete-from-loaded-dataset.3.query.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.
- */
-/*
- * Description     : Delete a portion of data from a loaded temporary dataset.
- * Expected Result : Success
- * Date            : 27th March 2015
- *
- */
-
-use dataverse test;
-  
-for $c in dataset('LineItem')
-order by $c.l_orderkey, $c.l_linenumber
-return $c 

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/drop-empty-secondary-indexes/drop-empty-secondary-indexes.1.ddl.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/drop-empty-secondary-indexes/drop-empty-secondary-indexes.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/drop-empty-secondary-indexes/drop-empty-secondary-indexes.1.ddl.aql
deleted file mode 100644
index 3b6c965..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/drop-empty-secondary-indexes/drop-empty-secondary-indexes.1.ddl.aql
+++ /dev/null
@@ -1,75 +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.
- */
-/*
- * Description     : Drop empty secondary indexes for a temporary dataset.
- * Expected Result : Success
- * Date            : 27th March 2015
- *
- */
-
-drop dataverse test if exists;
-create dataverse test;
-
-use dataverse test;
-create type Name as open {
-first : string,
-last : string
-}
-
-create type Person as open {
-name : Name
-}
-
-create type TestType as open {
-id : int32,
-name : string,
-locn : point,
-zip : string,
-person : Person
-}
-
-create temporary dataset t1(TestType) primary key id;
-
-create index rtree_index_point on t1(locn) type rtree;
-
-create index rtree_index_point_open on t1(open_locn:point?) type rtree enforced;
-
-create index keyWD_indx on t1(name) type keyword;
-
-create index keyWD_indx_open on t1(nickname:string?) type keyword enforced;
-
-create index secndIndx on t1(zip);
-
-create index nested on t1(person.name.first);
-
-create index secndIndx_open on t1(address:string?) enforced;
-
-drop index t1.rtree_index_point;
-
-drop index t1.rtree_index_point_open;
-
-drop index t1.keyWD_indx;
-
-drop index t1.keyWD_indx_open;
-
-drop index t1.secndIndx;
-
-drop index t1.nested;
-
-drop index t1.secndIndx_open;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/drop-empty-secondary-indexes/drop-empty-secondary-indexes.2.update.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/drop-empty-secondary-indexes/drop-empty-secondary-indexes.2.update.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/drop-empty-secondary-indexes/drop-empty-secondary-indexes.2.update.aql
deleted file mode 100644
index ebe3e6a..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/drop-empty-secondary-indexes/drop-empty-secondary-indexes.2.update.aql
+++ /dev/null
@@ -1,25 +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.
- */
-/*
- * Description     : Drop empty secondary indexes for a temporary dataset.
- * Expected Result : Success
- * Date            : 27th March 2015
- *
- */
-

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/drop-empty-secondary-indexes/drop-empty-secondary-indexes.3.query.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/drop-empty-secondary-indexes/drop-empty-secondary-indexes.3.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/drop-empty-secondary-indexes/drop-empty-secondary-indexes.3.query.aql
deleted file mode 100644
index 61f4309..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/drop-empty-secondary-indexes/drop-empty-secondary-indexes.3.query.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.
- */
-/*
- * Description     : Drop empty secondary indexes for a temporary dataset.
- * Expected Result : Success
- * Date            : 27th March 2015
- *
- */
-
-for $l in dataset('Metadata.Index')
-where $l.IsPrimary=false and $l.DataverseName='test'
-return $l;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/drop-index/drop-index.1.ddl.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/drop-index/drop-index.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/drop-index/drop-index.1.ddl.aql
deleted file mode 100644
index 887b004..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/drop-index/drop-index.1.ddl.aql
+++ /dev/null
@@ -1,51 +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.
- */
-/*
- * Description     : Drop secondary indexes for a temporary dataset.
- * Expected Result : Success
- * Date            : 27th March 2015
- *
- */
-
-drop dataverse test if exists;
-create dataverse test;
-
-use dataverse test;
-
-create type Schema as closed {
-unique1:  int64,
-unique2:  int64,
-two:  int64,
-four:  int64,
-ten:  int64,
-twenty:  int64,
-onePercent: int64,
-tenPercent:  int64,
-twentyPercent:  int64,
-fiftyPercent:  int64,
-unique3:  int64,
-evenOnePercent: int64,
-oddOnePercent:  int64,
-stringu1:  string,
-stringu2:  string,
-string4:  string
-}
-
-create temporary dataset t1(Schema) primary key unique2;
-

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/drop-index/drop-index.2.update.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/drop-index/drop-index.2.update.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/drop-index/drop-index.2.update.aql
deleted file mode 100644
index bf556a0..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/drop-index/drop-index.2.update.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.
- */
-/*
- * Description     : Drop secondary indexes for a temporary dataset.
- * Expected Result : Success
- * Date            : 27th March 2015
- *
- */
-
-use dataverse test;
-
-// Load data
-load dataset t1
-using localfs
-(("path"="asterix_nc1://data/wisc/onektup.adm"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
-

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/drop-index/drop-index.3.ddl.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/drop-index/drop-index.3.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/drop-index/drop-index.3.ddl.aql
deleted file mode 100644
index a66a9e5..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/drop-index/drop-index.3.ddl.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.
- */
-/*
- * Description     : Drop secondary indexes for a temporary dataset.
- * Expected Result : Success
- * Date            : 27th March 2015
- *
- */
-
-use dataverse test;
-
-// create secondary indexes
-create index idx_t1_str1 on t1(stringu1);
-create index idx_t1_unique1 on t1(unique1);
-
-// drop secondary indexes
-drop index t1.idx_t1_str1;
-drop index t1.idx_t1_unique1;
-

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/drop-index/drop-index.4.query.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/drop-index/drop-index.4.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/drop-index/drop-index.4.query.aql
deleted file mode 100644
index 7a17743..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/drop-index/drop-index.4.query.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.
- */
-/*
- * Description     : Drop secondary indexes for a temporary dataset.
- * Expected Result : Success
- * Date            : 27th March 2015
- *
- */
-
-use dataverse test;
-
-for $a in dataset('t1')
-where $a.unique1 > 10 and $a.stringu1="DGAAAAXXXXXXXXXXXXXXXXXXX"
-return $a

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/empty-load-with-index/empty-load-with-index.1.ddl.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/empty-load-with-index/empty-load-with-index.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/empty-load-with-index/empty-load-with-index.1.ddl.aql
deleted file mode 100644
index d1f5a7a..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/empty-load-with-index/empty-load-with-index.1.ddl.aql
+++ /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.
- */
-/*
- * Description     : Load a temporary dataset with previously created empty indexes.
- * Expected Result : Success
- * Date            : March 27 2015
- */
-
-drop dataverse test if exists;
-create dataverse test;
-
-use dataverse test;
-
-create type LineItemType as closed {
-  l_orderkey: int64,
-  l_partkey: int64,
-  l_suppkey: int64,
-  l_linenumber: int64,
-  l_quantity: double,
-  l_extendedprice: double,
-  l_discount: double,
-  l_tax: double,
-  l_returnflag: string,
-  l_linestatus: string,
-  l_shipdate: string,
-  l_commitdate: string,
-  l_receiptdate: string,
-  l_shipinstruct: string,
-  l_shipmode: string,
-  l_comment: string
-}
-
-create temporary dataset LineItem(LineItemType)
-  primary key l_orderkey, l_linenumber;
-
-create index part_index on LineItem(l_partkey);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/empty-load-with-index/empty-load-with-index.2.update.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/empty-load-with-index/empty-load-with-index.2.update.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/empty-load-with-index/empty-load-with-index.2.update.aql
deleted file mode 100644
index 7223583..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/empty-load-with-index/empty-load-with-index.2.update.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.
- */
-/*
- * Description     : Load a temporary dataset with previously created empty indexes.
- * Expected Result : Success
- * Date            : March 27 2015
- */
-
-use dataverse test;
-
-load dataset LineItem 
-using localfs
-(("path"="asterix_nc1://data/empty.adm"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
-
-load dataset LineItem 
-using localfs
-(("path"="asterix_nc1://data/tpch0.001/lineitem.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;


[10/12] asterixdb git commit: [ASTERIXDB-2171][STO] Remove Temporary Datasets

Posted by mh...@apache.org.
http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/delete-from-loaded-dataset-with-index/delete-from-loaded-dataset-with-index.5.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/delete-from-loaded-dataset-with-index/delete-from-loaded-dataset-with-index.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/delete-from-loaded-dataset-with-index/delete-from-loaded-dataset-with-index.5.query.sqlpp
deleted file mode 100644
index dff3496..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/delete-from-loaded-dataset-with-index/delete-from-loaded-dataset-with-index.5.query.sqlpp
+++ /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.
- */
-/*
- * Description     : Delete a portion of data from a loaded temporary dataset with indexes.
- * Expected Result : Success
- * Date            : 27th March 2015
- *
- */
-
-use test;
-
-
-select element c
-from  LineItem as c
-where (c.l_suppkey < 150)
-order by c.l_orderkey,c.l_linenumber
-;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/delete-from-loaded-dataset/delete-from-loaded-dataset.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/delete-from-loaded-dataset/delete-from-loaded-dataset.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/delete-from-loaded-dataset/delete-from-loaded-dataset.1.ddl.sqlpp
deleted file mode 100644
index 617f5c9..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/delete-from-loaded-dataset/delete-from-loaded-dataset.1.ddl.sqlpp
+++ /dev/null
@@ -1,53 +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.
- */
-/*
- * Description     : Delete a portion of data from a loaded temporary dataset.
- * Expected Result : Success
- * Date            : 27th March 2015
- *
- */
-
-drop  dataverse test if exists;
-create  dataverse test;
-
-use test;
-
-
-create type test.LineItemType as
- closed {
-  l_orderkey : bigint,
-  l_partkey : bigint,
-  l_suppkey : bigint,
-  l_linenumber : bigint,
-  l_quantity : bigint,
-  l_extendedprice : double,
-  l_discount : double,
-  l_tax : double,
-  l_returnflag : string,
-  l_linestatus : string,
-  l_shipdate : string,
-  l_commitdate : string,
-  l_receiptdate : string,
-  l_shipinstruct : string,
-  l_shipmode : string,
-  l_comment : string
-};
-
-create temporary dataset LineItem(LineItemType) primary key l_orderkey,l_linenumber;
-

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/delete-from-loaded-dataset/delete-from-loaded-dataset.2.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/delete-from-loaded-dataset/delete-from-loaded-dataset.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/delete-from-loaded-dataset/delete-from-loaded-dataset.2.update.sqlpp
deleted file mode 100644
index eb0f76e..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/delete-from-loaded-dataset/delete-from-loaded-dataset.2.update.sqlpp
+++ /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.
- */
-/*
- * Description     : Delete a portion of data from a loaded temporary dataset.
- * Expected Result : Success
- * Date            : 27th March 2015
- *
- */
-
-use test;
-
-
-load  dataset LineItem using localfs ((`path`=`asterix_nc1://data/tpch0.001/lineitem.tbl`),(`format`=`delimited-text`),(`delimiter`=`|`)) pre-sorted;
-
-delete from LineItem
- where l_orderkey >= 10;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/delete-from-loaded-dataset/delete-from-loaded-dataset.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/delete-from-loaded-dataset/delete-from-loaded-dataset.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/delete-from-loaded-dataset/delete-from-loaded-dataset.3.query.sqlpp
deleted file mode 100644
index aa13cc7..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/delete-from-loaded-dataset/delete-from-loaded-dataset.3.query.sqlpp
+++ /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.
- */
-/*
- * Description     : Delete a portion of data from a loaded temporary dataset.
- * Expected Result : Success
- * Date            : 27th March 2015
- *
- */
-
-use test;
-
-
-select element c
-from  LineItem as c
-order by c.l_orderkey,c.l_linenumber
-;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/drop-empty-secondary-indexes/drop-empty-secondary-indexes.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/drop-empty-secondary-indexes/drop-empty-secondary-indexes.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/drop-empty-secondary-indexes/drop-empty-secondary-indexes.1.ddl.sqlpp
deleted file mode 100644
index 4ca67fe..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/drop-empty-secondary-indexes/drop-empty-secondary-indexes.1.ddl.sqlpp
+++ /dev/null
@@ -1,77 +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.
- */
-/*
- * Description     : Drop empty secondary indexes for a temporary dataset.
- * Expected Result : Success
- * Date            : 27th March 2015
- *
- */
-
-drop  dataverse test if exists;
-create  dataverse test;
-
-use test;
-
-
-create type test.Name as
-{
-  first : string,
-  last : string
-};
-
-create type test.Person as
-{
-  name : Name
-};
-
-create type test.TestType as
-{
-  id : integer,
-  name : string,
-  locn : point,
-  zip : string,
-  person : Person
-};
-
-create temporary dataset t1(TestType) primary key id;
-
-create  index rtree_index_point  on t1 (locn) type rtree;
-
-create  index rtree_index_point_open  on t1 (open_locn:point?) type rtree enforced;
-
-create  index keyWD_indx  on t1 (name) type keyword;
-
-create  index keyWD_indx_open  on t1 (nickname:string?) type keyword enforced;
-
-create  index secndIndx  on t1 (zip) type btree;
-
-create  index nested  on t1 (person.name.first) type btree;
-
-create  index secndIndx_open  on t1 (address:string?) type btree enforced;
-
-create  primary index sec_primary_idx on t1;
-
-drop index t1.rtree_index_point;
-drop index t1.rtree_index_point_open;
-drop index t1.keyWD_indx;
-drop index t1.keyWD_indx_open;
-drop index t1.secndIndx;
-drop index t1.nested;
-drop index t1.secndIndx_open;
-drop index t1.sec_primary_idx;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/drop-empty-secondary-indexes/drop-empty-secondary-indexes.2.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/drop-empty-secondary-indexes/drop-empty-secondary-indexes.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/drop-empty-secondary-indexes/drop-empty-secondary-indexes.2.update.sqlpp
deleted file mode 100644
index ebe3e6a..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/drop-empty-secondary-indexes/drop-empty-secondary-indexes.2.update.sqlpp
+++ /dev/null
@@ -1,25 +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.
- */
-/*
- * Description     : Drop empty secondary indexes for a temporary dataset.
- * Expected Result : Success
- * Date            : 27th March 2015
- *
- */
-

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/drop-empty-secondary-indexes/drop-empty-secondary-indexes.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/drop-empty-secondary-indexes/drop-empty-secondary-indexes.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/drop-empty-secondary-indexes/drop-empty-secondary-indexes.3.query.sqlpp
deleted file mode 100644
index 78e103b..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/drop-empty-secondary-indexes/drop-empty-secondary-indexes.3.query.sqlpp
+++ /dev/null
@@ -1,29 +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.
- */
-/*
- * Description     : Drop empty secondary indexes for a temporary dataset.
- * Expected Result : Success
- * Date            : 27th March 2015
- *
- */
-
-select element l
-from  `Metadata.Index` as l
-where ((l.IsPrimary = false) and (l.DataverseName = 'test'))
-;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/drop-index/drop-index.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/drop-index/drop-index.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/drop-index/drop-index.1.ddl.sqlpp
deleted file mode 100644
index 00e7e20..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/drop-index/drop-index.1.ddl.sqlpp
+++ /dev/null
@@ -1,53 +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.
- */
-/*
- * Description     : Drop secondary indexes for a temporary dataset.
- * Expected Result : Success
- * Date            : 27th March 2015
- *
- */
-
-drop  dataverse test if exists;
-create  dataverse test;
-
-use test;
-
-
-create type test.Schema as
- closed {
-  unique1 : bigint,
-  unique2 : bigint,
-  two : bigint,
-  four : bigint,
-  ten : bigint,
-  twenty : bigint,
-  onePercent : bigint,
-  tenPercent : bigint,
-  twentyPercent : bigint,
-  fiftyPercent : bigint,
-  unique3 : bigint,
-  evenOnePercent : bigint,
-  oddOnePercent : bigint,
-  stringu1 : string,
-  stringu2 : string,
-  string4 : string
-};
-
-create temporary dataset t1(Schema) primary key unique2;
-

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/drop-index/drop-index.2.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/drop-index/drop-index.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/drop-index/drop-index.2.update.sqlpp
deleted file mode 100644
index 23906a4..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/drop-index/drop-index.2.update.sqlpp
+++ /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.
- */
-/*
- * Description     : Drop secondary indexes for a temporary dataset.
- * Expected Result : Success
- * Date            : 27th March 2015
- *
- */
-
-use test;
-
-
-load  dataset t1 using localfs ((`path`=`asterix_nc1://data/wisc/onektup.adm`),(`format`=`delimited-text`),(`delimiter`=`|`)) pre-sorted;
-

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/drop-index/drop-index.3.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/drop-index/drop-index.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/drop-index/drop-index.3.ddl.sqlpp
deleted file mode 100644
index f01f959..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/drop-index/drop-index.3.ddl.sqlpp
+++ /dev/null
@@ -1,37 +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.
- */
-/*
- * Description     : Drop secondary indexes for a temporary dataset.
- * Expected Result : Success
- * Date            : 27th March 2015
- *
- */
-
-use test;
-
-
-create  index idx_t1_str1  on t1 (stringu1) type btree;
-
-create  index idx_t1_unique1  on t1 (unique1) type btree;
-
-create  primary index sec_primary_idx  on t1;
-
-drop index t1.idx_t1_str1;
-drop index t1.idx_t1_unique1;
-drop index t1.sec_primary_idx;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/drop-index/drop-index.4.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/drop-index/drop-index.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/drop-index/drop-index.4.query.sqlpp
deleted file mode 100644
index 0950324..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/drop-index/drop-index.4.query.sqlpp
+++ /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.
- */
-/*
- * Description     : Drop secondary indexes for a temporary dataset.
- * Expected Result : Success
- * Date            : 27th March 2015
- *
- */
-
-use test;
-
-
-select element a
-from  t1 as a
-where ((a.unique1 > 10) and (a.stringu1 = 'DGAAAAXXXXXXXXXXXXXXXXXXX'))
-;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/empty-load-with-index/empty-load-with-index.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/empty-load-with-index/empty-load-with-index.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/empty-load-with-index/empty-load-with-index.1.ddl.sqlpp
deleted file mode 100644
index 2b169dd..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/empty-load-with-index/empty-load-with-index.1.ddl.sqlpp
+++ /dev/null
@@ -1,55 +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.
- */
-/*
- * Description     : Load a temporary dataset with previously created empty indexes.
- * Expected Result : Success
- * Date            : March 27 2015
- */
-
-drop  dataverse test if exists;
-create  dataverse test;
-
-use test;
-
-
-create type test.LineItemType as
- closed {
-  l_orderkey : bigint,
-  l_partkey : bigint,
-  l_suppkey : bigint,
-  l_linenumber : bigint,
-  l_quantity : double,
-  l_extendedprice : double,
-  l_discount : double,
-  l_tax : double,
-  l_returnflag : string,
-  l_linestatus : string,
-  l_shipdate : string,
-  l_commitdate : string,
-  l_receiptdate : string,
-  l_shipinstruct : string,
-  l_shipmode : string,
-  l_comment : string
-};
-
-create temporary dataset LineItem(LineItemType) primary key l_orderkey,l_linenumber;
-
-create  index part_index  on LineItem (l_partkey) type btree;
-
-create  primary index sec_primary_idx on LineItem ;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/empty-load-with-index/empty-load-with-index.2.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/empty-load-with-index/empty-load-with-index.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/empty-load-with-index/empty-load-with-index.2.update.sqlpp
deleted file mode 100644
index ef9dd8f..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/empty-load-with-index/empty-load-with-index.2.update.sqlpp
+++ /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.
- */
-/*
- * Description     : Load a temporary dataset with previously created empty indexes.
- * Expected Result : Success
- * Date            : March 27 2015
- */
-
-use test;
-
-
-load  dataset LineItem using localfs ((`path`=`asterix_nc1://data/empty.adm`),(`format`=`delimited-text`),(`delimiter`=`|`)) pre-sorted;
-
-load  dataset LineItem using localfs ((`path`=`asterix_nc1://data/tpch0.001/lineitem.tbl`),(`format`=`delimited-text`),(`delimiter`=`|`)) pre-sorted;
-

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/empty-load-with-index/empty-load-with-index.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/empty-load-with-index/empty-load-with-index.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/empty-load-with-index/empty-load-with-index.3.query.sqlpp
deleted file mode 100644
index a4c443a..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/empty-load-with-index/empty-load-with-index.3.query.sqlpp
+++ /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.
- */
-/*
- * Description     : Load a temporary dataset with previously created empty indexes.
- * Expected Result : Success
- * Date            : March 27 2015
- */
-
-use test;
-
-
-select element c
-from  LineItem as c
-order by c.l_orderkey,c.l_linenumber
-limit 1
-;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/insert-and-scan-dataset-with-correlated-index/insert-and-scan-dataset-with-correlated-index.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/insert-and-scan-dataset-with-correlated-index/insert-and-scan-dataset-with-correlated-index.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/insert-and-scan-dataset-with-correlated-index/insert-and-scan-dataset-with-correlated-index.1.ddl.sqlpp
deleted file mode 100644
index 6c1595d..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/insert-and-scan-dataset-with-correlated-index/insert-and-scan-dataset-with-correlated-index.1.ddl.sqlpp
+++ /dev/null
@@ -1,42 +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.
- */
-/*
- * Description     : This test is intended to test inserting into a temporary dataset that has a correlated
- * secondary index and scan the data at the same time where we insert a materializing to prevent the possibility
- * of deadlatch.
- * Expected Result : Success
- * Date            : June 8 2017
- */
-
-drop  dataverse test if exists;
-create  dataverse test;
-
-create type test.Emp as
- closed {
-  id : bigint,
-  fname : string,
-  lname : string,
-  age : bigint,
-  dept : string
-};
-
-create temporary dataset test.employee(Emp) primary key id
-using compaction policy `correlated-prefix`
-((`max-mergable-component-size`=`16384`),(`max-tolerance-component-count`=`3`));
-

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/insert-and-scan-dataset-with-correlated-index/insert-and-scan-dataset-with-correlated-index.2.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/insert-and-scan-dataset-with-correlated-index/insert-and-scan-dataset-with-correlated-index.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/insert-and-scan-dataset-with-correlated-index/insert-and-scan-dataset-with-correlated-index.2.update.sqlpp
deleted file mode 100644
index 969f8ef..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/insert-and-scan-dataset-with-correlated-index/insert-and-scan-dataset-with-correlated-index.2.update.sqlpp
+++ /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.
- */
-/*
- * Description     : This test is intended to test inserting into a temporary dataset that has a correlated
- * secondary index and scan the data at the same time where we insert a materializing to prevent the possibility
- * of deadlatch.
- * Expected Result : Success
- * Date            : June 8 2017
- */
-
-use test;
-
-
-load  dataset test.employee using localfs ((`path`=`asterix_nc1://data/names.adm`),(`format`=`delimited-text`),(`delimiter`=`|`));
-
-insert into test.employee
-select element {'id':(x.id + 10000),'fname':x.fname,'lname':x.lname,'age':x.age,'dept':x.dept}
-from  `test.employee` as x
-;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/insert-and-scan-dataset-with-correlated-index/insert-and-scan-dataset-with-correlated-index.3.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/insert-and-scan-dataset-with-correlated-index/insert-and-scan-dataset-with-correlated-index.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/insert-and-scan-dataset-with-correlated-index/insert-and-scan-dataset-with-correlated-index.3.ddl.sqlpp
deleted file mode 100644
index c3ee05f..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/insert-and-scan-dataset-with-correlated-index/insert-and-scan-dataset-with-correlated-index.3.ddl.sqlpp
+++ /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.
- */
-/*
- * Description     : This test is intended to test inserting into a temporary dataset that has a correlated
- * secondary index and scan the data at the same time where we insert a materializing to prevent the possibility
- * of deadlatch.
- * Expected Result : Success
- * Date            : June 8 2017
- */
-
-use test;
-
-create  index idx_employee_first_name  on test.employee (fname) type btree;
-
-create  primary index sec_primary_idx  on test.employee;
-

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/insert-and-scan-dataset-with-correlated-index/insert-and-scan-dataset-with-correlated-index.4.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/insert-and-scan-dataset-with-correlated-index/insert-and-scan-dataset-with-correlated-index.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/insert-and-scan-dataset-with-correlated-index/insert-and-scan-dataset-with-correlated-index.4.query.sqlpp
deleted file mode 100644
index a179e16..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/insert-and-scan-dataset-with-correlated-index/insert-and-scan-dataset-with-correlated-index.4.query.sqlpp
+++ /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.
- */
-/*
- * Description     : This test is intended to test inserting into a temporary dataset that has a correlated
- * secondary index and scan the data at the same time where we insert a materializing to prevent the possibility
- * of deadlatch.
- * Expected Result : Success
- * Date            : June 8 2017
- */
-
-use test;
-
-
-select element l
-from  `test.employee` as l
-order by l.id
-;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/insert-and-scan-dataset-with-index/insert-and-scan-dataset-with-index.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/insert-and-scan-dataset-with-index/insert-and-scan-dataset-with-index.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/insert-and-scan-dataset-with-index/insert-and-scan-dataset-with-index.1.ddl.sqlpp
deleted file mode 100644
index e9aae4a..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/insert-and-scan-dataset-with-index/insert-and-scan-dataset-with-index.1.ddl.sqlpp
+++ /dev/null
@@ -1,42 +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.
- */
-/*
- * Description     : This test is intended to test inserting into a temporary dataset that has a secondary index and scan
- * the data at the same time where we insert a materializing to prevent the possibility of deadlatch.
- * Expected Result : Success
- * Date            : March 27 2015
- */
-
-drop  dataverse test if exists;
-create  dataverse test;
-
-create type test.Emp as
- closed {
-  id : bigint,
-  fname : string,
-  lname : string,
-  age : bigint,
-  dept : string
-};
-
-create temporary dataset test.employee(Emp) primary key id;
-
-create  index idx_employee_first_name  on test.employee (fname) type btree;
-
-create  primary index sec_primary_idx  on test.employee;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/insert-and-scan-dataset-with-index/insert-and-scan-dataset-with-index.2.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/insert-and-scan-dataset-with-index/insert-and-scan-dataset-with-index.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/insert-and-scan-dataset-with-index/insert-and-scan-dataset-with-index.2.update.sqlpp
deleted file mode 100644
index da759b4..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/insert-and-scan-dataset-with-index/insert-and-scan-dataset-with-index.2.update.sqlpp
+++ /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.
- */
-/*
- * Description     : This test is intended to test inserting into a temporary dataset that has a secondary index and scan
- * the data at the same time where we insert a materializing to prevent the possibility of deadlatch.
- * Expected Result : Success
- * Date            : March 28 2015
- */
-
-use test;
-
-
-load  dataset test.employee using localfs ((`path`=`asterix_nc1://data/names.adm`),(`format`=`delimited-text`),(`delimiter`=`|`));
-
-insert into test.employee
-select element {'id':(x.id + 10000),'fname':x.fname,'lname':x.lname,'age':x.age,'dept':x.dept}
-from  `test.employee` as x
-;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/insert-and-scan-dataset-with-index/insert-and-scan-dataset-with-index.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/insert-and-scan-dataset-with-index/insert-and-scan-dataset-with-index.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/insert-and-scan-dataset-with-index/insert-and-scan-dataset-with-index.3.query.sqlpp
deleted file mode 100644
index 065e5b9..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/insert-and-scan-dataset-with-index/insert-and-scan-dataset-with-index.3.query.sqlpp
+++ /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.
- */
-/*
- * Description     : This test is intended to test inserting into a temporary dataset that has a secondary index and scan
- * the data at the same time where we insert a materializing to prevent the possibility of dead latch.
- * Expected Result : Success
- * Date            : March 27 2015
- */
-
-use test;
-
-
-select element l
-from  `test.employee` as l
-order by l.id
-;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/insert-and-scan-dataset/insert-and-scan-dataset.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/insert-and-scan-dataset/insert-and-scan-dataset.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/insert-and-scan-dataset/insert-and-scan-dataset.1.ddl.sqlpp
deleted file mode 100644
index c716bbe..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/insert-and-scan-dataset/insert-and-scan-dataset.1.ddl.sqlpp
+++ /dev/null
@@ -1,38 +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.
- */
-/*
- * Description     : This test is intended to test inserting into a temporary dataset and scan it at the same time
- * where we insert a materializing to prevent the possibility of deadlatch.
- * Expected Result : Success
- * Date            : July 11 2013
- */
-
-drop  dataverse test if exists;
-create  dataverse test;
-
-use test;
-
-
-create type test.myDataType as
-{
-  id : bigint
-};
-
-create temporary dataset myData(myDataType) primary key id;
-

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/insert-and-scan-dataset/insert-and-scan-dataset.2.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/insert-and-scan-dataset/insert-and-scan-dataset.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/insert-and-scan-dataset/insert-and-scan-dataset.2.update.sqlpp
deleted file mode 100644
index 06917df..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/insert-and-scan-dataset/insert-and-scan-dataset.2.update.sqlpp
+++ /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  : insert-and-scan-dataset.aql
- * Description     : This test is intended to test inserting into a temporary dataset and scan it at the same time
- * where we insert a materializing to prevent the possibility of deadlatch.
- * Expected Result : Success
- * Date            : March 27 2015
- */
-
-use test;
-
-
-load  dataset myData using localfs ((`path`=`asterix_nc1://data/odd-numbers.adm`),(`format`=`adm`)) pre-sorted;
-
-insert into myData
-select element {'id':(x.id + 1)}
-from  myData as x
-;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/insert-and-scan-dataset/insert-and-scan-dataset.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/insert-and-scan-dataset/insert-and-scan-dataset.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/insert-and-scan-dataset/insert-and-scan-dataset.3.query.sqlpp
deleted file mode 100644
index 4418b67..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/insert-and-scan-dataset/insert-and-scan-dataset.3.query.sqlpp
+++ /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  : insert-and-scan-dataset.aql
- * Description     : This test is intended to test inserting into a temporary dataset and scan it at the same time
- * where we insert a materializing to prevent the possibility of deadlatch.
- * Expected Result : Success
- * Date            : March 27 2015
- *
- */
-
-use test;
-
-
-select element c
-from  myData as c
-order by c.id
-;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/q01_pricing_summary_report_nt/q01_pricing_summary_report_nt.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/q01_pricing_summary_report_nt/q01_pricing_summary_report_nt.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/q01_pricing_summary_report_nt/q01_pricing_summary_report_nt.1.ddl.sqlpp
deleted file mode 100644
index 45cde93..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/q01_pricing_summary_report_nt/q01_pricing_summary_report_nt.1.ddl.sqlpp
+++ /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.
- */
-/*
- * Description     : Test a read query over a temporary dataset.
- * Expected Result : Success
- * Date            : March 27 2015
- */
-
-drop  dataverse tpch if exists;
-create  dataverse tpch;
-
-use tpch;
-
-
-create type tpch.LineItemType as
- closed {
-  l_orderkey : bigint,
-  l_partkey : bigint,
-  l_suppkey : bigint,
-  l_linenumber : bigint,
-  l_quantity : double,
-  l_extendedprice : double,
-  l_discount : double,
-  l_tax : double,
-  l_returnflag : string,
-  l_linestatus : string,
-  l_shipdate : string,
-  l_commitdate : string,
-  l_receiptdate : string,
-  l_shipinstruct : string,
-  l_shipmode : string,
-  l_comment : string
-};
-
-create temporary dataset LineItem(LineItemType) primary key l_orderkey,l_linenumber;
-

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/q01_pricing_summary_report_nt/q01_pricing_summary_report_nt.2.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/q01_pricing_summary_report_nt/q01_pricing_summary_report_nt.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/q01_pricing_summary_report_nt/q01_pricing_summary_report_nt.2.update.sqlpp
deleted file mode 100644
index 696a84b..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/q01_pricing_summary_report_nt/q01_pricing_summary_report_nt.2.update.sqlpp
+++ /dev/null
@@ -1,29 +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.
- */
-/*
- * Description     : Test a read query over a temporary dataset.
- * Expected Result : Success
- * Date            : March 27 2015
- */
-
-use tpch;
-
-
-load  dataset LineItem using localfs ((`path`=`asterix_nc1://data/tpch0.001/lineitem.tbl`),(`format`=`delimited-text`),(`delimiter`=`|`)) pre-sorted;
-

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/q01_pricing_summary_report_nt/q01_pricing_summary_report_nt.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/q01_pricing_summary_report_nt/q01_pricing_summary_report_nt.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/q01_pricing_summary_report_nt/q01_pricing_summary_report_nt.3.query.sqlpp
deleted file mode 100644
index 4a5eae4..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/q01_pricing_summary_report_nt/q01_pricing_summary_report_nt.3.query.sqlpp
+++ /dev/null
@@ -1,44 +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.
- */
-/*
- * Description     : Test a read query over a temporary dataset.
- * Expected Result : Success
- * Date            : March 27 2015
- */
-
-use tpch;
-
-
-set `import-private-functions` `true`;
-
-select element {'l_returnflag':l_returnflag,'l_linestatus':l_linestatus,
-    'sum_qty': sum(l.l_quantity),
-    'sum_base_price': sum(l.l_extendedprice),
-    'sum_disc_price': sum(l.l_extendedprice * (1 - l.l_discount)),
-    'sum_charge': sum(l.l_extendedprice * (1 - l.l_discount) * (1 + l.l_tax)),
-    'ave_qty': avg(l.l_quantity),
-    'ave_price': avg(l.l_extendedprice),
-    'ave_disc': avg(l.l_discount),
-    'count_order': count(*)}
-from  LineItem as l
-where (l.l_shipdate <= '1998-09-02')
-/* +hash */
-group by l.l_returnflag as l_returnflag,l.l_linestatus as l_linestatus
-order by l_returnflag,l_linestatus
-;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/scan-delete-rtree-secondary-index/scan-delete-rtree-secondary-index.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/scan-delete-rtree-secondary-index/scan-delete-rtree-secondary-index.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/scan-delete-rtree-secondary-index/scan-delete-rtree-secondary-index.1.ddl.sqlpp
deleted file mode 100644
index 7e0f9cc..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/scan-delete-rtree-secondary-index/scan-delete-rtree-secondary-index.1.ddl.sqlpp
+++ /dev/null
@@ -1,45 +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.
- */
-/*
- * Description     : Delete tuples from a temporary dataset with a R-tree index.
- * Expected Result : Success
- * Date            : March 27 2015
- */
-
-drop  dataverse test if exists;
-create  dataverse test;
-
-use test;
-
-
-create type test.MyRecord as
- closed {
-  id : bigint,
-  point : point,
-  kwds : string,
-  line1 : line,
-  line2 : line,
-  poly1 : polygon,
-  poly2 : polygon,
-  rec : rectangle,
-  circle : circle
-};
-
-create temporary dataset MyData(MyRecord) primary key id;
-

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/scan-delete-rtree-secondary-index/scan-delete-rtree-secondary-index.2.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/scan-delete-rtree-secondary-index/scan-delete-rtree-secondary-index.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/scan-delete-rtree-secondary-index/scan-delete-rtree-secondary-index.2.update.sqlpp
deleted file mode 100644
index 39eaee2..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/scan-delete-rtree-secondary-index/scan-delete-rtree-secondary-index.2.update.sqlpp
+++ /dev/null
@@ -1,29 +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.
- */
-/*
- * Description     : Delete tuples from a temporary dataset with a R-tree index.
- * Expected Result : Success
- * Date            : March 27 2015
- */
-
-use test;
-
-
-load  dataset MyData using localfs ((`path`=`asterix_nc1://data/spatial/spatialData.json`),(`format`=`adm`)) pre-sorted;
-

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/scan-delete-rtree-secondary-index/scan-delete-rtree-secondary-index.3.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/scan-delete-rtree-secondary-index/scan-delete-rtree-secondary-index.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/scan-delete-rtree-secondary-index/scan-delete-rtree-secondary-index.3.ddl.sqlpp
deleted file mode 100644
index 56a3186..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/scan-delete-rtree-secondary-index/scan-delete-rtree-secondary-index.3.ddl.sqlpp
+++ /dev/null
@@ -1,29 +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.
- */
-/*
- * Description     : Delete tuples from a temporary dataset with a R-tree index.
- * Expected Result : Success
- * Date            : March 27 2015
- */
-
-use test;
-
-
-create  index rtree_index_point  on MyData (point) type rtree;
-

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/scan-delete-rtree-secondary-index/scan-delete-rtree-secondary-index.4.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/scan-delete-rtree-secondary-index/scan-delete-rtree-secondary-index.4.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/scan-delete-rtree-secondary-index/scan-delete-rtree-secondary-index.4.update.sqlpp
deleted file mode 100644
index 5a6708a..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/scan-delete-rtree-secondary-index/scan-delete-rtree-secondary-index.4.update.sqlpp
+++ /dev/null
@@ -1,29 +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.
- */
-/*
- * Description     : Delete tuples from a temporary dataset with a R-tree index.
- * Expected Result : Success
- * Date            : March 27 2015
- */
-
-use test;
-
-
-delete from MyData
- where id > 10;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/scan-delete-rtree-secondary-index/scan-delete-rtree-secondary-index.5.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/scan-delete-rtree-secondary-index/scan-delete-rtree-secondary-index.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/scan-delete-rtree-secondary-index/scan-delete-rtree-secondary-index.5.query.sqlpp
deleted file mode 100644
index 7e2f12e..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/scan-delete-rtree-secondary-index/scan-delete-rtree-secondary-index.5.query.sqlpp
+++ /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.
- */
-/*
- * Description     : Delete tuples from a temporary dataset with a R-tree index.
- * Expected Result : Success
- * Date            : March 27 2015
- */
-
-use test;
-
-
-select element {'id':o.id}
-from  MyData as o
-where test.`spatial-intersect`(o.point,test.`create-polygon`([0.0,1.0,0.0,4.0,12.0,4.0,12.0,1.0]))
-order by o.id
-;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/scan-insert-persistent-to-temp/scan-insert-persistent-to-temp.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/scan-insert-persistent-to-temp/scan-insert-persistent-to-temp.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/scan-insert-persistent-to-temp/scan-insert-persistent-to-temp.1.ddl.sqlpp
deleted file mode 100644
index b352768..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/scan-insert-persistent-to-temp/scan-insert-persistent-to-temp.1.ddl.sqlpp
+++ /dev/null
@@ -1,54 +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.
- */
-/*
- * Description     : Port data from a persistent dataset into a temporary dataset.
- *                   Both datasets have R-tree indexes.
- * Expected Result : Success
- * Date            : March 27 2015
- */
-
-drop  dataverse test if exists;
-create  dataverse test;
-
-use test;
-
-
-create type test.MyRecord as
- closed {
-  id : bigint,
-  point : point,
-  kwds : string,
-  line1 : line,
-  line2 : line,
-  poly1 : polygon,
-  poly2 : polygon,
-  rec : rectangle,
-  circle : circle
-};
-
-create type test.MyMiniRecord as
- closed {
-  id : bigint,
-  point : point
-};
-
-create  dataset MyData(MyRecord) primary key id;
-
-create temporary dataset MyMiniData(MyMiniRecord) primary key id;
-

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/scan-insert-persistent-to-temp/scan-insert-persistent-to-temp.2.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/scan-insert-persistent-to-temp/scan-insert-persistent-to-temp.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/scan-insert-persistent-to-temp/scan-insert-persistent-to-temp.2.update.sqlpp
deleted file mode 100644
index c2214c9..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/scan-insert-persistent-to-temp/scan-insert-persistent-to-temp.2.update.sqlpp
+++ /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.
- */
-/*
- * Description     : Port data from a temporary dataset into yet-another temporary dataset.
- *                   Both datasets have R-tree indexes.
- * Expected Result : Success
- * Date            : March 27 2015
- */
-
-use test;
-
-
-load  dataset MyData using localfs ((`path`=`asterix_nc1://data/spatial/spatialData.json`),(`format`=`adm`)) pre-sorted;
-
-load  dataset MyMiniData using localfs ((`path`=`asterix_nc1://data/spatial/spatialData0.json`),(`format`=`adm`)) pre-sorted;
-

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/scan-insert-persistent-to-temp/scan-insert-persistent-to-temp.3.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/scan-insert-persistent-to-temp/scan-insert-persistent-to-temp.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/scan-insert-persistent-to-temp/scan-insert-persistent-to-temp.3.ddl.sqlpp
deleted file mode 100644
index 98bfaf8..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/scan-insert-persistent-to-temp/scan-insert-persistent-to-temp.3.ddl.sqlpp
+++ /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.
- */
-/*
- * Description     : Port data from a temporary dataset into yet-another temporary dataset.
- *                   Both datasets have R-tree indexes.
- * Expected Result : Success
- * Date            : March 27 2015
- */
-
-use test;
-
-
-create  index rtree_index_point_0  on MyData (point) type rtree;
-
-create  index rtree_index_point  on MyMiniData (point) type rtree;
-

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/scan-insert-persistent-to-temp/scan-insert-persistent-to-temp.4.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/scan-insert-persistent-to-temp/scan-insert-persistent-to-temp.4.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/scan-insert-persistent-to-temp/scan-insert-persistent-to-temp.4.update.sqlpp
deleted file mode 100644
index 7ab14fe..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/scan-insert-persistent-to-temp/scan-insert-persistent-to-temp.4.update.sqlpp
+++ /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.
- */
-/*
- * Description     : Port data from a temporary dataset into yet-another temporary dataset.
- *                   Both datasets have R-tree indexes.
- * Expected Result : Success
- * Date            : March 27 2015
- */
-
-use test;
-
-
-insert into MyMiniData
-select element {'id':m.id,'point':m.point}
-from  MyData as m
-;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/scan-insert-persistent-to-temp/scan-insert-persistent-to-temp.5.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/scan-insert-persistent-to-temp/scan-insert-persistent-to-temp.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/scan-insert-persistent-to-temp/scan-insert-persistent-to-temp.5.query.sqlpp
deleted file mode 100644
index 33c063d..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/scan-insert-persistent-to-temp/scan-insert-persistent-to-temp.5.query.sqlpp
+++ /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.
- */
-/*
- * Description     : Port data from a temporary dataset into yet-another temporary dataset.
- *                   Both datasets have R-tree indexes.
- * Expected Result : Success
- * Date            : March 27 2015
- */
-
-use test;
-
-
-select element {'id':o.id}
-from  MyMiniData as o
-where test.`spatial-intersect`(o.point,test.`create-polygon`([0.0,1.0,0.0,4.0,12.0,4.0,12.0,1.0]))
-order by o.id
-;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/scan-insert-rtree-secondary-index/scan-insert-rtree-secondary-index.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/scan-insert-rtree-secondary-index/scan-insert-rtree-secondary-index.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/scan-insert-rtree-secondary-index/scan-insert-rtree-secondary-index.1.ddl.sqlpp
deleted file mode 100644
index 8d1110a..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/scan-insert-rtree-secondary-index/scan-insert-rtree-secondary-index.1.ddl.sqlpp
+++ /dev/null
@@ -1,54 +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.
- */
-/*
- * Description     : Port data from a temporary dataset into yet-another temporary dataset.
- *                   Both datasets have R-tree indexes.
- * Expected Result : Success
- * Date            : March 27 2015
- */
-
-drop  dataverse test if exists;
-create  dataverse test;
-
-use test;
-
-
-create type test.MyRecord as
- closed {
-  id : bigint,
-  point : point,
-  kwds : string,
-  line1 : line,
-  line2 : line,
-  poly1 : polygon,
-  poly2 : polygon,
-  rec : rectangle,
-  circle : circle
-};
-
-create type test.MyMiniRecord as
- closed {
-  id : bigint,
-  point : point
-};
-
-create temporary dataset MyData(MyRecord) primary key id;
-
-create temporary dataset MyMiniData(MyMiniRecord) primary key id;
-

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/scan-insert-rtree-secondary-index/scan-insert-rtree-secondary-index.2.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/scan-insert-rtree-secondary-index/scan-insert-rtree-secondary-index.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/scan-insert-rtree-secondary-index/scan-insert-rtree-secondary-index.2.update.sqlpp
deleted file mode 100644
index c2214c9..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/scan-insert-rtree-secondary-index/scan-insert-rtree-secondary-index.2.update.sqlpp
+++ /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.
- */
-/*
- * Description     : Port data from a temporary dataset into yet-another temporary dataset.
- *                   Both datasets have R-tree indexes.
- * Expected Result : Success
- * Date            : March 27 2015
- */
-
-use test;
-
-
-load  dataset MyData using localfs ((`path`=`asterix_nc1://data/spatial/spatialData.json`),(`format`=`adm`)) pre-sorted;
-
-load  dataset MyMiniData using localfs ((`path`=`asterix_nc1://data/spatial/spatialData0.json`),(`format`=`adm`)) pre-sorted;
-

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/scan-insert-rtree-secondary-index/scan-insert-rtree-secondary-index.3.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/scan-insert-rtree-secondary-index/scan-insert-rtree-secondary-index.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/scan-insert-rtree-secondary-index/scan-insert-rtree-secondary-index.3.ddl.sqlpp
deleted file mode 100644
index 98bfaf8..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/scan-insert-rtree-secondary-index/scan-insert-rtree-secondary-index.3.ddl.sqlpp
+++ /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.
- */
-/*
- * Description     : Port data from a temporary dataset into yet-another temporary dataset.
- *                   Both datasets have R-tree indexes.
- * Expected Result : Success
- * Date            : March 27 2015
- */
-
-use test;
-
-
-create  index rtree_index_point_0  on MyData (point) type rtree;
-
-create  index rtree_index_point  on MyMiniData (point) type rtree;
-

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/scan-insert-rtree-secondary-index/scan-insert-rtree-secondary-index.4.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/scan-insert-rtree-secondary-index/scan-insert-rtree-secondary-index.4.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/scan-insert-rtree-secondary-index/scan-insert-rtree-secondary-index.4.update.sqlpp
deleted file mode 100644
index 7ab14fe..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/scan-insert-rtree-secondary-index/scan-insert-rtree-secondary-index.4.update.sqlpp
+++ /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.
- */
-/*
- * Description     : Port data from a temporary dataset into yet-another temporary dataset.
- *                   Both datasets have R-tree indexes.
- * Expected Result : Success
- * Date            : March 27 2015
- */
-
-use test;
-
-
-insert into MyMiniData
-select element {'id':m.id,'point':m.point}
-from  MyData as m
-;


[04/12] asterixdb git commit: [ASTERIXDB-2171][STO] Remove Temporary Datasets

Posted by mh...@apache.org.
http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_ngram_index/primary_plus_ngram_index.3.update.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_ngram_index/primary_plus_ngram_index.3.update.aql b/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_ngram_index/primary_plus_ngram_index.3.update.aql
deleted file mode 100644
index ce01110..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_ngram_index/primary_plus_ngram_index.3.update.aql
+++ /dev/null
@@ -1,29 +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  : primary_plus_default_secondary_index.aql
- * Description     : Check that abort from duplicate key exception works and crash recovery works after the abort.   
- * Expected Result : Success
- * Date            : Oct 15 2013
- */
-
-use dataverse recovery;
-
-load dataset Fragile_raw using localfs
-(("path"="asterix_nc1://../../../../../../asterix-app/data/csv/fragile_02.adm"),("format"="adm")) pre-sorted;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_ngram_index/primary_plus_ngram_index.4.txneu.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_ngram_index/primary_plus_ngram_index.4.txneu.aql b/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_ngram_index/primary_plus_ngram_index.4.txneu.aql
deleted file mode 100644
index 261e7b9..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_ngram_index/primary_plus_ngram_index.4.txneu.aql
+++ /dev/null
@@ -1,43 +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  : primary_plus_default_secondary_index.aql
- * Description     : Check that abort from duplicate key exception works and crash recovery works after the abort.   
- * Expected Result : Success
- * Date            : Oct 15 2013
- */
-
-use dataverse recovery;
-
-/* Load Fragile data from raw dataset into cleaned dataset */
-insert into dataset Fragile (
-  for $t in dataset Fragile_raw
-  return {
-    "row_id": $t.row_id % 28000,
-    "sid": $t.sid,
-    "date": date($t.date),
-    "day": $t.day,
-    "time": parse-time($t.time, "h:m:s"),
-    "bpm": $t.bpm,
-    "RR": $t.RR,
-    "text": $t.text,
-    "location": $t.location,
-    "text2": $t.text2
-  }
-);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_ngram_index/primary_plus_ngram_index.5.txnqbc.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_ngram_index/primary_plus_ngram_index.5.txnqbc.aql b/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_ngram_index/primary_plus_ngram_index.5.txnqbc.aql
deleted file mode 100644
index dd499be..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_ngram_index/primary_plus_ngram_index.5.txnqbc.aql
+++ /dev/null
@@ -1,29 +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  : primary_plus_default_secondary_index.aql
- * Description     : Check that abort from duplicate key exception works and crash recovery works after the abort.   
- * Expected Result : Success
- * Date            : Otc 15 2013
- */
-
-use dataverse recovery;
-
-count (for $x in dataset Fragile 
-where contains($x.text2, "location") return $x);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_ngram_index/primary_plus_ngram_index.6.script.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_ngram_index/primary_plus_ngram_index.6.script.aql b/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_ngram_index/primary_plus_ngram_index.6.script.aql
deleted file mode 100644
index 4583455..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_ngram_index/primary_plus_ngram_index.6.script.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.
-#
-kill_cc_and_nc.sh
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_ngram_index/primary_plus_ngram_index.7.script.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_ngram_index/primary_plus_ngram_index.7.script.aql b/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_ngram_index/primary_plus_ngram_index.7.script.aql
deleted file mode 100644
index 7087cd3..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_ngram_index/primary_plus_ngram_index.7.script.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.
-#
-stop_and_start.sh
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_ngram_index/primary_plus_ngram_index.8.txnqar.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_ngram_index/primary_plus_ngram_index.8.txnqar.aql b/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_ngram_index/primary_plus_ngram_index.8.txnqar.aql
deleted file mode 100644
index 07b177d..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_ngram_index/primary_plus_ngram_index.8.txnqar.aql
+++ /dev/null
@@ -1,29 +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  : primary_plus_default_secondary_index.aql
- * Description     : Check that abort from duplicate key exception works and crash recovery works after the abort.   
- * Expected Result : Success
- * Date            : Oct 15 2013
- */
-
-use dataverse recovery;
-
-count (for $x in dataset Fragile 
-where contains($x.text2, "location") return $x);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_ngram_index/primary_plus_ngram_index.9.script.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_ngram_index/primary_plus_ngram_index.9.script.aql b/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_ngram_index/primary_plus_ngram_index.9.script.aql
deleted file mode 100644
index 40df6fb..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_ngram_index/primary_plus_ngram_index.9.script.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.
-#
-stop_and_delete.sh
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_rtree_index/primary_plus_rtree_index.1.script.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_rtree_index/primary_plus_rtree_index.1.script.aql b/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_rtree_index/primary_plus_rtree_index.1.script.aql
deleted file mode 100644
index 7d441cd..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_rtree_index/primary_plus_rtree_index.1.script.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.
-#
-create_and_start.sh
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_rtree_index/primary_plus_rtree_index.2.ddl.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_rtree_index/primary_plus_rtree_index.2.ddl.aql b/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_rtree_index/primary_plus_rtree_index.2.ddl.aql
deleted file mode 100644
index 95a21b4..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_rtree_index/primary_plus_rtree_index.2.ddl.aql
+++ /dev/null
@@ -1,72 +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  : primary_plus_keyword_secondary_index.aql
- * Description     : Check that abort from duplicate key exception works and crash recovery works after the abort.   
- * Expected Result : Success
- * Date            : Oct 15 2013
- */
-
-drop dataverse recovery if exists;
-create dataverse recovery;
-use dataverse recovery;
-
-/* For raw Fragile data */
-create type FragileTypeRaw as closed {
-  row_id: int32,
-  sid: int32,
-  date: string,
-  day: int32,
-  time: string,
-  bpm: int32,
-  RR: float,
-  /* new string field and location field*/
-  text: string,
-  location: point,
-  text2: string
-  
-};
-
-/* For cleaned Fragile data */
-create type FragileType as closed {
-  row_id: int32,
-  sid: int32,
-  date: date,
-  day: int32,
-  time: time,
-  bpm: int32,
-  RR: float,
-  
-  /* new string field and location field*/
-  text: string,
-  location: point,
-  text2: string
-};
-
-/* Create dataset for loading raw Fragile data */
-create temporary dataset Fragile_raw (FragileTypeRaw)
-primary key row_id;
-
-/* Create dataset for cleaned Fragile data */
-create temporary dataset Fragile (FragileType)
-primary key row_id;
-
-
-/* Create rtree secondary index on dataset clean Fragile */
-create index cfLocation on Fragile(location) type rtree;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_rtree_index/primary_plus_rtree_index.3.update.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_rtree_index/primary_plus_rtree_index.3.update.aql b/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_rtree_index/primary_plus_rtree_index.3.update.aql
deleted file mode 100644
index 66adcef..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_rtree_index/primary_plus_rtree_index.3.update.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  : primary_plus_keyword_secondary_index.aql
- * Description     : Check that abort from duplicate key exception works and crash recovery works after the abort.   
- * Expected Result : Success
- * Date            : Oct 15 2013
- */
-
-use dataverse recovery;
-
-
-load dataset Fragile_raw using localfs
-(("path"="asterix_nc1://../../../../../../asterix-app/data/csv/fragile_02.adm"),("format"="adm")) pre-sorted;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_rtree_index/primary_plus_rtree_index.4.txneu.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_rtree_index/primary_plus_rtree_index.4.txneu.aql b/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_rtree_index/primary_plus_rtree_index.4.txneu.aql
deleted file mode 100644
index 7c1c623..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_rtree_index/primary_plus_rtree_index.4.txneu.aql
+++ /dev/null
@@ -1,43 +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  : primary_plus_keyword_secondary_index.aql
- * Description     : Check that abort from duplicate key exception works and crash recovery works after the abort.   
- * Expected Result : Success
- * Date            : Oct 15 2013
- */
-
-use dataverse recovery;
-
-/* Load Fragile data from raw dataset into cleaned dataset */
-insert into dataset Fragile (
-  for $t in dataset Fragile_raw
-  return {
-    "row_id": $t.row_id % 28000,
-    "sid": $t.sid,
-    "date": date($t.date),
-    "day": $t.day,
-    "time": parse-time($t.time, "h:m:s"),
-    "bpm": $t.bpm,
-    "RR": $t.RR,
-    "text": $t.text,
-    "location": $t.location,
-    "text2": $t.text2
-  }
-);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_rtree_index/primary_plus_rtree_index.5.txnqbc.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_rtree_index/primary_plus_rtree_index.5.txnqbc.aql b/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_rtree_index/primary_plus_rtree_index.5.txnqbc.aql
deleted file mode 100644
index dc20c89..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_rtree_index/primary_plus_rtree_index.5.txnqbc.aql
+++ /dev/null
@@ -1,23 +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.
- */
-use dataverse recovery;
-
-count (for $x in dataset Fragile where 
-spatial-intersect($x.location, create-polygon([0.0,0.0, 2.0,2.0, 0.0,2.0, 2.0,0.0]))
-return $x);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_rtree_index/primary_plus_rtree_index.6.script.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_rtree_index/primary_plus_rtree_index.6.script.aql b/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_rtree_index/primary_plus_rtree_index.6.script.aql
deleted file mode 100644
index 4583455..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_rtree_index/primary_plus_rtree_index.6.script.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.
-#
-kill_cc_and_nc.sh
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_rtree_index/primary_plus_rtree_index.7.script.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_rtree_index/primary_plus_rtree_index.7.script.aql b/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_rtree_index/primary_plus_rtree_index.7.script.aql
deleted file mode 100644
index 7087cd3..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_rtree_index/primary_plus_rtree_index.7.script.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.
-#
-stop_and_start.sh
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_rtree_index/primary_plus_rtree_index.8.txnqar.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_rtree_index/primary_plus_rtree_index.8.txnqar.aql b/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_rtree_index/primary_plus_rtree_index.8.txnqar.aql
deleted file mode 100644
index dc20c89..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_rtree_index/primary_plus_rtree_index.8.txnqar.aql
+++ /dev/null
@@ -1,23 +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.
- */
-use dataverse recovery;
-
-count (for $x in dataset Fragile where 
-spatial-intersect($x.location, create-polygon([0.0,0.0, 2.0,2.0, 0.0,2.0, 2.0,0.0]))
-return $x);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_rtree_index/primary_plus_rtree_index.9.script.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_rtree_index/primary_plus_rtree_index.9.script.aql b/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_rtree_index/primary_plus_rtree_index.9.script.aql
deleted file mode 100644
index 40df6fb..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_rtree_index/primary_plus_rtree_index.9.script.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.
-#
-stop_and_delete.sh
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_dataset_recovery/dataset_recovery.1.script.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_dataset_recovery/dataset_recovery.1.script.aql b/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_dataset_recovery/dataset_recovery.1.script.aql
deleted file mode 100644
index 7d441cd..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_dataset_recovery/dataset_recovery.1.script.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.
-#
-create_and_start.sh
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_dataset_recovery/dataset_recovery.2.ddl.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_dataset_recovery/dataset_recovery.2.ddl.aql b/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_dataset_recovery/dataset_recovery.2.ddl.aql
deleted file mode 100644
index fc78710..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_dataset_recovery/dataset_recovery.2.ddl.aql
+++ /dev/null
@@ -1,37 +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  : dataverse_recovery
- * Description     :
- * Expected Result :
- * Date            :
- */
-
-drop dataverse SampleDV if exists;
-create dataverse SampleDV;
-
-use dataverse SampleDV;
-
-create type SampleType as open {
-    id: int32,
-    text: string
-}
-
-create temporary dataset SampleDS(SampleType)
-primary key id;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_dataset_recovery/dataset_recovery.3.script.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_dataset_recovery/dataset_recovery.3.script.aql b/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_dataset_recovery/dataset_recovery.3.script.aql
deleted file mode 100644
index 4583455..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_dataset_recovery/dataset_recovery.3.script.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.
-#
-kill_cc_and_nc.sh
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_dataset_recovery/dataset_recovery.4.script.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_dataset_recovery/dataset_recovery.4.script.aql b/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_dataset_recovery/dataset_recovery.4.script.aql
deleted file mode 100644
index d844d13..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_dataset_recovery/dataset_recovery.4.script.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.
-#
-stop_and_start.sh

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_dataset_recovery/dataset_recovery.5.errddl.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_dataset_recovery/dataset_recovery.5.errddl.aql b/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_dataset_recovery/dataset_recovery.5.errddl.aql
deleted file mode 100644
index bbd8044..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_dataset_recovery/dataset_recovery.5.errddl.aql
+++ /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.
- */
-use dataverse SampleDV;
-drop dataset SampleDS;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_dataset_recovery/dataset_recovery.6.script.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_dataset_recovery/dataset_recovery.6.script.aql b/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_dataset_recovery/dataset_recovery.6.script.aql
deleted file mode 100644
index f43dec7..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_dataset_recovery/dataset_recovery.6.script.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.
-#
-stop_and_delete.sh

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_delete_after_recovery/delete_after_recovery.1.script.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_delete_after_recovery/delete_after_recovery.1.script.aql b/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_delete_after_recovery/delete_after_recovery.1.script.aql
deleted file mode 100644
index 7d441cd..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_delete_after_recovery/delete_after_recovery.1.script.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.
-#
-create_and_start.sh
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_delete_after_recovery/delete_after_recovery.2.ddl.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_delete_after_recovery/delete_after_recovery.2.ddl.aql b/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_delete_after_recovery/delete_after_recovery.2.ddl.aql
deleted file mode 100644
index 9d4c55d..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_delete_after_recovery/delete_after_recovery.2.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  : primary_index_only.aql
- * Description     : Check that abort from duplicate key exception works and crash recovery works after the abort.   
- * Expected Result : Success
- * Date            : September 25 2013
- */
-
-drop dataverse recovery if exists;
-create dataverse recovery;
-use dataverse recovery;
-
-/* For raw Fragile data */
-create type FragileTypeRaw as closed {
-  row_id: int32,
-  sid: int32,
-  date: string,
-  day: int32,
-  time: string,
-  bpm: int32,
-  RR: float
-};
-
-/* For cleaned Fragile data */
-create type FragileType as closed {
-  row_id: int32,
-  sid: int32,
-  date: date,
-  day: int32,
-  time: time,
-  bpm: int32,
-  RR: float
-};
-
-/* Create dataset for loading raw Fragile data */
-create temporary dataset Fragile_raw (FragileTypeRaw)
-primary key row_id;
-
-/* Create dataset for cleaned Fragile data */
-create temporary dataset Fragile (FragileType)
-primary key row_id;
-

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_delete_after_recovery/delete_after_recovery.3.update.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_delete_after_recovery/delete_after_recovery.3.update.aql b/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_delete_after_recovery/delete_after_recovery.3.update.aql
deleted file mode 100644
index ab07679..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_delete_after_recovery/delete_after_recovery.3.update.aql
+++ /dev/null
@@ -1,29 +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  : primary_index_only.aql
- * Description     : Check that abort from duplicate key exception works and crash recovery works after the abort.   
- * Expected Result : Success
- * Date            : September 25 2013
- */
-
-use dataverse recovery;
-
-load dataset Fragile_raw using localfs
-(("path"="asterix_nc1://../../../../../../asterix-app/data/csv/fragile_01.csv"),("format"="delimited-text"),("delimiter"=",")) pre-sorted;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_delete_after_recovery/delete_after_recovery.4.script.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_delete_after_recovery/delete_after_recovery.4.script.aql b/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_delete_after_recovery/delete_after_recovery.4.script.aql
deleted file mode 100644
index 4583455..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_delete_after_recovery/delete_after_recovery.4.script.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.
-#
-kill_cc_and_nc.sh
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_delete_after_recovery/delete_after_recovery.5.script.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_delete_after_recovery/delete_after_recovery.5.script.aql b/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_delete_after_recovery/delete_after_recovery.5.script.aql
deleted file mode 100644
index d844d13..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_delete_after_recovery/delete_after_recovery.5.script.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.
-#
-stop_and_start.sh

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_delete_after_recovery/delete_after_recovery.6.update.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_delete_after_recovery/delete_after_recovery.6.update.aql b/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_delete_after_recovery/delete_after_recovery.6.update.aql
deleted file mode 100644
index f303a93..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_delete_after_recovery/delete_after_recovery.6.update.aql
+++ /dev/null
@@ -1,23 +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.
- */
-use dataverse recovery;
-
-delete $r from dataset Fragile_raw 
-  where $r.row_id % 2 = 1
-;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_delete_after_recovery/delete_after_recovery.7.query.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_delete_after_recovery/delete_after_recovery.7.query.aql b/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_delete_after_recovery/delete_after_recovery.7.query.aql
deleted file mode 100644
index a290f58..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_delete_after_recovery/delete_after_recovery.7.query.aql
+++ /dev/null
@@ -1,29 +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  : primary_index_only.aql
- * Description     : Check that abort from duplicate key exception works and crash recovery works after the abort.   
- * Expected Result : Success
- * Date            : September 25 2013
- */
-
-use dataverse recovery;
-
-count (for $x in dataset Fragile_raw return $x);
-

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_delete_after_recovery/delete_after_recovery.8.script.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_delete_after_recovery/delete_after_recovery.8.script.aql b/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_delete_after_recovery/delete_after_recovery.8.script.aql
deleted file mode 100644
index 40df6fb..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_delete_after_recovery/delete_after_recovery.8.script.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.
-#
-stop_and_delete.sh
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_insert_after_recovery/insert_after_recovery.1.script.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_insert_after_recovery/insert_after_recovery.1.script.aql b/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_insert_after_recovery/insert_after_recovery.1.script.aql
deleted file mode 100644
index 7d441cd..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_insert_after_recovery/insert_after_recovery.1.script.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.
-#
-create_and_start.sh
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_insert_after_recovery/insert_after_recovery.2.ddl.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_insert_after_recovery/insert_after_recovery.2.ddl.aql b/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_insert_after_recovery/insert_after_recovery.2.ddl.aql
deleted file mode 100644
index 9d4c55d..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_insert_after_recovery/insert_after_recovery.2.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  : primary_index_only.aql
- * Description     : Check that abort from duplicate key exception works and crash recovery works after the abort.   
- * Expected Result : Success
- * Date            : September 25 2013
- */
-
-drop dataverse recovery if exists;
-create dataverse recovery;
-use dataverse recovery;
-
-/* For raw Fragile data */
-create type FragileTypeRaw as closed {
-  row_id: int32,
-  sid: int32,
-  date: string,
-  day: int32,
-  time: string,
-  bpm: int32,
-  RR: float
-};
-
-/* For cleaned Fragile data */
-create type FragileType as closed {
-  row_id: int32,
-  sid: int32,
-  date: date,
-  day: int32,
-  time: time,
-  bpm: int32,
-  RR: float
-};
-
-/* Create dataset for loading raw Fragile data */
-create temporary dataset Fragile_raw (FragileTypeRaw)
-primary key row_id;
-
-/* Create dataset for cleaned Fragile data */
-create temporary dataset Fragile (FragileType)
-primary key row_id;
-

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_insert_after_recovery/insert_after_recovery.3.update.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_insert_after_recovery/insert_after_recovery.3.update.aql b/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_insert_after_recovery/insert_after_recovery.3.update.aql
deleted file mode 100644
index ab07679..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_insert_after_recovery/insert_after_recovery.3.update.aql
+++ /dev/null
@@ -1,29 +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  : primary_index_only.aql
- * Description     : Check that abort from duplicate key exception works and crash recovery works after the abort.   
- * Expected Result : Success
- * Date            : September 25 2013
- */
-
-use dataverse recovery;
-
-load dataset Fragile_raw using localfs
-(("path"="asterix_nc1://../../../../../../asterix-app/data/csv/fragile_01.csv"),("format"="delimited-text"),("delimiter"=",")) pre-sorted;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_insert_after_recovery/insert_after_recovery.4.script.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_insert_after_recovery/insert_after_recovery.4.script.aql b/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_insert_after_recovery/insert_after_recovery.4.script.aql
deleted file mode 100644
index 4583455..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_insert_after_recovery/insert_after_recovery.4.script.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.
-#
-kill_cc_and_nc.sh
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_insert_after_recovery/insert_after_recovery.5.script.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_insert_after_recovery/insert_after_recovery.5.script.aql b/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_insert_after_recovery/insert_after_recovery.5.script.aql
deleted file mode 100644
index d844d13..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_insert_after_recovery/insert_after_recovery.5.script.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.
-#
-stop_and_start.sh

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_insert_after_recovery/insert_after_recovery.6.update.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_insert_after_recovery/insert_after_recovery.6.update.aql b/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_insert_after_recovery/insert_after_recovery.6.update.aql
deleted file mode 100644
index c4e167a..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_insert_after_recovery/insert_after_recovery.6.update.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.
- */
-use dataverse recovery;
-
-/* Load Fragile data from raw dataset into cleaned dataset */
-insert into dataset Fragile (
-  for $t in dataset Fragile_raw
-  return {
-    "row_id": $t.row_id,
-    "sid": $t.sid,
-    "date": date($t.date),
-    "day": $t.day,
-    "time": parse-time($t.time, "h:m:s"),
-    "bpm": $t.bpm,
-    "RR": $t.RR
-  }
-);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_insert_after_recovery/insert_after_recovery.7.query.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_insert_after_recovery/insert_after_recovery.7.query.aql b/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_insert_after_recovery/insert_after_recovery.7.query.aql
deleted file mode 100644
index c193c29..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_insert_after_recovery/insert_after_recovery.7.query.aql
+++ /dev/null
@@ -1,29 +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  : primary_index_only.aql
- * Description     : Check that abort from duplicate key exception works and crash recovery works after the abort.   
- * Expected Result : Success
- * Date            : September 25 2013
- */
-
-use dataverse recovery;
-
-count (for $x in dataset Fragile return $x);
-

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_insert_after_recovery/insert_after_recovery.8.script.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_insert_after_recovery/insert_after_recovery.8.script.aql b/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_insert_after_recovery/insert_after_recovery.8.script.aql
deleted file mode 100644
index 40df6fb..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_insert_after_recovery/insert_after_recovery.8.script.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.
-#
-stop_and_delete.sh
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_load_after_recovery/load_after_recovery.1.script.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_load_after_recovery/load_after_recovery.1.script.aql b/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_load_after_recovery/load_after_recovery.1.script.aql
deleted file mode 100644
index 7d441cd..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_load_after_recovery/load_after_recovery.1.script.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.
-#
-create_and_start.sh
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_load_after_recovery/load_after_recovery.2.ddl.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_load_after_recovery/load_after_recovery.2.ddl.aql b/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_load_after_recovery/load_after_recovery.2.ddl.aql
deleted file mode 100644
index 9d4c55d..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_load_after_recovery/load_after_recovery.2.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  : primary_index_only.aql
- * Description     : Check that abort from duplicate key exception works and crash recovery works after the abort.   
- * Expected Result : Success
- * Date            : September 25 2013
- */
-
-drop dataverse recovery if exists;
-create dataverse recovery;
-use dataverse recovery;
-
-/* For raw Fragile data */
-create type FragileTypeRaw as closed {
-  row_id: int32,
-  sid: int32,
-  date: string,
-  day: int32,
-  time: string,
-  bpm: int32,
-  RR: float
-};
-
-/* For cleaned Fragile data */
-create type FragileType as closed {
-  row_id: int32,
-  sid: int32,
-  date: date,
-  day: int32,
-  time: time,
-  bpm: int32,
-  RR: float
-};
-
-/* Create dataset for loading raw Fragile data */
-create temporary dataset Fragile_raw (FragileTypeRaw)
-primary key row_id;
-
-/* Create dataset for cleaned Fragile data */
-create temporary dataset Fragile (FragileType)
-primary key row_id;
-

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_load_after_recovery/load_after_recovery.3.script.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_load_after_recovery/load_after_recovery.3.script.aql b/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_load_after_recovery/load_after_recovery.3.script.aql
deleted file mode 100644
index 4583455..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_load_after_recovery/load_after_recovery.3.script.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.
-#
-kill_cc_and_nc.sh
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_load_after_recovery/load_after_recovery.4.script.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_load_after_recovery/load_after_recovery.4.script.aql b/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_load_after_recovery/load_after_recovery.4.script.aql
deleted file mode 100644
index 7087cd3..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_load_after_recovery/load_after_recovery.4.script.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.
-#
-stop_and_start.sh
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_load_after_recovery/load_after_recovery.5.update.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_load_after_recovery/load_after_recovery.5.update.aql b/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_load_after_recovery/load_after_recovery.5.update.aql
deleted file mode 100644
index ab07679..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_load_after_recovery/load_after_recovery.5.update.aql
+++ /dev/null
@@ -1,29 +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  : primary_index_only.aql
- * Description     : Check that abort from duplicate key exception works and crash recovery works after the abort.   
- * Expected Result : Success
- * Date            : September 25 2013
- */
-
-use dataverse recovery;
-
-load dataset Fragile_raw using localfs
-(("path"="asterix_nc1://../../../../../../asterix-app/data/csv/fragile_01.csv"),("format"="delimited-text"),("delimiter"=",")) pre-sorted;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_load_after_recovery/load_after_recovery.6.query.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_load_after_recovery/load_after_recovery.6.query.aql b/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_load_after_recovery/load_after_recovery.6.query.aql
deleted file mode 100644
index ebac011..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_load_after_recovery/load_after_recovery.6.query.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  : primary_index_only.aql
- * Description     : Check that abort from duplicate key exception works and crash recovery works after the abort.   
- * Expected Result : Success
- * Date            : September 25 2013
- */
-
-use dataverse recovery;
-
-count (for $x in dataset Fragile_raw return $x);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_load_after_recovery/load_after_recovery.7.script.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_load_after_recovery/load_after_recovery.7.script.aql b/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_load_after_recovery/load_after_recovery.7.script.aql
deleted file mode 100644
index 40df6fb..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_load_after_recovery/load_after_recovery.7.script.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.
-#
-stop_and_delete.sh
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_secondary_index_recovery/secondary_index_recovery.1.script.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_secondary_index_recovery/secondary_index_recovery.1.script.aql b/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_secondary_index_recovery/secondary_index_recovery.1.script.aql
deleted file mode 100644
index 7d441cd..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_secondary_index_recovery/secondary_index_recovery.1.script.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.
-#
-create_and_start.sh
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_secondary_index_recovery/secondary_index_recovery.2.ddl.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_secondary_index_recovery/secondary_index_recovery.2.ddl.aql b/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_secondary_index_recovery/secondary_index_recovery.2.ddl.aql
deleted file mode 100644
index 3bcfdde..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_secondary_index_recovery/secondary_index_recovery.2.ddl.aql
+++ /dev/null
@@ -1,39 +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  : dataverse_recovery
- * Description     :
- * Expected Result :
- * Date            :
- */
-
-drop dataverse SampleDV if exists;
-create dataverse SampleDV;
-
-use dataverse SampleDV;
-
-create type SampleType as open {
-    id: int32,
-    text: string
-}
-
-create temporary dataset SampleDS(SampleType)
-primary key id;
-
-create index SampleDSix on SampleDS(text);


[06/12] asterixdb git commit: [ASTERIXDB-2171][STO] Remove Temporary Datasets

Posted by mh...@apache.org.
http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/results/temp-dataset/insert-and-scan-dataset-with-index/insert-and-scan-dataset-with-index.1.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temp-dataset/insert-and-scan-dataset-with-index/insert-and-scan-dataset-with-index.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temp-dataset/insert-and-scan-dataset-with-index/insert-and-scan-dataset-with-index.1.adm
deleted file mode 100644
index 0eea6ff..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/temp-dataset/insert-and-scan-dataset-with-index/insert-and-scan-dataset-with-index.1.adm
+++ /dev/null
@@ -1,240 +0,0 @@
-{ "id": 101, "fname": "Javier", "lname": "Makuch", "age": 28, "dept": "IT" }
-{ "id": 110, "fname": "Allan", "lname": "Piland", "age": 29, "dept": "HR" }
-{ "id": 112, "fname": "Pearlie", "lname": "Aumann", "age": 31, "dept": "Payroll" }
-{ "id": 113, "fname": "Chandra", "lname": "Hase", "age": 34, "dept": "Sales" }
-{ "id": 114, "fname": "Christian", "lname": "Convery", "age": 28, "dept": "HR" }
-{ "id": 115, "fname": "Panther", "lname": "Ritch", "age": 26, "dept": "IT" }
-{ "id": 116, "fname": "Ted", "lname": "Elsea", "age": 26, "dept": "IT" }
-{ "id": 117, "fname": "Tabatha", "lname": "Bladen", "age": 25, "dept": "HR" }
-{ "id": 118, "fname": "Clayton", "lname": "Oltman", "age": 42, "dept": "Sales" }
-{ "id": 119, "fname": "Sharron", "lname": "Darwin", "age": 32, "dept": "Payroll" }
-{ "id": 210, "fname": "Clayton", "lname": "Durgin", "age": 52, "dept": "HR" }
-{ "id": 212, "fname": "Emilia", "lname": "Chenail", "age": 26, "dept": "Sales" }
-{ "id": 213, "fname": "Kenya", "lname": "Almquist", "age": 43, "dept": "Payroll" }
-{ "id": 214, "fname": "Alejandra", "lname": "Lacefield", "age": 41, "dept": "HR" }
-{ "id": 215, "fname": "Karina", "lname": "Michelsen", "age": 46, "dept": "IT" }
-{ "id": 216, "fname": "Katy", "lname": "Delillo", "age": 36, "dept": "IT" }
-{ "id": 217, "fname": "Benita", "lname": "Kleist", "age": 37, "dept": "HR" }
-{ "id": 218, "fname": "Earlene", "lname": "Paluch", "age": 31, "dept": "IT" }
-{ "id": 219, "fname": "Kurt", "lname": "Petermann", "age": 27, "dept": "Payroll" }
-{ "id": 299, "fname": "Julio", "lname": "Iorio", "age": 37, "dept": "IT" }
-{ "id": 363, "fname": "Cody", "lname": "Rodreguez", "age": 26, "dept": "IT" }
-{ "id": 404, "fname": "Emilia", "lname": "Square", "age": 32, "dept": "IT" }
-{ "id": 414, "fname": "Mathew", "lname": "Fuschetto", "age": 34, "dept": "HR" }
-{ "id": 424, "fname": "Allyson", "lname": "Remus", "age": 32, "dept": "IT" }
-{ "id": 434, "fname": "Earlene", "lname": "Linebarger", "age": 26, "dept": "Payroll" }
-{ "id": 444, "fname": "Clinton", "lname": "Sick", "age": 29, "dept": "IT" }
-{ "id": 454, "fname": "Ted", "lname": "Caba", "age": 28, "dept": "HR" }
-{ "id": 463, "fname": "Marcie", "lname": "States", "age": 28, "dept": "IT" }
-{ "id": 464, "fname": "Fernando", "lname": "Engelke", "age": 39, "dept": "IT" }
-{ "id": 474, "fname": "Mathew", "lname": "Courchesne", "age": 31, "dept": "IT" }
-{ "id": 484, "fname": "Cody", "lname": "Vinyard", "age": 36, "dept": "Payroll" }
-{ "id": 494, "fname": "Benita", "lname": "Fravel", "age": 33, "dept": "Sales" }
-{ "id": 504, "fname": "Erik", "lname": "Dobek", "age": 29, "dept": "IT" }
-{ "id": 514, "fname": "Julio", "lname": "Ruben", "age": 41, "dept": "IT" }
-{ "id": 524, "fname": "Benita", "lname": "Maltos", "age": 33, "dept": "IT" }
-{ "id": 534, "fname": "Kurt", "lname": "Biscoe", "age": 36, "dept": "HR" }
-{ "id": 538, "fname": "Milagros", "lname": "Forkey", "age": 34, "dept": "Sales" }
-{ "id": 544, "fname": "Loraine", "lname": "Housel", "age": 30, "dept": "Sales" }
-{ "id": 554, "fname": "Jamie", "lname": "Rachal", "age": 30, "dept": "IT" }
-{ "id": 564, "fname": "Liza", "lname": "Fredenburg", "age": 37, "dept": "IT" }
-{ "id": 574, "fname": "Ericka", "lname": "Feldmann", "age": 29, "dept": "Sales" }
-{ "id": 584, "fname": "Dollie", "lname": "Dattilo", "age": 32, "dept": "Payroll" }
-{ "id": 589, "fname": "Lorrie", "lname": "Sharon", "age": 27, "dept": "IT" }
-{ "id": 594, "fname": "Roxie", "lname": "Houghtaling", "age": 40, "dept": "Payroll" }
-{ "id": 601, "fname": "Neil", "lname": "Deforge", "age": 26, "dept": "HR" }
-{ "id": 611, "fname": "Earlene", "lname": "Marcy", "age": 32, "dept": "IT" }
-{ "id": 621, "fname": "Erik", "lname": "Lechuga", "age": 42, "dept": "Payroll" }
-{ "id": 631, "fname": "Tyrone", "lname": "Holtzclaw", "age": 34, "dept": "Sales" }
-{ "id": 641, "fname": "Lance", "lname": "Hankey", "age": 35, "dept": "Sales" }
-{ "id": 651, "fname": "Mallory", "lname": "Gladding", "age": 31, "dept": "HR" }
-{ "id": 661, "fname": "Tia", "lname": "Braaten", "age": 40, "dept": "IT" }
-{ "id": 671, "fname": "Julio", "lname": "Vanpatten", "age": 30, "dept": "Payroll" }
-{ "id": 681, "fname": "Max", "lname": "Teachout", "age": 34, "dept": "IT" }
-{ "id": 691, "fname": "Karina", "lname": "Wingerter", "age": 31, "dept": "IT" }
-{ "id": 711, "fname": "Hugh", "lname": "Lema", "age": 25, "dept": "HR" }
-{ "id": 721, "fname": "Schwan", "lname": "Phil", "age": 34, "dept": "Payroll" }
-{ "id": 732, "fname": "Noemi", "lname": "Eacret", "age": 56, "dept": "HR" }
-{ "id": 741, "fname": "Julio", "lname": "Mattocks", "age": 38, "dept": "Sales" }
-{ "id": 751, "fname": "Lance", "lname": "Kottke", "age": 34, "dept": "IT" }
-{ "id": 761, "fname": "Kurt", "lname": "Liz", "age": 32, "dept": "HR" }
-{ "id": 771, "fname": "Neva", "lname": "Barbeau", "age": 45, "dept": "Sales" }
-{ "id": 781, "fname": "Karina", "lname": "Tuthill", "age": 46, "dept": "Payroll" }
-{ "id": 791, "fname": "Maricela", "lname": "Cambron", "age": 36, "dept": "IT" }
-{ "id": 809, "fname": "Clayton", "lname": "Delany", "age": 23, "dept": "IT" }
-{ "id": 811, "fname": "Kubik", "lname": "Kuhn", "age": 27, "dept": "HR" }
-{ "id": 821, "fname": "Allan", "lname": "Tomes", "age": 29, "dept": "Payroll" }
-{ "id": 831, "fname": "Lonnie", "lname": "Aller", "age": 33, "dept": "Sales" }
-{ "id": 841, "fname": "Neil", "lname": "Hurrell", "age": 26, "dept": "IT" }
-{ "id": 851, "fname": "Clayton", "lname": "Engles", "age": 41, "dept": "HR" }
-{ "id": 861, "fname": "Javier", "lname": "Gabrielson", "age": 39, "dept": "Payroll" }
-{ "id": 871, "fname": "Allan", "lname": "Alejandre", "age": 48, "dept": "IT" }
-{ "id": 881, "fname": "Julio", "lname": "Isa", "age": 38, "dept": "Sales" }
-{ "id": 891, "fname": "Roslyn", "lname": "Simmerman", "age": 31, "dept": "IT" }
-{ "id": 915, "fname": "Starner", "lname": "Stuart", "age": 25, "dept": "Sales" }
-{ "id": 925, "fname": "Sofia", "lname": "Cuff", "age": 30, "dept": "HR" }
-{ "id": 935, "fname": "Milagros", "lname": "Murguia", "age": 31, "dept": "IT" }
-{ "id": 945, "fname": "Margery", "lname": "Haldeman", "age": 32, "dept": "IT" }
-{ "id": 955, "fname": "Max", "lname": "Mell", "age": 33, "dept": "HR" }
-{ "id": 965, "fname": "Micco", "lname": "Mercy", "age": 31, "dept": "Payroll" }
-{ "id": 975, "fname": "Clare", "lname": "Vangieson", "age": 34, "dept": "IT" }
-{ "id": 985, "fname": "Elnora", "lname": "Dimauro", "age": 35, "dept": "Sales" }
-{ "id": 995, "fname": "Pearlie", "lname": "Kocian", "age": 38, "dept": "HR" }
-{ "id": 1007, "fname": "Yingyi", "lname": "Bu", "age": 27, "dept": "IT" }
-{ "id": 1263, "fname": "Tania", "lname": "Loffredo", "age": 25, "dept": "IT" }
-{ "id": 1410, "fname": "Clinton", "lname": "Fredricks", "age": 34, "dept": "IT" }
-{ "id": 1411, "fname": "Lance", "lname": "Farquhar", "age": 32, "dept": "HR" }
-{ "id": 1412, "fname": "Tabatha", "lname": "Crisler", "age": 33, "dept": "IT" }
-{ "id": 1413, "fname": "Max", "lname": "Durney", "age": 29, "dept": "IT" }
-{ "id": 1414, "fname": "Carmella", "lname": "Strauser", "age": 30, "dept": "Payroll" }
-{ "id": 1415, "fname": "Kelly", "lname": "Carrales", "age": 40, "dept": "IT" }
-{ "id": 1416, "fname": "Guy", "lname": "Merten", "age": 29, "dept": "Sales" }
-{ "id": 1417, "fname": "Noreen", "lname": "Ruhland", "age": 29, "dept": "IT" }
-{ "id": 1418, "fname": "Julio", "lname": "Damore", "age": 27, "dept": "Sales" }
-{ "id": 1419, "fname": "Selena", "lname": "Truby", "age": 25, "dept": "HR" }
-{ "id": 1420, "fname": "Alejandra", "lname": "Commons", "age": 30, "dept": "Sales" }
-{ "id": 1421, "fname": "Allyson", "lname": "Balk", "age": 30, "dept": "IT" }
-{ "id": 1422, "fname": "Nelson", "lname": "Byun", "age": 40, "dept": "Sales" }
-{ "id": 1423, "fname": "Christian", "lname": "Reidhead", "age": 40, "dept": "IT" }
-{ "id": 1424, "fname": "Pearlie", "lname": "Hopkin", "age": 48, "dept": "Payroll" }
-{ "id": 1425, "fname": "Nelson", "lname": "Wohlers", "age": 41, "dept": "HR" }
-{ "id": 1426, "fname": "Marcie", "lname": "Rasnake", "age": 42, "dept": "Sales" }
-{ "id": 1427, "fname": "Hugh", "lname": "Marshburn", "age": 43, "dept": "Payroll" }
-{ "id": 1428, "fname": "Mathew", "lname": "Marasco", "age": 45, "dept": "Sales" }
-{ "id": 1429, "fname": "Kurt", "lname": "Veres", "age": 32, "dept": "IT" }
-{ "id": 1430, "fname": "Julio", "lname": "Barkett", "age": 39, "dept": "Sales" }
-{ "id": 1863, "fname": "Darren", "lname": "Thorington", "age": 32, "dept": "Sales" }
-{ "id": 1999, "fname": "Susan", "lname": "Malaika", "age": 42, "dept": "HR" }
-{ "id": 2333, "fname": "Chen", "lname": "Li", "age": 42, "dept": "HR" }
-{ "id": 2963, "fname": "Neil", "lname": "Gunnerson", "age": 34, "dept": "IT" }
-{ "id": 3563, "fname": "Hazeltine", "lname": "Susan", "age": 29, "dept": "Sales" }
-{ "id": 3666, "fname": "Young Seok", "lname": "Kim", "age": 35, "dept": "Payroll" }
-{ "id": 4727, "fname": "Michael", "lname": "Carey", "age": 50, "dept": "Payroll" }
-{ "id": 5438, "fname": "Lakisha", "lname": "Quashie", "age": 29, "dept": "HR" }
-{ "id": 7444, "fname": "Sharad", "lname": "Mehrotra", "age": 42, "dept": "Sales" }
-{ "id": 7663, "fname": "Annabelle", "lname": "Nimmo", "age": 30, "dept": "Payroll" }
-{ "id": 8301, "fname": "Earlene", "lname": "Wallick", "age": 26, "dept": "HR" }
-{ "id": 8338, "fname": "Julio", "lname": "Bosket", "age": 28, "dept": "Payroll" }
-{ "id": 9555, "fname": "Tony", "lname": "Givargis", "age": 40, "dept": "Sales" }
-{ "id": 9763, "fname": "Ted", "lname": "Saini", "age": 31, "dept": "IT" }
-{ "id": 9941, "fname": "Khurram Faraaz", "lname": "Mohammed", "age": 30, "dept": "HR" }
-{ "id": 10101, "fname": "Javier", "lname": "Makuch", "age": 28, "dept": "IT" }
-{ "id": 10110, "fname": "Allan", "lname": "Piland", "age": 29, "dept": "HR" }
-{ "id": 10112, "fname": "Pearlie", "lname": "Aumann", "age": 31, "dept": "Payroll" }
-{ "id": 10113, "fname": "Chandra", "lname": "Hase", "age": 34, "dept": "Sales" }
-{ "id": 10114, "fname": "Christian", "lname": "Convery", "age": 28, "dept": "HR" }
-{ "id": 10115, "fname": "Panther", "lname": "Ritch", "age": 26, "dept": "IT" }
-{ "id": 10116, "fname": "Ted", "lname": "Elsea", "age": 26, "dept": "IT" }
-{ "id": 10117, "fname": "Tabatha", "lname": "Bladen", "age": 25, "dept": "HR" }
-{ "id": 10118, "fname": "Clayton", "lname": "Oltman", "age": 42, "dept": "Sales" }
-{ "id": 10119, "fname": "Sharron", "lname": "Darwin", "age": 32, "dept": "Payroll" }
-{ "id": 10210, "fname": "Clayton", "lname": "Durgin", "age": 52, "dept": "HR" }
-{ "id": 10212, "fname": "Emilia", "lname": "Chenail", "age": 26, "dept": "Sales" }
-{ "id": 10213, "fname": "Kenya", "lname": "Almquist", "age": 43, "dept": "Payroll" }
-{ "id": 10214, "fname": "Alejandra", "lname": "Lacefield", "age": 41, "dept": "HR" }
-{ "id": 10215, "fname": "Karina", "lname": "Michelsen", "age": 46, "dept": "IT" }
-{ "id": 10216, "fname": "Katy", "lname": "Delillo", "age": 36, "dept": "IT" }
-{ "id": 10217, "fname": "Benita", "lname": "Kleist", "age": 37, "dept": "HR" }
-{ "id": 10218, "fname": "Earlene", "lname": "Paluch", "age": 31, "dept": "IT" }
-{ "id": 10219, "fname": "Kurt", "lname": "Petermann", "age": 27, "dept": "Payroll" }
-{ "id": 10299, "fname": "Julio", "lname": "Iorio", "age": 37, "dept": "IT" }
-{ "id": 10363, "fname": "Cody", "lname": "Rodreguez", "age": 26, "dept": "IT" }
-{ "id": 10404, "fname": "Emilia", "lname": "Square", "age": 32, "dept": "IT" }
-{ "id": 10414, "fname": "Mathew", "lname": "Fuschetto", "age": 34, "dept": "HR" }
-{ "id": 10424, "fname": "Allyson", "lname": "Remus", "age": 32, "dept": "IT" }
-{ "id": 10434, "fname": "Earlene", "lname": "Linebarger", "age": 26, "dept": "Payroll" }
-{ "id": 10444, "fname": "Clinton", "lname": "Sick", "age": 29, "dept": "IT" }
-{ "id": 10454, "fname": "Ted", "lname": "Caba", "age": 28, "dept": "HR" }
-{ "id": 10463, "fname": "Marcie", "lname": "States", "age": 28, "dept": "IT" }
-{ "id": 10464, "fname": "Fernando", "lname": "Engelke", "age": 39, "dept": "IT" }
-{ "id": 10474, "fname": "Mathew", "lname": "Courchesne", "age": 31, "dept": "IT" }
-{ "id": 10484, "fname": "Cody", "lname": "Vinyard", "age": 36, "dept": "Payroll" }
-{ "id": 10494, "fname": "Benita", "lname": "Fravel", "age": 33, "dept": "Sales" }
-{ "id": 10504, "fname": "Erik", "lname": "Dobek", "age": 29, "dept": "IT" }
-{ "id": 10514, "fname": "Julio", "lname": "Ruben", "age": 41, "dept": "IT" }
-{ "id": 10524, "fname": "Benita", "lname": "Maltos", "age": 33, "dept": "IT" }
-{ "id": 10534, "fname": "Kurt", "lname": "Biscoe", "age": 36, "dept": "HR" }
-{ "id": 10538, "fname": "Milagros", "lname": "Forkey", "age": 34, "dept": "Sales" }
-{ "id": 10544, "fname": "Loraine", "lname": "Housel", "age": 30, "dept": "Sales" }
-{ "id": 10554, "fname": "Jamie", "lname": "Rachal", "age": 30, "dept": "IT" }
-{ "id": 10564, "fname": "Liza", "lname": "Fredenburg", "age": 37, "dept": "IT" }
-{ "id": 10574, "fname": "Ericka", "lname": "Feldmann", "age": 29, "dept": "Sales" }
-{ "id": 10584, "fname": "Dollie", "lname": "Dattilo", "age": 32, "dept": "Payroll" }
-{ "id": 10589, "fname": "Lorrie", "lname": "Sharon", "age": 27, "dept": "IT" }
-{ "id": 10594, "fname": "Roxie", "lname": "Houghtaling", "age": 40, "dept": "Payroll" }
-{ "id": 10601, "fname": "Neil", "lname": "Deforge", "age": 26, "dept": "HR" }
-{ "id": 10611, "fname": "Earlene", "lname": "Marcy", "age": 32, "dept": "IT" }
-{ "id": 10621, "fname": "Erik", "lname": "Lechuga", "age": 42, "dept": "Payroll" }
-{ "id": 10631, "fname": "Tyrone", "lname": "Holtzclaw", "age": 34, "dept": "Sales" }
-{ "id": 10641, "fname": "Lance", "lname": "Hankey", "age": 35, "dept": "Sales" }
-{ "id": 10651, "fname": "Mallory", "lname": "Gladding", "age": 31, "dept": "HR" }
-{ "id": 10661, "fname": "Tia", "lname": "Braaten", "age": 40, "dept": "IT" }
-{ "id": 10671, "fname": "Julio", "lname": "Vanpatten", "age": 30, "dept": "Payroll" }
-{ "id": 10681, "fname": "Max", "lname": "Teachout", "age": 34, "dept": "IT" }
-{ "id": 10691, "fname": "Karina", "lname": "Wingerter", "age": 31, "dept": "IT" }
-{ "id": 10711, "fname": "Hugh", "lname": "Lema", "age": 25, "dept": "HR" }
-{ "id": 10721, "fname": "Schwan", "lname": "Phil", "age": 34, "dept": "Payroll" }
-{ "id": 10732, "fname": "Noemi", "lname": "Eacret", "age": 56, "dept": "HR" }
-{ "id": 10741, "fname": "Julio", "lname": "Mattocks", "age": 38, "dept": "Sales" }
-{ "id": 10751, "fname": "Lance", "lname": "Kottke", "age": 34, "dept": "IT" }
-{ "id": 10761, "fname": "Kurt", "lname": "Liz", "age": 32, "dept": "HR" }
-{ "id": 10771, "fname": "Neva", "lname": "Barbeau", "age": 45, "dept": "Sales" }
-{ "id": 10781, "fname": "Karina", "lname": "Tuthill", "age": 46, "dept": "Payroll" }
-{ "id": 10791, "fname": "Maricela", "lname": "Cambron", "age": 36, "dept": "IT" }
-{ "id": 10809, "fname": "Clayton", "lname": "Delany", "age": 23, "dept": "IT" }
-{ "id": 10811, "fname": "Kubik", "lname": "Kuhn", "age": 27, "dept": "HR" }
-{ "id": 10821, "fname": "Allan", "lname": "Tomes", "age": 29, "dept": "Payroll" }
-{ "id": 10831, "fname": "Lonnie", "lname": "Aller", "age": 33, "dept": "Sales" }
-{ "id": 10841, "fname": "Neil", "lname": "Hurrell", "age": 26, "dept": "IT" }
-{ "id": 10851, "fname": "Clayton", "lname": "Engles", "age": 41, "dept": "HR" }
-{ "id": 10861, "fname": "Javier", "lname": "Gabrielson", "age": 39, "dept": "Payroll" }
-{ "id": 10871, "fname": "Allan", "lname": "Alejandre", "age": 48, "dept": "IT" }
-{ "id": 10881, "fname": "Julio", "lname": "Isa", "age": 38, "dept": "Sales" }
-{ "id": 10891, "fname": "Roslyn", "lname": "Simmerman", "age": 31, "dept": "IT" }
-{ "id": 10915, "fname": "Starner", "lname": "Stuart", "age": 25, "dept": "Sales" }
-{ "id": 10925, "fname": "Sofia", "lname": "Cuff", "age": 30, "dept": "HR" }
-{ "id": 10935, "fname": "Milagros", "lname": "Murguia", "age": 31, "dept": "IT" }
-{ "id": 10945, "fname": "Margery", "lname": "Haldeman", "age": 32, "dept": "IT" }
-{ "id": 10955, "fname": "Max", "lname": "Mell", "age": 33, "dept": "HR" }
-{ "id": 10965, "fname": "Micco", "lname": "Mercy", "age": 31, "dept": "Payroll" }
-{ "id": 10975, "fname": "Clare", "lname": "Vangieson", "age": 34, "dept": "IT" }
-{ "id": 10985, "fname": "Elnora", "lname": "Dimauro", "age": 35, "dept": "Sales" }
-{ "id": 10995, "fname": "Pearlie", "lname": "Kocian", "age": 38, "dept": "HR" }
-{ "id": 11007, "fname": "Yingyi", "lname": "Bu", "age": 27, "dept": "IT" }
-{ "id": 11263, "fname": "Tania", "lname": "Loffredo", "age": 25, "dept": "IT" }
-{ "id": 11410, "fname": "Clinton", "lname": "Fredricks", "age": 34, "dept": "IT" }
-{ "id": 11411, "fname": "Lance", "lname": "Farquhar", "age": 32, "dept": "HR" }
-{ "id": 11412, "fname": "Tabatha", "lname": "Crisler", "age": 33, "dept": "IT" }
-{ "id": 11413, "fname": "Max", "lname": "Durney", "age": 29, "dept": "IT" }
-{ "id": 11414, "fname": "Carmella", "lname": "Strauser", "age": 30, "dept": "Payroll" }
-{ "id": 11415, "fname": "Kelly", "lname": "Carrales", "age": 40, "dept": "IT" }
-{ "id": 11416, "fname": "Guy", "lname": "Merten", "age": 29, "dept": "Sales" }
-{ "id": 11417, "fname": "Noreen", "lname": "Ruhland", "age": 29, "dept": "IT" }
-{ "id": 11418, "fname": "Julio", "lname": "Damore", "age": 27, "dept": "Sales" }
-{ "id": 11419, "fname": "Selena", "lname": "Truby", "age": 25, "dept": "HR" }
-{ "id": 11420, "fname": "Alejandra", "lname": "Commons", "age": 30, "dept": "Sales" }
-{ "id": 11421, "fname": "Allyson", "lname": "Balk", "age": 30, "dept": "IT" }
-{ "id": 11422, "fname": "Nelson", "lname": "Byun", "age": 40, "dept": "Sales" }
-{ "id": 11423, "fname": "Christian", "lname": "Reidhead", "age": 40, "dept": "IT" }
-{ "id": 11424, "fname": "Pearlie", "lname": "Hopkin", "age": 48, "dept": "Payroll" }
-{ "id": 11425, "fname": "Nelson", "lname": "Wohlers", "age": 41, "dept": "HR" }
-{ "id": 11426, "fname": "Marcie", "lname": "Rasnake", "age": 42, "dept": "Sales" }
-{ "id": 11427, "fname": "Hugh", "lname": "Marshburn", "age": 43, "dept": "Payroll" }
-{ "id": 11428, "fname": "Mathew", "lname": "Marasco", "age": 45, "dept": "Sales" }
-{ "id": 11429, "fname": "Kurt", "lname": "Veres", "age": 32, "dept": "IT" }
-{ "id": 11430, "fname": "Julio", "lname": "Barkett", "age": 39, "dept": "Sales" }
-{ "id": 11863, "fname": "Darren", "lname": "Thorington", "age": 32, "dept": "Sales" }
-{ "id": 11999, "fname": "Susan", "lname": "Malaika", "age": 42, "dept": "HR" }
-{ "id": 12333, "fname": "Chen", "lname": "Li", "age": 42, "dept": "HR" }
-{ "id": 12963, "fname": "Neil", "lname": "Gunnerson", "age": 34, "dept": "IT" }
-{ "id": 13563, "fname": "Hazeltine", "lname": "Susan", "age": 29, "dept": "Sales" }
-{ "id": 13666, "fname": "Young Seok", "lname": "Kim", "age": 35, "dept": "Payroll" }
-{ "id": 14727, "fname": "Michael", "lname": "Carey", "age": 50, "dept": "Payroll" }
-{ "id": 15438, "fname": "Lakisha", "lname": "Quashie", "age": 29, "dept": "HR" }
-{ "id": 17444, "fname": "Sharad", "lname": "Mehrotra", "age": 42, "dept": "Sales" }
-{ "id": 17663, "fname": "Annabelle", "lname": "Nimmo", "age": 30, "dept": "Payroll" }
-{ "id": 18301, "fname": "Earlene", "lname": "Wallick", "age": 26, "dept": "HR" }
-{ "id": 18338, "fname": "Julio", "lname": "Bosket", "age": 28, "dept": "Payroll" }
-{ "id": 19555, "fname": "Tony", "lname": "Givargis", "age": 40, "dept": "Sales" }
-{ "id": 19763, "fname": "Ted", "lname": "Saini", "age": 31, "dept": "IT" }
-{ "id": 19941, "fname": "Khurram Faraaz", "lname": "Mohammed", "age": 30, "dept": "HR" }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/results/temp-dataset/insert-and-scan-dataset/insert-and-scan-dataset.1.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temp-dataset/insert-and-scan-dataset/insert-and-scan-dataset.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temp-dataset/insert-and-scan-dataset/insert-and-scan-dataset.1.adm
deleted file mode 100644
index 55b9a4d..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/temp-dataset/insert-and-scan-dataset/insert-and-scan-dataset.1.adm
+++ /dev/null
@@ -1,20 +0,0 @@
-{ "id": 1 }
-{ "id": 2 }
-{ "id": 3 }
-{ "id": 4 }
-{ "id": 5 }
-{ "id": 6 }
-{ "id": 7 }
-{ "id": 8 }
-{ "id": 9 }
-{ "id": 10 }
-{ "id": 11 }
-{ "id": 12 }
-{ "id": 13 }
-{ "id": 14 }
-{ "id": 15 }
-{ "id": 16 }
-{ "id": 17 }
-{ "id": 18 }
-{ "id": 19 }
-{ "id": 20 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/results/temp-dataset/q01_pricing_summary_report_nt/q01_pricing_summary_report_nt.1.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temp-dataset/q01_pricing_summary_report_nt/q01_pricing_summary_report_nt.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temp-dataset/q01_pricing_summary_report_nt/q01_pricing_summary_report_nt.1.adm
deleted file mode 100644
index c39ad6b..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/temp-dataset/q01_pricing_summary_report_nt/q01_pricing_summary_report_nt.1.adm
+++ /dev/null
@@ -1,4 +0,0 @@
-{ "l_returnflag": "A", "l_linestatus": "F", "sum_qty": 37474.0, "sum_base_price": 3.7569624640000015E7, "sum_disc_price": 3.5676192097E7, "sum_charge": 3.710141622242399E7, "ave_qty": 25.354533152909337, "ave_price": 25419.231826792973, "ave_disc": 0.050866035182679264, "count_order": 1478 }
-{ "l_returnflag": "N", "l_linestatus": "F", "sum_qty": 1041.0, "sum_base_price": 1041301.0700000001, "sum_disc_price": 999060.898, "sum_charge": 1036450.80228, "ave_qty": 27.394736842105264, "ave_price": 27402.659736842106, "ave_disc": 0.04289473684210526, "count_order": 38 }
-{ "l_returnflag": "N", "l_linestatus": "O", "sum_qty": 75168.0, "sum_base_price": 7.538495536999999E7, "sum_disc_price": 7.165316630340001E7, "sum_charge": 7.449879813307299E7, "ave_qty": 25.558653519211152, "ave_price": 25632.422771166268, "ave_disc": 0.04969738184291062, "count_order": 2941 }
-{ "l_returnflag": "R", "l_linestatus": "F", "sum_qty": 36511.0, "sum_base_price": 3.657084124E7, "sum_disc_price": 3.47384728758E7, "sum_charge": 3.616906011219301E7, "ave_qty": 25.059025394646532, "ave_price": 25100.09693891558, "ave_disc": 0.05002745367192856, "count_order": 1457 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/results/temp-dataset/scan-delete-rtree-secondary-index/scan-delete-rtree-secondary-index.1.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temp-dataset/scan-delete-rtree-secondary-index/scan-delete-rtree-secondary-index.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temp-dataset/scan-delete-rtree-secondary-index/scan-delete-rtree-secondary-index.1.adm
deleted file mode 100644
index 0e739c3..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/temp-dataset/scan-delete-rtree-secondary-index/scan-delete-rtree-secondary-index.1.adm
+++ /dev/null
@@ -1 +0,0 @@
-{ "id": 10 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/results/temp-dataset/scan-insert-persistent-to-temp/scan-insert-persistent-to-temp.1.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temp-dataset/scan-insert-persistent-to-temp/scan-insert-persistent-to-temp.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temp-dataset/scan-insert-persistent-to-temp/scan-insert-persistent-to-temp.1.adm
deleted file mode 100644
index 0dd81cd..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/temp-dataset/scan-insert-persistent-to-temp/scan-insert-persistent-to-temp.1.adm
+++ /dev/null
@@ -1,3 +0,0 @@
-{ "id": 10 }
-{ "id": 12 }
-{ "id": 20 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/results/temp-dataset/scan-insert-rtree-secondary-index/scan-insert-rtree-secondary-index.1.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temp-dataset/scan-insert-rtree-secondary-index/scan-insert-rtree-secondary-index.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temp-dataset/scan-insert-rtree-secondary-index/scan-insert-rtree-secondary-index.1.adm
deleted file mode 100644
index 0dd81cd..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/temp-dataset/scan-insert-rtree-secondary-index/scan-insert-rtree-secondary-index.1.adm
+++ /dev/null
@@ -1,3 +0,0 @@
-{ "id": 10 }
-{ "id": 12 }
-{ "id": 20 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/results/temp-dataset/scan-insert-temp-to-persistent/scan-insert-persistent-to-temp.1.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temp-dataset/scan-insert-temp-to-persistent/scan-insert-persistent-to-temp.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temp-dataset/scan-insert-temp-to-persistent/scan-insert-persistent-to-temp.1.adm
deleted file mode 100644
index 0dd81cd..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/temp-dataset/scan-insert-temp-to-persistent/scan-insert-persistent-to-temp.1.adm
+++ /dev/null
@@ -1,3 +0,0 @@
-{ "id": 10 }
-{ "id": 12 }
-{ "id": 20 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/results/temp-dataset/temp_primary_plus_ngram_flush/temp_primary_plus_ngram_flush.3.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temp-dataset/temp_primary_plus_ngram_flush/temp_primary_plus_ngram_flush.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temp-dataset/temp_primary_plus_ngram_flush/temp_primary_plus_ngram_flush.3.adm
deleted file mode 100644
index 83b33d2..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/temp-dataset/temp_primary_plus_ngram_flush/temp_primary_plus_ngram_flush.3.adm
+++ /dev/null
@@ -1 +0,0 @@
-1000

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/delete-from-loaded-dataset-with-index/delete-from-loaded-dataset-with-index.1.ast
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/delete-from-loaded-dataset-with-index/delete-from-loaded-dataset-with-index.1.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/delete-from-loaded-dataset-with-index/delete-from-loaded-dataset-with-index.1.ast
deleted file mode 100644
index 2192c6c..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/delete-from-loaded-dataset-with-index/delete-from-loaded-dataset-with-index.1.ast
+++ /dev/null
@@ -1,22 +0,0 @@
-DataverseUse test
-TypeDecl LineItemType [
-  closed RecordType {
-    l_orderkey : bigint,
-    l_partkey : bigint,
-    l_suppkey : bigint,
-    l_linenumber : bigint,
-    l_quantity : bigint,
-    l_extendedprice : double,
-    l_discount : double,
-    l_tax : double,
-    l_returnflag : string,
-    l_linestatus : string,
-    l_shipdate : string,
-    l_commitdate : string,
-    l_receiptdate : string,
-    l_shipinstruct : string,
-    l_shipmode : string,
-    l_comment : string
-  }
-]
-DatasetDecl LineItem(LineItemType) partitioned by [[l_orderkey], [l_linenumber]]

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/delete-from-loaded-dataset-with-index/delete-from-loaded-dataset-with-index.2.ast
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/delete-from-loaded-dataset-with-index/delete-from-loaded-dataset-with-index.2.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/delete-from-loaded-dataset-with-index/delete-from-loaded-dataset-with-index.2.ast
deleted file mode 100644
index 916a59e..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/delete-from-loaded-dataset-with-index/delete-from-loaded-dataset-with-index.2.ast
+++ /dev/null
@@ -1 +0,0 @@
-DataverseUse test

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/delete-from-loaded-dataset-with-index/delete-from-loaded-dataset-with-index.3.ast
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/delete-from-loaded-dataset-with-index/delete-from-loaded-dataset-with-index.3.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/delete-from-loaded-dataset-with-index/delete-from-loaded-dataset-with-index.3.ast
deleted file mode 100644
index 916a59e..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/delete-from-loaded-dataset-with-index/delete-from-loaded-dataset-with-index.3.ast
+++ /dev/null
@@ -1 +0,0 @@
-DataverseUse test

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/delete-from-loaded-dataset-with-index/delete-from-loaded-dataset-with-index.4.ast
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/delete-from-loaded-dataset-with-index/delete-from-loaded-dataset-with-index.4.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/delete-from-loaded-dataset-with-index/delete-from-loaded-dataset-with-index.4.ast
deleted file mode 100644
index 916a59e..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/delete-from-loaded-dataset-with-index/delete-from-loaded-dataset-with-index.4.ast
+++ /dev/null
@@ -1 +0,0 @@
-DataverseUse test

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/delete-from-loaded-dataset-with-index/delete-from-loaded-dataset-with-index.5.ast
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/delete-from-loaded-dataset-with-index/delete-from-loaded-dataset-with-index.5.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/delete-from-loaded-dataset-with-index/delete-from-loaded-dataset-with-index.5.ast
deleted file mode 100644
index 39ed05e..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/delete-from-loaded-dataset-with-index/delete-from-loaded-dataset-with-index.5.ast
+++ /dev/null
@@ -1,31 +0,0 @@
-DataverseUse test
-Query:
-SELECT ELEMENT [
-Variable [ Name=$c ]
-]
-FROM [  FunctionCall asterix.dataset@1[
-    LiteralExpr [STRING] [LineItem]
-  ]
-  AS Variable [ Name=$c ]
-]
-Where
-  OperatorExpr [
-    FieldAccessor [
-      Variable [ Name=$c ]
-      Field=l_suppkey
-    ]
-    <
-    LiteralExpr [LONG] [150]
-  ]
-Orderby
-  FieldAccessor [
-    Variable [ Name=$c ]
-    Field=l_orderkey
-  ]
-  ASC
-  FieldAccessor [
-    Variable [ Name=$c ]
-    Field=l_linenumber
-  ]
-  ASC
-

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/delete-from-loaded-dataset/delete-from-loaded-dataset.1.ast
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/delete-from-loaded-dataset/delete-from-loaded-dataset.1.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/delete-from-loaded-dataset/delete-from-loaded-dataset.1.ast
deleted file mode 100644
index 2192c6c..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/delete-from-loaded-dataset/delete-from-loaded-dataset.1.ast
+++ /dev/null
@@ -1,22 +0,0 @@
-DataverseUse test
-TypeDecl LineItemType [
-  closed RecordType {
-    l_orderkey : bigint,
-    l_partkey : bigint,
-    l_suppkey : bigint,
-    l_linenumber : bigint,
-    l_quantity : bigint,
-    l_extendedprice : double,
-    l_discount : double,
-    l_tax : double,
-    l_returnflag : string,
-    l_linestatus : string,
-    l_shipdate : string,
-    l_commitdate : string,
-    l_receiptdate : string,
-    l_shipinstruct : string,
-    l_shipmode : string,
-    l_comment : string
-  }
-]
-DatasetDecl LineItem(LineItemType) partitioned by [[l_orderkey], [l_linenumber]]

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/delete-from-loaded-dataset/delete-from-loaded-dataset.2.ast
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/delete-from-loaded-dataset/delete-from-loaded-dataset.2.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/delete-from-loaded-dataset/delete-from-loaded-dataset.2.ast
deleted file mode 100644
index 916a59e..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/delete-from-loaded-dataset/delete-from-loaded-dataset.2.ast
+++ /dev/null
@@ -1 +0,0 @@
-DataverseUse test

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/delete-from-loaded-dataset/delete-from-loaded-dataset.3.ast
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/delete-from-loaded-dataset/delete-from-loaded-dataset.3.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/delete-from-loaded-dataset/delete-from-loaded-dataset.3.ast
deleted file mode 100644
index 8a19724..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/delete-from-loaded-dataset/delete-from-loaded-dataset.3.ast
+++ /dev/null
@@ -1,22 +0,0 @@
-DataverseUse test
-Query:
-SELECT ELEMENT [
-Variable [ Name=$c ]
-]
-FROM [  FunctionCall asterix.dataset@1[
-    LiteralExpr [STRING] [LineItem]
-  ]
-  AS Variable [ Name=$c ]
-]
-Orderby
-  FieldAccessor [
-    Variable [ Name=$c ]
-    Field=l_orderkey
-  ]
-  ASC
-  FieldAccessor [
-    Variable [ Name=$c ]
-    Field=l_linenumber
-  ]
-  ASC
-

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/drop-empty-secondary-indexes/drop-empty-secondary-indexes.1.ast
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/drop-empty-secondary-indexes/drop-empty-secondary-indexes.1.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/drop-empty-secondary-indexes/drop-empty-secondary-indexes.1.ast
deleted file mode 100644
index 27e4cf5..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/drop-empty-secondary-indexes/drop-empty-secondary-indexes.1.ast
+++ /dev/null
@@ -1,22 +0,0 @@
-DataverseUse test
-TypeDecl Name [
-  open RecordType {
-    first : string,
-    last : string
-  }
-]
-TypeDecl Person [
-  open RecordType {
-    name : Name
-  }
-]
-TypeDecl TestType [
-  open RecordType {
-    id : integer,
-    name : string,
-    locn : point,
-    zip : string,
-    person : Person
-  }
-]
-DatasetDecl t1(TestType) partitioned by [[id]]

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/drop-empty-secondary-indexes/drop-empty-secondary-indexes.2.ast
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/drop-empty-secondary-indexes/drop-empty-secondary-indexes.2.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/drop-empty-secondary-indexes/drop-empty-secondary-indexes.2.ast
deleted file mode 100644
index e69de29..0000000

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/drop-empty-secondary-indexes/drop-empty-secondary-indexes.3.ast
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/drop-empty-secondary-indexes/drop-empty-secondary-indexes.3.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/drop-empty-secondary-indexes/drop-empty-secondary-indexes.3.ast
deleted file mode 100644
index b647c08..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/drop-empty-secondary-indexes/drop-empty-secondary-indexes.3.ast
+++ /dev/null
@@ -1,29 +0,0 @@
-Query:
-SELECT ELEMENT [
-Variable [ Name=$l ]
-]
-FROM [  FunctionCall asterix.dataset@1[
-    LiteralExpr [STRING] [Metadata.Index]
-  ]
-  AS Variable [ Name=$l ]
-]
-Where
-  OperatorExpr [
-    OperatorExpr [
-      FieldAccessor [
-        Variable [ Name=$l ]
-        Field=IsPrimary
-      ]
-      =
-      LiteralExpr [FALSE]
-    ]
-    and
-    OperatorExpr [
-      FieldAccessor [
-        Variable [ Name=$l ]
-        Field=DataverseName
-      ]
-      =
-      LiteralExpr [STRING] [test]
-    ]
-  ]

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/drop-index/drop-index.1.ast
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/drop-index/drop-index.1.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/drop-index/drop-index.1.ast
deleted file mode 100644
index 13f1f01..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/drop-index/drop-index.1.ast
+++ /dev/null
@@ -1,22 +0,0 @@
-DataverseUse test
-TypeDecl Schema [
-  closed RecordType {
-    unique1 : bigint,
-    unique2 : bigint,
-    two : bigint,
-    four : bigint,
-    ten : bigint,
-    twenty : bigint,
-    onePercent : bigint,
-    tenPercent : bigint,
-    twentyPercent : bigint,
-    fiftyPercent : bigint,
-    unique3 : bigint,
-    evenOnePercent : bigint,
-    oddOnePercent : bigint,
-    stringu1 : string,
-    stringu2 : string,
-    string4 : string
-  }
-]
-DatasetDecl t1(Schema) partitioned by [[unique2]]

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/drop-index/drop-index.2.ast
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/drop-index/drop-index.2.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/drop-index/drop-index.2.ast
deleted file mode 100644
index 916a59e..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/drop-index/drop-index.2.ast
+++ /dev/null
@@ -1 +0,0 @@
-DataverseUse test

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/drop-index/drop-index.3.ast
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/drop-index/drop-index.3.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/drop-index/drop-index.3.ast
deleted file mode 100644
index 916a59e..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/drop-index/drop-index.3.ast
+++ /dev/null
@@ -1 +0,0 @@
-DataverseUse test

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/drop-index/drop-index.4.ast
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/drop-index/drop-index.4.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/drop-index/drop-index.4.ast
deleted file mode 100644
index da430f1..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/drop-index/drop-index.4.ast
+++ /dev/null
@@ -1,30 +0,0 @@
-DataverseUse test
-Query:
-SELECT ELEMENT [
-Variable [ Name=$a ]
-]
-FROM [  FunctionCall asterix.dataset@1[
-    LiteralExpr [STRING] [t1]
-  ]
-  AS Variable [ Name=$a ]
-]
-Where
-  OperatorExpr [
-    OperatorExpr [
-      FieldAccessor [
-        Variable [ Name=$a ]
-        Field=unique1
-      ]
-      >
-      LiteralExpr [LONG] [10]
-    ]
-    and
-    OperatorExpr [
-      FieldAccessor [
-        Variable [ Name=$a ]
-        Field=stringu1
-      ]
-      =
-      LiteralExpr [STRING] [DGAAAAXXXXXXXXXXXXXXXXXXX]
-    ]
-  ]

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/empty-load-with-index/empty-load-with-index.1.ast
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/empty-load-with-index/empty-load-with-index.1.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/empty-load-with-index/empty-load-with-index.1.ast
deleted file mode 100644
index 17dc8b5..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/empty-load-with-index/empty-load-with-index.1.ast
+++ /dev/null
@@ -1,22 +0,0 @@
-DataverseUse test
-TypeDecl LineItemType [
-  closed RecordType {
-    l_orderkey : bigint,
-    l_partkey : bigint,
-    l_suppkey : bigint,
-    l_linenumber : bigint,
-    l_quantity : double,
-    l_extendedprice : double,
-    l_discount : double,
-    l_tax : double,
-    l_returnflag : string,
-    l_linestatus : string,
-    l_shipdate : string,
-    l_commitdate : string,
-    l_receiptdate : string,
-    l_shipinstruct : string,
-    l_shipmode : string,
-    l_comment : string
-  }
-]
-DatasetDecl LineItem(LineItemType) partitioned by [[l_orderkey], [l_linenumber]]

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/empty-load-with-index/empty-load-with-index.2.ast
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/empty-load-with-index/empty-load-with-index.2.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/empty-load-with-index/empty-load-with-index.2.ast
deleted file mode 100644
index 916a59e..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/empty-load-with-index/empty-load-with-index.2.ast
+++ /dev/null
@@ -1 +0,0 @@
-DataverseUse test

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/empty-load-with-index/empty-load-with-index.3.ast
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/empty-load-with-index/empty-load-with-index.3.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/empty-load-with-index/empty-load-with-index.3.ast
deleted file mode 100644
index 4241899..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/empty-load-with-index/empty-load-with-index.3.ast
+++ /dev/null
@@ -1,24 +0,0 @@
-DataverseUse test
-Query:
-SELECT ELEMENT [
-Variable [ Name=$c ]
-]
-FROM [  FunctionCall asterix.dataset@1[
-    LiteralExpr [STRING] [LineItem]
-  ]
-  AS Variable [ Name=$c ]
-]
-Orderby
-  FieldAccessor [
-    Variable [ Name=$c ]
-    Field=l_orderkey
-  ]
-  ASC
-  FieldAccessor [
-    Variable [ Name=$c ]
-    Field=l_linenumber
-  ]
-  ASC
-
-Limit
-  LiteralExpr [LONG] [1]

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/insert-and-scan-dataset-with-index/insert-and-scan-dataset-with-index.1.ast
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/insert-and-scan-dataset-with-index/insert-and-scan-dataset-with-index.1.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/insert-and-scan-dataset-with-index/insert-and-scan-dataset-with-index.1.ast
deleted file mode 100644
index d8ba369..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/insert-and-scan-dataset-with-index/insert-and-scan-dataset-with-index.1.ast
+++ /dev/null
@@ -1,10 +0,0 @@
-TypeDecl Emp [
-  closed RecordType {
-    id : bigint,
-    fname : string,
-    lname : string,
-    age : bigint,
-    dept : string
-  }
-]
-DatasetDecl employee(Emp) partitioned by [[id]]

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/insert-and-scan-dataset-with-index/insert-and-scan-dataset-with-index.2.ast
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/insert-and-scan-dataset-with-index/insert-and-scan-dataset-with-index.2.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/insert-and-scan-dataset-with-index/insert-and-scan-dataset-with-index.2.ast
deleted file mode 100644
index 916a59e..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/insert-and-scan-dataset-with-index/insert-and-scan-dataset-with-index.2.ast
+++ /dev/null
@@ -1 +0,0 @@
-DataverseUse test

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/insert-and-scan-dataset-with-index/insert-and-scan-dataset-with-index.3.ast
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/insert-and-scan-dataset-with-index/insert-and-scan-dataset-with-index.3.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/insert-and-scan-dataset-with-index/insert-and-scan-dataset-with-index.3.ast
deleted file mode 100644
index 514247c..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/insert-and-scan-dataset-with-index/insert-and-scan-dataset-with-index.3.ast
+++ /dev/null
@@ -1,17 +0,0 @@
-DataverseUse test
-Query:
-SELECT ELEMENT [
-Variable [ Name=$l ]
-]
-FROM [  FunctionCall asterix.dataset@1[
-    LiteralExpr [STRING] [test.employee]
-  ]
-  AS Variable [ Name=$l ]
-]
-Orderby
-  FieldAccessor [
-    Variable [ Name=$l ]
-    Field=id
-  ]
-  ASC
-

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/insert-and-scan-dataset/insert-and-scan-dataset.1.ast
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/insert-and-scan-dataset/insert-and-scan-dataset.1.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/insert-and-scan-dataset/insert-and-scan-dataset.1.ast
deleted file mode 100644
index d546e56..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/insert-and-scan-dataset/insert-and-scan-dataset.1.ast
+++ /dev/null
@@ -1,7 +0,0 @@
-DataverseUse test
-TypeDecl myDataType [
-  open RecordType {
-    id : bigint
-  }
-]
-DatasetDecl myData(myDataType) partitioned by [[id]]

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/insert-and-scan-dataset/insert-and-scan-dataset.2.ast
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/insert-and-scan-dataset/insert-and-scan-dataset.2.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/insert-and-scan-dataset/insert-and-scan-dataset.2.ast
deleted file mode 100644
index 916a59e..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/insert-and-scan-dataset/insert-and-scan-dataset.2.ast
+++ /dev/null
@@ -1 +0,0 @@
-DataverseUse test

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/insert-and-scan-dataset/insert-and-scan-dataset.3.ast
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/insert-and-scan-dataset/insert-and-scan-dataset.3.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/insert-and-scan-dataset/insert-and-scan-dataset.3.ast
deleted file mode 100644
index c733ebc..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/insert-and-scan-dataset/insert-and-scan-dataset.3.ast
+++ /dev/null
@@ -1,17 +0,0 @@
-DataverseUse test
-Query:
-SELECT ELEMENT [
-Variable [ Name=$c ]
-]
-FROM [  FunctionCall asterix.dataset@1[
-    LiteralExpr [STRING] [myData]
-  ]
-  AS Variable [ Name=$c ]
-]
-Orderby
-  FieldAccessor [
-    Variable [ Name=$c ]
-    Field=id
-  ]
-  ASC
-

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/q01_pricing_summary_report_nt/q01_pricing_summary_report_nt.1.ast
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/q01_pricing_summary_report_nt/q01_pricing_summary_report_nt.1.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/q01_pricing_summary_report_nt/q01_pricing_summary_report_nt.1.ast
deleted file mode 100644
index c1ede5e..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/q01_pricing_summary_report_nt/q01_pricing_summary_report_nt.1.ast
+++ /dev/null
@@ -1,22 +0,0 @@
-DataverseUse tpch
-TypeDecl LineItemType [
-  closed RecordType {
-    l_orderkey : bigint,
-    l_partkey : bigint,
-    l_suppkey : bigint,
-    l_linenumber : bigint,
-    l_quantity : double,
-    l_extendedprice : double,
-    l_discount : double,
-    l_tax : double,
-    l_returnflag : string,
-    l_linestatus : string,
-    l_shipdate : string,
-    l_commitdate : string,
-    l_receiptdate : string,
-    l_shipinstruct : string,
-    l_shipmode : string,
-    l_comment : string
-  }
-]
-DatasetDecl LineItem(LineItemType) partitioned by [[l_orderkey], [l_linenumber]]

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/q01_pricing_summary_report_nt/q01_pricing_summary_report_nt.2.ast
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/q01_pricing_summary_report_nt/q01_pricing_summary_report_nt.2.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/q01_pricing_summary_report_nt/q01_pricing_summary_report_nt.2.ast
deleted file mode 100644
index 0a5cca1..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/q01_pricing_summary_report_nt/q01_pricing_summary_report_nt.2.ast
+++ /dev/null
@@ -1 +0,0 @@
-DataverseUse tpch

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/q01_pricing_summary_report_nt/q01_pricing_summary_report_nt.3.ast
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/q01_pricing_summary_report_nt/q01_pricing_summary_report_nt.3.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/q01_pricing_summary_report_nt/q01_pricing_summary_report_nt.3.ast
deleted file mode 100644
index 2d5f7ae..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/q01_pricing_summary_report_nt/q01_pricing_summary_report_nt.3.ast
+++ /dev/null
@@ -1,249 +0,0 @@
-DataverseUse tpch
-Set import-private-functions=true
-Query:
-SELECT ELEMENT [
-RecordConstructor [
-  (
-    LiteralExpr [STRING] [l_returnflag]
-    :
-    Variable [ Name=$l_returnflag ]
-  )
-  (
-    LiteralExpr [STRING] [l_linestatus]
-    :
-    Variable [ Name=$l_linestatus ]
-  )
-  (
-    LiteralExpr [STRING] [sum_qty]
-    :
-    FunctionCall asterix.sql-sum@1[
-      (
-        SELECT ELEMENT [
-        FieldAccessor [
-          FieldAccessor [
-            Variable [ Name=#2 ]
-            Field=l
-          ]
-          Field=l_quantity
-        ]
-        ]
-        FROM [          Variable [ Name=#1 ]
-          AS Variable [ Name=#2 ]
-        ]
-      )
-    ]
-  )
-  (
-    LiteralExpr [STRING] [sum_base_price]
-    :
-    FunctionCall asterix.sql-sum@1[
-      (
-        SELECT ELEMENT [
-        FieldAccessor [
-          FieldAccessor [
-            Variable [ Name=#3 ]
-            Field=l
-          ]
-          Field=l_extendedprice
-        ]
-        ]
-        FROM [          Variable [ Name=#1 ]
-          AS Variable [ Name=#3 ]
-        ]
-      )
-    ]
-  )
-  (
-    LiteralExpr [STRING] [sum_disc_price]
-    :
-    FunctionCall asterix.sql-sum@1[
-      (
-        SELECT ELEMENT [
-        OperatorExpr [
-          FieldAccessor [
-            FieldAccessor [
-              Variable [ Name=#4 ]
-              Field=l
-            ]
-            Field=l_extendedprice
-          ]
-          *
-          OperatorExpr [
-            LiteralExpr [LONG] [1]
-            -
-            FieldAccessor [
-              FieldAccessor [
-                Variable [ Name=#4 ]
-                Field=l
-              ]
-              Field=l_discount
-            ]
-          ]
-        ]
-        ]
-        FROM [          Variable [ Name=#1 ]
-          AS Variable [ Name=#4 ]
-        ]
-      )
-    ]
-  )
-  (
-    LiteralExpr [STRING] [sum_charge]
-    :
-    FunctionCall asterix.sql-sum@1[
-      (
-        SELECT ELEMENT [
-        OperatorExpr [
-          FieldAccessor [
-            FieldAccessor [
-              Variable [ Name=#5 ]
-              Field=l
-            ]
-            Field=l_extendedprice
-          ]
-          *
-          OperatorExpr [
-            LiteralExpr [LONG] [1]
-            -
-            FieldAccessor [
-              FieldAccessor [
-                Variable [ Name=#5 ]
-                Field=l
-              ]
-              Field=l_discount
-            ]
-          ]
-          *
-          OperatorExpr [
-            LiteralExpr [LONG] [1]
-            +
-            FieldAccessor [
-              FieldAccessor [
-                Variable [ Name=#5 ]
-                Field=l
-              ]
-              Field=l_tax
-            ]
-          ]
-        ]
-        ]
-        FROM [          Variable [ Name=#1 ]
-          AS Variable [ Name=#5 ]
-        ]
-      )
-    ]
-  )
-  (
-    LiteralExpr [STRING] [ave_qty]
-    :
-    FunctionCall asterix.sql-avg@1[
-      (
-        SELECT ELEMENT [
-        FieldAccessor [
-          FieldAccessor [
-            Variable [ Name=#6 ]
-            Field=l
-          ]
-          Field=l_quantity
-        ]
-        ]
-        FROM [          Variable [ Name=#1 ]
-          AS Variable [ Name=#6 ]
-        ]
-      )
-    ]
-  )
-  (
-    LiteralExpr [STRING] [ave_price]
-    :
-    FunctionCall asterix.sql-avg@1[
-      (
-        SELECT ELEMENT [
-        FieldAccessor [
-          FieldAccessor [
-            Variable [ Name=#7 ]
-            Field=l
-          ]
-          Field=l_extendedprice
-        ]
-        ]
-        FROM [          Variable [ Name=#1 ]
-          AS Variable [ Name=#7 ]
-        ]
-      )
-    ]
-  )
-  (
-    LiteralExpr [STRING] [ave_disc]
-    :
-    FunctionCall asterix.sql-avg@1[
-      (
-        SELECT ELEMENT [
-        FieldAccessor [
-          FieldAccessor [
-            Variable [ Name=#8 ]
-            Field=l
-          ]
-          Field=l_discount
-        ]
-        ]
-        FROM [          Variable [ Name=#1 ]
-          AS Variable [ Name=#8 ]
-        ]
-      )
-    ]
-  )
-  (
-    LiteralExpr [STRING] [count_order]
-    :
-    FunctionCall asterix.sql-count@1[
-      (
-        SELECT ELEMENT [
-        LiteralExpr [LONG] [1]
-        ]
-        FROM [          Variable [ Name=#1 ]
-          AS Variable [ Name=#9 ]
-        ]
-      )
-    ]
-  )
-]
-]
-FROM [  FunctionCall asterix.dataset@1[
-    LiteralExpr [STRING] [LineItem]
-  ]
-  AS Variable [ Name=$l ]
-]
-Where
-  OperatorExpr [
-    FieldAccessor [
-      Variable [ Name=$l ]
-      Field=l_shipdate
-    ]
-    <=
-    LiteralExpr [STRING] [1998-09-02]
-  ]
-Groupby
-  Variable [ Name=$l_returnflag ]
-  :=
-  FieldAccessor [
-    Variable [ Name=$l ]
-    Field=l_returnflag
-  ]
-  Variable [ Name=$l_linestatus ]
-  :=
-  FieldAccessor [
-    Variable [ Name=$l ]
-    Field=l_linestatus
-  ]
-  GROUP AS Variable [ Name=#1 ]
-  (
-    l:=Variable [ Name=$l ]
-  )
-
-Orderby
-  Variable [ Name=$l_returnflag ]
-  ASC
-  Variable [ Name=$l_linestatus ]
-  ASC
-

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/scan-delete-rtree-secondary-index/scan-delete-rtree-secondary-index.1.ast
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/scan-delete-rtree-secondary-index/scan-delete-rtree-secondary-index.1.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/scan-delete-rtree-secondary-index/scan-delete-rtree-secondary-index.1.ast
deleted file mode 100644
index 9e01d19..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/scan-delete-rtree-secondary-index/scan-delete-rtree-secondary-index.1.ast
+++ /dev/null
@@ -1,15 +0,0 @@
-DataverseUse test
-TypeDecl MyRecord [
-  closed RecordType {
-    id : bigint,
-    point : point,
-    kwds : string,
-    line1 : line,
-    line2 : line,
-    poly1 : polygon,
-    poly2 : polygon,
-    rec : rectangle,
-    circle : circle
-  }
-]
-DatasetDecl MyData(MyRecord) partitioned by [[id]]

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/scan-delete-rtree-secondary-index/scan-delete-rtree-secondary-index.2.ast
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/scan-delete-rtree-secondary-index/scan-delete-rtree-secondary-index.2.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/scan-delete-rtree-secondary-index/scan-delete-rtree-secondary-index.2.ast
deleted file mode 100644
index 916a59e..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/scan-delete-rtree-secondary-index/scan-delete-rtree-secondary-index.2.ast
+++ /dev/null
@@ -1 +0,0 @@
-DataverseUse test

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/scan-delete-rtree-secondary-index/scan-delete-rtree-secondary-index.3.ast
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/scan-delete-rtree-secondary-index/scan-delete-rtree-secondary-index.3.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/scan-delete-rtree-secondary-index/scan-delete-rtree-secondary-index.3.ast
deleted file mode 100644
index 916a59e..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/scan-delete-rtree-secondary-index/scan-delete-rtree-secondary-index.3.ast
+++ /dev/null
@@ -1 +0,0 @@
-DataverseUse test

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/scan-delete-rtree-secondary-index/scan-delete-rtree-secondary-index.4.ast
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/scan-delete-rtree-secondary-index/scan-delete-rtree-secondary-index.4.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/scan-delete-rtree-secondary-index/scan-delete-rtree-secondary-index.4.ast
deleted file mode 100644
index 916a59e..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/scan-delete-rtree-secondary-index/scan-delete-rtree-secondary-index.4.ast
+++ /dev/null
@@ -1 +0,0 @@
-DataverseUse test

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/scan-delete-rtree-secondary-index/scan-delete-rtree-secondary-index.5.ast
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/scan-delete-rtree-secondary-index/scan-delete-rtree-secondary-index.5.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/scan-delete-rtree-secondary-index/scan-delete-rtree-secondary-index.5.ast
deleted file mode 100644
index 6aa79ff..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/scan-delete-rtree-secondary-index/scan-delete-rtree-secondary-index.5.ast
+++ /dev/null
@@ -1,45 +0,0 @@
-DataverseUse test
-Query:
-SELECT ELEMENT [
-RecordConstructor [
-  (
-    LiteralExpr [STRING] [id]
-    :
-    FieldAccessor [
-      Variable [ Name=$o ]
-      Field=id
-    ]
-  )
-]
-]
-FROM [  FunctionCall asterix.dataset@1[
-    LiteralExpr [STRING] [MyData]
-  ]
-  AS Variable [ Name=$o ]
-]
-Where
-  FunctionCall test.spatial-intersect@2[
-    FieldAccessor [
-      Variable [ Name=$o ]
-      Field=point
-    ]
-    FunctionCall test.create-polygon@1[
-      OrderedListConstructor [
-        LiteralExpr [DOUBLE] [0.0]
-        LiteralExpr [DOUBLE] [1.0]
-        LiteralExpr [DOUBLE] [0.0]
-        LiteralExpr [DOUBLE] [4.0]
-        LiteralExpr [DOUBLE] [12.0]
-        LiteralExpr [DOUBLE] [4.0]
-        LiteralExpr [DOUBLE] [12.0]
-        LiteralExpr [DOUBLE] [1.0]
-      ]
-    ]
-  ]
-Orderby
-  FieldAccessor [
-    Variable [ Name=$o ]
-    Field=id
-  ]
-  ASC
-

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/scan-insert-persistent-to-temp/scan-insert-persistent-to-temp.1.ast
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/scan-insert-persistent-to-temp/scan-insert-persistent-to-temp.1.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/scan-insert-persistent-to-temp/scan-insert-persistent-to-temp.1.ast
deleted file mode 100644
index 79e3a90..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/scan-insert-persistent-to-temp/scan-insert-persistent-to-temp.1.ast
+++ /dev/null
@@ -1,22 +0,0 @@
-DataverseUse test
-TypeDecl MyRecord [
-  closed RecordType {
-    id : bigint,
-    point : point,
-    kwds : string,
-    line1 : line,
-    line2 : line,
-    poly1 : polygon,
-    poly2 : polygon,
-    rec : rectangle,
-    circle : circle
-  }
-]
-TypeDecl MyMiniRecord [
-  closed RecordType {
-    id : bigint,
-    point : point
-  }
-]
-DatasetDecl MyData(MyRecord) partitioned by [[id]]
-DatasetDecl MyMiniData(MyMiniRecord) partitioned by [[id]]

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/scan-insert-persistent-to-temp/scan-insert-persistent-to-temp.2.ast
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/scan-insert-persistent-to-temp/scan-insert-persistent-to-temp.2.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/scan-insert-persistent-to-temp/scan-insert-persistent-to-temp.2.ast
deleted file mode 100644
index 916a59e..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/scan-insert-persistent-to-temp/scan-insert-persistent-to-temp.2.ast
+++ /dev/null
@@ -1 +0,0 @@
-DataverseUse test

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/scan-insert-persistent-to-temp/scan-insert-persistent-to-temp.3.ast
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/scan-insert-persistent-to-temp/scan-insert-persistent-to-temp.3.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/scan-insert-persistent-to-temp/scan-insert-persistent-to-temp.3.ast
deleted file mode 100644
index 916a59e..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/scan-insert-persistent-to-temp/scan-insert-persistent-to-temp.3.ast
+++ /dev/null
@@ -1 +0,0 @@
-DataverseUse test

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/scan-insert-persistent-to-temp/scan-insert-persistent-to-temp.4.ast
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/scan-insert-persistent-to-temp/scan-insert-persistent-to-temp.4.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/scan-insert-persistent-to-temp/scan-insert-persistent-to-temp.4.ast
deleted file mode 100644
index 916a59e..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/scan-insert-persistent-to-temp/scan-insert-persistent-to-temp.4.ast
+++ /dev/null
@@ -1 +0,0 @@
-DataverseUse test

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/scan-insert-persistent-to-temp/scan-insert-persistent-to-temp.5.ast
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/scan-insert-persistent-to-temp/scan-insert-persistent-to-temp.5.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/scan-insert-persistent-to-temp/scan-insert-persistent-to-temp.5.ast
deleted file mode 100644
index 0b75e50..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/scan-insert-persistent-to-temp/scan-insert-persistent-to-temp.5.ast
+++ /dev/null
@@ -1,45 +0,0 @@
-DataverseUse test
-Query:
-SELECT ELEMENT [
-RecordConstructor [
-  (
-    LiteralExpr [STRING] [id]
-    :
-    FieldAccessor [
-      Variable [ Name=$o ]
-      Field=id
-    ]
-  )
-]
-]
-FROM [  FunctionCall asterix.dataset@1[
-    LiteralExpr [STRING] [MyMiniData]
-  ]
-  AS Variable [ Name=$o ]
-]
-Where
-  FunctionCall test.spatial-intersect@2[
-    FieldAccessor [
-      Variable [ Name=$o ]
-      Field=point
-    ]
-    FunctionCall test.create-polygon@1[
-      OrderedListConstructor [
-        LiteralExpr [DOUBLE] [0.0]
-        LiteralExpr [DOUBLE] [1.0]
-        LiteralExpr [DOUBLE] [0.0]
-        LiteralExpr [DOUBLE] [4.0]
-        LiteralExpr [DOUBLE] [12.0]
-        LiteralExpr [DOUBLE] [4.0]
-        LiteralExpr [DOUBLE] [12.0]
-        LiteralExpr [DOUBLE] [1.0]
-      ]
-    ]
-  ]
-Orderby
-  FieldAccessor [
-    Variable [ Name=$o ]
-    Field=id
-  ]
-  ASC
-

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/scan-insert-rtree-secondary-index/scan-insert-rtree-secondary-index.1.ast
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/scan-insert-rtree-secondary-index/scan-insert-rtree-secondary-index.1.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/scan-insert-rtree-secondary-index/scan-insert-rtree-secondary-index.1.ast
deleted file mode 100644
index 79e3a90..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/scan-insert-rtree-secondary-index/scan-insert-rtree-secondary-index.1.ast
+++ /dev/null
@@ -1,22 +0,0 @@
-DataverseUse test
-TypeDecl MyRecord [
-  closed RecordType {
-    id : bigint,
-    point : point,
-    kwds : string,
-    line1 : line,
-    line2 : line,
-    poly1 : polygon,
-    poly2 : polygon,
-    rec : rectangle,
-    circle : circle
-  }
-]
-TypeDecl MyMiniRecord [
-  closed RecordType {
-    id : bigint,
-    point : point
-  }
-]
-DatasetDecl MyData(MyRecord) partitioned by [[id]]
-DatasetDecl MyMiniData(MyMiniRecord) partitioned by [[id]]

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/scan-insert-rtree-secondary-index/scan-insert-rtree-secondary-index.2.ast
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/scan-insert-rtree-secondary-index/scan-insert-rtree-secondary-index.2.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/scan-insert-rtree-secondary-index/scan-insert-rtree-secondary-index.2.ast
deleted file mode 100644
index 916a59e..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/scan-insert-rtree-secondary-index/scan-insert-rtree-secondary-index.2.ast
+++ /dev/null
@@ -1 +0,0 @@
-DataverseUse test

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/scan-insert-rtree-secondary-index/scan-insert-rtree-secondary-index.3.ast
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/scan-insert-rtree-secondary-index/scan-insert-rtree-secondary-index.3.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/scan-insert-rtree-secondary-index/scan-insert-rtree-secondary-index.3.ast
deleted file mode 100644
index 916a59e..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/scan-insert-rtree-secondary-index/scan-insert-rtree-secondary-index.3.ast
+++ /dev/null
@@ -1 +0,0 @@
-DataverseUse test

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/scan-insert-rtree-secondary-index/scan-insert-rtree-secondary-index.4.ast
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/scan-insert-rtree-secondary-index/scan-insert-rtree-secondary-index.4.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/scan-insert-rtree-secondary-index/scan-insert-rtree-secondary-index.4.ast
deleted file mode 100644
index 916a59e..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/scan-insert-rtree-secondary-index/scan-insert-rtree-secondary-index.4.ast
+++ /dev/null
@@ -1 +0,0 @@
-DataverseUse test

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/scan-insert-rtree-secondary-index/scan-insert-rtree-secondary-index.5.ast
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/scan-insert-rtree-secondary-index/scan-insert-rtree-secondary-index.5.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/scan-insert-rtree-secondary-index/scan-insert-rtree-secondary-index.5.ast
deleted file mode 100644
index 0b75e50..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/scan-insert-rtree-secondary-index/scan-insert-rtree-secondary-index.5.ast
+++ /dev/null
@@ -1,45 +0,0 @@
-DataverseUse test
-Query:
-SELECT ELEMENT [
-RecordConstructor [
-  (
-    LiteralExpr [STRING] [id]
-    :
-    FieldAccessor [
-      Variable [ Name=$o ]
-      Field=id
-    ]
-  )
-]
-]
-FROM [  FunctionCall asterix.dataset@1[
-    LiteralExpr [STRING] [MyMiniData]
-  ]
-  AS Variable [ Name=$o ]
-]
-Where
-  FunctionCall test.spatial-intersect@2[
-    FieldAccessor [
-      Variable [ Name=$o ]
-      Field=point
-    ]
-    FunctionCall test.create-polygon@1[
-      OrderedListConstructor [
-        LiteralExpr [DOUBLE] [0.0]
-        LiteralExpr [DOUBLE] [1.0]
-        LiteralExpr [DOUBLE] [0.0]
-        LiteralExpr [DOUBLE] [4.0]
-        LiteralExpr [DOUBLE] [12.0]
-        LiteralExpr [DOUBLE] [4.0]
-        LiteralExpr [DOUBLE] [12.0]
-        LiteralExpr [DOUBLE] [1.0]
-      ]
-    ]
-  ]
-Orderby
-  FieldAccessor [
-    Variable [ Name=$o ]
-    Field=id
-  ]
-  ASC
-

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/scan-insert-temp-to-persistent/scan-insert-temp-to-persistent.1.ast
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/scan-insert-temp-to-persistent/scan-insert-temp-to-persistent.1.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/scan-insert-temp-to-persistent/scan-insert-temp-to-persistent.1.ast
deleted file mode 100644
index 79e3a90..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/scan-insert-temp-to-persistent/scan-insert-temp-to-persistent.1.ast
+++ /dev/null
@@ -1,22 +0,0 @@
-DataverseUse test
-TypeDecl MyRecord [
-  closed RecordType {
-    id : bigint,
-    point : point,
-    kwds : string,
-    line1 : line,
-    line2 : line,
-    poly1 : polygon,
-    poly2 : polygon,
-    rec : rectangle,
-    circle : circle
-  }
-]
-TypeDecl MyMiniRecord [
-  closed RecordType {
-    id : bigint,
-    point : point
-  }
-]
-DatasetDecl MyData(MyRecord) partitioned by [[id]]
-DatasetDecl MyMiniData(MyMiniRecord) partitioned by [[id]]

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/scan-insert-temp-to-persistent/scan-insert-temp-to-persistent.2.ast
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/scan-insert-temp-to-persistent/scan-insert-temp-to-persistent.2.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/scan-insert-temp-to-persistent/scan-insert-temp-to-persistent.2.ast
deleted file mode 100644
index 916a59e..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/scan-insert-temp-to-persistent/scan-insert-temp-to-persistent.2.ast
+++ /dev/null
@@ -1 +0,0 @@
-DataverseUse test

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/scan-insert-temp-to-persistent/scan-insert-temp-to-persistent.3.ast
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/scan-insert-temp-to-persistent/scan-insert-temp-to-persistent.3.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/scan-insert-temp-to-persistent/scan-insert-temp-to-persistent.3.ast
deleted file mode 100644
index 916a59e..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/scan-insert-temp-to-persistent/scan-insert-temp-to-persistent.3.ast
+++ /dev/null
@@ -1 +0,0 @@
-DataverseUse test

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/scan-insert-temp-to-persistent/scan-insert-temp-to-persistent.4.ast
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/scan-insert-temp-to-persistent/scan-insert-temp-to-persistent.4.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/scan-insert-temp-to-persistent/scan-insert-temp-to-persistent.4.ast
deleted file mode 100644
index 916a59e..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/scan-insert-temp-to-persistent/scan-insert-temp-to-persistent.4.ast
+++ /dev/null
@@ -1 +0,0 @@
-DataverseUse test

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/scan-insert-temp-to-persistent/scan-insert-temp-to-persistent.5.ast
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/scan-insert-temp-to-persistent/scan-insert-temp-to-persistent.5.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/scan-insert-temp-to-persistent/scan-insert-temp-to-persistent.5.ast
deleted file mode 100644
index 0b75e50..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temp-dataset/scan-insert-temp-to-persistent/scan-insert-temp-to-persistent.5.ast
+++ /dev/null
@@ -1,45 +0,0 @@
-DataverseUse test
-Query:
-SELECT ELEMENT [
-RecordConstructor [
-  (
-    LiteralExpr [STRING] [id]
-    :
-    FieldAccessor [
-      Variable [ Name=$o ]
-      Field=id
-    ]
-  )
-]
-]
-FROM [  FunctionCall asterix.dataset@1[
-    LiteralExpr [STRING] [MyMiniData]
-  ]
-  AS Variable [ Name=$o ]
-]
-Where
-  FunctionCall test.spatial-intersect@2[
-    FieldAccessor [
-      Variable [ Name=$o ]
-      Field=point
-    ]
-    FunctionCall test.create-polygon@1[
-      OrderedListConstructor [
-        LiteralExpr [DOUBLE] [0.0]
-        LiteralExpr [DOUBLE] [1.0]
-        LiteralExpr [DOUBLE] [0.0]
-        LiteralExpr [DOUBLE] [4.0]
-        LiteralExpr [DOUBLE] [12.0]
-        LiteralExpr [DOUBLE] [4.0]
-        LiteralExpr [DOUBLE] [12.0]
-        LiteralExpr [DOUBLE] [1.0]
-      ]
-    ]
-  ]
-Orderby
-  FieldAccessor [
-    Variable [ Name=$o ]
-    Field=id
-  ]
-  ASC
-

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/testsuite.xml
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite.xml b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite.xml
index 8987a7f..17b14dc 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite.xml
@@ -7428,76 +7428,4 @@
       </compilation-unit>
     </test-case>
   </test-group>
-  <test-group name="temp-dataset">
-    <test-case FilePath="temp-dataset">
-      <compilation-unit name="delete-from-loaded-dataset">
-        <output-dir compare="Text">delete-from-loaded-dataset</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="temp-dataset">
-      <compilation-unit name="delete-from-loaded-dataset-with-index">
-        <output-dir compare="Text">delete-from-loaded-dataset-with-index</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="temp-dataset">
-      <compilation-unit name="drop-empty-secondary-indexes">
-        <output-dir compare="Text">drop-empty-secondary-indexes</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="temp-dataset">
-      <compilation-unit name="drop-index">
-        <output-dir compare="Text">drop-index</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="temp-dataset">
-      <compilation-unit name="empty-load-with-index">
-        <output-dir compare="Text">empty-load-with-index</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="temp-dataset">
-      <compilation-unit name="insert-and-scan-dataset">
-        <output-dir compare="Text">insert-and-scan-dataset</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="temp-dataset">
-      <compilation-unit name="q01_pricing_summary_report_nt">
-        <output-dir compare="Text">q01_pricing_summary_report_nt</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="temp-dataset">
-      <compilation-unit name="scan-delete-rtree-secondary-index">
-        <output-dir compare="Text">scan-delete-rtree-secondary-index</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="temp-dataset">
-      <compilation-unit name="scan-insert-rtree-secondary-index">
-        <output-dir compare="Text">scan-insert-rtree-secondary-index</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="temp-dataset">
-      <compilation-unit name="scan-insert-persistent-to-temp">
-        <output-dir compare="Text">scan-insert-persistent-to-temp</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="temp-dataset">
-      <compilation-unit name="scan-insert-temp-to-persistent">
-        <output-dir compare="Text">scan-insert-temp-to-persistent</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="temp-dataset">
-      <compilation-unit name="insert-and-scan-dataset-with-index">
-        <output-dir compare="Text">insert-and-scan-dataset-with-index</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="temp-dataset">
-      <compilation-unit name="insert-and-scan-dataset-with-correlated-index">
-        <output-dir compare="Text">insert-and-scan-dataset-with-index</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="temp-dataset">
-      <compilation-unit name="temp_primary_plus_ngram_flush">
-        <output-dir compare="Text">temp_primary_plus_ngram_flush</output-dir>
-      </compilation-unit>
-    </test-case>
-  </test-group>
 </test-suite>


[05/12] asterixdb git commit: [ASTERIXDB-2171][STO] Remove Temporary Datasets

Posted by mh...@apache.org.
http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
index 484e395..71f1e15 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
@@ -9031,72 +9031,7 @@
       </compilation-unit>
     </test-case>
   </test-group>
-  <test-group name="temp-dataset">
-    <test-case FilePath="temp-dataset">
-      <compilation-unit name="delete-from-loaded-dataset">
-        <output-dir compare="Text">delete-from-loaded-dataset</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="temp-dataset">
-      <compilation-unit name="delete-from-loaded-dataset-with-index">
-        <output-dir compare="Text">delete-from-loaded-dataset-with-index</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="temp-dataset">
-      <compilation-unit name="drop-empty-secondary-indexes">
-        <output-dir compare="Text">drop-empty-secondary-indexes</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="temp-dataset">
-      <compilation-unit name="drop-index">
-        <output-dir compare="Text">drop-index</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="temp-dataset">
-      <compilation-unit name="empty-load-with-index">
-        <output-dir compare="Text">empty-load-with-index</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="temp-dataset">
-      <compilation-unit name="insert-and-scan-dataset">
-        <output-dir compare="Text">insert-and-scan-dataset</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="temp-dataset">
-      <compilation-unit name="q01_pricing_summary_report_nt">
-        <output-dir compare="Text">q01_pricing_summary_report_nt</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="temp-dataset">
-      <compilation-unit name="scan-delete-rtree-secondary-index">
-        <output-dir compare="Text">scan-delete-rtree-secondary-index</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="temp-dataset">
-      <compilation-unit name="scan-insert-rtree-secondary-index">
-        <output-dir compare="Text">scan-insert-rtree-secondary-index</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="temp-dataset">
-      <compilation-unit name="scan-insert-persistent-to-temp">
-        <output-dir compare="Text">scan-insert-persistent-to-temp</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="temp-dataset">
-      <compilation-unit name="scan-insert-temp-to-persistent">
-        <output-dir compare="Text">scan-insert-temp-to-persistent</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="temp-dataset">
-      <compilation-unit name="insert-and-scan-dataset-with-index">
-        <output-dir compare="Text">insert-and-scan-dataset-with-index</output-dir>
-      </compilation-unit>
-    </test-case>
-        <test-case FilePath="temp-dataset">
-      <compilation-unit name="insert-and-scan-dataset-with-correlated-index">
-        <output-dir compare="Text">insert-and-scan-dataset-with-index</output-dir>
-      </compilation-unit>
-    </test-case>
+  <test-group name="unnest">
     <test-case FilePath="unnest">
       <compilation-unit name="left-outer-unnest">
         <output-dir compare="Text">left-outer-unnest</output-dir>

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp_parser.xml
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp_parser.xml b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp_parser.xml
index fcd1a70..af30871 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp_parser.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp_parser.xml
@@ -6746,66 +6746,4 @@
       </compilation-unit>
     </test-case>
   </test-group>
-  <test-group name="temp-dataset">
-    <test-case FilePath="temp-dataset">
-      <compilation-unit name="delete-from-loaded-dataset">
-        <output-dir compare="AST">delete-from-loaded-dataset</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="temp-dataset">
-      <compilation-unit name="delete-from-loaded-dataset-with-index">
-        <output-dir compare="AST">delete-from-loaded-dataset-with-index</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="temp-dataset">
-      <compilation-unit name="drop-empty-secondary-indexes">
-        <output-dir compare="AST">drop-empty-secondary-indexes</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="temp-dataset">
-      <compilation-unit name="drop-index">
-        <output-dir compare="AST">drop-index</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="temp-dataset">
-      <compilation-unit name="empty-load-with-index">
-        <output-dir compare="AST">empty-load-with-index</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="temp-dataset">
-      <compilation-unit name="insert-and-scan-dataset">
-        <output-dir compare="AST">insert-and-scan-dataset</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="temp-dataset">
-      <compilation-unit name="q01_pricing_summary_report_nt">
-        <output-dir compare="AST">q01_pricing_summary_report_nt</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="temp-dataset">
-      <compilation-unit name="scan-delete-rtree-secondary-index">
-        <output-dir compare="AST">scan-delete-rtree-secondary-index</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="temp-dataset">
-      <compilation-unit name="scan-insert-rtree-secondary-index">
-        <output-dir compare="AST">scan-insert-rtree-secondary-index</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="temp-dataset">
-      <compilation-unit name="scan-insert-persistent-to-temp">
-        <output-dir compare="AST">scan-insert-persistent-to-temp</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="temp-dataset">
-      <compilation-unit name="scan-insert-temp-to-persistent">
-        <output-dir compare="AST">scan-insert-temp-to-persistent</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="temp-dataset">
-      <compilation-unit name="insert-and-scan-dataset-with-index">
-        <output-dir compare="AST">insert-and-scan-dataset-with-index</output-dir>
-      </compilation-unit>
-    </test-case>
-  </test-group>
 </test-suite>

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/utils/StoragePathUtil.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/utils/StoragePathUtil.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/utils/StoragePathUtil.java
index 0b9b94b..027f72c 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/utils/StoragePathUtil.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/utils/StoragePathUtil.java
@@ -35,7 +35,6 @@ import org.apache.log4j.Logger;
 public class StoragePathUtil {
     private static final Logger LOGGER = Logger.getLogger(StoragePathUtil.class.getName());
     public static final String PARTITION_DIR_PREFIX = "partition_";
-    public static final String TEMP_DATASETS_STORAGE_FOLDER = "temp";
     public static final String DATASET_INDEX_NAME_SEPARATOR = "_idx_";
 
     private StoragePathUtil() {

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_index_only/primary_index_only.1.script.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_index_only/primary_index_only.1.script.aql b/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_index_only/primary_index_only.1.script.aql
deleted file mode 100644
index 7d441cd..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_index_only/primary_index_only.1.script.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.
-#
-create_and_start.sh
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_index_only/primary_index_only.2.ddl.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_index_only/primary_index_only.2.ddl.aql b/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_index_only/primary_index_only.2.ddl.aql
deleted file mode 100644
index 9d4c55d..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_index_only/primary_index_only.2.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  : primary_index_only.aql
- * Description     : Check that abort from duplicate key exception works and crash recovery works after the abort.   
- * Expected Result : Success
- * Date            : September 25 2013
- */
-
-drop dataverse recovery if exists;
-create dataverse recovery;
-use dataverse recovery;
-
-/* For raw Fragile data */
-create type FragileTypeRaw as closed {
-  row_id: int32,
-  sid: int32,
-  date: string,
-  day: int32,
-  time: string,
-  bpm: int32,
-  RR: float
-};
-
-/* For cleaned Fragile data */
-create type FragileType as closed {
-  row_id: int32,
-  sid: int32,
-  date: date,
-  day: int32,
-  time: time,
-  bpm: int32,
-  RR: float
-};
-
-/* Create dataset for loading raw Fragile data */
-create temporary dataset Fragile_raw (FragileTypeRaw)
-primary key row_id;
-
-/* Create dataset for cleaned Fragile data */
-create temporary dataset Fragile (FragileType)
-primary key row_id;
-

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_index_only/primary_index_only.3.update.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_index_only/primary_index_only.3.update.aql b/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_index_only/primary_index_only.3.update.aql
deleted file mode 100644
index ab07679..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_index_only/primary_index_only.3.update.aql
+++ /dev/null
@@ -1,29 +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  : primary_index_only.aql
- * Description     : Check that abort from duplicate key exception works and crash recovery works after the abort.   
- * Expected Result : Success
- * Date            : September 25 2013
- */
-
-use dataverse recovery;
-
-load dataset Fragile_raw using localfs
-(("path"="asterix_nc1://../../../../../../asterix-app/data/csv/fragile_01.csv"),("format"="delimited-text"),("delimiter"=",")) pre-sorted;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_index_only/primary_index_only.4.txneu.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_index_only/primary_index_only.4.txneu.aql b/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_index_only/primary_index_only.4.txneu.aql
deleted file mode 100644
index 2c8886e..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_index_only/primary_index_only.4.txneu.aql
+++ /dev/null
@@ -1,40 +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  : primary_index_only.aql
- * Description     : Check that abort from duplicate key exception works and crash recovery works after the abort.   
- * Expected Result : Success
- * Date            : September 25 2013
- */
-
-use dataverse recovery;
-
-/* Load Fragile data from raw dataset into cleaned dataset */
-insert into dataset Fragile (
-  for $t in dataset Fragile_raw
-  return {
-    "row_id": $t.row_id % 28000,
-    "sid": $t.sid,
-    "date": date($t.date),
-    "day": $t.day,
-    "time": parse-time($t.time, "h:m:s"),
-    "bpm": $t.bpm,
-    "RR": $t.RR
-  }
-);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_index_only/primary_index_only.5.txnqbc.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_index_only/primary_index_only.5.txnqbc.aql b/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_index_only/primary_index_only.5.txnqbc.aql
deleted file mode 100644
index ff244d9..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_index_only/primary_index_only.5.txnqbc.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  : primary_index_only.aql
- * Description     : Check that abort from duplicate key exception works and crash recovery works after the abort.   
- * Expected Result : Success
- * Date            : September 25 2013
- */
-
-use dataverse recovery;
-
-count (for $x in dataset Fragile return $x);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_index_only/primary_index_only.6.script.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_index_only/primary_index_only.6.script.aql b/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_index_only/primary_index_only.6.script.aql
deleted file mode 100644
index 4583455..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_index_only/primary_index_only.6.script.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.
-#
-kill_cc_and_nc.sh
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_index_only/primary_index_only.7.script.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_index_only/primary_index_only.7.script.aql b/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_index_only/primary_index_only.7.script.aql
deleted file mode 100644
index 7087cd3..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_index_only/primary_index_only.7.script.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.
-#
-stop_and_start.sh
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_index_only/primary_index_only.8.txnqar.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_index_only/primary_index_only.8.txnqar.aql b/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_index_only/primary_index_only.8.txnqar.aql
deleted file mode 100644
index ff244d9..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_index_only/primary_index_only.8.txnqar.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  : primary_index_only.aql
- * Description     : Check that abort from duplicate key exception works and crash recovery works after the abort.   
- * Expected Result : Success
- * Date            : September 25 2013
- */
-
-use dataverse recovery;
-
-count (for $x in dataset Fragile return $x);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_index_only/primary_index_only.9.script.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_index_only/primary_index_only.9.script.aql b/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_index_only/primary_index_only.9.script.aql
deleted file mode 100644
index 40df6fb..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_index_only/primary_index_only.9.script.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.
-#
-stop_and_delete.sh
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_default_secondary_index/primary_plus_default_secondary_index.1.script.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_default_secondary_index/primary_plus_default_secondary_index.1.script.aql b/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_default_secondary_index/primary_plus_default_secondary_index.1.script.aql
deleted file mode 100644
index 7d441cd..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_default_secondary_index/primary_plus_default_secondary_index.1.script.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.
-#
-create_and_start.sh
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_default_secondary_index/primary_plus_default_secondary_index.2.ddl.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_default_secondary_index/primary_plus_default_secondary_index.2.ddl.aql b/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_default_secondary_index/primary_plus_default_secondary_index.2.ddl.aql
deleted file mode 100644
index df75adf..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_default_secondary_index/primary_plus_default_secondary_index.2.ddl.aql
+++ /dev/null
@@ -1,71 +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  : primary_plus_default_secondary_index.aql
- * Description     : Check that abort from duplicate key exception works and crash recovery works after the abort.   
- * Expected Result : Success
- * Date            : Oct 15 2013
- */
-
-drop dataverse recovery if exists;
-create dataverse recovery;
-use dataverse recovery;
-
-/* For raw Fragile data */
-create type FragileTypeRaw as closed {
-  row_id: int32,
-  sid: int32,
-  date: string,
-  day: int32,
-  time: string,
-  bpm: int32,
-  RR: float,
-  /* new string field and location field*/
-  text: string,
-  location: point,
-  text2: string
-};
-
-/* For cleaned Fragile data */
-create type FragileType as closed {
-  row_id: int32,
-  sid: int32,
-  date: date,
-  day: int32,
-  time: time,
-  bpm: int32,
-  RR: float,
-  
-  /* new string field and location field*/
-  text: string,
-  location: point,
-  text2: string
-};
-
-/* Create dataset for loading raw Fragile data */
-create temporary dataset Fragile_raw (FragileTypeRaw)
-primary key row_id;
-
-/* Create dataset for cleaned Fragile data */
-create temporary dataset Fragile (FragileType)
-primary key row_id;
-
-/* Create default secondary index on dataset clean Fragile */
-create index cfSidIdx on Fragile(sid);
-

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_default_secondary_index/primary_plus_default_secondary_index.3.update.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_default_secondary_index/primary_plus_default_secondary_index.3.update.aql b/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_default_secondary_index/primary_plus_default_secondary_index.3.update.aql
deleted file mode 100644
index ce01110..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_default_secondary_index/primary_plus_default_secondary_index.3.update.aql
+++ /dev/null
@@ -1,29 +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  : primary_plus_default_secondary_index.aql
- * Description     : Check that abort from duplicate key exception works and crash recovery works after the abort.   
- * Expected Result : Success
- * Date            : Oct 15 2013
- */
-
-use dataverse recovery;
-
-load dataset Fragile_raw using localfs
-(("path"="asterix_nc1://../../../../../../asterix-app/data/csv/fragile_02.adm"),("format"="adm")) pre-sorted;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_default_secondary_index/primary_plus_default_secondary_index.4.txneu.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_default_secondary_index/primary_plus_default_secondary_index.4.txneu.aql b/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_default_secondary_index/primary_plus_default_secondary_index.4.txneu.aql
deleted file mode 100644
index 261e7b9..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_default_secondary_index/primary_plus_default_secondary_index.4.txneu.aql
+++ /dev/null
@@ -1,43 +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  : primary_plus_default_secondary_index.aql
- * Description     : Check that abort from duplicate key exception works and crash recovery works after the abort.   
- * Expected Result : Success
- * Date            : Oct 15 2013
- */
-
-use dataverse recovery;
-
-/* Load Fragile data from raw dataset into cleaned dataset */
-insert into dataset Fragile (
-  for $t in dataset Fragile_raw
-  return {
-    "row_id": $t.row_id % 28000,
-    "sid": $t.sid,
-    "date": date($t.date),
-    "day": $t.day,
-    "time": parse-time($t.time, "h:m:s"),
-    "bpm": $t.bpm,
-    "RR": $t.RR,
-    "text": $t.text,
-    "location": $t.location,
-    "text2": $t.text2
-  }
-);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_default_secondary_index/primary_plus_default_secondary_index.5.txnqbc.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_default_secondary_index/primary_plus_default_secondary_index.5.txnqbc.aql b/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_default_secondary_index/primary_plus_default_secondary_index.5.txnqbc.aql
deleted file mode 100644
index 595acfb..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_default_secondary_index/primary_plus_default_secondary_index.5.txnqbc.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  : primary_plus_default_secondary_index.aql
- * Description     : Check that abort from duplicate key exception works and crash recovery works after the abort.   
- * Expected Result : Success
- * Date            : Otc 15 2013
- */
-
-use dataverse recovery;
-
-count (for $x in dataset Fragile where $x.sid=1 return $x);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_default_secondary_index/primary_plus_default_secondary_index.6.script.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_default_secondary_index/primary_plus_default_secondary_index.6.script.aql b/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_default_secondary_index/primary_plus_default_secondary_index.6.script.aql
deleted file mode 100644
index 4583455..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_default_secondary_index/primary_plus_default_secondary_index.6.script.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.
-#
-kill_cc_and_nc.sh
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_default_secondary_index/primary_plus_default_secondary_index.7.script.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_default_secondary_index/primary_plus_default_secondary_index.7.script.aql b/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_default_secondary_index/primary_plus_default_secondary_index.7.script.aql
deleted file mode 100644
index 7087cd3..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_default_secondary_index/primary_plus_default_secondary_index.7.script.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.
-#
-stop_and_start.sh
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_default_secondary_index/primary_plus_default_secondary_index.8.txnqar.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_default_secondary_index/primary_plus_default_secondary_index.8.txnqar.aql b/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_default_secondary_index/primary_plus_default_secondary_index.8.txnqar.aql
deleted file mode 100644
index 714fcbc..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_default_secondary_index/primary_plus_default_secondary_index.8.txnqar.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  : primary_plus_default_secondary_index.aql
- * Description     : Check that abort from duplicate key exception works and crash recovery works after the abort.   
- * Expected Result : Success
- * Date            : Oct 15 2013
- */
-
-use dataverse recovery;
-
-count (for $x in dataset Fragile where $x.sid=1 return $x);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_default_secondary_index/primary_plus_default_secondary_index.9.script.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_default_secondary_index/primary_plus_default_secondary_index.9.script.aql b/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_default_secondary_index/primary_plus_default_secondary_index.9.script.aql
deleted file mode 100644
index 40df6fb..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_default_secondary_index/primary_plus_default_secondary_index.9.script.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.
-#
-stop_and_delete.sh
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_keyword_secondary_index/primary_plus_keyword_secondary_index.1.script.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_keyword_secondary_index/primary_plus_keyword_secondary_index.1.script.aql b/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_keyword_secondary_index/primary_plus_keyword_secondary_index.1.script.aql
deleted file mode 100644
index 7d441cd..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_keyword_secondary_index/primary_plus_keyword_secondary_index.1.script.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.
-#
-create_and_start.sh
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_keyword_secondary_index/primary_plus_keyword_secondary_index.2.ddl.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_keyword_secondary_index/primary_plus_keyword_secondary_index.2.ddl.aql b/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_keyword_secondary_index/primary_plus_keyword_secondary_index.2.ddl.aql
deleted file mode 100644
index 107500e..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_keyword_secondary_index/primary_plus_keyword_secondary_index.2.ddl.aql
+++ /dev/null
@@ -1,72 +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  : primary_plus_keyword_secondary_index.aql
- * Description     : Check that abort from duplicate key exception works and crash recovery works after the abort.   
- * Expected Result : Success
- * Date            : Oct 15 2013
- */
-
-drop dataverse recovery if exists;
-create dataverse recovery;
-use dataverse recovery;
-
-/* For raw Fragile data */
-create type FragileTypeRaw as closed {
-  row_id: int32,
-  sid: int32,
-  date: string,
-  day: int32,
-  time: string,
-  bpm: int32,
-  RR: float,
-  /* new string field and location field*/
-  text: string,
-  location: point,
-  text2: string
-  
-};
-
-/* For cleaned Fragile data */
-create type FragileType as closed {
-  row_id: int32,
-  sid: int32,
-  date: date,
-  day: int32,
-  time: time,
-  bpm: int32,
-  RR: float,
-  
-  /* new string field and location field*/
-  text: string,
-  location: point,
-  text2: string
-};
-
-/* Create dataset for loading raw Fragile data */
-create temporary dataset Fragile_raw (FragileTypeRaw)
-primary key row_id;
-
-/* Create dataset for cleaned Fragile data */
-create temporary dataset Fragile (FragileType)
-primary key row_id;
-
-/* Create keyword secondary index on dataset clean Fragile */
-create index cfText on Fragile(text) type keyword;
-

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_keyword_secondary_index/primary_plus_keyword_secondary_index.3.update.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_keyword_secondary_index/primary_plus_keyword_secondary_index.3.update.aql b/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_keyword_secondary_index/primary_plus_keyword_secondary_index.3.update.aql
deleted file mode 100644
index 66adcef..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_keyword_secondary_index/primary_plus_keyword_secondary_index.3.update.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  : primary_plus_keyword_secondary_index.aql
- * Description     : Check that abort from duplicate key exception works and crash recovery works after the abort.   
- * Expected Result : Success
- * Date            : Oct 15 2013
- */
-
-use dataverse recovery;
-
-
-load dataset Fragile_raw using localfs
-(("path"="asterix_nc1://../../../../../../asterix-app/data/csv/fragile_02.adm"),("format"="adm")) pre-sorted;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_keyword_secondary_index/primary_plus_keyword_secondary_index.4.txneu.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_keyword_secondary_index/primary_plus_keyword_secondary_index.4.txneu.aql b/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_keyword_secondary_index/primary_plus_keyword_secondary_index.4.txneu.aql
deleted file mode 100644
index 7c1c623..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_keyword_secondary_index/primary_plus_keyword_secondary_index.4.txneu.aql
+++ /dev/null
@@ -1,43 +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  : primary_plus_keyword_secondary_index.aql
- * Description     : Check that abort from duplicate key exception works and crash recovery works after the abort.   
- * Expected Result : Success
- * Date            : Oct 15 2013
- */
-
-use dataverse recovery;
-
-/* Load Fragile data from raw dataset into cleaned dataset */
-insert into dataset Fragile (
-  for $t in dataset Fragile_raw
-  return {
-    "row_id": $t.row_id % 28000,
-    "sid": $t.sid,
-    "date": date($t.date),
-    "day": $t.day,
-    "time": parse-time($t.time, "h:m:s"),
-    "bpm": $t.bpm,
-    "RR": $t.RR,
-    "text": $t.text,
-    "location": $t.location,
-    "text2": $t.text2
-  }
-);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_keyword_secondary_index/primary_plus_keyword_secondary_index.5.txnqbc.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_keyword_secondary_index/primary_plus_keyword_secondary_index.5.txnqbc.aql b/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_keyword_secondary_index/primary_plus_keyword_secondary_index.5.txnqbc.aql
deleted file mode 100644
index 4d48e81..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_keyword_secondary_index/primary_plus_keyword_secondary_index.5.txnqbc.aql
+++ /dev/null
@@ -1,23 +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.
- */
-use dataverse recovery;
-
-for $x in dataset Fragile 
-where word-tokens($x.text) ~= word-tokens(" 1 20130417 1")
-return $x.text;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_keyword_secondary_index/primary_plus_keyword_secondary_index.6.script.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_keyword_secondary_index/primary_plus_keyword_secondary_index.6.script.aql b/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_keyword_secondary_index/primary_plus_keyword_secondary_index.6.script.aql
deleted file mode 100644
index 4583455..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_keyword_secondary_index/primary_plus_keyword_secondary_index.6.script.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.
-#
-kill_cc_and_nc.sh
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_keyword_secondary_index/primary_plus_keyword_secondary_index.7.script.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_keyword_secondary_index/primary_plus_keyword_secondary_index.7.script.aql b/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_keyword_secondary_index/primary_plus_keyword_secondary_index.7.script.aql
deleted file mode 100644
index 7087cd3..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_keyword_secondary_index/primary_plus_keyword_secondary_index.7.script.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.
-#
-stop_and_start.sh
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_keyword_secondary_index/primary_plus_keyword_secondary_index.8.txnqar.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_keyword_secondary_index/primary_plus_keyword_secondary_index.8.txnqar.aql b/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_keyword_secondary_index/primary_plus_keyword_secondary_index.8.txnqar.aql
deleted file mode 100644
index 4d48e81..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_keyword_secondary_index/primary_plus_keyword_secondary_index.8.txnqar.aql
+++ /dev/null
@@ -1,23 +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.
- */
-use dataverse recovery;
-
-for $x in dataset Fragile 
-where word-tokens($x.text) ~= word-tokens(" 1 20130417 1")
-return $x.text;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_keyword_secondary_index/primary_plus_keyword_secondary_index.9.script.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_keyword_secondary_index/primary_plus_keyword_secondary_index.9.script.aql b/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_keyword_secondary_index/primary_plus_keyword_secondary_index.9.script.aql
deleted file mode 100644
index 40df6fb..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_keyword_secondary_index/primary_plus_keyword_secondary_index.9.script.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.
-#
-stop_and_delete.sh
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_multiple_secondary_indices/primary_plus_multiple_secondary_indices.1.script.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_multiple_secondary_indices/primary_plus_multiple_secondary_indices.1.script.aql b/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_multiple_secondary_indices/primary_plus_multiple_secondary_indices.1.script.aql
deleted file mode 100644
index 7d441cd..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_multiple_secondary_indices/primary_plus_multiple_secondary_indices.1.script.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.
-#
-create_and_start.sh
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_multiple_secondary_indices/primary_plus_multiple_secondary_indices.2.ddl.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_multiple_secondary_indices/primary_plus_multiple_secondary_indices.2.ddl.aql b/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_multiple_secondary_indices/primary_plus_multiple_secondary_indices.2.ddl.aql
deleted file mode 100644
index 071f4e0..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_multiple_secondary_indices/primary_plus_multiple_secondary_indices.2.ddl.aql
+++ /dev/null
@@ -1,78 +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  : primary_plus_keyword_secondary_index.aql
- * Description     : Check that abort from duplicate key exception works and crash recovery works after the abort.   
- * Expected Result : Success
- * Date            : Oct 15 2013
- */
-
-drop dataverse recovery if exists;
-create dataverse recovery;
-use dataverse recovery;
-
-/* For raw Fragile data */
-create type FragileTypeRaw as closed {
-  row_id: int32,
-  sid: int32,
-  date: string,
-  day: int32,
-  time: string,
-  bpm: int32,
-  RR: float,
-  /* new string field and location field*/
-  text: string,
-  location: point,
-  text2: string
-  
-};
-
-/* For cleaned Fragile data */
-create type FragileType as closed {
-  row_id: int32,
-  sid: int32,
-  date: date,
-  day: int32,
-  time: time,
-  bpm: int32,
-  RR: float,
-  
-  /* new string field and location field*/
-  text: string,
-  location: point,
-  text2: string
-};
-
-/* Create dataset for loading raw Fragile data */
-create temporary dataset Fragile_raw (FragileTypeRaw)
-primary key row_id;
-
-/* Create dataset for cleaned Fragile data */
-create temporary dataset Fragile (FragileType)
-primary key row_id;
-
-
-/* Create rtree secondary index on dataset clean Fragile */
-create index cfLocation on Fragile(location) type rtree;
-
-create index cfText on Fragile(text) type keyword;
-
-create index cfSidIdx on Fragile(sid);
-
-create index cfText2Ix on Fragile(text2) type ngram(3);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_multiple_secondary_indices/primary_plus_multiple_secondary_indices.3.update.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_multiple_secondary_indices/primary_plus_multiple_secondary_indices.3.update.aql b/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_multiple_secondary_indices/primary_plus_multiple_secondary_indices.3.update.aql
deleted file mode 100644
index 66adcef..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_multiple_secondary_indices/primary_plus_multiple_secondary_indices.3.update.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  : primary_plus_keyword_secondary_index.aql
- * Description     : Check that abort from duplicate key exception works and crash recovery works after the abort.   
- * Expected Result : Success
- * Date            : Oct 15 2013
- */
-
-use dataverse recovery;
-
-
-load dataset Fragile_raw using localfs
-(("path"="asterix_nc1://../../../../../../asterix-app/data/csv/fragile_02.adm"),("format"="adm")) pre-sorted;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_multiple_secondary_indices/primary_plus_multiple_secondary_indices.4.txneu.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_multiple_secondary_indices/primary_plus_multiple_secondary_indices.4.txneu.aql b/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_multiple_secondary_indices/primary_plus_multiple_secondary_indices.4.txneu.aql
deleted file mode 100644
index 7c1c623..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_multiple_secondary_indices/primary_plus_multiple_secondary_indices.4.txneu.aql
+++ /dev/null
@@ -1,43 +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  : primary_plus_keyword_secondary_index.aql
- * Description     : Check that abort from duplicate key exception works and crash recovery works after the abort.   
- * Expected Result : Success
- * Date            : Oct 15 2013
- */
-
-use dataverse recovery;
-
-/* Load Fragile data from raw dataset into cleaned dataset */
-insert into dataset Fragile (
-  for $t in dataset Fragile_raw
-  return {
-    "row_id": $t.row_id % 28000,
-    "sid": $t.sid,
-    "date": date($t.date),
-    "day": $t.day,
-    "time": parse-time($t.time, "h:m:s"),
-    "bpm": $t.bpm,
-    "RR": $t.RR,
-    "text": $t.text,
-    "location": $t.location,
-    "text2": $t.text2
-  }
-);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_multiple_secondary_indices/primary_plus_multiple_secondary_indices.5.txnqbc.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_multiple_secondary_indices/primary_plus_multiple_secondary_indices.5.txnqbc.aql b/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_multiple_secondary_indices/primary_plus_multiple_secondary_indices.5.txnqbc.aql
deleted file mode 100644
index ca06f14..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_multiple_secondary_indices/primary_plus_multiple_secondary_indices.5.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  : primary_plus_keyword_secondary_index.aql
- * Description     : Check that abort from duplicate key exception works and crash recovery works after the abort.   
- * Expected Result : Success
- * Date            : Otc 15 2013
- */
-
-
-use dataverse recovery;
-
-count (for $x in dataset Fragile 
-where contains($x.text2, "location") return $x);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_multiple_secondary_indices/primary_plus_multiple_secondary_indices.6.script.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_multiple_secondary_indices/primary_plus_multiple_secondary_indices.6.script.aql b/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_multiple_secondary_indices/primary_plus_multiple_secondary_indices.6.script.aql
deleted file mode 100644
index 4583455..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_multiple_secondary_indices/primary_plus_multiple_secondary_indices.6.script.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.
-#
-kill_cc_and_nc.sh
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_multiple_secondary_indices/primary_plus_multiple_secondary_indices.7.script.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_multiple_secondary_indices/primary_plus_multiple_secondary_indices.7.script.aql b/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_multiple_secondary_indices/primary_plus_multiple_secondary_indices.7.script.aql
deleted file mode 100644
index 7087cd3..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_multiple_secondary_indices/primary_plus_multiple_secondary_indices.7.script.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.
-#
-stop_and_start.sh
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_multiple_secondary_indices/primary_plus_multiple_secondary_indices.8.txnqar.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_multiple_secondary_indices/primary_plus_multiple_secondary_indices.8.txnqar.aql b/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_multiple_secondary_indices/primary_plus_multiple_secondary_indices.8.txnqar.aql
deleted file mode 100644
index 4738d05..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_multiple_secondary_indices/primary_plus_multiple_secondary_indices.8.txnqar.aql
+++ /dev/null
@@ -1,23 +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.
- */
-
-use dataverse recovery;
-
-count (for $x in dataset Fragile 
-where contains($x.text2, "location") return $x);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_multiple_secondary_indices/primary_plus_multiple_secondary_indices.9.script.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_multiple_secondary_indices/primary_plus_multiple_secondary_indices.9.script.aql b/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_multiple_secondary_indices/primary_plus_multiple_secondary_indices.9.script.aql
deleted file mode 100644
index 40df6fb..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_multiple_secondary_indices/primary_plus_multiple_secondary_indices.9.script.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.
-#
-stop_and_delete.sh
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_ngram_index/primary_plus_ngram_index.1.script.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_ngram_index/primary_plus_ngram_index.1.script.aql b/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_ngram_index/primary_plus_ngram_index.1.script.aql
deleted file mode 100644
index 7d441cd..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_ngram_index/primary_plus_ngram_index.1.script.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.
-#
-create_and_start.sh
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_ngram_index/primary_plus_ngram_index.2.ddl.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_ngram_index/primary_plus_ngram_index.2.ddl.aql b/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_ngram_index/primary_plus_ngram_index.2.ddl.aql
deleted file mode 100644
index 4d215d3..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recover_after_abort/temp_primary_plus_ngram_index/primary_plus_ngram_index.2.ddl.aql
+++ /dev/null
@@ -1,71 +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  : primary_plus_default_secondary_index.aql
- * Description     : Check that abort from duplicate key exception works and crash recovery works after the abort.   
- * Expected Result : Success
- * Date            : Oct 15 2013
- */
-
-drop dataverse recovery if exists;
-create dataverse recovery;
-use dataverse recovery;
-
-/* For raw Fragile data */
-create type FragileTypeRaw as closed {
-  row_id: int32,
-  sid: int32,
-  date: string,
-  day: int32,
-  time: string,
-  bpm: int32,
-  RR: float,
-  /* new string field and location field*/
-  text: string,
-  location: point,
-  text2: string
-};
-
-/* For cleaned Fragile data */
-create type FragileType as closed {
-  row_id: int32,
-  sid: int32,
-  date: date,
-  day: int32,
-  time: time,
-  bpm: int32,
-  RR: float,
-  
-  /* new string field and location field*/
-  text: string,
-  location: point,
-  text2: string
-};
-
-/* Create dataset for loading raw Fragile data */
-create temporary dataset Fragile_raw (FragileTypeRaw)
-primary key row_id;
-
-/* Create dataset for cleaned Fragile data */
-create temporary dataset Fragile (FragileType)
-primary key row_id;
-
-/* Create default secondary index on dataset clean Fragile */
-create index cfText2Ix on Fragile(text2) type ngram(3);
-


[08/12] asterixdb git commit: [ASTERIXDB-2171][STO] Remove Temporary Datasets

Posted by mh...@apache.org.
http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/results/temp-dataset/compact-dataset-and-its-indexes/compact-dataset-and-its-indexes.1.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temp-dataset/compact-dataset-and-its-indexes/compact-dataset-and-its-indexes.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temp-dataset/compact-dataset-and-its-indexes/compact-dataset-and-its-indexes.1.adm
deleted file mode 100644
index 0d69984..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/temp-dataset/compact-dataset-and-its-indexes/compact-dataset-and-its-indexes.1.adm
+++ /dev/null
@@ -1,167 +0,0 @@
-{ "l_orderkey": 36, "l_partkey": 120, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 42, "l_extendedprice": 42845.04d, "l_discount": 0.09d, "l_tax": 0.0d, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-03", "l_commitdate": "1996-01-21", "l_receiptdate": "1996-02-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": " careful courts. special " }
-{ "l_orderkey": 68, "l_partkey": 8, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 3, "l_extendedprice": 2724.0d, "l_discount": 0.05d, "l_tax": 0.02d, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-04", "l_commitdate": "1998-06-05", "l_receiptdate": "1998-07-21", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "fully special instructions cajole. furious" }
-{ "l_orderkey": 162, "l_partkey": 190, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 2, "l_extendedprice": 2180.38d, "l_discount": 0.02d, "l_tax": 0.01d, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-02", "l_commitdate": "1995-06-17", "l_receiptdate": "1995-09-08", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "es! final somas integrate" }
-{ "l_orderkey": 192, "l_partkey": 98, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 23, "l_extendedprice": 22956.07d, "l_discount": 0.0d, "l_tax": 0.0d, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-05", "l_commitdate": "1998-02-06", "l_receiptdate": "1998-03-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "ly pending theodolites haggle quickly fluf" }
-{ "l_orderkey": 197, "l_partkey": 99, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 39, "l_extendedprice": 38964.51d, "l_discount": 0.02d, "l_tax": 0.04d, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-21", "l_commitdate": "1995-07-01", "l_receiptdate": "1995-08-14", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "press accounts. daringly sp" }
-{ "l_orderkey": 227, "l_partkey": 166, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 19, "l_extendedprice": 20257.04d, "l_discount": 0.05d, "l_tax": 0.06d, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-10", "l_commitdate": "1996-01-30", "l_receiptdate": "1995-12-26", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "s cajole furiously a" }
-{ "l_orderkey": 290, "l_partkey": 6, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 35, "l_extendedprice": 31710.0d, "l_discount": 0.01d, "l_tax": 0.02d, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-04-01", "l_commitdate": "1994-02-05", "l_receiptdate": "1994-04-27", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ove the final foxes detect slyly fluffily" }
-{ "l_orderkey": 325, "l_partkey": 159, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 34, "l_extendedprice": 36011.1d, "l_discount": 0.09d, "l_tax": 0.04d, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-28", "l_commitdate": "1993-12-13", "l_receiptdate": "1993-11-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "ly bold deposits. always iron" }
-{ "l_orderkey": 355, "l_partkey": 114, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 31, "l_extendedprice": 31437.41d, "l_discount": 0.09d, "l_tax": 0.07d, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-07-13", "l_commitdate": "1994-08-18", "l_receiptdate": "1994-07-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "y unusual, ironic" }
-{ "l_orderkey": 389, "l_partkey": 190, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 2, "l_extendedprice": 2180.38d, "l_discount": 0.09d, "l_tax": 0.0d, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-04-13", "l_commitdate": "1994-04-10", "l_receiptdate": "1994-04-25", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "fts. courts eat blithely even dependenc" }
-{ "l_orderkey": 391, "l_partkey": 122, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 14, "l_extendedprice": 14309.68d, "l_discount": 0.09d, "l_tax": 0.02d, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-02-11", "l_commitdate": "1995-02-03", "l_receiptdate": "1995-02-13", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": " escapades sleep furiously about " }
-{ "l_orderkey": 417, "l_partkey": 40, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 39, "l_extendedprice": 36661.56d, "l_discount": 0.01d, "l_tax": 0.02d, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-05-31", "l_commitdate": "1994-05-02", "l_receiptdate": "1994-06-06", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "y regular requests wake along " }
-{ "l_orderkey": 453, "l_partkey": 198, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 45, "l_extendedprice": 49418.55d, "l_discount": 0.01d, "l_tax": 0.0d, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-30", "l_commitdate": "1997-08-20", "l_receiptdate": "1997-07-19", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ifts wake carefully." }
-{ "l_orderkey": 485, "l_partkey": 150, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 50, "l_extendedprice": 52507.5d, "l_discount": 0.01d, "l_tax": 0.0d, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-28", "l_commitdate": "1997-05-26", "l_receiptdate": "1997-04-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "iously quick excuses. carefully final f" }
-{ "l_orderkey": 545, "l_partkey": 170, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 4, "l_extendedprice": 4280.68d, "l_discount": 0.02d, "l_tax": 0.0d, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-23", "l_commitdate": "1995-12-16", "l_receiptdate": "1996-03-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": ", ironic grouches cajole over" }
-{ "l_orderkey": 581, "l_partkey": 64, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 41, "l_extendedprice": 39526.46d, "l_discount": 0.09d, "l_tax": 0.07d, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-26", "l_commitdate": "1997-04-06", "l_receiptdate": "1997-06-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "nts. quickly" }
-{ "l_orderkey": 647, "l_partkey": 17, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 41, "l_extendedprice": 37597.41d, "l_discount": 0.08d, "l_tax": 0.08d, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-19", "l_commitdate": "1997-09-24", "l_receiptdate": "1997-12-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "r instructions. quickly unusu" }
-{ "l_orderkey": 704, "l_partkey": 190, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 40, "l_extendedprice": 43607.6d, "l_discount": 0.05d, "l_tax": 0.05d, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-30", "l_commitdate": "1997-01-10", "l_receiptdate": "1997-02-20", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ggle quickly. r" }
-{ "l_orderkey": 738, "l_partkey": 198, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 34, "l_extendedprice": 37338.46d, "l_discount": 0.0d, "l_tax": 0.06d, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-09", "l_commitdate": "1993-04-15", "l_receiptdate": "1993-07-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "s against the ironic exc" }
-{ "l_orderkey": 773, "l_partkey": 100, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 5, "l_extendedprice": 5000.5d, "l_discount": 0.06d, "l_tax": 0.04d, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-11-21", "l_commitdate": "1993-12-19", "l_receiptdate": "1993-12-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ar requests. regular, thin packages u" }
-{ "l_orderkey": 800, "l_partkey": 72, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 38, "l_extendedprice": 36938.66d, "l_discount": 0.0d, "l_tax": 0.05d, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-21", "l_commitdate": "1998-09-25", "l_receiptdate": "1998-08-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "according to the bold, final dependencies " }
-{ "l_orderkey": 900, "l_partkey": 199, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 44, "l_extendedprice": 48364.36d, "l_discount": 0.01d, "l_tax": 0.06d, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-15", "l_commitdate": "1994-12-03", "l_receiptdate": "1994-12-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": " detect quick" }
-{ "l_orderkey": 931, "l_partkey": 40, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 18, "l_extendedprice": 16920.72d, "l_discount": 0.0d, "l_tax": 0.05d, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-04-04", "l_commitdate": "1993-01-11", "l_receiptdate": "1993-04-13", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "slyly ironic re" }
-{ "l_orderkey": 932, "l_partkey": 44, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 41, "l_extendedprice": 38705.64d, "l_discount": 0.01d, "l_tax": 0.05d, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-05", "l_commitdate": "1997-07-22", "l_receiptdate": "1997-06-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "foxes. ironic pl" }
-{ "l_orderkey": 965, "l_partkey": 108, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 20, "l_extendedprice": 20162.0d, "l_discount": 0.04d, "l_tax": 0.05d, "l_returnflag": "N", "l_linestatus": "F", "l_shipdate": "1995-06-16", "l_commitdate": "1995-07-20", "l_receiptdate": "1995-07-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "kly. carefully pending requ" }
-{ "l_orderkey": 995, "l_partkey": 173, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 15, "l_extendedprice": 16097.55d, "l_discount": 0.08d, "l_tax": 0.05d, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-06-30", "l_commitdate": "1995-08-04", "l_receiptdate": "1995-07-27", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "uses. fluffily fina" }
-{ "l_orderkey": 1025, "l_partkey": 150, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 36, "l_extendedprice": 37805.4d, "l_discount": 0.03d, "l_tax": 0.04d, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-05-15", "l_commitdate": "1995-07-05", "l_receiptdate": "1995-06-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "e unusual, regular instr" }
-{ "l_orderkey": 1027, "l_partkey": 156, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 43, "l_extendedprice": 45414.45d, "l_discount": 0.07d, "l_tax": 0.08d, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-06-17", "l_commitdate": "1992-08-28", "l_receiptdate": "1992-07-10", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "oxes. carefully regular deposits" }
-{ "l_orderkey": 1155, "l_partkey": 70, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 4, "l_extendedprice": 3880.28d, "l_discount": 0.09d, "l_tax": 0.05d, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-19", "l_commitdate": "1997-12-09", "l_receiptdate": "1997-11-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ic foxes according to the carefully final " }
-{ "l_orderkey": 1185, "l_partkey": 72, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 8, "l_extendedprice": 7776.56d, "l_discount": 0.01d, "l_tax": 0.06d, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-12-05", "l_commitdate": "1992-10-05", "l_receiptdate": "1992-12-28", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ely according to the furiously regular r" }
-{ "l_orderkey": 1216, "l_partkey": 97, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 8, "l_extendedprice": 7976.72d, "l_discount": 0.03d, "l_tax": 0.04d, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-02-01", "l_commitdate": "1993-03-06", "l_receiptdate": "1993-02-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": " of the carefully express" }
-{ "l_orderkey": 1223, "l_partkey": 100, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 28, "l_extendedprice": 28002.8d, "l_discount": 0.1d, "l_tax": 0.06d, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-07", "l_commitdate": "1996-07-24", "l_receiptdate": "1996-08-13", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": " quickly ironic requests. furious" }
-{ "l_orderkey": 1381, "l_partkey": 144, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 47, "l_extendedprice": 49074.58d, "l_discount": 0.08d, "l_tax": 0.04d, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-09-22", "l_commitdate": "1998-08-12", "l_receiptdate": "1998-10-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "ly ironic deposits" }
-{ "l_orderkey": 1409, "l_partkey": 99, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 23, "l_extendedprice": 22979.07d, "l_discount": 0.01d, "l_tax": 0.03d, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-04-18", "l_commitdate": "1993-02-25", "l_receiptdate": "1993-05-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ions. slyly ironic packages wake quick" }
-{ "l_orderkey": 1445, "l_partkey": 100, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 24, "l_extendedprice": 24002.4d, "l_discount": 0.01d, "l_tax": 0.0d, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-02-21", "l_commitdate": "1995-02-22", "l_receiptdate": "1995-03-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "al accounts use furiously a" }
-{ "l_orderkey": 1477, "l_partkey": 72, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 31, "l_extendedprice": 30134.17d, "l_discount": 0.0d, "l_tax": 0.06d, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-16", "l_commitdate": "1997-09-30", "l_receiptdate": "1997-12-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": " requests. fluffily final " }
-{ "l_orderkey": 1540, "l_partkey": 173, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 38, "l_extendedprice": 40780.46d, "l_discount": 0.03d, "l_tax": 0.01d, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-09-30", "l_commitdate": "1992-10-27", "l_receiptdate": "1992-10-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": " final grouches bo" }
-{ "l_orderkey": 1568, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 36, "l_extendedprice": 35643.24d, "l_discount": 0.02d, "l_tax": 0.03d, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-31", "l_commitdate": "1997-04-22", "l_receiptdate": "1997-06-21", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "platelets-- furiously sly excu" }
-{ "l_orderkey": 1605, "l_partkey": 142, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 47, "l_extendedprice": 48980.58d, "l_discount": 0.0d, "l_tax": 0.01d, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-29", "l_commitdate": "1998-06-12", "l_receiptdate": "1998-05-20", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": ". carefully r" }
-{ "l_orderkey": 1607, "l_partkey": 190, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 2, "l_extendedprice": 2180.38d, "l_discount": 0.02d, "l_tax": 0.0d, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-11", "l_commitdate": "1996-02-15", "l_receiptdate": "1996-01-19", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "packages haggle. regular requests boost s" }
-{ "l_orderkey": 1635, "l_partkey": 71, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 3, "l_extendedprice": 2913.21d, "l_discount": 0.06d, "l_tax": 0.08d, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-13", "l_commitdate": "1997-03-25", "l_receiptdate": "1997-03-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": " quickly ironic r" }
-{ "l_orderkey": 1700, "l_partkey": 140, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 38, "l_extendedprice": 39525.32d, "l_discount": 0.04d, "l_tax": 0.04d, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-03", "l_commitdate": "1996-07-27", "l_receiptdate": "1996-10-22", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "ular dependencies engage slyly " }
-{ "l_orderkey": 1796, "l_partkey": 10, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 28, "l_extendedprice": 25480.28d, "l_discount": 0.08d, "l_tax": 0.04d, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-12-01", "l_commitdate": "1993-01-01", "l_receiptdate": "1992-12-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "y quickly ironic accounts." }
-{ "l_orderkey": 1825, "l_partkey": 156, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 43, "l_extendedprice": 45414.45d, "l_discount": 0.05d, "l_tax": 0.05d, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-02-18", "l_commitdate": "1994-02-19", "l_receiptdate": "1994-03-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": " accounts breach fluffily spe" }
-{ "l_orderkey": 1827, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 47, "l_extendedprice": 46534.23d, "l_discount": 0.0d, "l_tax": 0.01d, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-01", "l_commitdate": "1996-08-07", "l_receiptdate": "1996-08-23", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": ". pending courts about the even e" }
-{ "l_orderkey": 1893, "l_partkey": 99, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 43, "l_extendedprice": 42960.87d, "l_discount": 0.1d, "l_tax": 0.0d, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-25", "l_commitdate": "1998-01-06", "l_receiptdate": "1998-02-14", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "he carefully regular " }
-{ "l_orderkey": 1924, "l_partkey": 73, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 7, "l_extendedprice": 6811.49d, "l_discount": 0.06d, "l_tax": 0.07d, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-01", "l_commitdate": "1996-12-02", "l_receiptdate": "1997-01-08", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "osits. even accounts nag furious" }
-{ "l_orderkey": 1953, "l_partkey": 128, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 25, "l_extendedprice": 25703.0d, "l_discount": 0.07d, "l_tax": 0.06d, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-07", "l_commitdate": "1994-01-28", "l_receiptdate": "1994-01-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ular, regular i" }
-{ "l_orderkey": 1985, "l_partkey": 28, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 33, "l_extendedprice": 30624.66d, "l_discount": 0.1d, "l_tax": 0.03d, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-04", "l_commitdate": "1994-11-01", "l_receiptdate": "1994-12-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "s are express packages. pendin" }
-{ "l_orderkey": 1988, "l_partkey": 72, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 36, "l_extendedprice": 34994.52d, "l_discount": 0.09d, "l_tax": 0.04d, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-21", "l_commitdate": "1995-11-24", "l_receiptdate": "1996-01-27", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "gular theodolites. " }
-{ "l_orderkey": 2048, "l_partkey": 35, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 7, "l_extendedprice": 6545.21d, "l_discount": 0.06d, "l_tax": 0.01d, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-12-07", "l_commitdate": "1994-01-31", "l_receiptdate": "1994-01-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "lent platelets boost deposits. carefully sp" }
-{ "l_orderkey": 2086, "l_partkey": 60, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 22, "l_extendedprice": 21121.32d, "l_discount": 0.03d, "l_tax": 0.07d, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-04", "l_commitdate": "1994-12-16", "l_receiptdate": "1994-12-20", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "idly busy acc" }
-{ "l_orderkey": 2118, "l_partkey": 160, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 24, "l_extendedprice": 25443.84d, "l_discount": 0.1d, "l_tax": 0.03d, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-06", "l_commitdate": "1996-12-14", "l_receiptdate": "1997-01-14", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "about the slyly bold depende" }
-{ "l_orderkey": 2183, "l_partkey": 71, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 29, "l_extendedprice": 28161.03d, "l_discount": 0.05d, "l_tax": 0.01d, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-21", "l_commitdate": "1996-08-24", "l_receiptdate": "1996-08-15", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ly unusual deposits sleep carefully" }
-{ "l_orderkey": 2211, "l_partkey": 48, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 25, "l_extendedprice": 23701.0d, "l_discount": 0.04d, "l_tax": 0.01d, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-10-09", "l_commitdate": "1994-08-04", "l_receiptdate": "1994-11-03", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "deas. carefully special theodolites along" }
-{ "l_orderkey": 2215, "l_partkey": 73, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 33, "l_extendedprice": 32111.31d, "l_discount": 0.0d, "l_tax": 0.0d, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-19", "l_commitdate": "1996-08-10", "l_receiptdate": "1996-07-30", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "dolites cajole b" }
-{ "l_orderkey": 2272, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 18, "l_extendedprice": 17821.62d, "l_discount": 0.04d, "l_tax": 0.0d, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-08-01", "l_commitdate": "1993-07-06", "l_receiptdate": "1993-08-25", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ons along the blithely e" }
-{ "l_orderkey": 2342, "l_partkey": 42, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 12, "l_extendedprice": 11304.48d, "l_discount": 0.0d, "l_tax": 0.08d, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-31", "l_commitdate": "1996-07-26", "l_receiptdate": "1996-08-14", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "print blithely even deposits. carefull" }
-{ "l_orderkey": 2343, "l_partkey": 110, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 27, "l_extendedprice": 27272.97d, "l_discount": 0.0d, "l_tax": 0.0d, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-10", "l_commitdate": "1995-11-17", "l_receiptdate": "1995-12-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "old theodolites." }
-{ "l_orderkey": 2439, "l_partkey": 164, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 2, "l_extendedprice": 2128.32d, "l_discount": 0.09d, "l_tax": 0.03d, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-14", "l_commitdate": "1997-06-11", "l_receiptdate": "1997-05-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "courts boos" }
-{ "l_orderkey": 2469, "l_partkey": 166, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 11, "l_extendedprice": 11727.76d, "l_discount": 0.0d, "l_tax": 0.04d, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-09", "l_commitdate": "1997-01-26", "l_receiptdate": "1997-02-16", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ies wake carefully b" }
-{ "l_orderkey": 2592, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 7, "l_extendedprice": 6930.63d, "l_discount": 0.1d, "l_tax": 0.04d, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-03-13", "l_commitdate": "1993-04-25", "l_receiptdate": "1993-04-01", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": " carefully special theodolites integrate " }
-{ "l_orderkey": 2625, "l_partkey": 20, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 42, "l_extendedprice": 38640.84d, "l_discount": 0.02d, "l_tax": 0.04d, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-10-18", "l_commitdate": "1992-11-17", "l_receiptdate": "1992-10-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": " even accounts haggle furiously" }
-{ "l_orderkey": 2659, "l_partkey": 42, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 28, "l_extendedprice": 26377.12d, "l_discount": 0.08d, "l_tax": 0.05d, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-03-17", "l_commitdate": "1994-01-24", "l_receiptdate": "1994-03-19", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "idle tithes" }
-{ "l_orderkey": 2689, "l_partkey": 6, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 45, "l_extendedprice": 40770.0d, "l_discount": 0.02d, "l_tax": 0.04d, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-04-29", "l_commitdate": "1992-06-22", "l_receiptdate": "1992-04-30", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "e quickly. carefully silent" }
-{ "l_orderkey": 2690, "l_partkey": 140, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 44, "l_extendedprice": 45766.16d, "l_discount": 0.05d, "l_tax": 0.06d, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-30", "l_commitdate": "1996-05-19", "l_receiptdate": "1996-06-26", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "ly alongside of th" }
-{ "l_orderkey": 2692, "l_partkey": 17, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 3, "l_extendedprice": 2751.03d, "l_discount": 0.1d, "l_tax": 0.04d, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-25", "l_commitdate": "1998-01-29", "l_receiptdate": "1998-03-27", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "equests. bold, even foxes haggle slyl" }
-{ "l_orderkey": 2694, "l_partkey": 153, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 30, "l_extendedprice": 31594.5d, "l_discount": 0.02d, "l_tax": 0.06d, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-20", "l_commitdate": "1996-06-01", "l_receiptdate": "1996-07-15", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "oxes. never iro" }
-{ "l_orderkey": 2759, "l_partkey": 59, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 10, "l_extendedprice": 9590.5d, "l_discount": 0.1d, "l_tax": 0.03d, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-12-14", "l_commitdate": "1994-01-08", "l_receiptdate": "1994-01-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "s. busily ironic theodo" }
-{ "l_orderkey": 2819, "l_partkey": 70, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 17, "l_extendedprice": 16491.19d, "l_discount": 0.08d, "l_tax": 0.08d, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-07-16", "l_commitdate": "1994-07-15", "l_receiptdate": "1994-07-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "en deposits above the f" }
-{ "l_orderkey": 2850, "l_partkey": 97, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 43, "l_extendedprice": 42874.87d, "l_discount": 0.02d, "l_tax": 0.05d, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-11", "l_commitdate": "1996-11-03", "l_receiptdate": "1997-02-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "unusual accounts" }
-{ "l_orderkey": 2886, "l_partkey": 60, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 1, "l_extendedprice": 960.06d, "l_discount": 0.09d, "l_tax": 0.05d, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-02-01", "l_commitdate": "1994-12-18", "l_receiptdate": "1995-02-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "eposits fr" }
-{ "l_orderkey": 2912, "l_partkey": 122, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 8, "l_extendedprice": 8176.96d, "l_discount": 0.06d, "l_tax": 0.04d, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-04-09", "l_commitdate": "1992-04-19", "l_receiptdate": "1992-04-26", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "hs cajole over the slyl" }
-{ "l_orderkey": 2944, "l_partkey": 120, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 44, "l_extendedprice": 44885.28d, "l_discount": 0.08d, "l_tax": 0.05d, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-25", "l_commitdate": "1997-10-28", "l_receiptdate": "1998-01-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ickly special theodolit" }
-{ "l_orderkey": 2947, "l_partkey": 10, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 37, "l_extendedprice": 33670.37d, "l_discount": 0.09d, "l_tax": 0.07d, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-09", "l_commitdate": "1995-07-05", "l_receiptdate": "1995-08-20", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "e accounts: expres" }
-{ "l_orderkey": 2950, "l_partkey": 130, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 32, "l_extendedprice": 32964.16d, "l_discount": 0.01d, "l_tax": 0.05d, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-21", "l_commitdate": "1997-08-25", "l_receiptdate": "1997-10-08", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "its wake carefully slyly final ideas." }
-{ "l_orderkey": 2978, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 29, "l_extendedprice": 28712.61d, "l_discount": 0.0d, "l_tax": 0.08d, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-06-03", "l_commitdate": "1995-07-25", "l_receiptdate": "1995-06-06", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ecial ideas promise slyly" }
-{ "l_orderkey": 3143, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 22, "l_extendedprice": 21781.98d, "l_discount": 0.02d, "l_tax": 0.0d, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-11", "l_commitdate": "1993-03-26", "l_receiptdate": "1993-05-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "l, special instructions nag " }
-{ "l_orderkey": 3264, "l_partkey": 200, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 39, "l_extendedprice": 42907.8d, "l_discount": 0.06d, "l_tax": 0.06d, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-07", "l_commitdate": "1996-12-12", "l_receiptdate": "1996-11-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "sleep carefully after the slyly final" }
-{ "l_orderkey": 3266, "l_partkey": 64, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 31, "l_extendedprice": 29885.86d, "l_discount": 0.09d, "l_tax": 0.02d, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-06-19", "l_commitdate": "1995-05-04", "l_receiptdate": "1995-07-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "grate among the quickly express deposits" }
-{ "l_orderkey": 3270, "l_partkey": 35, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 11, "l_extendedprice": 10285.33d, "l_discount": 0.07d, "l_tax": 0.06d, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-29", "l_commitdate": "1997-08-11", "l_receiptdate": "1997-08-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": " solve at the regular deposits. " }
-{ "l_orderkey": 3364, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 49, "l_extendedprice": 48514.41d, "l_discount": 0.03d, "l_tax": 0.05d, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-17", "l_commitdate": "1997-08-23", "l_receiptdate": "1997-10-06", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "d accounts? caref" }
-{ "l_orderkey": 3366, "l_partkey": 40, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 4, "l_extendedprice": 3760.16d, "l_discount": 0.07d, "l_tax": 0.01d, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-20", "l_commitdate": "1997-06-25", "l_receiptdate": "1997-06-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": " carefully about " }
-{ "l_orderkey": 3425, "l_partkey": 120, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 11, "l_extendedprice": 11221.32d, "l_discount": 0.03d, "l_tax": 0.08d, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-24", "l_commitdate": "1996-05-29", "l_receiptdate": "1996-05-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ckly final deposits use quickly?" }
-{ "l_orderkey": 3460, "l_partkey": 11, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 40, "l_extendedprice": 36440.4d, "l_discount": 0.1d, "l_tax": 0.06d, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-28", "l_commitdate": "1995-12-14", "l_receiptdate": "1996-01-02", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "odolites are slyly bold deposits" }
-{ "l_orderkey": 3494, "l_partkey": 117, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 40, "l_extendedprice": 40684.4d, "l_discount": 0.05d, "l_tax": 0.04d, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-07-10", "l_commitdate": "1993-06-01", "l_receiptdate": "1993-07-25", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "lites haggle furiously about the fin" }
-{ "l_orderkey": 3520, "l_partkey": 28, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 30, "l_extendedprice": 27840.6d, "l_discount": 0.04d, "l_tax": 0.02d, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-11", "l_commitdate": "1997-10-02", "l_receiptdate": "1997-12-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "deas should solve blithely among the ironi" }
-{ "l_orderkey": 3559, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 29, "l_extendedprice": 28712.61d, "l_discount": 0.0d, "l_tax": 0.07d, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-12-10", "l_commitdate": "1992-12-03", "l_receiptdate": "1992-12-20", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "l, regular accounts wake flu" }
-{ "l_orderkey": 3585, "l_partkey": 122, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 21, "l_extendedprice": 21464.52d, "l_discount": 0.05d, "l_tax": 0.04d, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-04", "l_commitdate": "1994-12-25", "l_receiptdate": "1995-01-01", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ounts use. express, final platelets us" }
-{ "l_orderkey": 3618, "l_partkey": 140, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 38, "l_extendedprice": 39525.32d, "l_discount": 0.08d, "l_tax": 0.0d, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-22", "l_commitdate": "1998-02-23", "l_receiptdate": "1998-01-03", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "nts haggle fluffily above the regular " }
-{ "l_orderkey": 3715, "l_partkey": 97, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 13, "l_extendedprice": 12962.17d, "l_discount": 0.0d, "l_tax": 0.03d, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-11", "l_commitdate": "1996-04-25", "l_receiptdate": "1996-06-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "e quickly ironic" }
-{ "l_orderkey": 3844, "l_partkey": 135, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 2, "l_extendedprice": 2070.26d, "l_discount": 0.03d, "l_tax": 0.07d, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-02-24", "l_commitdate": "1995-02-03", "l_receiptdate": "1995-03-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "es haggle final acco" }
-{ "l_orderkey": 3878, "l_partkey": 200, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 6, "l_extendedprice": 6601.2d, "l_discount": 0.07d, "l_tax": 0.04d, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-21", "l_commitdate": "1997-05-22", "l_receiptdate": "1997-07-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "s. regular instru" }
-{ "l_orderkey": 3906, "l_partkey": 153, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 42, "l_extendedprice": 44232.3d, "l_discount": 0.0d, "l_tax": 0.04d, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-09-03", "l_commitdate": "1992-07-22", "l_receiptdate": "1992-09-04", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "jole blithely after the furiously regular " }
-{ "l_orderkey": 3974, "l_partkey": 22, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 47, "l_extendedprice": 43334.94d, "l_discount": 0.1d, "l_tax": 0.03d, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-03", "l_commitdate": "1996-05-08", "l_receiptdate": "1996-06-28", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "dencies above the re" }
-{ "l_orderkey": 4001, "l_partkey": 106, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 26, "l_extendedprice": 26158.6d, "l_discount": 0.0d, "l_tax": 0.01d, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-26", "l_commitdate": "1997-06-18", "l_receiptdate": "1997-08-08", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "tegrate blithely" }
-{ "l_orderkey": 4005, "l_partkey": 4, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 26, "l_extendedprice": 23504.0d, "l_discount": 0.09d, "l_tax": 0.05d, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-01", "l_commitdate": "1997-02-03", "l_receiptdate": "1996-12-15", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": " to the quic" }
-{ "l_orderkey": 4033, "l_partkey": 110, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 27, "l_extendedprice": 27272.97d, "l_discount": 0.01d, "l_tax": 0.04d, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-08-08", "l_commitdate": "1993-08-14", "l_receiptdate": "1993-08-09", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "pinto beans" }
-{ "l_orderkey": 4034, "l_partkey": 190, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 48, "l_extendedprice": 52329.12d, "l_discount": 0.03d, "l_tax": 0.03d, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-03-01", "l_commitdate": "1994-01-16", "l_receiptdate": "1994-03-16", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": " blithely regular requests play carefull" }
-{ "l_orderkey": 4036, "l_partkey": 6, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 46, "l_extendedprice": 41676.0d, "l_discount": 0.09d, "l_tax": 0.0d, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-21", "l_commitdate": "1997-05-29", "l_receiptdate": "1997-07-18", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "usly across the even th" }
-{ "l_orderkey": 4064, "l_partkey": 199, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 3, "l_extendedprice": 3297.57d, "l_discount": 0.1d, "l_tax": 0.04d, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-04", "l_commitdate": "1997-01-01", "l_receiptdate": "1997-01-23", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "its! quickly sp" }
-{ "l_orderkey": 4067, "l_partkey": 180, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 18, "l_extendedprice": 19443.24d, "l_discount": 0.03d, "l_tax": 0.08d, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-01-24", "l_commitdate": "1992-12-23", "l_receiptdate": "1993-02-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "e the slyly final packages d" }
-{ "l_orderkey": 4068, "l_partkey": 110, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 43, "l_extendedprice": 43434.73d, "l_discount": 0.05d, "l_tax": 0.06d, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-28", "l_commitdate": "1996-11-16", "l_receiptdate": "1996-12-22", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ructions. regular, special packag" }
-{ "l_orderkey": 4098, "l_partkey": 200, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 46, "l_extendedprice": 50609.2d, "l_discount": 0.07d, "l_tax": 0.03d, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-26", "l_commitdate": "1997-01-27", "l_receiptdate": "1997-02-13", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "e slyly blithely silent deposits. fluff" }
-{ "l_orderkey": 4192, "l_partkey": 11, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 36, "l_extendedprice": 32796.36d, "l_discount": 0.06d, "l_tax": 0.08d, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-25", "l_commitdate": "1998-05-26", "l_receiptdate": "1998-05-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "eodolites sleep" }
-{ "l_orderkey": 4194, "l_partkey": 197, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 43, "l_extendedprice": 47179.17d, "l_discount": 0.08d, "l_tax": 0.06d, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-11-06", "l_commitdate": "1994-12-09", "l_receiptdate": "1994-11-16", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "olites are after the exp" }
-{ "l_orderkey": 4261, "l_partkey": 110, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 12, "l_extendedprice": 12121.32d, "l_discount": 0.05d, "l_tax": 0.01d, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-11-01", "l_commitdate": "1993-01-01", "l_receiptdate": "1992-11-12", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "into beans " }
-{ "l_orderkey": 4418, "l_partkey": 35, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 32, "l_extendedprice": 29920.96d, "l_discount": 0.02d, "l_tax": 0.06d, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-28", "l_commitdate": "1993-06-02", "l_receiptdate": "1993-05-30", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ly. bold pinto b" }
-{ "l_orderkey": 4422, "l_partkey": 135, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 5, "l_extendedprice": 5175.65d, "l_discount": 0.09d, "l_tax": 0.07d, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-17", "l_commitdate": "1995-08-13", "l_receiptdate": "1995-07-25", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "e furiously about t" }
-{ "l_orderkey": 4486, "l_partkey": 135, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 46, "l_extendedprice": 47615.98d, "l_discount": 0.08d, "l_tax": 0.0d, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-02", "l_commitdate": "1998-04-05", "l_receiptdate": "1998-05-08", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ackages. specia" }
-{ "l_orderkey": 4512, "l_partkey": 162, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 30, "l_extendedprice": 31864.8d, "l_discount": 0.07d, "l_tax": 0.07d, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-28", "l_commitdate": "1995-12-22", "l_receiptdate": "1996-02-22", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ly unusual package" }
-{ "l_orderkey": 4513, "l_partkey": 170, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 29, "l_extendedprice": 31034.93d, "l_discount": 0.03d, "l_tax": 0.01d, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-18", "l_commitdate": "1996-05-23", "l_receiptdate": "1996-06-08", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "cajole. regular packages boost. s" }
-{ "l_orderkey": 4545, "l_partkey": 173, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 38, "l_extendedprice": 40780.46d, "l_discount": 0.06d, "l_tax": 0.06d, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-27", "l_commitdate": "1993-03-01", "l_receiptdate": "1993-02-04", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "nts serve according to th" }
-{ "l_orderkey": 4549, "l_partkey": 159, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 44, "l_extendedprice": 46602.6d, "l_discount": 0.08d, "l_tax": 0.0d, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-13", "l_commitdate": "1998-04-15", "l_receiptdate": "1998-03-27", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ding to the regular, silent requests" }
-{ "l_orderkey": 4551, "l_partkey": 11, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 6, "l_extendedprice": 5466.06d, "l_discount": 0.08d, "l_tax": 0.08d, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-18", "l_commitdate": "1996-04-23", "l_receiptdate": "1996-06-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "fily silent fo" }
-{ "l_orderkey": 4576, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 5, "l_extendedprice": 4950.45d, "l_discount": 0.09d, "l_tax": 0.03d, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-23", "l_commitdate": "1996-11-08", "l_receiptdate": "1996-09-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "ly express, special asymptote" }
-{ "l_orderkey": 4608, "l_partkey": 173, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 30, "l_extendedprice": 32195.1d, "l_discount": 0.08d, "l_tax": 0.05d, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-10-08", "l_commitdate": "1994-07-18", "l_receiptdate": "1994-10-25", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "s cajole. slyly " }
-{ "l_orderkey": 4641, "l_partkey": 190, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 45, "l_extendedprice": 49058.55d, "l_discount": 0.07d, "l_tax": 0.03d, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-05-11", "l_commitdate": "1993-04-19", "l_receiptdate": "1993-05-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": " about the close " }
-{ "l_orderkey": 4679, "l_partkey": 190, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 7, "l_extendedprice": 7631.33d, "l_discount": 0.1d, "l_tax": 0.05d, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-05-11", "l_commitdate": "1993-04-11", "l_receiptdate": "1993-05-16", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "kages. bold, regular packa" }
-{ "l_orderkey": 4769, "l_partkey": 35, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 16, "l_extendedprice": 14960.48d, "l_discount": 0.08d, "l_tax": 0.05d, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-16", "l_commitdate": "1995-07-05", "l_receiptdate": "1995-07-22", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": " deposits. slyly even asymptote" }
-{ "l_orderkey": 4802, "l_partkey": 40, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 6, "l_extendedprice": 5640.24d, "l_discount": 0.0d, "l_tax": 0.06d, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-16", "l_commitdate": "1997-03-25", "l_receiptdate": "1997-04-21", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "unusual accounts wake blithely. b" }
-{ "l_orderkey": 4804, "l_partkey": 128, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 44, "l_extendedprice": 45237.28d, "l_discount": 0.06d, "l_tax": 0.08d, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-05-02", "l_commitdate": "1992-03-24", "l_receiptdate": "1992-05-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "aggle quickly among the slyly fi" }
-{ "l_orderkey": 4805, "l_partkey": 150, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 7, "l_extendedprice": 7351.05d, "l_discount": 0.09d, "l_tax": 0.03d, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-05-01", "l_commitdate": "1992-07-09", "l_receiptdate": "1992-05-09", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": " requests. regular deposit" }
-{ "l_orderkey": 4807, "l_partkey": 122, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 9, "l_extendedprice": 9199.08d, "l_discount": 0.04d, "l_tax": 0.08d, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-23", "l_commitdate": "1997-03-01", "l_receiptdate": "1997-05-15", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "may are blithely. carefully even pinto b" }
-{ "l_orderkey": 4836, "l_partkey": 162, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 22, "l_extendedprice": 23367.52d, "l_discount": 0.01d, "l_tax": 0.03d, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-03", "l_commitdate": "1997-02-23", "l_receiptdate": "1997-03-04", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "al pinto beans. care" }
-{ "l_orderkey": 4837, "l_partkey": 42, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 16, "l_extendedprice": 15072.64d, "l_discount": 0.09d, "l_tax": 0.04d, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-12", "l_commitdate": "1998-06-06", "l_receiptdate": "1998-08-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ing requests are blithely regular instructi" }
-{ "l_orderkey": 4898, "l_partkey": 72, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 44, "l_extendedprice": 42771.08d, "l_discount": 0.07d, "l_tax": 0.02d, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-09-13", "l_commitdate": "1994-08-18", "l_receiptdate": "1994-09-16", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "y regular grouches about" }
-{ "l_orderkey": 4928, "l_partkey": 100, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 4, "l_extendedprice": 4000.4d, "l_discount": 0.04d, "l_tax": 0.02d, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-10-25", "l_commitdate": "1993-12-24", "l_receiptdate": "1993-11-16", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "bout the slyly final accounts. carefull" }
-{ "l_orderkey": 4929, "l_partkey": 14, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 20, "l_extendedprice": 18280.2d, "l_discount": 0.0d, "l_tax": 0.04d, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-12", "l_commitdate": "1996-05-23", "l_receiptdate": "1996-03-20", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": " final pinto beans detect. final," }
-{ "l_orderkey": 4967, "l_partkey": 71, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 50, "l_extendedprice": 48553.5d, "l_discount": 0.07d, "l_tax": 0.01d, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-27", "l_commitdate": "1997-05-13", "l_receiptdate": "1997-06-12", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "kages. final, unusual accounts c" }
-{ "l_orderkey": 4996, "l_partkey": 56, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 35, "l_extendedprice": 33461.75d, "l_discount": 0.07d, "l_tax": 0.01d, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-10-30", "l_commitdate": "1992-10-27", "l_receiptdate": "1992-11-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "s. unusual, regular dolphins integrate care" }
-{ "l_orderkey": 5028, "l_partkey": 14, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 15, "l_extendedprice": 13710.15d, "l_discount": 0.07d, "l_tax": 0.07d, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-07-17", "l_commitdate": "1992-07-16", "l_receiptdate": "1992-08-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "es are quickly final pains. furiously pend" }
-{ "l_orderkey": 5031, "l_partkey": 50, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 15, "l_extendedprice": 14250.75d, "l_discount": 0.02d, "l_tax": 0.05d, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-04-01", "l_commitdate": "1995-02-24", "l_receiptdate": "1995-04-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "yly pending theodolites." }
-{ "l_orderkey": 5092, "l_partkey": 164, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 30, "l_extendedprice": 31924.8d, "l_discount": 0.06d, "l_tax": 0.0d, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-27", "l_commitdate": "1995-12-08", "l_receiptdate": "1996-01-09", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ss, ironic deposits. furiously stea" }
-{ "l_orderkey": 5153, "l_partkey": 35, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 42, "l_extendedprice": 39271.26d, "l_discount": 0.03d, "l_tax": 0.01d, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-03", "l_commitdate": "1995-11-09", "l_receiptdate": "1995-10-11", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "re thinly. ironic" }
-{ "l_orderkey": 5154, "l_partkey": 190, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 11, "l_extendedprice": 11992.09d, "l_discount": 0.02d, "l_tax": 0.05d, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-06", "l_commitdate": "1997-06-30", "l_receiptdate": "1997-09-04", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "luffily bold foxes. final" }
-{ "l_orderkey": 5185, "l_partkey": 197, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 37, "l_extendedprice": 40596.03d, "l_discount": 0.0d, "l_tax": 0.04d, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-08", "l_commitdate": "1997-09-08", "l_receiptdate": "1997-08-14", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "gainst the courts dazzle care" }
-{ "l_orderkey": 5187, "l_partkey": 11, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 49, "l_extendedprice": 44639.49d, "l_discount": 0.04d, "l_tax": 0.06d, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-20", "l_commitdate": "1997-10-12", "l_receiptdate": "1997-10-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "l, regular platelets instead of the foxes w" }
-{ "l_orderkey": 5190, "l_partkey": 56, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 43, "l_extendedprice": 41110.15d, "l_discount": 0.09d, "l_tax": 0.06d, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-08-19", "l_commitdate": "1992-06-10", "l_receiptdate": "1992-09-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "encies use fluffily unusual requests? hoc" }
-{ "l_orderkey": 5217, "l_partkey": 80, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 50, "l_extendedprice": 49004.0d, "l_discount": 0.05d, "l_tax": 0.02d, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-26", "l_commitdate": "1995-11-21", "l_receiptdate": "1996-01-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "s. express, express accounts c" }
-{ "l_orderkey": 5281, "l_partkey": 114, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 37, "l_extendedprice": 37522.07d, "l_discount": 0.05d, "l_tax": 0.02d, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-10", "l_commitdate": "1996-01-31", "l_receiptdate": "1995-11-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ronic dependencies. fluffily final p" }
-{ "l_orderkey": 5284, "l_partkey": 173, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 16, "l_extendedprice": 17170.72d, "l_discount": 0.04d, "l_tax": 0.02d, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-17", "l_commitdate": "1995-08-23", "l_receiptdate": "1995-08-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "unts detect furiously even d" }
-{ "l_orderkey": 5286, "l_partkey": 199, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 1, "l_extendedprice": 1099.19d, "l_discount": 0.01d, "l_tax": 0.07d, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-25", "l_commitdate": "1997-11-07", "l_receiptdate": "1997-12-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ly! furiously final pack" }
-{ "l_orderkey": 5313, "l_partkey": 17, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 34, "l_extendedprice": 31178.34d, "l_discount": 0.1d, "l_tax": 0.02d, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-07", "l_commitdate": "1997-08-12", "l_receiptdate": "1997-08-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ccording to the blithely final account" }
-{ "l_orderkey": 5315, "l_partkey": 35, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 12, "l_extendedprice": 11220.36d, "l_discount": 0.08d, "l_tax": 0.06d, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-12-18", "l_commitdate": "1993-01-16", "l_receiptdate": "1993-01-10", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ccounts. furiously ironi" }
-{ "l_orderkey": 5316, "l_partkey": 108, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 29, "l_extendedprice": 29234.9d, "l_discount": 0.1d, "l_tax": 0.05d, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-28", "l_commitdate": "1994-04-29", "l_receiptdate": "1994-04-09", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "ckly unusual foxes bo" }
-{ "l_orderkey": 5379, "l_partkey": 199, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 40, "l_extendedprice": 43967.6d, "l_discount": 0.01d, "l_tax": 0.08d, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-01", "l_commitdate": "1995-10-19", "l_receiptdate": "1995-10-30", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "carefully final accounts haggle blithely. " }
-{ "l_orderkey": 5441, "l_partkey": 164, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 3, "l_extendedprice": 3192.48d, "l_discount": 0.0d, "l_tax": 0.02d, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-08-12", "l_commitdate": "1994-10-14", "l_receiptdate": "1994-09-01", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "are. unusual, " }
-{ "l_orderkey": 5445, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 33, "l_extendedprice": 32672.97d, "l_discount": 0.08d, "l_tax": 0.07d, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-21", "l_commitdate": "1993-10-14", "l_receiptdate": "1993-10-28", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "ests. final instructions" }
-{ "l_orderkey": 5446, "l_partkey": 190, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 27, "l_extendedprice": 29435.13d, "l_discount": 0.05d, "l_tax": 0.07d, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-07-21", "l_commitdate": "1994-08-25", "l_receiptdate": "1994-08-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ously across the quic" }
-{ "l_orderkey": 5447, "l_partkey": 99, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 31, "l_extendedprice": 30971.79d, "l_discount": 0.09d, "l_tax": 0.03d, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-14", "l_commitdate": "1996-05-07", "l_receiptdate": "1996-07-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": " foxes sleep. blithely unusual accounts det" }
-{ "l_orderkey": 5476, "l_partkey": 48, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 13, "l_extendedprice": 12324.52d, "l_discount": 0.01d, "l_tax": 0.04d, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-27", "l_commitdate": "1997-12-08", "l_receiptdate": "1997-12-29", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "iously special ac" }
-{ "l_orderkey": 5506, "l_partkey": 140, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 2, "l_extendedprice": 2080.28d, "l_discount": 0.0d, "l_tax": 0.03d, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-04", "l_commitdate": "1994-01-13", "l_receiptdate": "1994-02-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "onic theodolites are fluffil" }
-{ "l_orderkey": 5536, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 14, "l_extendedprice": 13861.26d, "l_discount": 0.08d, "l_tax": 0.06d, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-18", "l_commitdate": "1998-05-08", "l_receiptdate": "1998-06-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "instructions sleep " }
-{ "l_orderkey": 5572, "l_partkey": 22, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 24, "l_extendedprice": 22128.48d, "l_discount": 0.08d, "l_tax": 0.08d, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-10-30", "l_commitdate": "1994-10-02", "l_receiptdate": "1994-11-27", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "ests cajole. evenly ironic exc" }
-{ "l_orderkey": 5664, "l_partkey": 122, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 25, "l_extendedprice": 25553.0d, "l_discount": 0.0d, "l_tax": 0.06d, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-10-29", "l_commitdate": "1998-09-23", "l_receiptdate": "1998-11-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "eposits: furiously ironic grouch" }
-{ "l_orderkey": 5670, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 27, "l_extendedprice": 26732.43d, "l_discount": 0.1d, "l_tax": 0.06d, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-05-09", "l_commitdate": "1993-05-30", "l_receiptdate": "1993-06-06", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": " ideas promise bli" }
-{ "l_orderkey": 5728, "l_partkey": 44, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 47, "l_extendedprice": 44369.88d, "l_discount": 0.1d, "l_tax": 0.05d, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-13", "l_commitdate": "1995-01-25", "l_receiptdate": "1994-12-25", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "nd the bravely final deposits. final ideas" }
-{ "l_orderkey": 5735, "l_partkey": 60, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 41, "l_extendedprice": 39362.46d, "l_discount": 0.01d, "l_tax": 0.01d, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-23", "l_commitdate": "1995-02-10", "l_receiptdate": "1995-01-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "lthily ruthless i" }
-{ "l_orderkey": 5826, "l_partkey": 144, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 4, "l_extendedprice": 4176.56d, "l_discount": 0.03d, "l_tax": 0.06d, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-31", "l_commitdate": "1998-09-10", "l_receiptdate": "1998-08-27", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": " packages across the fluffily spec" }
-{ "l_orderkey": 5829, "l_partkey": 40, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 4, "l_extendedprice": 3760.16d, "l_discount": 0.01d, "l_tax": 0.02d, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-01", "l_commitdate": "1997-02-17", "l_receiptdate": "1997-03-22", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ithely; accounts cajole ideas. regular foxe" }
-{ "l_orderkey": 5856, "l_partkey": 4, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 1, "l_extendedprice": 904.0d, "l_discount": 0.03d, "l_tax": 0.02d, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-29", "l_commitdate": "1995-01-07", "l_receiptdate": "1995-01-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "tly. special deposits wake blithely even" }
-{ "l_orderkey": 5926, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 8, "l_extendedprice": 7920.72d, "l_discount": 0.02d, "l_tax": 0.0d, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-07-17", "l_commitdate": "1994-07-20", "l_receiptdate": "1994-08-11", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "gle furiously express foxes. bo" }
-{ "l_orderkey": 5927, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 44, "l_extendedprice": 43563.96d, "l_discount": 0.04d, "l_tax": 0.05d, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-29", "l_commitdate": "1997-11-21", "l_receiptdate": "1997-12-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "rding to the special, final decoy" }
-{ "l_orderkey": 5955, "l_partkey": 140, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 14, "l_extendedprice": 14561.96d, "l_discount": 0.08d, "l_tax": 0.08d, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-06-22", "l_commitdate": "1995-05-23", "l_receiptdate": "1995-06-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": " unusual, bold theodolit" }
-{ "l_orderkey": 5959, "l_partkey": 135, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 49, "l_extendedprice": 50721.37d, "l_discount": 0.07d, "l_tax": 0.03d, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-07-16", "l_commitdate": "1992-08-09", "l_receiptdate": "1992-08-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "usual packages haggle slyly pi" }
-{ "l_orderkey": 5988, "l_partkey": 172, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 41, "l_extendedprice": 43958.97d, "l_discount": 0.08d, "l_tax": 0.03d, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-01-20", "l_commitdate": "1994-02-06", "l_receiptdate": "1994-02-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "the pending, express reque" }


[09/12] asterixdb git commit: [ASTERIXDB-2171][STO] Remove Temporary Datasets

Posted by mh...@apache.org.
http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/scan-insert-rtree-secondary-index/scan-insert-rtree-secondary-index.5.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/scan-insert-rtree-secondary-index/scan-insert-rtree-secondary-index.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/scan-insert-rtree-secondary-index/scan-insert-rtree-secondary-index.5.query.sqlpp
deleted file mode 100644
index 33c063d..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/scan-insert-rtree-secondary-index/scan-insert-rtree-secondary-index.5.query.sqlpp
+++ /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.
- */
-/*
- * Description     : Port data from a temporary dataset into yet-another temporary dataset.
- *                   Both datasets have R-tree indexes.
- * Expected Result : Success
- * Date            : March 27 2015
- */
-
-use test;
-
-
-select element {'id':o.id}
-from  MyMiniData as o
-where test.`spatial-intersect`(o.point,test.`create-polygon`([0.0,1.0,0.0,4.0,12.0,4.0,12.0,1.0]))
-order by o.id
-;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/scan-insert-temp-to-persistent/scan-insert-temp-to-persistent.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/scan-insert-temp-to-persistent/scan-insert-temp-to-persistent.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/scan-insert-temp-to-persistent/scan-insert-temp-to-persistent.1.ddl.sqlpp
deleted file mode 100644
index f7e32ab..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/scan-insert-temp-to-persistent/scan-insert-temp-to-persistent.1.ddl.sqlpp
+++ /dev/null
@@ -1,54 +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.
- */
-/*
- * Description     : Port data from a temporary dataset into a persistent dataset.
- *                   Both datasets have R-tree indexes.
- * Expected Result : Success
- * Date            : March 27 2015
- */
-
-drop  dataverse test if exists;
-create  dataverse test;
-
-use test;
-
-
-create type test.MyRecord as
- closed {
-  id : bigint,
-  point : point,
-  kwds : string,
-  line1 : line,
-  line2 : line,
-  poly1 : polygon,
-  poly2 : polygon,
-  rec : rectangle,
-  circle : circle
-};
-
-create type test.MyMiniRecord as
- closed {
-  id : bigint,
-  point : point
-};
-
-create temporary dataset MyData(MyRecord) primary key id;
-
-create  dataset MyMiniData(MyMiniRecord) primary key id;
-

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/scan-insert-temp-to-persistent/scan-insert-temp-to-persistent.2.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/scan-insert-temp-to-persistent/scan-insert-temp-to-persistent.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/scan-insert-temp-to-persistent/scan-insert-temp-to-persistent.2.update.sqlpp
deleted file mode 100644
index c2214c9..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/scan-insert-temp-to-persistent/scan-insert-temp-to-persistent.2.update.sqlpp
+++ /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.
- */
-/*
- * Description     : Port data from a temporary dataset into yet-another temporary dataset.
- *                   Both datasets have R-tree indexes.
- * Expected Result : Success
- * Date            : March 27 2015
- */
-
-use test;
-
-
-load  dataset MyData using localfs ((`path`=`asterix_nc1://data/spatial/spatialData.json`),(`format`=`adm`)) pre-sorted;
-
-load  dataset MyMiniData using localfs ((`path`=`asterix_nc1://data/spatial/spatialData0.json`),(`format`=`adm`)) pre-sorted;
-

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/scan-insert-temp-to-persistent/scan-insert-temp-to-persistent.3.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/scan-insert-temp-to-persistent/scan-insert-temp-to-persistent.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/scan-insert-temp-to-persistent/scan-insert-temp-to-persistent.3.ddl.sqlpp
deleted file mode 100644
index 98bfaf8..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/scan-insert-temp-to-persistent/scan-insert-temp-to-persistent.3.ddl.sqlpp
+++ /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.
- */
-/*
- * Description     : Port data from a temporary dataset into yet-another temporary dataset.
- *                   Both datasets have R-tree indexes.
- * Expected Result : Success
- * Date            : March 27 2015
- */
-
-use test;
-
-
-create  index rtree_index_point_0  on MyData (point) type rtree;
-
-create  index rtree_index_point  on MyMiniData (point) type rtree;
-

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/scan-insert-temp-to-persistent/scan-insert-temp-to-persistent.4.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/scan-insert-temp-to-persistent/scan-insert-temp-to-persistent.4.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/scan-insert-temp-to-persistent/scan-insert-temp-to-persistent.4.update.sqlpp
deleted file mode 100644
index 7ab14fe..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/scan-insert-temp-to-persistent/scan-insert-temp-to-persistent.4.update.sqlpp
+++ /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.
- */
-/*
- * Description     : Port data from a temporary dataset into yet-another temporary dataset.
- *                   Both datasets have R-tree indexes.
- * Expected Result : Success
- * Date            : March 27 2015
- */
-
-use test;
-
-
-insert into MyMiniData
-select element {'id':m.id,'point':m.point}
-from  MyData as m
-;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/scan-insert-temp-to-persistent/scan-insert-temp-to-persistent.5.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/scan-insert-temp-to-persistent/scan-insert-temp-to-persistent.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/scan-insert-temp-to-persistent/scan-insert-temp-to-persistent.5.query.sqlpp
deleted file mode 100644
index 33c063d..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/scan-insert-temp-to-persistent/scan-insert-temp-to-persistent.5.query.sqlpp
+++ /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.
- */
-/*
- * Description     : Port data from a temporary dataset into yet-another temporary dataset.
- *                   Both datasets have R-tree indexes.
- * Expected Result : Success
- * Date            : March 27 2015
- */
-
-use test;
-
-
-select element {'id':o.id}
-from  MyMiniData as o
-where test.`spatial-intersect`(o.point,test.`create-polygon`([0.0,1.0,0.0,4.0,12.0,4.0,12.0,1.0]))
-order by o.id
-;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/results/rebalance/all_datasets/all_datasets.11.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/rebalance/all_datasets/all_datasets.11.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/rebalance/all_datasets/all_datasets.11.adm
index 8325e98..f57b230 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/rebalance/all_datasets/all_datasets.11.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/rebalance/all_datasets/all_datasets.11.adm
@@ -1 +1 @@
-{"temp":false,"keys":"l_orderkey,l_linenumber","type":{"type":"org.apache.asterix.om.types.ARecordType","name":"LineItemType","open":false,"fields":[{"l_orderkey":{"type":"AInt64"}},{"l_partkey":{"type":"AInt64"}},{"l_suppkey":{"type":"AInt64"}},{"l_linenumber":{"type":"AInt64"}},{"l_quantity":{"type":"ADouble"}},{"l_extendedprice":{"type":"ADouble"}},{"l_discount":{"type":"ADouble"}},{"l_tax":{"type":"ADouble"}},{"l_returnflag":{"type":"AString"}},{"l_linestatus":{"type":"AString"}},{"l_shipdate":{"type":"AString"}},{"l_commitdate":{"type":"AString"}},{"l_receiptdate":{"type":"AString"}},{"l_shipinstruct":{"type":"AString"}},{"l_shipmode":{"type":"AString"}},{"l_comment":{"type":"AString"}}]},"splits":[{"ip":"127.0.0.1","path":"storage/partition_0/tpch1/2/LineItem_idx_LineItem"},{"ip":"127.0.0.1","path":"storage/partition_1/tpch1/2/LineItem_idx_LineItem"},{"ip":"127.0.0.1","path":"storage/partition_2/tpch1/2/LineItem_idx_LineItem"},{"ip":"127.0.0.1","path":"storage/partition_3/tpch
 1/2/LineItem_idx_LineItem"}]}
\ No newline at end of file
+{"keys":"l_orderkey,l_linenumber","type":{"type":"org.apache.asterix.om.types.ARecordType","name":"LineItemType","open":false,"fields":[{"l_orderkey":{"type":"AInt64"}},{"l_partkey":{"type":"AInt64"}},{"l_suppkey":{"type":"AInt64"}},{"l_linenumber":{"type":"AInt64"}},{"l_quantity":{"type":"ADouble"}},{"l_extendedprice":{"type":"ADouble"}},{"l_discount":{"type":"ADouble"}},{"l_tax":{"type":"ADouble"}},{"l_returnflag":{"type":"AString"}},{"l_linestatus":{"type":"AString"}},{"l_shipdate":{"type":"AString"}},{"l_commitdate":{"type":"AString"}},{"l_receiptdate":{"type":"AString"}},{"l_shipinstruct":{"type":"AString"}},{"l_shipmode":{"type":"AString"}},{"l_comment":{"type":"AString"}}]},"splits":[{"ip":"127.0.0.1","path":"storage/partition_0/tpch1/2/LineItem_idx_LineItem"},{"ip":"127.0.0.1","path":"storage/partition_1/tpch1/2/LineItem_idx_LineItem"},{"ip":"127.0.0.1","path":"storage/partition_2/tpch1/2/LineItem_idx_LineItem"},{"ip":"127.0.0.1","path":"storage/partition_3/tpch1/2/LineItem_
 idx_LineItem"}]}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/results/rebalance/all_datasets/all_datasets.12.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/rebalance/all_datasets/all_datasets.12.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/rebalance/all_datasets/all_datasets.12.adm
index b93b62c..39cad23 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/rebalance/all_datasets/all_datasets.12.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/rebalance/all_datasets/all_datasets.12.adm
@@ -1 +1 @@
-{"temp":false,"keys":"o_orderkey","type":{"type":"org.apache.asterix.om.types.ARecordType","name":"OrderType","open":false,"fields":[{"o_orderkey":{"type":"AInt64"}},{"o_custkey":{"type":"AInt64"}},{"o_orderstatus":{"type":"AString"}},{"o_totalprice":{"type":"ADouble"}},{"o_orderdate":{"type":"AString"}},{"o_orderpriority":{"type":"AString"}},{"o_clerk":{"type":"AString"}},{"o_shippriority":{"type":"AInt64"}},{"o_comment":{"type":"AString"}}]},"splits":[{"ip":"127.0.0.1","path":"storage/partition_0/tpch2/2/Orders_idx_Orders"},{"ip":"127.0.0.1","path":"storage/partition_1/tpch2/2/Orders_idx_Orders"},{"ip":"127.0.0.1","path":"storage/partition_2/tpch2/2/Orders_idx_Orders"},{"ip":"127.0.0.1","path":"storage/partition_3/tpch2/2/Orders_idx_Orders"}]}
\ No newline at end of file
+{"keys":"o_orderkey","type":{"type":"org.apache.asterix.om.types.ARecordType","name":"OrderType","open":false,"fields":[{"o_orderkey":{"type":"AInt64"}},{"o_custkey":{"type":"AInt64"}},{"o_orderstatus":{"type":"AString"}},{"o_totalprice":{"type":"ADouble"}},{"o_orderdate":{"type":"AString"}},{"o_orderpriority":{"type":"AString"}},{"o_clerk":{"type":"AString"}},{"o_shippriority":{"type":"AInt64"}},{"o_comment":{"type":"AString"}}]},"splits":[{"ip":"127.0.0.1","path":"storage/partition_0/tpch2/2/Orders_idx_Orders"},{"ip":"127.0.0.1","path":"storage/partition_1/tpch2/2/Orders_idx_Orders"},{"ip":"127.0.0.1","path":"storage/partition_2/tpch2/2/Orders_idx_Orders"},{"ip":"127.0.0.1","path":"storage/partition_3/tpch2/2/Orders_idx_Orders"}]}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/results/rebalance/all_datasets/all_datasets.5.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/rebalance/all_datasets/all_datasets.5.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/rebalance/all_datasets/all_datasets.5.adm
index abe4c45..fa5763e 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/rebalance/all_datasets/all_datasets.5.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/rebalance/all_datasets/all_datasets.5.adm
@@ -1 +1 @@
-{"temp":false,"keys":"l_orderkey,l_linenumber","type":{"type":"org.apache.asterix.om.types.ARecordType","name":"LineItemType","open":false,"fields":[{"l_orderkey":{"type":"AInt64"}},{"l_partkey":{"type":"AInt64"}},{"l_suppkey":{"type":"AInt64"}},{"l_linenumber":{"type":"AInt64"}},{"l_quantity":{"type":"ADouble"}},{"l_extendedprice":{"type":"ADouble"}},{"l_discount":{"type":"ADouble"}},{"l_tax":{"type":"ADouble"}},{"l_returnflag":{"type":"AString"}},{"l_linestatus":{"type":"AString"}},{"l_shipdate":{"type":"AString"}},{"l_commitdate":{"type":"AString"}},{"l_receiptdate":{"type":"AString"}},{"l_shipinstruct":{"type":"AString"}},{"l_shipmode":{"type":"AString"}},{"l_comment":{"type":"AString"}}]},"splits":[{"ip":"127.0.0.1","path":"storage/partition_0/tpch1/1/LineItem_idx_LineItem"},{"ip":"127.0.0.1","path":"storage/partition_1/tpch1/1/LineItem_idx_LineItem"}]}
\ No newline at end of file
+{"keys":"l_orderkey,l_linenumber","type":{"type":"org.apache.asterix.om.types.ARecordType","name":"LineItemType","open":false,"fields":[{"l_orderkey":{"type":"AInt64"}},{"l_partkey":{"type":"AInt64"}},{"l_suppkey":{"type":"AInt64"}},{"l_linenumber":{"type":"AInt64"}},{"l_quantity":{"type":"ADouble"}},{"l_extendedprice":{"type":"ADouble"}},{"l_discount":{"type":"ADouble"}},{"l_tax":{"type":"ADouble"}},{"l_returnflag":{"type":"AString"}},{"l_linestatus":{"type":"AString"}},{"l_shipdate":{"type":"AString"}},{"l_commitdate":{"type":"AString"}},{"l_receiptdate":{"type":"AString"}},{"l_shipinstruct":{"type":"AString"}},{"l_shipmode":{"type":"AString"}},{"l_comment":{"type":"AString"}}]},"splits":[{"ip":"127.0.0.1","path":"storage/partition_0/tpch1/1/LineItem_idx_LineItem"},{"ip":"127.0.0.1","path":"storage/partition_1/tpch1/1/LineItem_idx_LineItem"}]}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/results/rebalance/all_datasets/all_datasets.6.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/rebalance/all_datasets/all_datasets.6.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/rebalance/all_datasets/all_datasets.6.adm
index 9b37249..b3dea05 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/rebalance/all_datasets/all_datasets.6.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/rebalance/all_datasets/all_datasets.6.adm
@@ -1 +1 @@
-{"temp":false,"keys":"o_orderkey","type":{"type":"org.apache.asterix.om.types.ARecordType","name":"OrderType","open":false,"fields":[{"o_orderkey":{"type":"AInt64"}},{"o_custkey":{"type":"AInt64"}},{"o_orderstatus":{"type":"AString"}},{"o_totalprice":{"type":"ADouble"}},{"o_orderdate":{"type":"AString"}},{"o_orderpriority":{"type":"AString"}},{"o_clerk":{"type":"AString"}},{"o_shippriority":{"type":"AInt64"}},{"o_comment":{"type":"AString"}}]},"splits":[{"ip":"127.0.0.1","path":"storage/partition_0/tpch2/1/Orders_idx_Orders"},{"ip":"127.0.0.1","path":"storage/partition_1/tpch2/1/Orders_idx_Orders"}]}
\ No newline at end of file
+{"keys":"o_orderkey","type":{"type":"org.apache.asterix.om.types.ARecordType","name":"OrderType","open":false,"fields":[{"o_orderkey":{"type":"AInt64"}},{"o_custkey":{"type":"AInt64"}},{"o_orderstatus":{"type":"AString"}},{"o_totalprice":{"type":"ADouble"}},{"o_orderdate":{"type":"AString"}},{"o_orderpriority":{"type":"AString"}},{"o_clerk":{"type":"AString"}},{"o_shippriority":{"type":"AInt64"}},{"o_comment":{"type":"AString"}}]},"splits":[{"ip":"127.0.0.1","path":"storage/partition_0/tpch2/1/Orders_idx_Orders"},{"ip":"127.0.0.1","path":"storage/partition_1/tpch2/1/Orders_idx_Orders"}]}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/results/rebalance/single_dataset/single_dataset.5.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/rebalance/single_dataset/single_dataset.5.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/rebalance/single_dataset/single_dataset.5.adm
index 4f0990e..8ff99de 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/rebalance/single_dataset/single_dataset.5.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/rebalance/single_dataset/single_dataset.5.adm
@@ -1 +1 @@
-{"temp":false,"keys":"l_orderkey,l_linenumber","type":{"type":"org.apache.asterix.om.types.ARecordType","name":"LineItemType","open":false,"fields":[{"l_orderkey":{"type":"AInt64"}},{"l_partkey":{"type":"AInt64"}},{"l_suppkey":{"type":"AInt64"}},{"l_linenumber":{"type":"AInt64"}},{"l_quantity":{"type":"ADouble"}},{"l_extendedprice":{"type":"ADouble"}},{"l_discount":{"type":"ADouble"}},{"l_tax":{"type":"ADouble"}},{"l_returnflag":{"type":"AString"}},{"l_linestatus":{"type":"AString"}},{"l_shipdate":{"type":"AString"}},{"l_commitdate":{"type":"AString"}},{"l_receiptdate":{"type":"AString"}},{"l_shipinstruct":{"type":"AString"}},{"l_shipmode":{"type":"AString"}},{"l_comment":{"type":"AString"}}]},"splits":[{"ip":"127.0.0.1","path":"storage/partition_0/tpch/1/LineItem_idx_LineItem"},{"ip":"127.0.0.1","path":"storage/partition_1/tpch/1/LineItem_idx_LineItem"}]}
+{"keys":"l_orderkey,l_linenumber","type":{"type":"org.apache.asterix.om.types.ARecordType","name":"LineItemType","open":false,"fields":[{"l_orderkey":{"type":"AInt64"}},{"l_partkey":{"type":"AInt64"}},{"l_suppkey":{"type":"AInt64"}},{"l_linenumber":{"type":"AInt64"}},{"l_quantity":{"type":"ADouble"}},{"l_extendedprice":{"type":"ADouble"}},{"l_discount":{"type":"ADouble"}},{"l_tax":{"type":"ADouble"}},{"l_returnflag":{"type":"AString"}},{"l_linestatus":{"type":"AString"}},{"l_shipdate":{"type":"AString"}},{"l_commitdate":{"type":"AString"}},{"l_receiptdate":{"type":"AString"}},{"l_shipinstruct":{"type":"AString"}},{"l_shipmode":{"type":"AString"}},{"l_comment":{"type":"AString"}}]},"splits":[{"ip":"127.0.0.1","path":"storage/partition_0/tpch/1/LineItem_idx_LineItem"},{"ip":"127.0.0.1","path":"storage/partition_1/tpch/1/LineItem_idx_LineItem"}]}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/results/rebalance/single_dataset/single_dataset.9.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/rebalance/single_dataset/single_dataset.9.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/rebalance/single_dataset/single_dataset.9.adm
index 44c244c..6eece47 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/rebalance/single_dataset/single_dataset.9.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/rebalance/single_dataset/single_dataset.9.adm
@@ -1 +1 @@
-{"temp":false,"keys":"l_orderkey,l_linenumber","type":{"type":"org.apache.asterix.om.types.ARecordType","name":"LineItemType","open":false,"fields":[{"l_orderkey":{"type":"AInt64"}},{"l_partkey":{"type":"AInt64"}},{"l_suppkey":{"type":"AInt64"}},{"l_linenumber":{"type":"AInt64"}},{"l_quantity":{"type":"ADouble"}},{"l_extendedprice":{"type":"ADouble"}},{"l_discount":{"type":"ADouble"}},{"l_tax":{"type":"ADouble"}},{"l_returnflag":{"type":"AString"}},{"l_linestatus":{"type":"AString"}},{"l_shipdate":{"type":"AString"}},{"l_commitdate":{"type":"AString"}},{"l_receiptdate":{"type":"AString"}},{"l_shipinstruct":{"type":"AString"}},{"l_shipmode":{"type":"AString"}},{"l_comment":{"type":"AString"}}]},"splits":[{"ip":"127.0.0.1","path":"storage/partition_0/tpch/2/LineItem_idx_LineItem"},{"ip":"127.0.0.1","path":"storage/partition_1/tpch/2/LineItem_idx_LineItem"},{"ip":"127.0.0.1","path":"storage/partition_2/tpch/2/LineItem_idx_LineItem"},{"ip":"127.0.0.1","path":"storage/partition_3/tpch/2/
 LineItem_idx_LineItem"}]}
\ No newline at end of file
+{"keys":"l_orderkey,l_linenumber","type":{"type":"org.apache.asterix.om.types.ARecordType","name":"LineItemType","open":false,"fields":[{"l_orderkey":{"type":"AInt64"}},{"l_partkey":{"type":"AInt64"}},{"l_suppkey":{"type":"AInt64"}},{"l_linenumber":{"type":"AInt64"}},{"l_quantity":{"type":"ADouble"}},{"l_extendedprice":{"type":"ADouble"}},{"l_discount":{"type":"ADouble"}},{"l_tax":{"type":"ADouble"}},{"l_returnflag":{"type":"AString"}},{"l_linestatus":{"type":"AString"}},{"l_shipdate":{"type":"AString"}},{"l_commitdate":{"type":"AString"}},{"l_receiptdate":{"type":"AString"}},{"l_shipinstruct":{"type":"AString"}},{"l_shipmode":{"type":"AString"}},{"l_comment":{"type":"AString"}}]},"splits":[{"ip":"127.0.0.1","path":"storage/partition_0/tpch/2/LineItem_idx_LineItem"},{"ip":"127.0.0.1","path":"storage/partition_1/tpch/2/LineItem_idx_LineItem"},{"ip":"127.0.0.1","path":"storage/partition_2/tpch/2/LineItem_idx_LineItem"},{"ip":"127.0.0.1","path":"storage/partition_3/tpch/2/LineItem_idx_
 LineItem"}]}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/results/rebalance/single_dataset_with_index/single_dataset_with_index.10.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/rebalance/single_dataset_with_index/single_dataset_with_index.10.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/rebalance/single_dataset_with_index/single_dataset_with_index.10.adm
index 44c244c..6eece47 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/rebalance/single_dataset_with_index/single_dataset_with_index.10.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/rebalance/single_dataset_with_index/single_dataset_with_index.10.adm
@@ -1 +1 @@
-{"temp":false,"keys":"l_orderkey,l_linenumber","type":{"type":"org.apache.asterix.om.types.ARecordType","name":"LineItemType","open":false,"fields":[{"l_orderkey":{"type":"AInt64"}},{"l_partkey":{"type":"AInt64"}},{"l_suppkey":{"type":"AInt64"}},{"l_linenumber":{"type":"AInt64"}},{"l_quantity":{"type":"ADouble"}},{"l_extendedprice":{"type":"ADouble"}},{"l_discount":{"type":"ADouble"}},{"l_tax":{"type":"ADouble"}},{"l_returnflag":{"type":"AString"}},{"l_linestatus":{"type":"AString"}},{"l_shipdate":{"type":"AString"}},{"l_commitdate":{"type":"AString"}},{"l_receiptdate":{"type":"AString"}},{"l_shipinstruct":{"type":"AString"}},{"l_shipmode":{"type":"AString"}},{"l_comment":{"type":"AString"}}]},"splits":[{"ip":"127.0.0.1","path":"storage/partition_0/tpch/2/LineItem_idx_LineItem"},{"ip":"127.0.0.1","path":"storage/partition_1/tpch/2/LineItem_idx_LineItem"},{"ip":"127.0.0.1","path":"storage/partition_2/tpch/2/LineItem_idx_LineItem"},{"ip":"127.0.0.1","path":"storage/partition_3/tpch/2/
 LineItem_idx_LineItem"}]}
\ No newline at end of file
+{"keys":"l_orderkey,l_linenumber","type":{"type":"org.apache.asterix.om.types.ARecordType","name":"LineItemType","open":false,"fields":[{"l_orderkey":{"type":"AInt64"}},{"l_partkey":{"type":"AInt64"}},{"l_suppkey":{"type":"AInt64"}},{"l_linenumber":{"type":"AInt64"}},{"l_quantity":{"type":"ADouble"}},{"l_extendedprice":{"type":"ADouble"}},{"l_discount":{"type":"ADouble"}},{"l_tax":{"type":"ADouble"}},{"l_returnflag":{"type":"AString"}},{"l_linestatus":{"type":"AString"}},{"l_shipdate":{"type":"AString"}},{"l_commitdate":{"type":"AString"}},{"l_receiptdate":{"type":"AString"}},{"l_shipinstruct":{"type":"AString"}},{"l_shipmode":{"type":"AString"}},{"l_comment":{"type":"AString"}}]},"splits":[{"ip":"127.0.0.1","path":"storage/partition_0/tpch/2/LineItem_idx_LineItem"},{"ip":"127.0.0.1","path":"storage/partition_1/tpch/2/LineItem_idx_LineItem"},{"ip":"127.0.0.1","path":"storage/partition_2/tpch/2/LineItem_idx_LineItem"},{"ip":"127.0.0.1","path":"storage/partition_3/tpch/2/LineItem_idx_
 LineItem"}]}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/results/rebalance/single_dataset_with_index/single_dataset_with_index.5.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/rebalance/single_dataset_with_index/single_dataset_with_index.5.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/rebalance/single_dataset_with_index/single_dataset_with_index.5.adm
index 4f0990e..8ff99de 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/rebalance/single_dataset_with_index/single_dataset_with_index.5.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/rebalance/single_dataset_with_index/single_dataset_with_index.5.adm
@@ -1 +1 @@
-{"temp":false,"keys":"l_orderkey,l_linenumber","type":{"type":"org.apache.asterix.om.types.ARecordType","name":"LineItemType","open":false,"fields":[{"l_orderkey":{"type":"AInt64"}},{"l_partkey":{"type":"AInt64"}},{"l_suppkey":{"type":"AInt64"}},{"l_linenumber":{"type":"AInt64"}},{"l_quantity":{"type":"ADouble"}},{"l_extendedprice":{"type":"ADouble"}},{"l_discount":{"type":"ADouble"}},{"l_tax":{"type":"ADouble"}},{"l_returnflag":{"type":"AString"}},{"l_linestatus":{"type":"AString"}},{"l_shipdate":{"type":"AString"}},{"l_commitdate":{"type":"AString"}},{"l_receiptdate":{"type":"AString"}},{"l_shipinstruct":{"type":"AString"}},{"l_shipmode":{"type":"AString"}},{"l_comment":{"type":"AString"}}]},"splits":[{"ip":"127.0.0.1","path":"storage/partition_0/tpch/1/LineItem_idx_LineItem"},{"ip":"127.0.0.1","path":"storage/partition_1/tpch/1/LineItem_idx_LineItem"}]}
+{"keys":"l_orderkey,l_linenumber","type":{"type":"org.apache.asterix.om.types.ARecordType","name":"LineItemType","open":false,"fields":[{"l_orderkey":{"type":"AInt64"}},{"l_partkey":{"type":"AInt64"}},{"l_suppkey":{"type":"AInt64"}},{"l_linenumber":{"type":"AInt64"}},{"l_quantity":{"type":"ADouble"}},{"l_extendedprice":{"type":"ADouble"}},{"l_discount":{"type":"ADouble"}},{"l_tax":{"type":"ADouble"}},{"l_returnflag":{"type":"AString"}},{"l_linestatus":{"type":"AString"}},{"l_shipdate":{"type":"AString"}},{"l_commitdate":{"type":"AString"}},{"l_receiptdate":{"type":"AString"}},{"l_shipinstruct":{"type":"AString"}},{"l_shipmode":{"type":"AString"}},{"l_comment":{"type":"AString"}}]},"splits":[{"ip":"127.0.0.1","path":"storage/partition_0/tpch/1/LineItem_idx_LineItem"},{"ip":"127.0.0.1","path":"storage/partition_1/tpch/1/LineItem_idx_LineItem"}]}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/results/rebalance/single_dataverse/single_dataverse.11.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/rebalance/single_dataverse/single_dataverse.11.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/rebalance/single_dataverse/single_dataverse.11.adm
index 44c244c..6eece47 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/rebalance/single_dataverse/single_dataverse.11.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/rebalance/single_dataverse/single_dataverse.11.adm
@@ -1 +1 @@
-{"temp":false,"keys":"l_orderkey,l_linenumber","type":{"type":"org.apache.asterix.om.types.ARecordType","name":"LineItemType","open":false,"fields":[{"l_orderkey":{"type":"AInt64"}},{"l_partkey":{"type":"AInt64"}},{"l_suppkey":{"type":"AInt64"}},{"l_linenumber":{"type":"AInt64"}},{"l_quantity":{"type":"ADouble"}},{"l_extendedprice":{"type":"ADouble"}},{"l_discount":{"type":"ADouble"}},{"l_tax":{"type":"ADouble"}},{"l_returnflag":{"type":"AString"}},{"l_linestatus":{"type":"AString"}},{"l_shipdate":{"type":"AString"}},{"l_commitdate":{"type":"AString"}},{"l_receiptdate":{"type":"AString"}},{"l_shipinstruct":{"type":"AString"}},{"l_shipmode":{"type":"AString"}},{"l_comment":{"type":"AString"}}]},"splits":[{"ip":"127.0.0.1","path":"storage/partition_0/tpch/2/LineItem_idx_LineItem"},{"ip":"127.0.0.1","path":"storage/partition_1/tpch/2/LineItem_idx_LineItem"},{"ip":"127.0.0.1","path":"storage/partition_2/tpch/2/LineItem_idx_LineItem"},{"ip":"127.0.0.1","path":"storage/partition_3/tpch/2/
 LineItem_idx_LineItem"}]}
\ No newline at end of file
+{"keys":"l_orderkey,l_linenumber","type":{"type":"org.apache.asterix.om.types.ARecordType","name":"LineItemType","open":false,"fields":[{"l_orderkey":{"type":"AInt64"}},{"l_partkey":{"type":"AInt64"}},{"l_suppkey":{"type":"AInt64"}},{"l_linenumber":{"type":"AInt64"}},{"l_quantity":{"type":"ADouble"}},{"l_extendedprice":{"type":"ADouble"}},{"l_discount":{"type":"ADouble"}},{"l_tax":{"type":"ADouble"}},{"l_returnflag":{"type":"AString"}},{"l_linestatus":{"type":"AString"}},{"l_shipdate":{"type":"AString"}},{"l_commitdate":{"type":"AString"}},{"l_receiptdate":{"type":"AString"}},{"l_shipinstruct":{"type":"AString"}},{"l_shipmode":{"type":"AString"}},{"l_comment":{"type":"AString"}}]},"splits":[{"ip":"127.0.0.1","path":"storage/partition_0/tpch/2/LineItem_idx_LineItem"},{"ip":"127.0.0.1","path":"storage/partition_1/tpch/2/LineItem_idx_LineItem"},{"ip":"127.0.0.1","path":"storage/partition_2/tpch/2/LineItem_idx_LineItem"},{"ip":"127.0.0.1","path":"storage/partition_3/tpch/2/LineItem_idx_
 LineItem"}]}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/results/rebalance/single_dataverse/single_dataverse.12.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/rebalance/single_dataverse/single_dataverse.12.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/rebalance/single_dataverse/single_dataverse.12.adm
index 32a8aa9..5a549ef 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/rebalance/single_dataverse/single_dataverse.12.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/rebalance/single_dataverse/single_dataverse.12.adm
@@ -1 +1 @@
-{"temp":false,"keys":"o_orderkey","type":{"type":"org.apache.asterix.om.types.ARecordType","name":"OrderType","open":false,"fields":[{"o_orderkey":{"type":"AInt64"}},{"o_custkey":{"type":"AInt64"}},{"o_orderstatus":{"type":"AString"}},{"o_totalprice":{"type":"ADouble"}},{"o_orderdate":{"type":"AString"}},{"o_orderpriority":{"type":"AString"}},{"o_clerk":{"type":"AString"}},{"o_shippriority":{"type":"AInt64"}},{"o_comment":{"type":"AString"}}]},"splits":[{"ip":"127.0.0.1","path":"storage/partition_0/tpch/2/Orders_idx_Orders"},{"ip":"127.0.0.1","path":"storage/partition_1/tpch/2/Orders_idx_Orders"},{"ip":"127.0.0.1","path":"storage/partition_2/tpch/2/Orders_idx_Orders"},{"ip":"127.0.0.1","path":"storage/partition_3/tpch/2/Orders_idx_Orders"}]}
\ No newline at end of file
+{"keys":"o_orderkey","type":{"type":"org.apache.asterix.om.types.ARecordType","name":"OrderType","open":false,"fields":[{"o_orderkey":{"type":"AInt64"}},{"o_custkey":{"type":"AInt64"}},{"o_orderstatus":{"type":"AString"}},{"o_totalprice":{"type":"ADouble"}},{"o_orderdate":{"type":"AString"}},{"o_orderpriority":{"type":"AString"}},{"o_clerk":{"type":"AString"}},{"o_shippriority":{"type":"AInt64"}},{"o_comment":{"type":"AString"}}]},"splits":[{"ip":"127.0.0.1","path":"storage/partition_0/tpch/2/Orders_idx_Orders"},{"ip":"127.0.0.1","path":"storage/partition_1/tpch/2/Orders_idx_Orders"},{"ip":"127.0.0.1","path":"storage/partition_2/tpch/2/Orders_idx_Orders"},{"ip":"127.0.0.1","path":"storage/partition_3/tpch/2/Orders_idx_Orders"}]}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/results/rebalance/single_dataverse/single_dataverse.5.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/rebalance/single_dataverse/single_dataverse.5.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/rebalance/single_dataverse/single_dataverse.5.adm
index 4f0990e..8ff99de 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/rebalance/single_dataverse/single_dataverse.5.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/rebalance/single_dataverse/single_dataverse.5.adm
@@ -1 +1 @@
-{"temp":false,"keys":"l_orderkey,l_linenumber","type":{"type":"org.apache.asterix.om.types.ARecordType","name":"LineItemType","open":false,"fields":[{"l_orderkey":{"type":"AInt64"}},{"l_partkey":{"type":"AInt64"}},{"l_suppkey":{"type":"AInt64"}},{"l_linenumber":{"type":"AInt64"}},{"l_quantity":{"type":"ADouble"}},{"l_extendedprice":{"type":"ADouble"}},{"l_discount":{"type":"ADouble"}},{"l_tax":{"type":"ADouble"}},{"l_returnflag":{"type":"AString"}},{"l_linestatus":{"type":"AString"}},{"l_shipdate":{"type":"AString"}},{"l_commitdate":{"type":"AString"}},{"l_receiptdate":{"type":"AString"}},{"l_shipinstruct":{"type":"AString"}},{"l_shipmode":{"type":"AString"}},{"l_comment":{"type":"AString"}}]},"splits":[{"ip":"127.0.0.1","path":"storage/partition_0/tpch/1/LineItem_idx_LineItem"},{"ip":"127.0.0.1","path":"storage/partition_1/tpch/1/LineItem_idx_LineItem"}]}
+{"keys":"l_orderkey,l_linenumber","type":{"type":"org.apache.asterix.om.types.ARecordType","name":"LineItemType","open":false,"fields":[{"l_orderkey":{"type":"AInt64"}},{"l_partkey":{"type":"AInt64"}},{"l_suppkey":{"type":"AInt64"}},{"l_linenumber":{"type":"AInt64"}},{"l_quantity":{"type":"ADouble"}},{"l_extendedprice":{"type":"ADouble"}},{"l_discount":{"type":"ADouble"}},{"l_tax":{"type":"ADouble"}},{"l_returnflag":{"type":"AString"}},{"l_linestatus":{"type":"AString"}},{"l_shipdate":{"type":"AString"}},{"l_commitdate":{"type":"AString"}},{"l_receiptdate":{"type":"AString"}},{"l_shipinstruct":{"type":"AString"}},{"l_shipmode":{"type":"AString"}},{"l_comment":{"type":"AString"}}]},"splits":[{"ip":"127.0.0.1","path":"storage/partition_0/tpch/1/LineItem_idx_LineItem"},{"ip":"127.0.0.1","path":"storage/partition_1/tpch/1/LineItem_idx_LineItem"}]}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/results/rebalance/single_dataverse/single_dataverse.6.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/rebalance/single_dataverse/single_dataverse.6.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/rebalance/single_dataverse/single_dataverse.6.adm
index c5639c6..6889a70 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/rebalance/single_dataverse/single_dataverse.6.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/rebalance/single_dataverse/single_dataverse.6.adm
@@ -1 +1 @@
-{"temp":false,"keys":"o_orderkey","type":{"type":"org.apache.asterix.om.types.ARecordType","name":"OrderType","open":false,"fields":[{"o_orderkey":{"type":"AInt64"}},{"o_custkey":{"type":"AInt64"}},{"o_orderstatus":{"type":"AString"}},{"o_totalprice":{"type":"ADouble"}},{"o_orderdate":{"type":"AString"}},{"o_orderpriority":{"type":"AString"}},{"o_clerk":{"type":"AString"}},{"o_shippriority":{"type":"AInt64"}},{"o_comment":{"type":"AString"}}]},"splits":[{"ip":"127.0.0.1","path":"storage/partition_0/tpch/1/Orders_idx_Orders"},{"ip":"127.0.0.1","path":"storage/partition_1/tpch/1/Orders_idx_Orders"}]}
\ No newline at end of file
+{"keys":"o_orderkey","type":{"type":"org.apache.asterix.om.types.ARecordType","name":"OrderType","open":false,"fields":[{"o_orderkey":{"type":"AInt64"}},{"o_custkey":{"type":"AInt64"}},{"o_orderstatus":{"type":"AString"}},{"o_totalprice":{"type":"ADouble"}},{"o_orderdate":{"type":"AString"}},{"o_orderpriority":{"type":"AString"}},{"o_clerk":{"type":"AString"}},{"o_shippriority":{"type":"AInt64"}},{"o_comment":{"type":"AString"}}]},"splits":[{"ip":"127.0.0.1","path":"storage/partition_0/tpch/1/Orders_idx_Orders"},{"ip":"127.0.0.1","path":"storage/partition_1/tpch/1/Orders_idx_Orders"}]}
\ No newline at end of file


[02/12] asterixdb git commit: [ASTERIXDB-2171][STO] Remove Temporary Datasets

Posted by mh...@apache.org.
http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-lang-aql/src/main/javacc/AQL.jj
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-lang-aql/src/main/javacc/AQL.jj b/asterixdb/asterix-lang-aql/src/main/javacc/AQL.jj
index 0cc6789..59ed8ae 100644
--- a/asterixdb/asterix-lang-aql/src/main/javacc/AQL.jj
+++ b/asterixdb/asterix-lang-aql/src/main/javacc/AQL.jj
@@ -448,7 +448,6 @@ DatasetDecl DatasetSpecification() throws ParseException:
   DatasetDecl dsetDecl = null;
   boolean autogenerated = false;
   String compactionPolicy = null;
-  boolean temp = false;
   Pair<Integer, List<String>> filterField = null;
   Pair<Identifier,Identifier> metaTypeComponents = new Pair<Identifier, Identifier>(null, null);
 }
@@ -480,10 +479,7 @@ DatasetDecl DatasetSpecification() throws ParseException:
                                    ifNotExists);
       }
 
-    | (<INTERNAL> | <TEMPORARY> {
-            temp = token.image.toLowerCase().equals("temporary");
-        }
-      )?
+    | ( <INTERNAL> )?
     <DATASET> nameComponents = QualifiedName()
     <LEFTPAREN> typeComponents = TypeName() <RIGHTPAREN>
     (
@@ -511,8 +507,7 @@ DatasetDecl DatasetSpecification() throws ParseException:
         InternalDetailsDecl idd = new InternalDetailsDecl(primaryKeyFields.second,
                                                           primaryKeyFields.first,
                                                           autogenerated,
-                                                          filterField == null? null : filterField.second,
-                                                          temp);
+                                                          filterField == null? null : filterField.second);
         dsetDecl = new DatasetDecl(nameComponents.first,
                                    nameComponents.second,
                                    typeComponents.first,
@@ -2729,7 +2724,7 @@ TOKEN :
   | <START: "start">
   | <STOP: "stop">
   | <SOME : "some">
-  | <TEMPORARY : "temporary">
+  | <TEMPORARY : "temporary"> // intentionally not used but reserved for future usage
   | <THEN : "then">
   | <TO : "to">
   | <TYPE : "type">

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/ExternalDetailsDecl.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/ExternalDetailsDecl.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/ExternalDetailsDecl.java
index b15579c..db599c4 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/ExternalDetailsDecl.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/ExternalDetailsDecl.java
@@ -40,9 +40,4 @@ public class ExternalDetailsDecl implements IDatasetDetailsDecl {
         return properties;
     }
 
-    @Override
-    public boolean isTemp() {
-        return false;
-    }
-
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/IDatasetDetailsDecl.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/IDatasetDetailsDecl.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/IDatasetDetailsDecl.java
index da27f17..7f48c32 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/IDatasetDetailsDecl.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/IDatasetDetailsDecl.java
@@ -20,6 +20,4 @@ package org.apache.asterix.lang.common.statement;
 
 public interface IDatasetDetailsDecl {
 
-    public boolean isTemp();
-
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/InternalDetailsDecl.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/InternalDetailsDecl.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/InternalDetailsDecl.java
index 41d97a8..f312ddd 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/InternalDetailsDecl.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/InternalDetailsDecl.java
@@ -24,16 +24,14 @@ public class InternalDetailsDecl implements IDatasetDetailsDecl {
     private final List<List<String>> partitioningExprs;
     private final List<Integer> keySourceIndicators;
     private final boolean autogenerated;
-    private final boolean temp;
     private final List<String> filterField;
 
     public InternalDetailsDecl(List<List<String>> partitioningExpr, List<Integer> keySourceIndicators,
-            boolean autogenerated, List<String> filterField, boolean temp) {
+            boolean autogenerated, List<String> filterField) {
         this.partitioningExprs = partitioningExpr;
         this.keySourceIndicators = keySourceIndicators;
         this.autogenerated = autogenerated;
         this.filterField = filterField;
-        this.temp = temp;
     }
 
     public List<List<String>> getPartitioningExprs() {
@@ -48,11 +46,6 @@ public class InternalDetailsDecl implements IDatasetDetailsDecl {
         return autogenerated;
     }
 
-    @Override
-    public boolean isTemp() {
-        return temp;
-    }
-
     public List<String> getFilterField() {
         return filterField;
     }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/FormatPrintVisitor.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/FormatPrintVisitor.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/FormatPrintVisitor.java
index 5a3a530..21a4b76 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/FormatPrintVisitor.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/FormatPrintVisitor.java
@@ -468,8 +468,7 @@ public class FormatPrintVisitor implements ILangVisitor<Void, Integer> {
     @Override
     public Void visit(DatasetDecl dd, Integer step) throws CompilationException {
         if (dd.getDatasetType() == DatasetType.INTERNAL) {
-            String temp = dd.getDatasetDetailsDecl().isTemp() ? "temporary" : "";
-            out.print(skip(step) + "create " + temp + datasetSymbol + generateFullName(dd.getDataverse(), dd.getName())
+            out.print(skip(step) + "create " + datasetSymbol + generateFullName(dd.getDataverse(), dd.getName())
                     + generateIfNotExists(dd.getIfNotExists()) + "(" + dd.getQualifiedTypeName() + ")"
                     + " primary key ");
             printDelimitedKeys(((InternalDetailsDecl) dd.getDatasetDetailsDecl()).getPartitioningExprs(), ",");

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj b/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj
index a58ce0f..7ac8c57 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj
+++ b/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj
@@ -501,7 +501,6 @@ DatasetDecl DatasetSpecification() throws ParseException:
   DatasetDecl dsetDecl = null;
   boolean autogenerated = false;
   String compactionPolicy = null;
-  boolean temp = false;
   Pair<Integer, List<String>> filterField = null;
   Pair<Identifier,Identifier> metaTypeComponents = new Pair<Identifier, Identifier>(null, null);
 }
@@ -533,7 +532,7 @@ DatasetDecl DatasetSpecification() throws ParseException:
                                    ifNotExists);
       }
 
-    | (<INTERNAL> | <TEMPORARY> { temp = true; })?
+    | ( <INTERNAL> )?
     Dataset() nameComponents = QualifiedName()
     <LEFTPAREN> typeComponents = TypeName() <RIGHTPAREN>
     (
@@ -561,8 +560,7 @@ DatasetDecl DatasetSpecification() throws ParseException:
         InternalDetailsDecl idd = new InternalDetailsDecl(primaryKeyFields.second,
                                                           primaryKeyFields.first,
                                                           autogenerated,
-                                                          filterField == null? null : filterField.second,
-                                                          temp);
+                                                          filterField == null? null : filterField.second);
         dsetDecl = new DatasetDecl(nameComponents.first,
                                    nameComponents.second,
                                    typeComponents.first,
@@ -3265,7 +3263,7 @@ TOKEN [IGNORE_CASE]:
   | <SOME : "some">
   | <START : "start">
   | <STOP : "stop">
-  | <TEMPORARY : "temporary">
+  | <TEMPORARY : "temporary"> // intentionally not used but reserved for future usage
   | <THEN : "then">
   | <TYPE : "type">
   | <TO : "to">

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/GarbageCollector.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/GarbageCollector.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/GarbageCollector.java
deleted file mode 100644
index a97e22a..0000000
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/GarbageCollector.java
+++ /dev/null
@@ -1,67 +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.metadata;
-
-import java.util.concurrent.TimeUnit;
-import java.util.logging.Level;
-import java.util.logging.Logger;
-
-/**
- * Periodically recycle temporary datasets.
- *
- * @author yingyib
- */
-public class GarbageCollector implements Runnable {
-    private static final Logger LOGGER = Logger.getLogger(GarbageCollector.class.getName());
-
-    // TODO(mblow): make this configurable
-    private static final long CLEANUP_PERIOD = 1;
-    private static final TimeUnit CLEANUP_PERIOD_UNIT = TimeUnit.DAYS;
-
-    static {
-        // Starts the garbage collector thread which
-        // should always be running.
-        Thread gcThread = new Thread(new GarbageCollector(), "Metadata GC");
-        gcThread.setDaemon(true);
-        gcThread.start();
-    }
-
-    @Override
-    @SuppressWarnings({"squid:S2142", "squid:S2189"}) // rethrow/interrupt thread on InterruptedException, endless loop
-    public void run() {
-        LOGGER.info("Starting Metadata GC");
-        while (true) {
-            try {
-                synchronized (this) {
-                    CLEANUP_PERIOD_UNIT.timedWait(this, CLEANUP_PERIOD);
-                }
-                MetadataManager.INSTANCE.cleanupTempDatasets();
-            } catch (InterruptedException e) {
-                break;
-            } catch (Exception e) {
-                LOGGER.log(Level.WARNING, "Exception cleaning temp datasets", e);
-            }
-        }
-        LOGGER.info("Exiting Metadata GC");
-    }
-
-    public static void ensure() {
-        // no need to do anything, <clinit> does the work
-    }
-}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/IDatasetDetails.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/IDatasetDetails.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/IDatasetDetails.java
index b9ced54..47c6432 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/IDatasetDetails.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/IDatasetDetails.java
@@ -26,28 +26,7 @@ import org.apache.hyracks.api.exceptions.HyracksDataException;
 
 public interface IDatasetDetails extends Serializable {
 
-    public DatasetType getDatasetType();
+    DatasetType getDatasetType();
 
-    public void writeDatasetDetailsRecordType(DataOutput out) throws HyracksDataException;
-
-    /**
-     * @return if the dataset is a temporary dataset.
-     *         Here is a summary of temporary datasets:
-     *         1. Different from a persistent dataset, reads and writes over a temporary dataset do not require any lock.
-     *         Writes over a temporary dataset do not generate any write-ahead update and commit log but generate
-     *         flush log and job commit log.
-     *         2. A temporary dataset can only be an internal dataset, stored in partitioned LSM-Btrees.
-     *         3. All secondary indexes for persistent datasets are supported for temporary datasets.
-     *         4. A temporary dataset will be automatically garbage collected if it is not active in the past 30 days.
-     *         A temporary dataset could be used for the following scenarios:
-     *         1. A data scientist wants to run some one-time data analysis queries over a dataset that s/he pre-processed
-     *         and the dataset is only used by her/himself in an one-query-at-a-time manner.
-     *         2. Articulate AQL with external systems such as Pregelix/IMRU/Spark. A user can first run an AQL
-     *         query to populate a temporary dataset, then kick off an external runtime to read this dataset,
-     *         dump the results of the external runtime to yet-another-temporary dataset, and finally run yet-another AQL
-     *         over the second temporary dataset.
-     */
-    public boolean isTemp();
-
-    public long getLastAccessTime();
+    void writeDatasetDetailsRecordType(DataOutput out) throws HyracksDataException;
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataCache.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataCache.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataCache.java
index 8f88ca7..4f827f5 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataCache.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataCache.java
@@ -50,8 +50,6 @@ import org.apache.asterix.metadata.utils.IndexUtil;
  */
 public class MetadataCache {
 
-    // Default life time period of a temp dataset. It is 30 days.
-    private final static long TEMP_DATASET_INACTIVE_TIME_THRESHOLD = 3600 * 24 * 30 * 1000L;
     // Key is dataverse name.
     protected final Map<String, Dataverse> dataverses = new HashMap<>();
     // Key is dataverse name. Key of value map is dataset name.
@@ -574,28 +572,6 @@ public class MetadataCache {
     }
 
     /**
-     * Clean up temp datasets that are expired.
-     * The garbage collection will pause other dataset operations.
-     */
-    public void cleanupTempDatasets() {
-        synchronized (datasets) {
-            for (Map<String, Dataset> map : datasets.values()) {
-                Iterator<Dataset> datasetIterator = map.values().iterator();
-                while (datasetIterator.hasNext()) {
-                    Dataset dataset = datasetIterator.next();
-                    if (dataset.getDatasetDetails().isTemp()) {
-                        long currentTime = System.currentTimeMillis();
-                        long duration = currentTime - dataset.getDatasetDetails().getLastAccessTime();
-                        if (duration > TEMP_DATASET_INACTIVE_TIME_THRESHOLD) {
-                            datasetIterator.remove();
-                        }
-                    }
-                }
-            }
-        }
-    }
-
-    /**
      * Represents a logical operation against the metadata.
      */
     protected class MetadataLogicalOperation {

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataManager.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataManager.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataManager.java
index 1384d26..f8adc81 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataManager.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataManager.java
@@ -118,7 +118,7 @@ public class MetadataManager implements IMetadataManager {
 
     @Override
     public void init() throws HyracksDataException {
-        GarbageCollector.ensure();
+        // no op
     }
 
     @Override
@@ -215,18 +215,6 @@ public class MetadataManager implements IMetadataManager {
     public List<Dataset> getDataverseDatasets(MetadataTransactionContext ctx, String dataverseName)
             throws AlgebricksException {
         List<Dataset> dataverseDatasets = new ArrayList<>();
-        // add uncommitted temporary datasets
-        for (Dataset dataset : ctx.getDataverseDatasets(dataverseName)) {
-            if (dataset.getDatasetDetails().isTemp()) {
-                dataverseDatasets.add(dataset);
-            }
-        }
-        // add the committed temporary datasets with the cache
-        for (Dataset dataset : cache.getDataverseDatasets(dataverseName)) {
-            if (dataset.getDatasetDetails().isTemp()) {
-                dataverseDatasets.add(dataset);
-            }
-        }
         try {
             // Assuming that the transaction can read its own writes on the
             // metadata node.
@@ -242,14 +230,11 @@ public class MetadataManager implements IMetadataManager {
     @Override
     public void addDataset(MetadataTransactionContext ctx, Dataset dataset) throws AlgebricksException {
         // add dataset into metadataNode
-        if (!dataset.getDatasetDetails().isTemp()) {
-            try {
-                metadataNode.addDataset(ctx.getTxnId(), dataset);
-            } catch (RemoteException e) {
-                throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
-            }
+        try {
+            metadataNode.addDataset(ctx.getTxnId(), dataset);
+        } catch (RemoteException e) {
+            throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
         }
-
         // reflect the dataset into the cache
         ctx.addDataset(dataset);
     }
@@ -257,16 +242,11 @@ public class MetadataManager implements IMetadataManager {
     @Override
     public void dropDataset(MetadataTransactionContext ctx, String dataverseName, String datasetName)
             throws AlgebricksException {
-        Dataset dataset = findDataset(ctx, dataverseName, datasetName);
-        // If a dataset is not in the cache, then it could not be a temp dataset
-        if (dataset == null || !dataset.getDatasetDetails().isTemp()) {
-            try {
-                metadataNode.dropDataset(ctx.getTxnId(), dataverseName, datasetName);
-            } catch (RemoteException e) {
-                throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
-            }
+        try {
+            metadataNode.dropDataset(ctx.getTxnId(), dataverseName, datasetName);
+        } catch (RemoteException e) {
+            throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
         }
-
         // Drops the dataset from cache
         ctx.dropDataset(dataverseName, datasetName);
     }
@@ -315,16 +295,10 @@ public class MetadataManager implements IMetadataManager {
         if (dataset == null) {
             return datasetIndexes;
         }
-        if (dataset.getDatasetDetails().isTemp()) {
-            // for temp datsets
-            datasetIndexes = cache.getDatasetIndexes(dataverseName, datasetName);
-        } else {
-            try {
-                // for persistent datasets
-                datasetIndexes = metadataNode.getDatasetIndexes(ctx.getTxnId(), dataverseName, datasetName);
-            } catch (RemoteException e) {
-                throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
-            }
+        try {
+            datasetIndexes = metadataNode.getDatasetIndexes(ctx.getTxnId(), dataverseName, datasetName);
+        } catch (RemoteException e) {
+            throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
         }
         return datasetIndexes;
     }
@@ -423,15 +397,10 @@ public class MetadataManager implements IMetadataManager {
 
     @Override
     public void addIndex(MetadataTransactionContext ctx, Index index) throws AlgebricksException {
-        String dataverseName = index.getDataverseName();
-        String datasetName = index.getDatasetName();
-        Dataset dataset = findDataset(ctx, dataverseName, datasetName);
-        if (dataset == null || !dataset.getDatasetDetails().isTemp()) {
-            try {
-                metadataNode.addIndex(ctx.getTxnId(), index);
-            } catch (RemoteException e) {
-                throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
-            }
+        try {
+            metadataNode.addIndex(ctx.getTxnId(), index);
+        } catch (RemoteException e) {
+            throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
         }
         ctx.addIndex(index);
     }
@@ -450,17 +419,10 @@ public class MetadataManager implements IMetadataManager {
     @Override
     public void dropIndex(MetadataTransactionContext ctx, String dataverseName, String datasetName, String indexName)
             throws AlgebricksException {
-        Dataset dataset = findDataset(ctx, dataverseName, datasetName);
-        // If a dataset is not in the cache, then it could be an unloaded persistent
-        // dataset.
-        // If the dataset is a temp dataset, then we do not need to call any MedataNode
-        // operations.
-        if (dataset == null || !dataset.getDatasetDetails().isTemp()) {
-            try {
-                metadataNode.dropIndex(ctx.getTxnId(), dataverseName, datasetName, indexName);
-            } catch (RemoteException e) {
-                throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
-            }
+        try {
+            metadataNode.dropIndex(ctx.getTxnId(), dataverseName, datasetName, indexName);
+        } catch (RemoteException e) {
+            throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
         }
         ctx.dropIndex(dataverseName, datasetName, indexName);
     }
@@ -982,11 +944,6 @@ public class MetadataManager implements IMetadataManager {
         ctx.addDataset(dataset);
     }
 
-    @Override
-    public void cleanupTempDatasets() {
-        cache.cleanupTempDatasets();
-    }
-
     public Dataset findDataset(MetadataTransactionContext ctx, String dataverseName, String datasetName) {
         Dataset dataset = ctx.getDataset(dataverseName, datasetName);
         if (dataset == null) {

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataManager.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataManager.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataManager.java
index 43a927b..b2d0d3e 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataManager.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataManager.java
@@ -661,13 +661,6 @@ public interface IMetadataManager extends IMetadataBootstrap {
     void updateDataset(MetadataTransactionContext ctx, Dataset dataset) throws AlgebricksException;
 
     /**
-     * Clean up temporary datasets that have not been active for a long time.
-     *
-     * @throws AlgebricksException
-     */
-    void cleanupTempDatasets() throws AlgebricksException;
-
-    /**
      * Add an extension entity to its extension dataset under the ongoing metadata
      * transaction
      *

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataBootstrap.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataBootstrap.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataBootstrap.java
index bff721e..4c98904 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataBootstrap.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataBootstrap.java
@@ -206,7 +206,7 @@ public class MetadataBootstrap {
         for (int i = 0; i < indexes.length; i++) {
             IDatasetDetails id = new InternalDatasetDetails(FileStructure.BTREE, PartitioningStrategy.HASH,
                     indexes[i].getPartitioningExpr(), indexes[i].getPartitioningExpr(), null,
-                    indexes[i].getPartitioningExprType(), false, null, false);
+                    indexes[i].getPartitioningExprType(), false, null);
             MetadataManager.INSTANCE.addDataset(mdTxnCtx,
                     new Dataset(indexes[i].getDataverseName(), indexes[i].getIndexedDatasetName(),
                             indexes[i].getDataverseName(), indexes[i].getPayloadRecordType().getTypeName(),

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/BTreeResourceFactoryProvider.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/BTreeResourceFactoryProvider.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/BTreeResourceFactoryProvider.java
index bfc6a8e..301aafb 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/BTreeResourceFactoryProvider.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/BTreeResourceFactoryProvider.java
@@ -69,7 +69,6 @@ public class BTreeResourceFactoryProvider implements IResourceFactoryProvider {
         ITypeTraits[] typeTraits = getTypeTraits(mdProvider, dataset, index, recordType, metaType);
         IBinaryComparatorFactory[] cmpFactories = getCmpFactories(mdProvider, dataset, index, recordType, metaType);
         int[] bloomFilterFields = getBloomFilterFields(dataset, index);
-        boolean durable = !dataset.isTemp();
         double bloomFilterFalsePositiveRate = mdProvider.getStorageProperties().getBloomFilterFalsePositiveRate();
         ILSMOperationTrackerFactory opTrackerFactory = dataset.getIndexOperationTrackerFactory(index);
         ILSMIOOperationCallbackFactory ioOpCallbackFactory = dataset.getIoOperationCallbackFactory(index);
@@ -84,12 +83,12 @@ public class BTreeResourceFactoryProvider implements IResourceFactoryProvider {
                         ? new ExternalBTreeLocalResourceFactory(storageManager, typeTraits, cmpFactories,
                                 filterTypeTraits, filterCmpFactories, filterFields, opTrackerFactory,
                                 ioOpCallbackFactory, metadataPageManagerFactory, ioSchedulerProvider,
-                                mergePolicyFactory, mergePolicyProperties, durable, bloomFilterFields,
+                                mergePolicyFactory, mergePolicyProperties, true, bloomFilterFields,
                                 bloomFilterFalsePositiveRate, false, btreeFields)
                         : new ExternalBTreeWithBuddyLocalResourceFactory(storageManager, typeTraits, cmpFactories,
                                 filterTypeTraits, filterCmpFactories, filterFields, opTrackerFactory,
                                 ioOpCallbackFactory, metadataPageManagerFactory, ioSchedulerProvider,
-                                mergePolicyFactory, mergePolicyProperties, durable, bloomFilterFields,
+                                mergePolicyFactory, mergePolicyProperties, true, bloomFilterFields,
                                 bloomFilterFalsePositiveRate, false, btreeFields);
             case INTERNAL:
                 AsterixVirtualBufferCacheProvider vbcProvider =
@@ -97,7 +96,7 @@ public class BTreeResourceFactoryProvider implements IResourceFactoryProvider {
                 return new LSMBTreeLocalResourceFactory(storageManager, typeTraits, cmpFactories, filterTypeTraits,
                         filterCmpFactories, filterFields, opTrackerFactory, ioOpCallbackFactory,
                         metadataPageManagerFactory, vbcProvider, ioSchedulerProvider, mergePolicyFactory,
-                        mergePolicyProperties, durable, bloomFilterFields, bloomFilterFalsePositiveRate,
+                        mergePolicyProperties, true, bloomFilterFields, bloomFilterFalsePositiveRate,
                         index.isPrimaryIndex(), btreeFields);
             default:
                 throw new CompilationException(ErrorCode.COMPILATION_UNKNOWN_DATASET_TYPE,

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataProvider.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataProvider.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataProvider.java
index 71ed913..305cdfa 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataProvider.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataProvider.java
@@ -158,7 +158,6 @@ public class MetadataProvider implements IMetadataProvider<DataSourceId, String>
     private IResultSerializerFactoryProvider resultSerializerFactoryProvider;
     private TxnId txnId;
     private Map<String, Integer> externalDataLocks;
-    private boolean isTemporaryDatasetWriteJob = true;
     private boolean blockingOperatorDisabled = false;
 
     public MetadataProvider(ICcApplicationContext appCtx, Dataverse defaultDataverse) {
@@ -257,11 +256,6 @@ public class MetadataProvider implements IMetadataProvider<DataSourceId, String>
         return isWriteTransaction;
     }
 
-    public boolean isTemporaryDatasetWriteJob() {
-        // The transaction only writes temporary datasets.
-        return isTemporaryDatasetWriteJob;
-    }
-
     public IFunctionManager getFunctionManager() {
         return functionManager;
     }
@@ -569,8 +563,6 @@ public class MetadataProvider implements IMetadataProvider<DataSourceId, String>
             fieldPermutation[numKeys + 1] = idx;
         }
 
-        boolean temp = dataset.getDatasetDetails().isTemp();
-        isTemporaryDatasetWriteJob = isTemporaryDatasetWriteJob && temp;
         Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint =
                 getSplitProviderAndConstraints(dataset);
         long numElementsHint = getCardinalityPerPartitionHint(dataset);
@@ -820,8 +812,6 @@ public class MetadataProvider implements IMetadataProvider<DataSourceId, String>
             throw new AlgebricksException(
                     "Unknown dataset " + datasetName + " in dataverse " + dataSource.getId().getDataverseName());
         }
-        boolean temp = dataset.getDatasetDetails().isTemp();
-        isTemporaryDatasetWriteJob = isTemporaryDatasetWriteJob && temp;
         int numKeys = primaryKeys.size();
         int numFilterFields = DatasetUtil.getFilterField(dataset) == null ? 0 : 1;
         int numOfAdditionalFields = additionalNonFilterFields == null ? 0 : additionalNonFilterFields.size();
@@ -933,9 +923,6 @@ public class MetadataProvider implements IMetadataProvider<DataSourceId, String>
         String datasetName = dataSource.getId().getDatasourceName();
         Dataset dataset =
                 MetadataManagerUtil.findExistingDataset(mdTxnCtx, dataSource.getId().getDataverseName(), datasetName);
-        boolean temp = dataset.getDatasetDetails().isTemp();
-        isTemporaryDatasetWriteJob = isTemporaryDatasetWriteJob && temp;
-
         int numKeys = keys.size();
         int numFilterFields = DatasetUtil.getFilterField(dataset) == null ? 0 : 1;
         // Move key fields to front.
@@ -1040,9 +1027,6 @@ public class MetadataProvider implements IMetadataProvider<DataSourceId, String>
             JobSpecification spec, IndexOperation indexOp, boolean bulkload, List<LogicalVariable> prevSecondaryKeys,
             List<LogicalVariable> prevAdditionalFilteringKeys) throws AlgebricksException {
         Dataset dataset = MetadataManagerUtil.findExistingDataset(mdTxnCtx, dataverseName, datasetName);
-        boolean temp = dataset.getDatasetDetails().isTemp();
-        isTemporaryDatasetWriteJob = isTemporaryDatasetWriteJob && temp;
-
         int numKeys = primaryKeys.size() + secondaryKeys.size();
         int numFilterFields = DatasetUtil.getFilterField(dataset) == null ? 0 : 1;
 
@@ -1128,9 +1112,6 @@ public class MetadataProvider implements IMetadataProvider<DataSourceId, String>
             JobSpecification spec, IndexOperation indexOp, boolean bulkload, List<LogicalVariable> prevSecondaryKeys,
             List<LogicalVariable> prevAdditionalFilteringKeys) throws AlgebricksException {
         Dataset dataset = MetadataManagerUtil.findExistingDataset(mdTxnCtx, dataverseName, datasetName);
-        boolean temp = dataset.getDatasetDetails().isTemp();
-        isTemporaryDatasetWriteJob = isTemporaryDatasetWriteJob && temp;
-
         String itemTypeName = dataset.getItemTypeName();
         IAType itemType = MetadataManager.INSTANCE
                 .getDatatype(mdTxnCtx, dataset.getItemTypeDataverseName(), itemTypeName).getDatatype();
@@ -1245,9 +1226,6 @@ public class MetadataProvider implements IMetadataProvider<DataSourceId, String>
             throw new AlgebricksException("Cannot create composite inverted index on multiple fields.");
         }
         Dataset dataset = MetadataManagerUtil.findExistingDataset(mdTxnCtx, dataverseName, datasetName);
-        boolean temp = dataset.getDatasetDetails().isTemp();
-        isTemporaryDatasetWriteJob = isTemporaryDatasetWriteJob && temp;
-
         // For tokenization, sorting and loading.
         // One token (+ optional partitioning field) + primary keys: [token,
         // number of token, PK]

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Dataset.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Dataset.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Dataset.java
index 48a6e6a..e6c0de8 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Dataset.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Dataset.java
@@ -30,8 +30,8 @@ import java.util.stream.IntStream;
 import org.apache.asterix.active.IActiveEntityEventsListener;
 import org.apache.asterix.active.IActiveNotificationHandler;
 import org.apache.asterix.common.config.DatasetConfig.DatasetType;
-import org.apache.asterix.common.context.DatasetLSMComponentIdGeneratorFactory;
 import org.apache.asterix.common.context.CorrelatedPrefixMergePolicyFactory;
+import org.apache.asterix.common.context.DatasetLSMComponentIdGeneratorFactory;
 import org.apache.asterix.common.context.IStorageComponentProvider;
 import org.apache.asterix.common.dataflow.NoOpFrameOperationCallbackFactory;
 import org.apache.asterix.common.exceptions.CompilationException;
@@ -76,8 +76,6 @@ import org.apache.asterix.transaction.management.opcallbacks.PrimaryIndexOperati
 import org.apache.asterix.transaction.management.opcallbacks.SecondaryIndexModificationOperationCallbackFactory;
 import org.apache.asterix.transaction.management.opcallbacks.SecondaryIndexOperationTrackerFactory;
 import org.apache.asterix.transaction.management.opcallbacks.SecondaryIndexSearchOperationCallbackFactory;
-import org.apache.asterix.transaction.management.opcallbacks.TempDatasetPrimaryIndexModificationOperationCallbackFactory;
-import org.apache.asterix.transaction.management.opcallbacks.TempDatasetSecondaryIndexModificationOperationCallbackFactory;
 import org.apache.asterix.transaction.management.opcallbacks.UpsertOperationCallbackFactory;
 import org.apache.asterix.transaction.management.resource.DatasetLocalResourceFactory;
 import org.apache.asterix.transaction.management.runtime.CommitRuntimeFactory;
@@ -556,17 +554,15 @@ public class Dataset implements IMetadataEntity<Dataset>, IDataset {
      */
     public ISearchOperationCallbackFactory getSearchCallbackFactory(IStorageComponentProvider storageComponentProvider,
             Index index, TxnId txnId, IndexOperation op, int[] primaryKeyFields) throws AlgebricksException {
-        if (getDatasetDetails().isTemp()) {
-            return NoOpOperationCallbackFactory.INSTANCE;
-        } else if (index.isPrimaryIndex()) {
-            /**
+        if (index.isPrimaryIndex()) {
+            /*
              * Due to the read-committed isolation level,
              * we may acquire very short duration lock(i.e., instant lock) for readers.
              */
-            return (op == IndexOperation.UPSERT)
-                    ? new LockThenSearchOperationCallbackFactory(txnId, getDatasetId(), primaryKeyFields,
-                            storageComponentProvider.getTransactionSubsystemProvider(), ResourceType.LSM_BTREE)
-                    : new PrimaryIndexInstantSearchOperationCallbackFactory(txnId, getDatasetId(), primaryKeyFields,
+            return (op == IndexOperation.UPSERT) ?
+                    new LockThenSearchOperationCallbackFactory(txnId, getDatasetId(), primaryKeyFields,
+                            storageComponentProvider.getTransactionSubsystemProvider(), ResourceType.LSM_BTREE) :
+                    new PrimaryIndexInstantSearchOperationCallbackFactory(txnId, getDatasetId(), primaryKeyFields,
                             storageComponentProvider.getTransactionSubsystemProvider(), ResourceType.LSM_BTREE);
         } else if (index.getKeyFieldNames().isEmpty()) {
             // this is the case where the index is secondary primary index and locking is required
@@ -596,32 +592,22 @@ public class Dataset implements IMetadataEntity<Dataset>, IDataset {
     public IModificationOperationCallbackFactory getModificationCallbackFactory(
             IStorageComponentProvider componentProvider, Index index, TxnId txnId, IndexOperation op,
             int[] primaryKeyFields) throws AlgebricksException {
-        if (getDatasetDetails().isTemp()) {
-            return op == IndexOperation.DELETE || op == IndexOperation.INSERT || op == IndexOperation.UPSERT
-                    ? index.isPrimaryIndex()
-                            ? new TempDatasetPrimaryIndexModificationOperationCallbackFactory(txnId, datasetId,
-                                    primaryKeyFields, componentProvider.getTransactionSubsystemProvider(),
-                                    Operation.get(op), index.resourceType())
-                            : new TempDatasetSecondaryIndexModificationOperationCallbackFactory(txnId, getDatasetId(),
-                                    primaryKeyFields, componentProvider.getTransactionSubsystemProvider(),
-                                    Operation.get(op), index.resourceType())
-                    : NoOpOperationCallbackFactory.INSTANCE;
-        } else if (index.isPrimaryIndex()) {
-            return op == IndexOperation.UPSERT
-                    ? new UpsertOperationCallbackFactory(txnId, getDatasetId(), primaryKeyFields,
+        if (index.isPrimaryIndex()) {
+            return op == IndexOperation.UPSERT ?
+                    new UpsertOperationCallbackFactory(txnId, getDatasetId(), primaryKeyFields,
                             componentProvider.getTransactionSubsystemProvider(), Operation.get(op),
-                            index.resourceType())
-                    : op == IndexOperation.DELETE || op == IndexOperation.INSERT
-                            ? new PrimaryIndexModificationOperationCallbackFactory(txnId, getDatasetId(),
+                            index.resourceType()) :
+                    op == IndexOperation.DELETE || op == IndexOperation.INSERT ?
+                            new PrimaryIndexModificationOperationCallbackFactory(txnId, getDatasetId(),
                                     primaryKeyFields, componentProvider.getTransactionSubsystemProvider(),
-                                    Operation.get(op), index.resourceType())
-                            : NoOpOperationCallbackFactory.INSTANCE;
+                                    Operation.get(op), index.resourceType()) :
+                            NoOpOperationCallbackFactory.INSTANCE;
         } else {
-            return op == IndexOperation.DELETE || op == IndexOperation.INSERT || op == IndexOperation.UPSERT
-                    ? new SecondaryIndexModificationOperationCallbackFactory(txnId, getDatasetId(), primaryKeyFields,
+            return op == IndexOperation.DELETE || op == IndexOperation.INSERT || op == IndexOperation.UPSERT ?
+                    new SecondaryIndexModificationOperationCallbackFactory(txnId, getDatasetId(), primaryKeyFields,
                             componentProvider.getTransactionSubsystemProvider(), Operation.get(op),
-                            index.resourceType())
-                    : NoOpOperationCallbackFactory.INSTANCE;
+                            index.resourceType()) :
+                    NoOpOperationCallbackFactory.INSTANCE;
         }
     }
 
@@ -678,18 +664,13 @@ public class Dataset implements IMetadataEntity<Dataset>, IDataset {
             int[] primaryKeyFieldPermutation, boolean isSink) throws AlgebricksException {
         int[] datasetPartitions = getDatasetPartitions(metadataProvider);
         return new CommitRuntimeFactory(txnId, datasetId, primaryKeyFieldPermutation,
-                metadataProvider.isTemporaryDatasetWriteJob(), metadataProvider.isWriteTransaction(), datasetPartitions,
-                isSink);
+                metadataProvider.isWriteTransaction(), datasetPartitions, isSink);
     }
 
     public IFrameOperationCallbackFactory getFrameOpCallbackFactory() {
         return NoOpFrameOperationCallbackFactory.INSTANCE;
     }
 
-    public boolean isTemp() {
-        return getDatasetDetails().isTemp();
-    }
-
     public boolean isCorrelated() {
         return CorrelatedPrefixMergePolicyFactory.NAME.equals(compactionPolicyFactory);
     }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/ExternalDatasetDetails.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/ExternalDatasetDetails.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/ExternalDatasetDetails.java
index b9b4cd9..ea514e4 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/ExternalDatasetDetails.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/ExternalDatasetDetails.java
@@ -54,7 +54,6 @@ public class ExternalDatasetDetails implements IDatasetDetails {
     private static final Logger LOGGER = Logger.getLogger(ExternalDatasetDetails.class.getName());
     private final String adapter;
     private final Map<String, String> properties;
-    private final long addToCacheTime;
     private Date lastRefreshTime;
     private TransactionState state;
 
@@ -62,7 +61,6 @@ public class ExternalDatasetDetails implements IDatasetDetails {
             TransactionState state) {
         this.properties = properties;
         this.adapter = adapter;
-        this.addToCacheTime = System.currentTimeMillis();
         this.lastRefreshTime = lastRefreshTime;
         this.state = state;
     }
@@ -132,16 +130,6 @@ public class ExternalDatasetDetails implements IDatasetDetails {
         externalRecordBuilder.write(out, true);
     }
 
-    @Override
-    public boolean isTemp() {
-        return false;
-    }
-
-    @Override
-    public long getLastAccessTime() {
-        return addToCacheTime;
-    }
-
     public Date getTimestamp() {
         return lastRefreshTime;
     }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/InternalDatasetDetails.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/InternalDatasetDetails.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/InternalDatasetDetails.java
index 4b64f47..80c4f3d 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/InternalDatasetDetails.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/InternalDatasetDetails.java
@@ -61,8 +61,6 @@ public class InternalDatasetDetails implements IDatasetDetails {
     private final List<List<String>> primaryKeys;
     private final List<IAType> primaryKeyTypes;
     private final boolean autogenerated;
-    private final boolean temp;
-    private long lastAccessTime;
     private final List<String> filterField;
     private final List<Integer> keySourceIndicators;
 
@@ -71,7 +69,7 @@ public class InternalDatasetDetails implements IDatasetDetails {
 
     public InternalDatasetDetails(FileStructure fileStructure, PartitioningStrategy partitioningStrategy,
             List<List<String>> partitioningKey, List<List<String>> primaryKey, List<Integer> keyFieldIndicators,
-            List<IAType> primaryKeyType, boolean autogenerated, List<String> filterField, boolean temp) {
+            List<IAType> primaryKeyType, boolean autogenerated, List<String> filterField) {
         this.fileStructure = fileStructure;
         this.partitioningStrategy = partitioningStrategy;
         this.partitioningKeys = partitioningKey;
@@ -87,8 +85,6 @@ public class InternalDatasetDetails implements IDatasetDetails {
         this.primaryKeyTypes = primaryKeyType;
         this.autogenerated = autogenerated;
         this.filterField = filterField;
-        this.temp = temp;
-        this.lastAccessTime = System.currentTimeMillis();
     }
 
     public List<List<String>> getPartitioningKey() {
@@ -125,21 +121,10 @@ public class InternalDatasetDetails implements IDatasetDetails {
 
     @Override
     public DatasetType getDatasetType() {
-        lastAccessTime = System.currentTimeMillis();
         return DatasetType.INTERNAL;
     }
 
     @Override
-    public long getLastAccessTime() {
-        return lastAccessTime;
-    }
-
-    @Override
-    public boolean isTemp() {
-        return temp;
-    }
-
-    @Override
     public void writeDatasetDetailsRecordType(DataOutput out) throws HyracksDataException {
 
         IARecordBuilder internalRecordBuilder = new RecordBuilder();

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DatasetTupleTranslator.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DatasetTupleTranslator.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DatasetTupleTranslator.java
index c274833..d5e179b 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DatasetTupleTranslator.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DatasetTupleTranslator.java
@@ -207,13 +207,8 @@ public class DatasetTupleTranslator extends AbstractTupleTranslator<Dataset> {
                     }
                 }
 
-                // Temporary dataset only lives in the compiler therefore the temp field is
-                // false.
-                // DatasetTupleTranslator always read from the metadata node, so the temp flag
-                // should be always false.
                 datasetDetails = new InternalDatasetDetails(fileStructure, partitioningStrategy, partitioningKey,
-                        partitioningKey, keyFieldSourceIndicator, partitioningKeyType, autogenerated, filterField,
-                        false);
+                        partitioningKey, keyFieldSourceIndicator, partitioningKeyType, autogenerated, filterField);
                 break;
             }
 

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/DatasetUtil.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/DatasetUtil.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/DatasetUtil.java
index e4c4860..5973c06 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/DatasetUtil.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/DatasetUtil.java
@@ -88,7 +88,6 @@ import org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory
 import org.apache.hyracks.storage.am.common.dataflow.IndexCreateOperatorDescriptor;
 import org.apache.hyracks.storage.am.common.dataflow.IndexDataflowHelperFactory;
 import org.apache.hyracks.storage.am.common.dataflow.IndexDropOperatorDescriptor;
-import org.apache.hyracks.storage.am.common.impls.NoOpOperationCallbackFactory;
 import org.apache.hyracks.storage.am.common.ophelpers.IndexOperation;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicyFactory;
 import org.apache.hyracks.storage.am.lsm.common.dataflow.LSMTreeIndexCompactOperatorDescriptor;
@@ -297,7 +296,7 @@ public class DatasetUtil {
                 compactionInfo.first, compactionInfo.second);
         IndexBuilderFactory indexBuilderFactory =
                 new IndexBuilderFactory(metadataProvider.getStorageComponentProvider().getStorageManager(),
-                        splitsAndConstraint.first, resourceFactory, !dataset.isTemp());
+                        splitsAndConstraint.first, resourceFactory, true);
         IndexCreateOperatorDescriptor indexCreateOp = new IndexCreateOperatorDescriptor(spec, indexBuilderFactory);
         AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, indexCreateOp,
                 splitsAndConstraint.second);
@@ -352,9 +351,8 @@ public class DatasetUtil {
         // +Infinity
         int[] highKeyFields = null;
         ITransactionSubsystemProvider txnSubsystemProvider = TransactionSubsystemProvider.INSTANCE;
-        boolean temp = dataset.getDatasetDetails().isTemp();
-        ISearchOperationCallbackFactory searchCallbackFactory = temp ? NoOpOperationCallbackFactory.INSTANCE
-                : new PrimaryIndexInstantSearchOperationCallbackFactory(txnId, dataset.getDatasetId(),
+        ISearchOperationCallbackFactory searchCallbackFactory =
+                new PrimaryIndexInstantSearchOperationCallbackFactory(txnId, dataset.getDatasetId(),
                         dataset.getPrimaryBloomFilterFields(), txnSubsystemProvider,
                         IRecoveryManager.ResourceType.LSM_BTREE);
         IndexDataflowHelperFactory indexHelperFactory = new IndexDataflowHelperFactory(

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/ExternalIndexingOperations.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/ExternalIndexingOperations.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/ExternalIndexingOperations.java
index 5bb0aa9..d625dd0 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/ExternalIndexingOperations.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/ExternalIndexingOperations.java
@@ -202,7 +202,7 @@ public class ExternalIndexingOperations {
         IResourceFactory resourceFactory = dataset.getResourceFactory(metadataProvider, fileIndex, recordType, null,
                 mergePolicyFactory, mergePolicyProperties);
         IIndexBuilderFactory indexBuilderFactory = new IndexBuilderFactory(storageComponentProvider.getStorageManager(),
-                secondaryFileSplitProvider, resourceFactory, !dataset.isTemp());
+                secondaryFileSplitProvider, resourceFactory, true);
         IIndexDataflowHelperFactory dataflowHelperFactory =
                 new IndexDataflowHelperFactory(storageComponentProvider.getStorageManager(), secondaryFileSplitProvider);
         ExternalFilesIndexCreateOperatorDescriptor externalFilesOp = new ExternalFilesIndexCreateOperatorDescriptor(

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/InvertedIndexResourceFactoryProvider.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/InvertedIndexResourceFactoryProvider.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/InvertedIndexResourceFactoryProvider.java
index 362305e..43cc0ec 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/InvertedIndexResourceFactoryProvider.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/InvertedIndexResourceFactoryProvider.java
@@ -110,7 +110,6 @@ public class InvertedIndexResourceFactoryProvider implements IResourceFactoryPro
         AsterixVirtualBufferCacheProvider vbcProvider = new AsterixVirtualBufferCacheProvider(dataset.getDatasetId());
         ILSMIOOperationSchedulerProvider ioSchedulerProvider =
                 storageComponentProvider.getIoOperationSchedulerProvider();
-        boolean durable = !dataset.isTemp();
         double bloomFilterFalsePositiveRate = mdProvider.getStorageProperties().getBloomFilterFalsePositiveRate();
         ITypeTraits[] typeTraits = getInvListTypeTraits(mdProvider, dataset, recordType, metaType);
         IBinaryComparatorFactory[] cmpFactories =
@@ -122,7 +121,7 @@ public class InvertedIndexResourceFactoryProvider implements IResourceFactoryPro
         return new LSMInvertedIndexLocalResourceFactory(storageManager, typeTraits, cmpFactories, filterTypeTraits,
                 filterCmpFactories, secondaryFilterFields, opTrackerFactory, ioOpCallbackFactory,
                 metadataPageManagerFactory, vbcProvider, ioSchedulerProvider, mergePolicyFactory, mergePolicyProperties,
-                durable, tokenTypeTraits, tokenCmpFactories, tokenizerFactory, isPartitioned, invertedIndexFields,
+                true, tokenTypeTraits, tokenCmpFactories, tokenizerFactory, isPartitioned, invertedIndexFields,
                 secondaryFilterFieldsForNonBulkLoadOps, invertedIndexFieldsForNonBulkLoadOps,
                 bloomFilterFalsePositiveRate);
     }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/RTreeResourceFactoryProvider.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/RTreeResourceFactoryProvider.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/RTreeResourceFactoryProvider.java
index 425f8a1..5075adc 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/RTreeResourceFactoryProvider.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/RTreeResourceFactoryProvider.java
@@ -142,7 +142,6 @@ public class RTreeResourceFactoryProvider implements IResourceFactoryProvider {
                 storageComponentProvider.getMetadataPageManagerFactory();
         ILSMIOOperationSchedulerProvider ioSchedulerProvider =
                 storageComponentProvider.getIoOperationSchedulerProvider();
-        boolean durable = !dataset.isTemp();
         ILinearizeComparatorFactory linearizeCmpFactory =
                 MetadataProvider.proposeLinearizer(keyType, secondaryComparatorFactories.length);
         ITypeTraits[] typeTraits = getTypeTraits(mdProvider, dataset, index, recordType, metaType);
@@ -159,12 +158,12 @@ public class RTreeResourceFactoryProvider implements IResourceFactoryProvider {
             return new LSMRTreeWithAntiMatterLocalResourceFactory(storageManager, typeTraits, rtreeCmpFactories,
                     filterTypeTraits, filterCmpFactories, secondaryFilterFields, opTrackerFactory, ioOpCallbackFactory,
                     metadataPageManagerFactory, vbcProvider, ioSchedulerProvider, mergePolicyFactory,
-                    mergePolicyProperties, durable, valueProviderFactories, rTreePolicyType, linearizeCmpFactory,
+                    mergePolicyProperties, true, valueProviderFactories, rTreePolicyType, linearizeCmpFactory,
                     rtreeFields, isPointMBR, btreeCompFactories);
         } else {
             return new ExternalRTreeLocalResourceFactory(storageManager, typeTraits, rtreeCmpFactories,
                     filterTypeTraits, filterCmpFactories, secondaryFilterFields, opTrackerFactory, ioOpCallbackFactory,
-                    metadataPageManagerFactory, ioSchedulerProvider, mergePolicyFactory, mergePolicyProperties, durable,
+                    metadataPageManagerFactory, ioSchedulerProvider, mergePolicyFactory, mergePolicyProperties, true,
                     btreeCompFactories, valueProviderFactories, rTreePolicyType, linearizeCmpFactory, rtreeFields,
                     new int[] { numNestedSecondaryKeyFields }, isPointMBR,
                     mdProvider.getStorageProperties().getBloomFilterFalsePositiveRate());

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedTreeIndexOperationsHelper.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedTreeIndexOperationsHelper.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedTreeIndexOperationsHelper.java
index 994370c..2a4a952 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedTreeIndexOperationsHelper.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedTreeIndexOperationsHelper.java
@@ -56,7 +56,6 @@ import org.apache.hyracks.dataflow.common.data.marshalling.IntegerSerializerDese
 import org.apache.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
 import org.apache.hyracks.storage.am.common.api.ISearchOperationCallbackFactory;
 import org.apache.hyracks.storage.am.common.dataflow.IndexDataflowHelperFactory;
-import org.apache.hyracks.storage.am.common.impls.NoOpOperationCallbackFactory;
 import org.apache.hyracks.storage.am.lsm.btree.dataflow.LSMBTreeDiskComponentScanOperatorDescriptor;
 
 /**
@@ -276,9 +275,8 @@ public abstract class SecondaryCorrelatedTreeIndexOperationsHelper extends Secon
     protected IOperatorDescriptor createPrimaryIndexScanDiskComponentsOp(JobSpecification spec,
             MetadataProvider metadataProvider, RecordDescriptor outRecDesc, TxnId txnId) throws AlgebricksException {
         ITransactionSubsystemProvider txnSubsystemProvider = TransactionSubsystemProvider.INSTANCE;
-        boolean temp = dataset.getDatasetDetails().isTemp();
-        ISearchOperationCallbackFactory searchCallbackFactory = temp ? NoOpOperationCallbackFactory.INSTANCE
-                : new PrimaryIndexInstantSearchOperationCallbackFactory(txnId, dataset.getDatasetId(),
+        ISearchOperationCallbackFactory searchCallbackFactory =
+                new PrimaryIndexInstantSearchOperationCallbackFactory(txnId, dataset.getDatasetId(),
                         dataset.getPrimaryBloomFilterFields(), txnSubsystemProvider,
                         IRecoveryManager.ResourceType.LSM_BTREE);
         IndexDataflowHelperFactory indexHelperFactory = new IndexDataflowHelperFactory(

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryTreeIndexOperationsHelper.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryTreeIndexOperationsHelper.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryTreeIndexOperationsHelper.java
index b63ea16..185cf8c 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryTreeIndexOperationsHelper.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryTreeIndexOperationsHelper.java
@@ -59,7 +59,7 @@ public abstract class SecondaryTreeIndexOperationsHelper extends SecondaryIndexO
         IResourceFactory resourceFactory = dataset.getResourceFactory(metadataProvider, index, itemType, metaType,
                 mergePolicyFactory, mergePolicyProperties);
         IIndexBuilderFactory indexBuilderFactory = new IndexBuilderFactory(storageComponentProvider.getStorageManager(),
-                secondaryFileSplitProvider, resourceFactory, !dataset.isTemp());
+                secondaryFileSplitProvider, resourceFactory, true);
         IndexCreateOperatorDescriptor secondaryIndexCreateOp =
                 new IndexCreateOperatorDescriptor(spec, indexBuilderFactory);
         AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, secondaryIndexCreateOp,

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SplitsAndConstraintsUtil.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SplitsAndConstraintsUtil.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SplitsAndConstraintsUtil.java
index 98cfc57..e5f5c19 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SplitsAndConstraintsUtil.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SplitsAndConstraintsUtil.java
@@ -83,9 +83,7 @@ public class SplitsAndConstraintsUtil {
             for (int k = 0; k < numPartitions; k++) {
                 // format: 'storage dir name'/partition_#/dataverse/dataset_idx_index
                 File f = new File(StoragePathUtil.prepareStoragePartitionPath(storageDirName,
-                        nodePartitions[k].getPartitionId())
-                        + (dataset.isTemp() ? (File.separator + StoragePathUtil.TEMP_DATASETS_STORAGE_FOLDER) : "")
-                        + File.separator + relPathFile);
+                        nodePartitions[k].getPartitionId()) + File.separator + relPathFile);
                 splits.add(StoragePathUtil.getFileSplitForClusterPartition(nodePartitions[k], f.getPath()));
             }
         }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-metadata/src/test/java/org/apache/asterix/metadata/entitytupletranslators/DatasetTupleTranslatorTest.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-metadata/src/test/java/org/apache/asterix/metadata/entitytupletranslators/DatasetTupleTranslatorTest.java b/asterixdb/asterix-metadata/src/test/java/org/apache/asterix/metadata/entitytupletranslators/DatasetTupleTranslatorTest.java
index 0198d74..b87ef2a 100644
--- a/asterixdb/asterix-metadata/src/test/java/org/apache/asterix/metadata/entitytupletranslators/DatasetTupleTranslatorTest.java
+++ b/asterixdb/asterix-metadata/src/test/java/org/apache/asterix/metadata/entitytupletranslators/DatasetTupleTranslatorTest.java
@@ -49,7 +49,7 @@ public class DatasetTupleTranslatorTest {
                     Collections.singletonList(Collections.singletonList("row_id")),
                     Collections.singletonList(Collections.singletonList("row_id")),
                     indicator == null ? null : Collections.singletonList(indicator),
-                    Collections.singletonList(BuiltinType.AINT64), false, Collections.emptyList(), false);
+                    Collections.singletonList(BuiltinType.AINT64), false, Collections.emptyList());
 
             Dataset dataset =
                     new Dataset("test", "log", "foo", "LogType", "CB", "MetaType", "DEFAULT_NG_ALL_NODES", "prefix",

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-metadata/src/test/java/org/apache/asterix/metadata/entitytupletranslators/IndexTupleTranslatorTest.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-metadata/src/test/java/org/apache/asterix/metadata/entitytupletranslators/IndexTupleTranslatorTest.java b/asterixdb/asterix-metadata/src/test/java/org/apache/asterix/metadata/entitytupletranslators/IndexTupleTranslatorTest.java
index 32f65e1..ab4229c 100644
--- a/asterixdb/asterix-metadata/src/test/java/org/apache/asterix/metadata/entitytupletranslators/IndexTupleTranslatorTest.java
+++ b/asterixdb/asterix-metadata/src/test/java/org/apache/asterix/metadata/entitytupletranslators/IndexTupleTranslatorTest.java
@@ -60,7 +60,7 @@ public class IndexTupleTranslatorTest {
                     Collections.singletonList(Collections.singletonList("row_id")),
                     Collections.singletonList(Collections.singletonList("row_id")),
                     indicator == null ? null : Collections.singletonList(indicator),
-                    Collections.singletonList(BuiltinType.AINT64), false, Collections.emptyList(), false);
+                    Collections.singletonList(BuiltinType.AINT64), false, Collections.emptyList());
 
             Dataset dataset =
                     new Dataset("test", "d1", "foo", "LogType", "CB", "MetaType", "DEFAULT_NG_ALL_NODES", "prefix",

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/TempDatasetIndexModificationOperationCallback.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/TempDatasetIndexModificationOperationCallback.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/TempDatasetIndexModificationOperationCallback.java
deleted file mode 100644
index c609fc6..0000000
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/TempDatasetIndexModificationOperationCallback.java
+++ /dev/null
@@ -1,54 +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.transaction.management.opcallbacks;
-
-import org.apache.asterix.common.transactions.DatasetId;
-import org.apache.asterix.common.transactions.ILockManager;
-import org.apache.asterix.common.transactions.ITransactionContext;
-import org.apache.asterix.common.transactions.ITransactionSubsystem;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
-
-/**
- * This class is the operation callback for temporary datasets.
- * A temporary dataset does not require any lock and does not generate any write-ahead update and commit log
- * but generates flush log and job commit log.
- * The "before" and "found" method in this callback is empty so that no locking is requested for accessing a temporary
- * dataset and no write-ahead log is written for update operations.
- */
-public class TempDatasetIndexModificationOperationCallback extends AbstractIndexModificationOperationCallback {
-
-    public TempDatasetIndexModificationOperationCallback(DatasetId datasetId, int[] primaryKeyFields,
-            ITransactionContext txnCtx, ILockManager lockManager, ITransactionSubsystem txnSubsystem, long resourceId,
-            int resourcePartition, byte resourceType, Operation indexOp) {
-        super(datasetId, primaryKeyFields, txnCtx, lockManager, txnSubsystem, resourceId, resourcePartition,
-                resourceType, indexOp);
-    }
-
-    @Override
-    public void before(ITupleReference tuple) throws HyracksDataException {
-
-    }
-
-    @Override
-    public void found(ITupleReference before, ITupleReference after) throws HyracksDataException {
-
-    }
-}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/TempDatasetPrimaryIndexModificationOperationCallbackFactory.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/TempDatasetPrimaryIndexModificationOperationCallbackFactory.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/TempDatasetPrimaryIndexModificationOperationCallbackFactory.java
deleted file mode 100644
index 735d7ea..0000000
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/TempDatasetPrimaryIndexModificationOperationCallbackFactory.java
+++ /dev/null
@@ -1,80 +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.transaction.management.opcallbacks;
-
-import org.apache.asterix.common.api.IJobEventListenerFactory;
-import org.apache.asterix.common.context.ITransactionSubsystemProvider;
-import org.apache.asterix.common.dataflow.DatasetLocalResource;
-import org.apache.asterix.common.exceptions.ACIDException;
-import org.apache.asterix.common.transactions.AbstractOperationCallbackFactory;
-import org.apache.asterix.common.transactions.DatasetId;
-import org.apache.asterix.common.transactions.ITransactionContext;
-import org.apache.asterix.common.transactions.ITransactionSubsystem;
-import org.apache.asterix.common.transactions.TxnId;
-import org.apache.asterix.transaction.management.opcallbacks.AbstractIndexModificationOperationCallback.Operation;
-import org.apache.hyracks.api.context.IHyracksTaskContext;
-import org.apache.hyracks.api.dataflow.IOperatorNodePushable;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.api.job.IJobletEventListenerFactory;
-import org.apache.hyracks.storage.am.common.api.IModificationOperationCallbackFactory;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
-import org.apache.hyracks.storage.common.IModificationOperationCallback;
-import org.apache.hyracks.storage.common.IResourceLifecycleManager;
-import org.apache.hyracks.storage.common.LocalResource;
-
-public class TempDatasetPrimaryIndexModificationOperationCallbackFactory extends AbstractOperationCallbackFactory
-        implements IModificationOperationCallbackFactory {
-
-    private static final long serialVersionUID = 1L;
-    private final Operation indexOp;
-
-    public TempDatasetPrimaryIndexModificationOperationCallbackFactory(TxnId txnId, int datasetId,
-            int[] primaryKeyFields, ITransactionSubsystemProvider txnSubsystemProvider, Operation indexOp,
-            byte resourceType) {
-        super(txnId, datasetId, primaryKeyFields, txnSubsystemProvider, resourceType);
-        this.indexOp = indexOp;
-    }
-
-    @Override
-    public IModificationOperationCallback createModificationOperationCallback(LocalResource resource,
-            IHyracksTaskContext ctx, IOperatorNodePushable operatorNodePushable) throws HyracksDataException {
-        ITransactionSubsystem txnSubsystem = txnSubsystemProvider.getTransactionSubsystem(ctx);
-        IResourceLifecycleManager indexLifeCycleManager =
-                txnSubsystem.getAsterixAppRuntimeContextProvider().getDatasetLifecycleManager();
-        ILSMIndex index = (ILSMIndex) indexLifeCycleManager.get(resource.getPath());
-        if (index == null) {
-            throw new HyracksDataException("Index(id:" + resource.getId() + ") is not registered.");
-        }
-
-        try {
-            IJobletEventListenerFactory fact = ctx.getJobletContext().getJobletEventListenerFactory();
-            ITransactionContext txnCtx = txnSubsystem.getTransactionManager()
-                    .getTransactionContext(((IJobEventListenerFactory) fact).getTxnId(txnId));
-            DatasetLocalResource aResource = (DatasetLocalResource) resource.getResource();
-            IModificationOperationCallback modCallback = new TempDatasetIndexModificationOperationCallback(
-                    new DatasetId(datasetId), primaryKeyFields, txnCtx, txnSubsystem.getLockManager(), txnSubsystem,
-                    resource.getId(), aResource.getPartition(), resourceType, indexOp);
-            txnCtx.register(resource.getId(), index, modCallback, true);
-            return modCallback;
-        } catch (ACIDException e) {
-            throw new HyracksDataException(e);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/TempDatasetSecondaryIndexModificationOperationCallbackFactory.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/TempDatasetSecondaryIndexModificationOperationCallbackFactory.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/TempDatasetSecondaryIndexModificationOperationCallbackFactory.java
deleted file mode 100644
index b744606..0000000
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/TempDatasetSecondaryIndexModificationOperationCallbackFactory.java
+++ /dev/null
@@ -1,81 +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.transaction.management.opcallbacks;
-
-import org.apache.asterix.common.api.IJobEventListenerFactory;
-import org.apache.asterix.common.context.ITransactionSubsystemProvider;
-import org.apache.asterix.common.dataflow.DatasetLocalResource;
-import org.apache.asterix.common.exceptions.ACIDException;
-import org.apache.asterix.common.transactions.AbstractOperationCallbackFactory;
-import org.apache.asterix.common.transactions.DatasetId;
-import org.apache.asterix.common.transactions.ITransactionContext;
-import org.apache.asterix.common.transactions.ITransactionSubsystem;
-import org.apache.asterix.common.transactions.TxnId;
-import org.apache.asterix.transaction.management.opcallbacks.AbstractIndexModificationOperationCallback.Operation;
-import org.apache.hyracks.api.context.IHyracksTaskContext;
-import org.apache.hyracks.api.dataflow.IOperatorNodePushable;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.api.job.IJobletEventListenerFactory;
-import org.apache.hyracks.storage.am.common.api.IModificationOperationCallbackFactory;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
-import org.apache.hyracks.storage.common.IIndex;
-import org.apache.hyracks.storage.common.IModificationOperationCallback;
-import org.apache.hyracks.storage.common.IResourceLifecycleManager;
-import org.apache.hyracks.storage.common.LocalResource;
-
-public class TempDatasetSecondaryIndexModificationOperationCallbackFactory extends AbstractOperationCallbackFactory
-        implements IModificationOperationCallbackFactory {
-
-    private static final long serialVersionUID = 1L;
-    private final Operation indexOp;
-
-    public TempDatasetSecondaryIndexModificationOperationCallbackFactory(TxnId txnId, int datasetId,
-            int[] primaryKeyFields, ITransactionSubsystemProvider txnSubsystemProvider, Operation indexOp,
-            byte resourceType) {
-        super(txnId, datasetId, primaryKeyFields, txnSubsystemProvider, resourceType);
-        this.indexOp = indexOp;
-    }
-
-    @Override
-    public IModificationOperationCallback createModificationOperationCallback(LocalResource resource,
-            IHyracksTaskContext ctx, IOperatorNodePushable operatorNodePushable) throws HyracksDataException {
-        DatasetLocalResource aResource = (DatasetLocalResource) resource.getResource();
-        ITransactionSubsystem txnSubsystem = txnSubsystemProvider.getTransactionSubsystem(ctx);
-        IResourceLifecycleManager<IIndex> indexLifeCycleManager =
-                txnSubsystem.getAsterixAppRuntimeContextProvider().getDatasetLifecycleManager();
-        ILSMIndex index = (ILSMIndex) indexLifeCycleManager.get(resource.getPath());
-        if (index == null) {
-            throw new HyracksDataException("Index(id:" + resource.getId() + ") is not registered.");
-        }
-
-        try {
-            IJobletEventListenerFactory fact = ctx.getJobletContext().getJobletEventListenerFactory();
-            ITransactionContext txnCtx = txnSubsystem.getTransactionManager()
-                    .getTransactionContext(((IJobEventListenerFactory) fact).getTxnId(txnId));
-            IModificationOperationCallback modCallback = new TempDatasetIndexModificationOperationCallback(
-                    new DatasetId(datasetId), primaryKeyFields, txnCtx, txnSubsystem.getLockManager(), txnSubsystem,
-                    resource.getId(), aResource.getPartition(), resourceType, indexOp);
-            txnCtx.register(resource.getId(), index, modCallback, false);
-            return modCallback;
-        } catch (ACIDException e) {
-            throw new HyracksDataException(e);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/PersistentLocalResourceRepository.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/PersistentLocalResourceRepository.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/PersistentLocalResourceRepository.java
index 5666b48..db3647e 100644
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/PersistentLocalResourceRepository.java
+++ b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/PersistentLocalResourceRepository.java
@@ -380,25 +380,14 @@ public class PersistentLocalResourceRepository implements ILocalResourceReposito
 
     private void createReplicationJob(ReplicationOperation operation, FileReference fileRef)
             throws HyracksDataException {
-        /**
-         * Durable resources path format:
-         * /partition/dataverse/idx/fileName
-         * Temporary resources path format:
-         * /partition/TEMP_DATASETS_STORAGE_FOLDER/dataverse/idx/fileName
-         */
-        String[] fileNameTokens = fileRef.getAbsolutePath().split(File.separator);
-        String partitionDir = fileNameTokens[fileNameTokens.length - 4];
-        //exclude temporary datasets resources
-        if (!partitionDir.equals(StoragePathUtil.TEMP_DATASETS_STORAGE_FOLDER)) {
-            filesToBeReplicated.clear();
-            filesToBeReplicated.add(fileRef.getAbsolutePath());
-            ReplicationJob job = new ReplicationJob(ReplicationJobType.METADATA, operation,
-                    ReplicationExecutionType.SYNC, filesToBeReplicated);
-            try {
-                replicationManager.submitJob(job);
-            } catch (IOException e) {
-                throw new HyracksDataException(e);
-            }
+        filesToBeReplicated.clear();
+        filesToBeReplicated.add(fileRef.getAbsolutePath());
+        ReplicationJob job = new ReplicationJob(ReplicationJobType.METADATA, operation, ReplicationExecutionType.SYNC,
+                filesToBeReplicated);
+        try {
+            replicationManager.submitJob(job);
+        } catch (IOException e) {
+            throw new HyracksDataException(e);
         }
     }
 


[11/12] asterixdb git commit: [ASTERIXDB-2171][STO] Remove Temporary Datasets

Posted by mh...@apache.org.
http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/empty-load-with-index/empty-load-with-index.3.query.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/empty-load-with-index/empty-load-with-index.3.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/empty-load-with-index/empty-load-with-index.3.query.aql
deleted file mode 100644
index 04f1679..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/empty-load-with-index/empty-load-with-index.3.query.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.
- */
-/*
- * Description     : Load a temporary dataset with previously created empty indexes.
- * Expected Result : Success
- * Date            : March 27 2015
- */
-
-use dataverse test;
-
-for $c in dataset('LineItem')
-order by $c.l_orderkey, $c.l_linenumber
-limit 1
-return $c

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/insert-and-scan-dataset-with-correlated-index/insert-and-scan-dataset-with-correlated-index.1.ddl.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/insert-and-scan-dataset-with-correlated-index/insert-and-scan-dataset-with-correlated-index.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/insert-and-scan-dataset-with-correlated-index/insert-and-scan-dataset-with-correlated-index.1.ddl.aql
deleted file mode 100644
index 6834cde..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/insert-and-scan-dataset-with-correlated-index/insert-and-scan-dataset-with-correlated-index.1.ddl.aql
+++ /dev/null
@@ -1,39 +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.
- */
-/*
- * Description     : This test is intended to test inserting into a temporary dataset that has a correlated
- * secondary index and scan the data.
- * Expected Result : Success
- * Date            : June 8 2017
- */
-
-drop dataverse test if exists;
-create dataverse test;
-
-create type test.Emp as closed {
-id: int64,
-fname: string,
-lname: string,
-age: int64,
-dept: string
-}
-
-create temporary dataset test.employee(Emp) primary key id
-using compaction policy "correlated-prefix"
-(("max-mergable-component-size"="16384"),("max-tolerance-component-count"="3"));
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/insert-and-scan-dataset-with-correlated-index/insert-and-scan-dataset-with-correlated-index.2.update.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/insert-and-scan-dataset-with-correlated-index/insert-and-scan-dataset-with-correlated-index.2.update.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/insert-and-scan-dataset-with-correlated-index/insert-and-scan-dataset-with-correlated-index.2.update.aql
deleted file mode 100644
index c68a59e..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/insert-and-scan-dataset-with-correlated-index/insert-and-scan-dataset-with-correlated-index.2.update.aql
+++ /dev/null
@@ -1,42 +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.
- */
-/*
- * Description     : This test is intended to test inserting into a temporary dataset that has a correlated
- * secondary index and scan the data.
- * Expected Result : Success
- * Date            : June 8 2017
- */
-
-use dataverse test;
-
-load dataset test.employee
-using localfs
-(("path"="asterix_nc1://data/names.adm"),("format"="delimited-text"),("delimiter"="|"));
-
-
-insert into dataset test.employee (
-for $x in dataset test.employee
-return {
-    "id": $x.id + 10000,
-    "fname": $x.fname,
-    "lname": $x.lname,
-    "age": $x.age,
-    "dept": $x.dept
-}
-);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/insert-and-scan-dataset-with-correlated-index/insert-and-scan-dataset-with-correlated-index.3.ddl.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/insert-and-scan-dataset-with-correlated-index/insert-and-scan-dataset-with-correlated-index.3.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/insert-and-scan-dataset-with-correlated-index/insert-and-scan-dataset-with-correlated-index.3.ddl.aql
deleted file mode 100644
index 00b0850..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/insert-and-scan-dataset-with-correlated-index/insert-and-scan-dataset-with-correlated-index.3.ddl.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.
- */
-/*
- * Description     : This test is intended to test inserting into a temporary dataset that has a correlated
- * secondary index and scan the data.
- * Expected Result : Success
- * Date            : June 8 2017
- */
-
-use dataverse test;
-
-create index idx_employee_first_name on test.employee(fname);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/insert-and-scan-dataset-with-correlated-index/insert-and-scan-dataset-with-correlated-index.4.query.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/insert-and-scan-dataset-with-correlated-index/insert-and-scan-dataset-with-correlated-index.4.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/insert-and-scan-dataset-with-correlated-index/insert-and-scan-dataset-with-correlated-index.4.query.aql
deleted file mode 100644
index 195f11e..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/insert-and-scan-dataset-with-correlated-index/insert-and-scan-dataset-with-correlated-index.4.query.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.
- */
-/*
- * Description     : This test is intended to test inserting into a temporary dataset that has a correlated
- * secondary index and scan the data.
- * Expected Result : Success
- * Date            : June 8 2017
- */
-
-use dataverse test;
-
-for $l in dataset('test.employee')
-order by $l.id
-return $l

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/insert-and-scan-dataset-with-index/insert-and-scan-dataset-with-index.1.ddl.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/insert-and-scan-dataset-with-index/insert-and-scan-dataset-with-index.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/insert-and-scan-dataset-with-index/insert-and-scan-dataset-with-index.1.ddl.aql
deleted file mode 100644
index e6865e2..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/insert-and-scan-dataset-with-index/insert-and-scan-dataset-with-index.1.ddl.aql
+++ /dev/null
@@ -1,39 +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.
- */
-/*
- * Description     : This test is intended to test inserting into a temporary dataset that has a secondary index and scan
- * the data at the same time where we insert a materializing to prevent the possibility of deadlatch.
- * Expected Result : Success
- * Date            : March 27 2015
- */
-
-drop dataverse test if exists;
-create dataverse test;
-
-create type test.Emp as closed {
-id: int64,
-fname: string,
-lname: string,
-age: int64,
-dept: string
-}
-
-create temporary dataset test.employee(Emp) primary key id;
-
-create index idx_employee_first_name on test.employee(fname);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/insert-and-scan-dataset-with-index/insert-and-scan-dataset-with-index.2.update.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/insert-and-scan-dataset-with-index/insert-and-scan-dataset-with-index.2.update.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/insert-and-scan-dataset-with-index/insert-and-scan-dataset-with-index.2.update.aql
deleted file mode 100644
index 5c9e569..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/insert-and-scan-dataset-with-index/insert-and-scan-dataset-with-index.2.update.aql
+++ /dev/null
@@ -1,42 +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.
- */
-/*
- * Description     : This test is intended to test inserting into a temporary dataset that has a secondary index and scan
- * the data at the same time where we insert a materializing to prevent the possibility of deadlatch.
- * Expected Result : Success
- * Date            : March 28 2015
- */
-
-use dataverse test;
-
-load dataset test.employee
-using localfs
-(("path"="asterix_nc1://data/names.adm"),("format"="delimited-text"),("delimiter"="|"));
-
-
-insert into dataset test.employee (
-for $x in dataset test.employee
-return {
-    "id": $x.id + 10000,
-    "fname": $x.fname,
-    "lname": $x.lname,
-    "age": $x.age,
-    "dept": $x.dept
-}
-);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/insert-and-scan-dataset-with-index/insert-and-scan-dataset-with-index.3.query.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/insert-and-scan-dataset-with-index/insert-and-scan-dataset-with-index.3.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/insert-and-scan-dataset-with-index/insert-and-scan-dataset-with-index.3.query.aql
deleted file mode 100644
index 1b15209..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/insert-and-scan-dataset-with-index/insert-and-scan-dataset-with-index.3.query.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.
- */
-/*
- * Description     : This test is intended to test inserting into a temporary dataset that has a secondary index and scan
- * the data at the same time where we insert a materializing to prevent the possibility of dead latch.
- * Expected Result : Success
- * Date            : March 27 2015
- */
- 
- 
-use dataverse test;
-
-for $l in dataset('test.employee')
-order by $l.id
-return $l

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/insert-and-scan-dataset/insert-and-scan-dataset.1.ddl.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/insert-and-scan-dataset/insert-and-scan-dataset.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/insert-and-scan-dataset/insert-and-scan-dataset.1.ddl.aql
deleted file mode 100644
index 4b17b3e..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/insert-and-scan-dataset/insert-and-scan-dataset.1.ddl.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.
- */
-/*
- * Description     : This test is intended to test inserting into a temporary dataset and scan it at the same time
- * where we insert a materializing to prevent the possibility of deadlatch.
- * Expected Result : Success
- * Date            : July 11 2013
- */
-
-drop dataverse test if exists;
-create dataverse test;
-
-use dataverse test;
-
-create type myDataType as open {
-  id: int64
-}
-
-create temporary dataset myData(myDataType)
-  primary key id;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/insert-and-scan-dataset/insert-and-scan-dataset.2.update.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/insert-and-scan-dataset/insert-and-scan-dataset.2.update.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/insert-and-scan-dataset/insert-and-scan-dataset.2.update.aql
deleted file mode 100644
index c61edcb..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/insert-and-scan-dataset/insert-and-scan-dataset.2.update.aql
+++ /dev/null
@@ -1,39 +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  : insert-and-scan-dataset.aql
- * Description     : This test is intended to test inserting into a temporary dataset and scan it at the same time
- * where we insert a materializing to prevent the possibility of deadlatch.
- * Expected Result : Success
- * Date            : March 27 2015
- */
-
-use dataverse test;
-
-load dataset myData
-using localfs
-(("path"="asterix_nc1://data/odd-numbers.adm"),("format"="adm"))pre-sorted;
-
-
-insert into dataset myData (
-for $x in dataset myData
-return {
-    "id": $x.id + 1
-}
-);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/insert-and-scan-dataset/insert-and-scan-dataset.3.query.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/insert-and-scan-dataset/insert-and-scan-dataset.3.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/insert-and-scan-dataset/insert-and-scan-dataset.3.query.aql
deleted file mode 100644
index 37ceca0..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/insert-and-scan-dataset/insert-and-scan-dataset.3.query.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  : insert-and-scan-dataset.aql
- * Description     : This test is intended to test inserting into a temporary dataset and scan it at the same time
- * where we insert a materializing to prevent the possibility of deadlatch.
- * Expected Result : Success
- * Date            : March 27 2015
- *
- */
- 
-use dataverse test;
-
-for $c in dataset('myData')
-order by $c.id
-return $c 

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/q01_pricing_summary_report_nt/q01_pricing_summary_report_nt.1.ddl.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/q01_pricing_summary_report_nt/q01_pricing_summary_report_nt.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/q01_pricing_summary_report_nt/q01_pricing_summary_report_nt.1.ddl.aql
deleted file mode 100644
index 78c0221..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/q01_pricing_summary_report_nt/q01_pricing_summary_report_nt.1.ddl.aql
+++ /dev/null
@@ -1,51 +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.
- */
-/*
- * Description     : Test a read query over a temporary dataset.
- * Expected Result : Success
- * Date            : March 27 2015
- */
-
-drop dataverse tpch if exists;
-create dataverse tpch;
-
-use dataverse tpch;
-
-create type LineItemType as closed {
-  l_orderkey: int64,
-  l_partkey: int64,
-  l_suppkey: int64,
-  l_linenumber: int64,
-  l_quantity: double,
-  l_extendedprice: double,
-  l_discount: double,
-  l_tax: double,
-  l_returnflag: string,
-  l_linestatus: string,
-  l_shipdate: string,
-  l_commitdate: string,
-  l_receiptdate: string,
-  l_shipinstruct: string,
-  l_shipmode: string,
-  l_comment: string
-}
-
-create temporary dataset LineItem(LineItemType)
-  primary key l_orderkey, l_linenumber;
-

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/q01_pricing_summary_report_nt/q01_pricing_summary_report_nt.2.update.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/q01_pricing_summary_report_nt/q01_pricing_summary_report_nt.2.update.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/q01_pricing_summary_report_nt/q01_pricing_summary_report_nt.2.update.aql
deleted file mode 100644
index 5a99e55..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/q01_pricing_summary_report_nt/q01_pricing_summary_report_nt.2.update.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.
- */
-/*
- * Description     : Test a read query over a temporary dataset.
- * Expected Result : Success
- * Date            : March 27 2015
- */
-
-use dataverse tpch;
-
-load dataset LineItem 
-using localfs
-(("path"="asterix_nc1://data/tpch0.001/lineitem.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
-

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/q01_pricing_summary_report_nt/q01_pricing_summary_report_nt.3.query.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/q01_pricing_summary_report_nt/q01_pricing_summary_report_nt.3.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/q01_pricing_summary_report_nt/q01_pricing_summary_report_nt.3.query.aql
deleted file mode 100644
index 75f6f6a..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/q01_pricing_summary_report_nt/q01_pricing_summary_report_nt.3.query.aql
+++ /dev/null
@@ -1,46 +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.
- */
-/*
- * Description     : Test a read query over a temporary dataset.
- * Expected Result : Success
- * Date            : March 27 2015
- */
-
-use dataverse tpch;
-set import-private-functions 'true';
- 
-for $l in dataset('LineItem')
-where $l.l_shipdate <= '1998-09-02'
-/*+ hash*/
-group by $l_returnflag := $l.l_returnflag,
-         $l_linestatus := $l.l_linestatus  
-  with $l
-order by $l_returnflag, $l_linestatus
-return {
-  "l_returnflag": $l_returnflag,
-  "l_linestatus": $l_linestatus,
-  "sum_qty": sum(for $i in $l return $i.l_quantity),
-  "sum_base_price": sum(for $i in $l return $i.l_extendedprice),
-  "sum_disc_price": sum(for $i in $l return $i.l_extendedprice * (1 - $i.l_discount)),
-  "sum_charge": sum(for $i in $l return $i.l_extendedprice * (1 - $i.l_discount) * (1 + $i.l_tax)),
-  "ave_qty": avg(for $i in $l return $i.l_quantity),  
-  "ave_price": avg(for $i in $l return $i.l_extendedprice),
-  "ave_disc": avg(for $i in $l return $i.l_discount),
-  "count_order": count($l)
-}   

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/scan-delete-rtree-secondary-index/scan-delete-rtree-secondary-index.1.ddl.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/scan-delete-rtree-secondary-index/scan-delete-rtree-secondary-index.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/scan-delete-rtree-secondary-index/scan-delete-rtree-secondary-index.1.ddl.aql
deleted file mode 100644
index 9bf5118..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/scan-delete-rtree-secondary-index/scan-delete-rtree-secondary-index.1.ddl.aql
+++ /dev/null
@@ -1,44 +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.
- */
-/*
- * Description     : Delete tuples from a temporary dataset with a R-tree index.
- * Expected Result : Success
- * Date            : March 27 2015
- */
-
-drop dataverse test if exists;
-create dataverse test;
-
-use dataverse test;
-
-create type MyRecord as closed {
-  id: int64,
-  point: point,
-  kwds: string,
-  line1: line,
-  line2: line,
-  poly1: polygon,
-  poly2: polygon,
-  rec: rectangle,
-  circle: circle
-}
-
-create temporary dataset MyData(MyRecord)
-  primary key id;
-

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/scan-delete-rtree-secondary-index/scan-delete-rtree-secondary-index.2.update.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/scan-delete-rtree-secondary-index/scan-delete-rtree-secondary-index.2.update.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/scan-delete-rtree-secondary-index/scan-delete-rtree-secondary-index.2.update.aql
deleted file mode 100644
index 24d8d3d..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/scan-delete-rtree-secondary-index/scan-delete-rtree-secondary-index.2.update.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.
- */
-/*
- * Description     : Delete tuples from a temporary dataset with a R-tree index.
- * Expected Result : Success
- * Date            : March 27 2015
- */
-
-use dataverse test;
-
-load dataset MyData 
-using localfs
-(("path"="asterix_nc1://data/spatial/spatialData.json"),("format"="adm")) pre-sorted;
-

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/scan-delete-rtree-secondary-index/scan-delete-rtree-secondary-index.3.ddl.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/scan-delete-rtree-secondary-index/scan-delete-rtree-secondary-index.3.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/scan-delete-rtree-secondary-index/scan-delete-rtree-secondary-index.3.ddl.aql
deleted file mode 100644
index 3fff78a..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/scan-delete-rtree-secondary-index/scan-delete-rtree-secondary-index.3.ddl.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.
- */
-/*
- * Description     : Delete tuples from a temporary dataset with a R-tree index.
- * Expected Result : Success
- * Date            : March 27 2015
- */
-
-use dataverse test;
-
-create index rtree_index_point on MyData(point) type rtree;
-

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/scan-delete-rtree-secondary-index/scan-delete-rtree-secondary-index.4.update.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/scan-delete-rtree-secondary-index/scan-delete-rtree-secondary-index.4.update.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/scan-delete-rtree-secondary-index/scan-delete-rtree-secondary-index.4.update.aql
deleted file mode 100644
index 2698483..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/scan-delete-rtree-secondary-index/scan-delete-rtree-secondary-index.4.update.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.
- */
-/*
- * Description     : Delete tuples from a temporary dataset with a R-tree index.
- * Expected Result : Success
- * Date            : March 27 2015
- */
-
-use dataverse test;
-
-delete $m from dataset MyData where $m.id>10;
-

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/scan-delete-rtree-secondary-index/scan-delete-rtree-secondary-index.5.query.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/scan-delete-rtree-secondary-index/scan-delete-rtree-secondary-index.5.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/scan-delete-rtree-secondary-index/scan-delete-rtree-secondary-index.5.query.aql
deleted file mode 100644
index db8d975..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/scan-delete-rtree-secondary-index/scan-delete-rtree-secondary-index.5.query.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.
- */
-/*
- * Description     : Delete tuples from a temporary dataset with a R-tree index.
- * Expected Result : Success
- * Date            : March 27 2015
- */
-
-use dataverse test;
-
-for $o in dataset('MyData')
-where spatial-intersect($o.point, create-polygon([0.0,1.0,0.0,4.0,12.0,4.0,12.0,1.0]))
-order by $o.id
-return {"id":$o.id}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/scan-insert-persistent-to-temp/scan-insert-persistent-to-temp.1.ddl.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/scan-insert-persistent-to-temp/scan-insert-persistent-to-temp.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/scan-insert-persistent-to-temp/scan-insert-persistent-to-temp.1.ddl.aql
deleted file mode 100644
index 82430ed..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/scan-insert-persistent-to-temp/scan-insert-persistent-to-temp.1.ddl.aql
+++ /dev/null
@@ -1,53 +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.
- */
-/*
- * Description     : Port data from a persistent dataset into a temporary dataset. 
- *                   Both datasets have R-tree indexes.
- * Expected Result : Success
- * Date            : March 27 2015
- */
-
-drop dataverse test if exists;
-create dataverse test;
-
-use dataverse test;
-
-create type MyRecord as closed {
-  id: int64,
-  point: point,
-  kwds: string,
-  line1: line,
-  line2: line,
-  poly1: polygon,
-  poly2: polygon,
-  rec: rectangle,
-  circle: circle
-}
-
-create type MyMiniRecord as closed {
-  id: int64,
-  point: point
-}
-
-create dataset MyData(MyRecord)
-  primary key id;
-
-create temporary dataset MyMiniData(MyMiniRecord)
-  primary key id;
-

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/scan-insert-persistent-to-temp/scan-insert-persistent-to-temp.2.update.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/scan-insert-persistent-to-temp/scan-insert-persistent-to-temp.2.update.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/scan-insert-persistent-to-temp/scan-insert-persistent-to-temp.2.update.aql
deleted file mode 100644
index 8fffba1..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/scan-insert-persistent-to-temp/scan-insert-persistent-to-temp.2.update.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.
- */
-/*
- * Description     : Port data from a temporary dataset into yet-another temporary dataset. 
- *                   Both datasets have R-tree indexes.
- * Expected Result : Success
- * Date            : March 27 2015
- */
-
-use dataverse test;
-
-load dataset MyData 
-using localfs
-(("path"="asterix_nc1://data/spatial/spatialData.json"),("format"="adm")) pre-sorted;
-
-load dataset MyMiniData 
-using localfs
-(("path"="asterix_nc1://data/spatial/spatialData0.json"),("format"="adm")) pre-sorted;
-

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/scan-insert-persistent-to-temp/scan-insert-persistent-to-temp.3.ddl.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/scan-insert-persistent-to-temp/scan-insert-persistent-to-temp.3.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/scan-insert-persistent-to-temp/scan-insert-persistent-to-temp.3.ddl.aql
deleted file mode 100644
index 6c00a8d..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/scan-insert-persistent-to-temp/scan-insert-persistent-to-temp.3.ddl.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.
- */
-/*
- * Description     : Port data from a temporary dataset into yet-another temporary dataset. 
- *                   Both datasets have R-tree indexes.
- * Expected Result : Success
- * Date            : March 27 2015
- */
-
-use dataverse test;
-
-create index rtree_index_point_0 on MyData(point) type rtree;
-create index rtree_index_point on MyMiniData(point) type rtree;
-

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/scan-insert-persistent-to-temp/scan-insert-persistent-to-temp.4.update.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/scan-insert-persistent-to-temp/scan-insert-persistent-to-temp.4.update.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/scan-insert-persistent-to-temp/scan-insert-persistent-to-temp.4.update.aql
deleted file mode 100644
index 913be14..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/scan-insert-persistent-to-temp/scan-insert-persistent-to-temp.4.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.
- */
-/*
- * Description     : Port data from a temporary dataset into yet-another temporary dataset.
- *                   Both datasets have R-tree indexes.
- * Expected Result : Success
- * Date            : March 27 2015
- */
-
-use dataverse test;
-
-insert into dataset MyMiniData
-(
-    for $m in dataset('MyData')
-    return {
-        "id": $m.id,
-        "point": $m.point
-    }
-);
-

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/scan-insert-persistent-to-temp/scan-insert-persistent-to-temp.5.query.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/scan-insert-persistent-to-temp/scan-insert-persistent-to-temp.5.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/scan-insert-persistent-to-temp/scan-insert-persistent-to-temp.5.query.aql
deleted file mode 100644
index 35906d1..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/scan-insert-persistent-to-temp/scan-insert-persistent-to-temp.5.query.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.
- */
-/*
- * Description     : Port data from a temporary dataset into yet-another temporary dataset. 
- *                   Both datasets have R-tree indexes.
- * Expected Result : Success
- * Date            : March 27 2015
- */
-
-use dataverse test;
-
-for $o in dataset('MyMiniData')
-where spatial-intersect($o.point, create-polygon([0.0,1.0,0.0,4.0,12.0,4.0,12.0,1.0]))
-order by $o.id
-return {"id":$o.id}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/scan-insert-rtree-secondary-index/scan-insert-rtree-secondary-index.1.ddl.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/scan-insert-rtree-secondary-index/scan-insert-rtree-secondary-index.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/scan-insert-rtree-secondary-index/scan-insert-rtree-secondary-index.1.ddl.aql
deleted file mode 100644
index 66c5c48..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/scan-insert-rtree-secondary-index/scan-insert-rtree-secondary-index.1.ddl.aql
+++ /dev/null
@@ -1,53 +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.
- */
-/*
- * Description     : Port data from a temporary dataset into yet-another temporary dataset. 
- *                   Both datasets have R-tree indexes.
- * Expected Result : Success
- * Date            : March 27 2015
- */
-
-drop dataverse test if exists;
-create dataverse test;
-
-use dataverse test;
-
-create type MyRecord as closed {
-  id: int64,
-  point: point,
-  kwds: string,
-  line1: line,
-  line2: line,
-  poly1: polygon,
-  poly2: polygon,
-  rec: rectangle,
-  circle: circle
-}
-
-create type MyMiniRecord as closed {
-  id: int64,
-  point: point
-}
-
-create temporary dataset MyData(MyRecord)
-  primary key id;
-
-create temporary dataset MyMiniData(MyMiniRecord)
-  primary key id;
-

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/scan-insert-rtree-secondary-index/scan-insert-rtree-secondary-index.2.update.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/scan-insert-rtree-secondary-index/scan-insert-rtree-secondary-index.2.update.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/scan-insert-rtree-secondary-index/scan-insert-rtree-secondary-index.2.update.aql
deleted file mode 100644
index 8fffba1..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/scan-insert-rtree-secondary-index/scan-insert-rtree-secondary-index.2.update.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.
- */
-/*
- * Description     : Port data from a temporary dataset into yet-another temporary dataset. 
- *                   Both datasets have R-tree indexes.
- * Expected Result : Success
- * Date            : March 27 2015
- */
-
-use dataverse test;
-
-load dataset MyData 
-using localfs
-(("path"="asterix_nc1://data/spatial/spatialData.json"),("format"="adm")) pre-sorted;
-
-load dataset MyMiniData 
-using localfs
-(("path"="asterix_nc1://data/spatial/spatialData0.json"),("format"="adm")) pre-sorted;
-

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/scan-insert-rtree-secondary-index/scan-insert-rtree-secondary-index.3.ddl.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/scan-insert-rtree-secondary-index/scan-insert-rtree-secondary-index.3.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/scan-insert-rtree-secondary-index/scan-insert-rtree-secondary-index.3.ddl.aql
deleted file mode 100644
index 6c00a8d..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/scan-insert-rtree-secondary-index/scan-insert-rtree-secondary-index.3.ddl.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.
- */
-/*
- * Description     : Port data from a temporary dataset into yet-another temporary dataset. 
- *                   Both datasets have R-tree indexes.
- * Expected Result : Success
- * Date            : March 27 2015
- */
-
-use dataverse test;
-
-create index rtree_index_point_0 on MyData(point) type rtree;
-create index rtree_index_point on MyMiniData(point) type rtree;
-

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/scan-insert-rtree-secondary-index/scan-insert-rtree-secondary-index.4.update.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/scan-insert-rtree-secondary-index/scan-insert-rtree-secondary-index.4.update.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/scan-insert-rtree-secondary-index/scan-insert-rtree-secondary-index.4.update.aql
deleted file mode 100644
index 913be14..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/scan-insert-rtree-secondary-index/scan-insert-rtree-secondary-index.4.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.
- */
-/*
- * Description     : Port data from a temporary dataset into yet-another temporary dataset.
- *                   Both datasets have R-tree indexes.
- * Expected Result : Success
- * Date            : March 27 2015
- */
-
-use dataverse test;
-
-insert into dataset MyMiniData
-(
-    for $m in dataset('MyData')
-    return {
-        "id": $m.id,
-        "point": $m.point
-    }
-);
-

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/scan-insert-rtree-secondary-index/scan-insert-rtree-secondary-index.5.query.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/scan-insert-rtree-secondary-index/scan-insert-rtree-secondary-index.5.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/scan-insert-rtree-secondary-index/scan-insert-rtree-secondary-index.5.query.aql
deleted file mode 100644
index 35906d1..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/scan-insert-rtree-secondary-index/scan-insert-rtree-secondary-index.5.query.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.
- */
-/*
- * Description     : Port data from a temporary dataset into yet-another temporary dataset. 
- *                   Both datasets have R-tree indexes.
- * Expected Result : Success
- * Date            : March 27 2015
- */
-
-use dataverse test;
-
-for $o in dataset('MyMiniData')
-where spatial-intersect($o.point, create-polygon([0.0,1.0,0.0,4.0,12.0,4.0,12.0,1.0]))
-order by $o.id
-return {"id":$o.id}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/scan-insert-temp-to-persistent/scan-insert-temp-to-persistent.1.ddl.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/scan-insert-temp-to-persistent/scan-insert-temp-to-persistent.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/scan-insert-temp-to-persistent/scan-insert-temp-to-persistent.1.ddl.aql
deleted file mode 100644
index a770f78..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/scan-insert-temp-to-persistent/scan-insert-temp-to-persistent.1.ddl.aql
+++ /dev/null
@@ -1,53 +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.
- */
-/*
- * Description     : Port data from a temporary dataset into a persistent dataset. 
- *                   Both datasets have R-tree indexes.
- * Expected Result : Success
- * Date            : March 27 2015
- */
-
-drop dataverse test if exists;
-create dataverse test;
-
-use dataverse test;
-
-create type MyRecord as closed {
-  id: int64,
-  point: point,
-  kwds: string,
-  line1: line,
-  line2: line,
-  poly1: polygon,
-  poly2: polygon,
-  rec: rectangle,
-  circle: circle
-}
-
-create type MyMiniRecord as closed {
-  id: int64,
-  point: point
-}
-
-create temporary dataset MyData(MyRecord)
-  primary key id;
-
-create dataset MyMiniData(MyMiniRecord)
-  primary key id;
-

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/scan-insert-temp-to-persistent/scan-insert-temp-to-persistent.2.update.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/scan-insert-temp-to-persistent/scan-insert-temp-to-persistent.2.update.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/scan-insert-temp-to-persistent/scan-insert-temp-to-persistent.2.update.aql
deleted file mode 100644
index 8fffba1..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/scan-insert-temp-to-persistent/scan-insert-temp-to-persistent.2.update.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.
- */
-/*
- * Description     : Port data from a temporary dataset into yet-another temporary dataset. 
- *                   Both datasets have R-tree indexes.
- * Expected Result : Success
- * Date            : March 27 2015
- */
-
-use dataverse test;
-
-load dataset MyData 
-using localfs
-(("path"="asterix_nc1://data/spatial/spatialData.json"),("format"="adm")) pre-sorted;
-
-load dataset MyMiniData 
-using localfs
-(("path"="asterix_nc1://data/spatial/spatialData0.json"),("format"="adm")) pre-sorted;
-

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/scan-insert-temp-to-persistent/scan-insert-temp-to-persistent.3.ddl.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/scan-insert-temp-to-persistent/scan-insert-temp-to-persistent.3.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/scan-insert-temp-to-persistent/scan-insert-temp-to-persistent.3.ddl.aql
deleted file mode 100644
index 6c00a8d..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/scan-insert-temp-to-persistent/scan-insert-temp-to-persistent.3.ddl.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.
- */
-/*
- * Description     : Port data from a temporary dataset into yet-another temporary dataset. 
- *                   Both datasets have R-tree indexes.
- * Expected Result : Success
- * Date            : March 27 2015
- */
-
-use dataverse test;
-
-create index rtree_index_point_0 on MyData(point) type rtree;
-create index rtree_index_point on MyMiniData(point) type rtree;
-

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/scan-insert-temp-to-persistent/scan-insert-temp-to-persistent.4.update.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/scan-insert-temp-to-persistent/scan-insert-temp-to-persistent.4.update.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/scan-insert-temp-to-persistent/scan-insert-temp-to-persistent.4.update.aql
deleted file mode 100644
index 913be14..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/scan-insert-temp-to-persistent/scan-insert-temp-to-persistent.4.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.
- */
-/*
- * Description     : Port data from a temporary dataset into yet-another temporary dataset.
- *                   Both datasets have R-tree indexes.
- * Expected Result : Success
- * Date            : March 27 2015
- */
-
-use dataverse test;
-
-insert into dataset MyMiniData
-(
-    for $m in dataset('MyData')
-    return {
-        "id": $m.id,
-        "point": $m.point
-    }
-);
-

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/scan-insert-temp-to-persistent/scan-insert-temp-to-persistent.5.query.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/scan-insert-temp-to-persistent/scan-insert-temp-to-persistent.5.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/scan-insert-temp-to-persistent/scan-insert-temp-to-persistent.5.query.aql
deleted file mode 100644
index 35906d1..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/scan-insert-temp-to-persistent/scan-insert-temp-to-persistent.5.query.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.
- */
-/*
- * Description     : Port data from a temporary dataset into yet-another temporary dataset. 
- *                   Both datasets have R-tree indexes.
- * Expected Result : Success
- * Date            : March 27 2015
- */
-
-use dataverse test;
-
-for $o in dataset('MyMiniData')
-where spatial-intersect($o.point, create-polygon([0.0,1.0,0.0,4.0,12.0,4.0,12.0,1.0]))
-order by $o.id
-return {"id":$o.id}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/temp_primary_plus_ngram_flush/temp_primary_plus_ngram_flush.1.ddl.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/temp_primary_plus_ngram_flush/temp_primary_plus_ngram_flush.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/temp_primary_plus_ngram_flush/temp_primary_plus_ngram_flush.1.ddl.aql
deleted file mode 100644
index 671e5a2..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/temp_primary_plus_ngram_flush/temp_primary_plus_ngram_flush.1.ddl.aql
+++ /dev/null
@@ -1,67 +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  : temp_primary_plus_ngram_flush.aql
- * Description     : Check that flush for temporary datasets with ngram indexes succeeds.
- * Expected Result : Success
- * Date            : Apr 4 2016
- */
-
-drop dataverse recovery if exists;
-create dataverse recovery;
-use dataverse recovery;
-
-/* For raw Fragile data */
-create type FragileTypeRaw as closed {
-row_id: int32,
-sid: int32,
-date: string,
-day: int32,
-time: string,
-bpm: int32,
-RR: float,
-text: string,
-location: point,
-text2: string
-};
-
-/* For cleaned Fragile data */
-create type FragileType as closed {
-row_id: int32,
-sid: int32,
-date: date,
-day: int32,
-time: time,
-bpm: int32,
-RR: float,
-text: string,
-location: point,
-text2: string
-};
-
-/* Create dataset for loading raw Fragile data */
-create temporary dataset Fragile_raw (FragileTypeRaw)
-primary key row_id;
-
-/* Create dataset for cleaned Fragile data */
-create temporary dataset Fragile (FragileType)
-primary key row_id;
-
-/* Create default secondary index on dataset clean Fragile */
-create index cfText2Ix on Fragile(text2) type ngram(3);
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/temp_primary_plus_ngram_flush/temp_primary_plus_ngram_flush.2.update.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/temp_primary_plus_ngram_flush/temp_primary_plus_ngram_flush.2.update.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/temp_primary_plus_ngram_flush/temp_primary_plus_ngram_flush.2.update.aql
deleted file mode 100644
index 1b3cbd3..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/temp_primary_plus_ngram_flush/temp_primary_plus_ngram_flush.2.update.aql
+++ /dev/null
@@ -1,46 +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  : temp_primary_plus_ngram_flush.aql
- * Description     : Check that flush for temporary datasets with ngram indexes succeeds.
- * Expected Result : Success
- * Date            : Apr 4 2016
- */
-
-use dataverse recovery;
-
-load dataset Fragile_raw using localfs
-(("path"="asterix_nc1://data/csv/fragile_02.adm"),("format"="adm")) pre-sorted;
-
-/* Load Fragile data from raw dataset into cleaned dataset */
-insert into dataset Fragile (
-for $t in dataset Fragile_raw
-where $t.row_id <= 1000
-return {
-"row_id": $t.row_id,
-"sid": $t.sid,
-"date": date($t.date),
-"day": $t.day,
-"time": parse-time($t.time, "h:m:s"),
-"bpm": $t.bpm,
-"RR": $t.RR,
-"text": $t.text,
-"location": $t.location,
-"text2": $t.text2}
-);
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/temp_primary_plus_ngram_flush/temp_primary_plus_ngram_flush.3.query.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/temp_primary_plus_ngram_flush/temp_primary_plus_ngram_flush.3.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/temp_primary_plus_ngram_flush/temp_primary_plus_ngram_flush.3.query.aql
deleted file mode 100644
index 67c41d6..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temp-dataset/temp_primary_plus_ngram_flush/temp_primary_plus_ngram_flush.3.query.aql
+++ /dev/null
@@ -1,29 +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  : temp_primary_plus_ngram_flush.aql
- * Description     : Check that flush for temporary datasets with ngram indexes succeeds.
- * Expected Result : Success
- * Date            : Apr 4 2016
- */
-
-use dataverse recovery;
-
-count (for $x in dataset Fragile
-where contains($x.text2, "location") return $x);
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/delete-from-loaded-dataset-with-index/delete-from-loaded-dataset-with-index.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/delete-from-loaded-dataset-with-index/delete-from-loaded-dataset-with-index.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/delete-from-loaded-dataset-with-index/delete-from-loaded-dataset-with-index.1.ddl.sqlpp
deleted file mode 100644
index 016b37e..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/delete-from-loaded-dataset-with-index/delete-from-loaded-dataset-with-index.1.ddl.sqlpp
+++ /dev/null
@@ -1,53 +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.
- */
-/*
- * Description     : Delete a portion of data from a loaded temporary dataset with indexes.
- * Expected Result : Success
- * Date            : 27th March 2015
- *
- */
-
-drop  dataverse test if exists;
-create  dataverse test;
-
-use test;
-
-
-create type test.LineItemType as
- closed {
-  l_orderkey : bigint,
-  l_partkey : bigint,
-  l_suppkey : bigint,
-  l_linenumber : bigint,
-  l_quantity : bigint,
-  l_extendedprice : double,
-  l_discount : double,
-  l_tax : double,
-  l_returnflag : string,
-  l_linestatus : string,
-  l_shipdate : string,
-  l_commitdate : string,
-  l_receiptdate : string,
-  l_shipinstruct : string,
-  l_shipmode : string,
-  l_comment : string
-};
-
-create temporary dataset LineItem(LineItemType) primary key l_orderkey,l_linenumber;
-

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/delete-from-loaded-dataset-with-index/delete-from-loaded-dataset-with-index.2.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/delete-from-loaded-dataset-with-index/delete-from-loaded-dataset-with-index.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/delete-from-loaded-dataset-with-index/delete-from-loaded-dataset-with-index.2.update.sqlpp
deleted file mode 100644
index 5ebf862..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/delete-from-loaded-dataset-with-index/delete-from-loaded-dataset-with-index.2.update.sqlpp
+++ /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.
- */
-/*
- * Description     : Delete a portion of data from a loaded temporary dataset with indexes.
- * Expected Result : Success
- * Date            : 27th March 2015
- *
- */
-
-use test;
-
-
-load  dataset LineItem using localfs ((`path`=`asterix_nc1://data/tpch0.001/lineitem.tbl`),(`format`=`delimited-text`),(`delimiter`=`|`)) pre-sorted;
-

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/delete-from-loaded-dataset-with-index/delete-from-loaded-dataset-with-index.3.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/delete-from-loaded-dataset-with-index/delete-from-loaded-dataset-with-index.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/delete-from-loaded-dataset-with-index/delete-from-loaded-dataset-with-index.3.ddl.sqlpp
deleted file mode 100644
index 55f14eb..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/delete-from-loaded-dataset-with-index/delete-from-loaded-dataset-with-index.3.ddl.sqlpp
+++ /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.
- */
-/*
- * Description     : Delete a portion of data from a loaded temporary dataset with indexes.
- * Expected Result : Success
- * Date            : 27th March 2015
- *
- */
-
-use test;
-
-
-create  index idx_LineItem_partkey  on LineItem (l_linenumber) type btree;
-
-create  index idx_LineItem_suppkey  on LineItem (l_suppkey) type btree;
-
-create  primary index sec_primary_idx  on LineItem;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/delete-from-loaded-dataset-with-index/delete-from-loaded-dataset-with-index.4.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/delete-from-loaded-dataset-with-index/delete-from-loaded-dataset-with-index.4.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/delete-from-loaded-dataset-with-index/delete-from-loaded-dataset-with-index.4.update.sqlpp
deleted file mode 100644
index 71a5f53..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temp-dataset/delete-from-loaded-dataset-with-index/delete-from-loaded-dataset-with-index.4.update.sqlpp
+++ /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.
- */
-/*
- * Description     : Delete a portion of data from a loaded temporary dataset with indexes.
- * Expected Result : Success
- * Date            : 27th March 2015
- *
- */
-
-use test;
-
-
-delete from LineItem
- where l_suppkey >= 2 or l_linenumber > 1;


[07/12] asterixdb git commit: [ASTERIXDB-2171][STO] Remove Temporary Datasets

Posted by mh...@apache.org.
http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/results/temp-dataset/delete-from-loaded-dataset-with-index/delete-from-loaded-dataset-with-index.1.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temp-dataset/delete-from-loaded-dataset-with-index/delete-from-loaded-dataset-with-index.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temp-dataset/delete-from-loaded-dataset-with-index/delete-from-loaded-dataset-with-index.1.adm
deleted file mode 100644
index 199ac15..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/temp-dataset/delete-from-loaded-dataset-with-index/delete-from-loaded-dataset-with-index.1.adm
+++ /dev/null
@@ -1,167 +0,0 @@
-{ "l_orderkey": 36, "l_partkey": 120, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 42, "l_extendedprice": 42845.04, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-03", "l_commitdate": "1996-01-21", "l_receiptdate": "1996-02-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": " careful courts. special " }
-{ "l_orderkey": 68, "l_partkey": 8, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 3, "l_extendedprice": 2724.0, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-04", "l_commitdate": "1998-06-05", "l_receiptdate": "1998-07-21", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "fully special instructions cajole. furious" }
-{ "l_orderkey": 162, "l_partkey": 190, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 2, "l_extendedprice": 2180.38, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-02", "l_commitdate": "1995-06-17", "l_receiptdate": "1995-09-08", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "es! final somas integrate" }
-{ "l_orderkey": 192, "l_partkey": 98, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 23, "l_extendedprice": 22956.07, "l_discount": 0.0, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-05", "l_commitdate": "1998-02-06", "l_receiptdate": "1998-03-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "ly pending theodolites haggle quickly fluf" }
-{ "l_orderkey": 197, "l_partkey": 99, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 39, "l_extendedprice": 38964.51, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-21", "l_commitdate": "1995-07-01", "l_receiptdate": "1995-08-14", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "press accounts. daringly sp" }
-{ "l_orderkey": 227, "l_partkey": 166, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 19, "l_extendedprice": 20257.04, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-10", "l_commitdate": "1996-01-30", "l_receiptdate": "1995-12-26", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "s cajole furiously a" }
-{ "l_orderkey": 290, "l_partkey": 6, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 35, "l_extendedprice": 31710.0, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-04-01", "l_commitdate": "1994-02-05", "l_receiptdate": "1994-04-27", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ove the final foxes detect slyly fluffily" }
-{ "l_orderkey": 325, "l_partkey": 159, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 34, "l_extendedprice": 36011.1, "l_discount": 0.09, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-28", "l_commitdate": "1993-12-13", "l_receiptdate": "1993-11-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "ly bold deposits. always iron" }
-{ "l_orderkey": 355, "l_partkey": 114, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 31, "l_extendedprice": 31437.41, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-07-13", "l_commitdate": "1994-08-18", "l_receiptdate": "1994-07-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "y unusual, ironic" }
-{ "l_orderkey": 389, "l_partkey": 190, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 2, "l_extendedprice": 2180.38, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-04-13", "l_commitdate": "1994-04-10", "l_receiptdate": "1994-04-25", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "fts. courts eat blithely even dependenc" }
-{ "l_orderkey": 391, "l_partkey": 122, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 14, "l_extendedprice": 14309.68, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-02-11", "l_commitdate": "1995-02-03", "l_receiptdate": "1995-02-13", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": " escapades sleep furiously about " }
-{ "l_orderkey": 417, "l_partkey": 40, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 39, "l_extendedprice": 36661.56, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-05-31", "l_commitdate": "1994-05-02", "l_receiptdate": "1994-06-06", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "y regular requests wake along " }
-{ "l_orderkey": 453, "l_partkey": 198, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 45, "l_extendedprice": 49418.55, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-30", "l_commitdate": "1997-08-20", "l_receiptdate": "1997-07-19", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ifts wake carefully." }
-{ "l_orderkey": 485, "l_partkey": 150, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 50, "l_extendedprice": 52507.5, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-28", "l_commitdate": "1997-05-26", "l_receiptdate": "1997-04-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "iously quick excuses. carefully final f" }
-{ "l_orderkey": 545, "l_partkey": 170, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 4, "l_extendedprice": 4280.68, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-23", "l_commitdate": "1995-12-16", "l_receiptdate": "1996-03-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": ", ironic grouches cajole over" }
-{ "l_orderkey": 581, "l_partkey": 64, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 41, "l_extendedprice": 39526.46, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-26", "l_commitdate": "1997-04-06", "l_receiptdate": "1997-06-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "nts. quickly" }
-{ "l_orderkey": 647, "l_partkey": 17, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 41, "l_extendedprice": 37597.41, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-19", "l_commitdate": "1997-09-24", "l_receiptdate": "1997-12-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "r instructions. quickly unusu" }
-{ "l_orderkey": 704, "l_partkey": 190, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 40, "l_extendedprice": 43607.6, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-30", "l_commitdate": "1997-01-10", "l_receiptdate": "1997-02-20", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ggle quickly. r" }
-{ "l_orderkey": 738, "l_partkey": 198, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 34, "l_extendedprice": 37338.46, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-09", "l_commitdate": "1993-04-15", "l_receiptdate": "1993-07-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "s against the ironic exc" }
-{ "l_orderkey": 773, "l_partkey": 100, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 5, "l_extendedprice": 5000.5, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-11-21", "l_commitdate": "1993-12-19", "l_receiptdate": "1993-12-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ar requests. regular, thin packages u" }
-{ "l_orderkey": 800, "l_partkey": 72, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 38, "l_extendedprice": 36938.66, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-21", "l_commitdate": "1998-09-25", "l_receiptdate": "1998-08-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "according to the bold, final dependencies " }
-{ "l_orderkey": 900, "l_partkey": 199, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 44, "l_extendedprice": 48364.36, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-15", "l_commitdate": "1994-12-03", "l_receiptdate": "1994-12-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": " detect quick" }
-{ "l_orderkey": 931, "l_partkey": 40, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 18, "l_extendedprice": 16920.72, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-04-04", "l_commitdate": "1993-01-11", "l_receiptdate": "1993-04-13", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "slyly ironic re" }
-{ "l_orderkey": 932, "l_partkey": 44, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 41, "l_extendedprice": 38705.64, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-05", "l_commitdate": "1997-07-22", "l_receiptdate": "1997-06-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "foxes. ironic pl" }
-{ "l_orderkey": 965, "l_partkey": 108, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 20, "l_extendedprice": 20162.0, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "F", "l_shipdate": "1995-06-16", "l_commitdate": "1995-07-20", "l_receiptdate": "1995-07-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "kly. carefully pending requ" }
-{ "l_orderkey": 995, "l_partkey": 173, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 15, "l_extendedprice": 16097.55, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-06-30", "l_commitdate": "1995-08-04", "l_receiptdate": "1995-07-27", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "uses. fluffily fina" }
-{ "l_orderkey": 1025, "l_partkey": 150, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 36, "l_extendedprice": 37805.4, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-05-15", "l_commitdate": "1995-07-05", "l_receiptdate": "1995-06-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "e unusual, regular instr" }
-{ "l_orderkey": 1027, "l_partkey": 156, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 43, "l_extendedprice": 45414.45, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-06-17", "l_commitdate": "1992-08-28", "l_receiptdate": "1992-07-10", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "oxes. carefully regular deposits" }
-{ "l_orderkey": 1155, "l_partkey": 70, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 4, "l_extendedprice": 3880.28, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-19", "l_commitdate": "1997-12-09", "l_receiptdate": "1997-11-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ic foxes according to the carefully final " }
-{ "l_orderkey": 1185, "l_partkey": 72, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 8, "l_extendedprice": 7776.56, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-12-05", "l_commitdate": "1992-10-05", "l_receiptdate": "1992-12-28", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ely according to the furiously regular r" }
-{ "l_orderkey": 1216, "l_partkey": 97, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 8, "l_extendedprice": 7976.72, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-02-01", "l_commitdate": "1993-03-06", "l_receiptdate": "1993-02-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": " of the carefully express" }
-{ "l_orderkey": 1223, "l_partkey": 100, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 28, "l_extendedprice": 28002.8, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-07", "l_commitdate": "1996-07-24", "l_receiptdate": "1996-08-13", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": " quickly ironic requests. furious" }
-{ "l_orderkey": 1381, "l_partkey": 144, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 47, "l_extendedprice": 49074.58, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-09-22", "l_commitdate": "1998-08-12", "l_receiptdate": "1998-10-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "ly ironic deposits" }
-{ "l_orderkey": 1409, "l_partkey": 99, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 23, "l_extendedprice": 22979.07, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-04-18", "l_commitdate": "1993-02-25", "l_receiptdate": "1993-05-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ions. slyly ironic packages wake quick" }
-{ "l_orderkey": 1445, "l_partkey": 100, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 24, "l_extendedprice": 24002.4, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-02-21", "l_commitdate": "1995-02-22", "l_receiptdate": "1995-03-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "al accounts use furiously a" }
-{ "l_orderkey": 1477, "l_partkey": 72, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 31, "l_extendedprice": 30134.17, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-16", "l_commitdate": "1997-09-30", "l_receiptdate": "1997-12-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": " requests. fluffily final " }
-{ "l_orderkey": 1540, "l_partkey": 173, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 38, "l_extendedprice": 40780.46, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-09-30", "l_commitdate": "1992-10-27", "l_receiptdate": "1992-10-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": " final grouches bo" }
-{ "l_orderkey": 1568, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 36, "l_extendedprice": 35643.24, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-31", "l_commitdate": "1997-04-22", "l_receiptdate": "1997-06-21", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "platelets-- furiously sly excu" }
-{ "l_orderkey": 1605, "l_partkey": 142, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 47, "l_extendedprice": 48980.58, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-29", "l_commitdate": "1998-06-12", "l_receiptdate": "1998-05-20", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": ". carefully r" }
-{ "l_orderkey": 1607, "l_partkey": 190, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 2, "l_extendedprice": 2180.38, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-11", "l_commitdate": "1996-02-15", "l_receiptdate": "1996-01-19", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "packages haggle. regular requests boost s" }
-{ "l_orderkey": 1635, "l_partkey": 71, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 3, "l_extendedprice": 2913.21, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-13", "l_commitdate": "1997-03-25", "l_receiptdate": "1997-03-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": " quickly ironic r" }
-{ "l_orderkey": 1700, "l_partkey": 140, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 38, "l_extendedprice": 39525.32, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-03", "l_commitdate": "1996-07-27", "l_receiptdate": "1996-10-22", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "ular dependencies engage slyly " }
-{ "l_orderkey": 1796, "l_partkey": 10, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 28, "l_extendedprice": 25480.28, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-12-01", "l_commitdate": "1993-01-01", "l_receiptdate": "1992-12-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "y quickly ironic accounts." }
-{ "l_orderkey": 1825, "l_partkey": 156, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 43, "l_extendedprice": 45414.45, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-02-18", "l_commitdate": "1994-02-19", "l_receiptdate": "1994-03-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": " accounts breach fluffily spe" }
-{ "l_orderkey": 1827, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 47, "l_extendedprice": 46534.23, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-01", "l_commitdate": "1996-08-07", "l_receiptdate": "1996-08-23", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": ". pending courts about the even e" }
-{ "l_orderkey": 1893, "l_partkey": 99, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 43, "l_extendedprice": 42960.87, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-25", "l_commitdate": "1998-01-06", "l_receiptdate": "1998-02-14", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "he carefully regular " }
-{ "l_orderkey": 1924, "l_partkey": 73, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 7, "l_extendedprice": 6811.49, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-01", "l_commitdate": "1996-12-02", "l_receiptdate": "1997-01-08", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "osits. even accounts nag furious" }
-{ "l_orderkey": 1953, "l_partkey": 128, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 25, "l_extendedprice": 25703.0, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-07", "l_commitdate": "1994-01-28", "l_receiptdate": "1994-01-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ular, regular i" }
-{ "l_orderkey": 1985, "l_partkey": 28, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 33, "l_extendedprice": 30624.66, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-04", "l_commitdate": "1994-11-01", "l_receiptdate": "1994-12-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "s are express packages. pendin" }
-{ "l_orderkey": 1988, "l_partkey": 72, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 36, "l_extendedprice": 34994.52, "l_discount": 0.09, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-21", "l_commitdate": "1995-11-24", "l_receiptdate": "1996-01-27", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "gular theodolites. " }
-{ "l_orderkey": 2048, "l_partkey": 35, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 7, "l_extendedprice": 6545.21, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-12-07", "l_commitdate": "1994-01-31", "l_receiptdate": "1994-01-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "lent platelets boost deposits. carefully sp" }
-{ "l_orderkey": 2086, "l_partkey": 60, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 22, "l_extendedprice": 21121.32, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-04", "l_commitdate": "1994-12-16", "l_receiptdate": "1994-12-20", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "idly busy acc" }
-{ "l_orderkey": 2118, "l_partkey": 160, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 24, "l_extendedprice": 25443.84, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-06", "l_commitdate": "1996-12-14", "l_receiptdate": "1997-01-14", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "about the slyly bold depende" }
-{ "l_orderkey": 2183, "l_partkey": 71, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 29, "l_extendedprice": 28161.03, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-21", "l_commitdate": "1996-08-24", "l_receiptdate": "1996-08-15", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ly unusual deposits sleep carefully" }
-{ "l_orderkey": 2211, "l_partkey": 48, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 25, "l_extendedprice": 23701.0, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-10-09", "l_commitdate": "1994-08-04", "l_receiptdate": "1994-11-03", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "deas. carefully special theodolites along" }
-{ "l_orderkey": 2215, "l_partkey": 73, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 33, "l_extendedprice": 32111.31, "l_discount": 0.0, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-19", "l_commitdate": "1996-08-10", "l_receiptdate": "1996-07-30", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "dolites cajole b" }
-{ "l_orderkey": 2272, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 18, "l_extendedprice": 17821.62, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-08-01", "l_commitdate": "1993-07-06", "l_receiptdate": "1993-08-25", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ons along the blithely e" }
-{ "l_orderkey": 2342, "l_partkey": 42, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 12, "l_extendedprice": 11304.48, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-31", "l_commitdate": "1996-07-26", "l_receiptdate": "1996-08-14", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "print blithely even deposits. carefull" }
-{ "l_orderkey": 2343, "l_partkey": 110, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 27, "l_extendedprice": 27272.97, "l_discount": 0.0, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-10", "l_commitdate": "1995-11-17", "l_receiptdate": "1995-12-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "old theodolites." }
-{ "l_orderkey": 2439, "l_partkey": 164, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 2, "l_extendedprice": 2128.32, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-14", "l_commitdate": "1997-06-11", "l_receiptdate": "1997-05-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "courts boos" }
-{ "l_orderkey": 2469, "l_partkey": 166, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 11, "l_extendedprice": 11727.76, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-09", "l_commitdate": "1997-01-26", "l_receiptdate": "1997-02-16", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ies wake carefully b" }
-{ "l_orderkey": 2592, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 7, "l_extendedprice": 6930.63, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-03-13", "l_commitdate": "1993-04-25", "l_receiptdate": "1993-04-01", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": " carefully special theodolites integrate " }
-{ "l_orderkey": 2625, "l_partkey": 20, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 42, "l_extendedprice": 38640.84, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-10-18", "l_commitdate": "1992-11-17", "l_receiptdate": "1992-10-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": " even accounts haggle furiously" }
-{ "l_orderkey": 2659, "l_partkey": 42, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 28, "l_extendedprice": 26377.12, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-03-17", "l_commitdate": "1994-01-24", "l_receiptdate": "1994-03-19", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "idle tithes" }
-{ "l_orderkey": 2689, "l_partkey": 6, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 45, "l_extendedprice": 40770.0, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-04-29", "l_commitdate": "1992-06-22", "l_receiptdate": "1992-04-30", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "e quickly. carefully silent" }
-{ "l_orderkey": 2690, "l_partkey": 140, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 44, "l_extendedprice": 45766.16, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-30", "l_commitdate": "1996-05-19", "l_receiptdate": "1996-06-26", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "ly alongside of th" }
-{ "l_orderkey": 2692, "l_partkey": 17, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 3, "l_extendedprice": 2751.03, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-25", "l_commitdate": "1998-01-29", "l_receiptdate": "1998-03-27", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "equests. bold, even foxes haggle slyl" }
-{ "l_orderkey": 2694, "l_partkey": 153, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 30, "l_extendedprice": 31594.5, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-20", "l_commitdate": "1996-06-01", "l_receiptdate": "1996-07-15", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "oxes. never iro" }
-{ "l_orderkey": 2759, "l_partkey": 59, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 10, "l_extendedprice": 9590.5, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-12-14", "l_commitdate": "1994-01-08", "l_receiptdate": "1994-01-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "s. busily ironic theodo" }
-{ "l_orderkey": 2819, "l_partkey": 70, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 17, "l_extendedprice": 16491.19, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-07-16", "l_commitdate": "1994-07-15", "l_receiptdate": "1994-07-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "en deposits above the f" }
-{ "l_orderkey": 2850, "l_partkey": 97, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 43, "l_extendedprice": 42874.87, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-11", "l_commitdate": "1996-11-03", "l_receiptdate": "1997-02-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "unusual accounts" }
-{ "l_orderkey": 2886, "l_partkey": 60, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 1, "l_extendedprice": 960.06, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-02-01", "l_commitdate": "1994-12-18", "l_receiptdate": "1995-02-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "eposits fr" }
-{ "l_orderkey": 2912, "l_partkey": 122, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 8, "l_extendedprice": 8176.96, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-04-09", "l_commitdate": "1992-04-19", "l_receiptdate": "1992-04-26", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "hs cajole over the slyl" }
-{ "l_orderkey": 2944, "l_partkey": 120, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 44, "l_extendedprice": 44885.28, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-25", "l_commitdate": "1997-10-28", "l_receiptdate": "1998-01-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ickly special theodolit" }
-{ "l_orderkey": 2947, "l_partkey": 10, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 37, "l_extendedprice": 33670.37, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-09", "l_commitdate": "1995-07-05", "l_receiptdate": "1995-08-20", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "e accounts: expres" }
-{ "l_orderkey": 2950, "l_partkey": 130, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 32, "l_extendedprice": 32964.16, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-21", "l_commitdate": "1997-08-25", "l_receiptdate": "1997-10-08", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "its wake carefully slyly final ideas." }
-{ "l_orderkey": 2978, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 29, "l_extendedprice": 28712.61, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-06-03", "l_commitdate": "1995-07-25", "l_receiptdate": "1995-06-06", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ecial ideas promise slyly" }
-{ "l_orderkey": 3143, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 22, "l_extendedprice": 21781.98, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-11", "l_commitdate": "1993-03-26", "l_receiptdate": "1993-05-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "l, special instructions nag " }
-{ "l_orderkey": 3264, "l_partkey": 200, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 39, "l_extendedprice": 42907.8, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-07", "l_commitdate": "1996-12-12", "l_receiptdate": "1996-11-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "sleep carefully after the slyly final" }
-{ "l_orderkey": 3266, "l_partkey": 64, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 31, "l_extendedprice": 29885.86, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-06-19", "l_commitdate": "1995-05-04", "l_receiptdate": "1995-07-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "grate among the quickly express deposits" }
-{ "l_orderkey": 3270, "l_partkey": 35, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 11, "l_extendedprice": 10285.33, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-29", "l_commitdate": "1997-08-11", "l_receiptdate": "1997-08-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": " solve at the regular deposits. " }
-{ "l_orderkey": 3364, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 49, "l_extendedprice": 48514.41, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-17", "l_commitdate": "1997-08-23", "l_receiptdate": "1997-10-06", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "d accounts? caref" }
-{ "l_orderkey": 3366, "l_partkey": 40, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 4, "l_extendedprice": 3760.16, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-20", "l_commitdate": "1997-06-25", "l_receiptdate": "1997-06-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": " carefully about " }
-{ "l_orderkey": 3425, "l_partkey": 120, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 11, "l_extendedprice": 11221.32, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-24", "l_commitdate": "1996-05-29", "l_receiptdate": "1996-05-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ckly final deposits use quickly?" }
-{ "l_orderkey": 3460, "l_partkey": 11, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 40, "l_extendedprice": 36440.4, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-28", "l_commitdate": "1995-12-14", "l_receiptdate": "1996-01-02", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "odolites are slyly bold deposits" }
-{ "l_orderkey": 3494, "l_partkey": 117, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 40, "l_extendedprice": 40684.4, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-07-10", "l_commitdate": "1993-06-01", "l_receiptdate": "1993-07-25", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "lites haggle furiously about the fin" }
-{ "l_orderkey": 3520, "l_partkey": 28, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 30, "l_extendedprice": 27840.6, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-11", "l_commitdate": "1997-10-02", "l_receiptdate": "1997-12-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "deas should solve blithely among the ironi" }
-{ "l_orderkey": 3559, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 29, "l_extendedprice": 28712.61, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-12-10", "l_commitdate": "1992-12-03", "l_receiptdate": "1992-12-20", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "l, regular accounts wake flu" }
-{ "l_orderkey": 3585, "l_partkey": 122, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 21, "l_extendedprice": 21464.52, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-04", "l_commitdate": "1994-12-25", "l_receiptdate": "1995-01-01", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ounts use. express, final platelets us" }
-{ "l_orderkey": 3618, "l_partkey": 140, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 38, "l_extendedprice": 39525.32, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-22", "l_commitdate": "1998-02-23", "l_receiptdate": "1998-01-03", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "nts haggle fluffily above the regular " }
-{ "l_orderkey": 3715, "l_partkey": 97, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 13, "l_extendedprice": 12962.17, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-11", "l_commitdate": "1996-04-25", "l_receiptdate": "1996-06-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "e quickly ironic" }
-{ "l_orderkey": 3844, "l_partkey": 135, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 2, "l_extendedprice": 2070.26, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-02-24", "l_commitdate": "1995-02-03", "l_receiptdate": "1995-03-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "es haggle final acco" }
-{ "l_orderkey": 3878, "l_partkey": 200, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 6, "l_extendedprice": 6601.2, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-21", "l_commitdate": "1997-05-22", "l_receiptdate": "1997-07-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "s. regular instru" }
-{ "l_orderkey": 3906, "l_partkey": 153, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 42, "l_extendedprice": 44232.3, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-09-03", "l_commitdate": "1992-07-22", "l_receiptdate": "1992-09-04", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "jole blithely after the furiously regular " }
-{ "l_orderkey": 3974, "l_partkey": 22, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 47, "l_extendedprice": 43334.94, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-03", "l_commitdate": "1996-05-08", "l_receiptdate": "1996-06-28", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "dencies above the re" }
-{ "l_orderkey": 4001, "l_partkey": 106, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 26, "l_extendedprice": 26158.6, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-26", "l_commitdate": "1997-06-18", "l_receiptdate": "1997-08-08", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "tegrate blithely" }
-{ "l_orderkey": 4005, "l_partkey": 4, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 26, "l_extendedprice": 23504.0, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-01", "l_commitdate": "1997-02-03", "l_receiptdate": "1996-12-15", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": " to the quic" }
-{ "l_orderkey": 4033, "l_partkey": 110, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 27, "l_extendedprice": 27272.97, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-08-08", "l_commitdate": "1993-08-14", "l_receiptdate": "1993-08-09", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "pinto beans" }
-{ "l_orderkey": 4034, "l_partkey": 190, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 48, "l_extendedprice": 52329.12, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-03-01", "l_commitdate": "1994-01-16", "l_receiptdate": "1994-03-16", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": " blithely regular requests play carefull" }
-{ "l_orderkey": 4036, "l_partkey": 6, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 46, "l_extendedprice": 41676.0, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-21", "l_commitdate": "1997-05-29", "l_receiptdate": "1997-07-18", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "usly across the even th" }
-{ "l_orderkey": 4064, "l_partkey": 199, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 3, "l_extendedprice": 3297.57, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-04", "l_commitdate": "1997-01-01", "l_receiptdate": "1997-01-23", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "its! quickly sp" }
-{ "l_orderkey": 4067, "l_partkey": 180, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 18, "l_extendedprice": 19443.24, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-01-24", "l_commitdate": "1992-12-23", "l_receiptdate": "1993-02-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "e the slyly final packages d" }
-{ "l_orderkey": 4068, "l_partkey": 110, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 43, "l_extendedprice": 43434.73, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-28", "l_commitdate": "1996-11-16", "l_receiptdate": "1996-12-22", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ructions. regular, special packag" }
-{ "l_orderkey": 4098, "l_partkey": 200, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 46, "l_extendedprice": 50609.2, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-26", "l_commitdate": "1997-01-27", "l_receiptdate": "1997-02-13", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "e slyly blithely silent deposits. fluff" }
-{ "l_orderkey": 4192, "l_partkey": 11, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 36, "l_extendedprice": 32796.36, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-25", "l_commitdate": "1998-05-26", "l_receiptdate": "1998-05-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "eodolites sleep" }
-{ "l_orderkey": 4194, "l_partkey": 197, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 43, "l_extendedprice": 47179.17, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-11-06", "l_commitdate": "1994-12-09", "l_receiptdate": "1994-11-16", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "olites are after the exp" }
-{ "l_orderkey": 4261, "l_partkey": 110, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 12, "l_extendedprice": 12121.32, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-11-01", "l_commitdate": "1993-01-01", "l_receiptdate": "1992-11-12", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "into beans " }
-{ "l_orderkey": 4418, "l_partkey": 35, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 32, "l_extendedprice": 29920.96, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-28", "l_commitdate": "1993-06-02", "l_receiptdate": "1993-05-30", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ly. bold pinto b" }
-{ "l_orderkey": 4422, "l_partkey": 135, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 5, "l_extendedprice": 5175.65, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-17", "l_commitdate": "1995-08-13", "l_receiptdate": "1995-07-25", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "e furiously about t" }
-{ "l_orderkey": 4486, "l_partkey": 135, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 46, "l_extendedprice": 47615.98, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-02", "l_commitdate": "1998-04-05", "l_receiptdate": "1998-05-08", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ackages. specia" }
-{ "l_orderkey": 4512, "l_partkey": 162, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 30, "l_extendedprice": 31864.8, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-28", "l_commitdate": "1995-12-22", "l_receiptdate": "1996-02-22", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ly unusual package" }
-{ "l_orderkey": 4513, "l_partkey": 170, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 29, "l_extendedprice": 31034.93, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-18", "l_commitdate": "1996-05-23", "l_receiptdate": "1996-06-08", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "cajole. regular packages boost. s" }
-{ "l_orderkey": 4545, "l_partkey": 173, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 38, "l_extendedprice": 40780.46, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-27", "l_commitdate": "1993-03-01", "l_receiptdate": "1993-02-04", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "nts serve according to th" }
-{ "l_orderkey": 4549, "l_partkey": 159, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 44, "l_extendedprice": 46602.6, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-13", "l_commitdate": "1998-04-15", "l_receiptdate": "1998-03-27", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ding to the regular, silent requests" }
-{ "l_orderkey": 4551, "l_partkey": 11, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 6, "l_extendedprice": 5466.06, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-18", "l_commitdate": "1996-04-23", "l_receiptdate": "1996-06-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "fily silent fo" }
-{ "l_orderkey": 4576, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 5, "l_extendedprice": 4950.45, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-23", "l_commitdate": "1996-11-08", "l_receiptdate": "1996-09-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "ly express, special asymptote" }
-{ "l_orderkey": 4608, "l_partkey": 173, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 30, "l_extendedprice": 32195.1, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-10-08", "l_commitdate": "1994-07-18", "l_receiptdate": "1994-10-25", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "s cajole. slyly " }
-{ "l_orderkey": 4641, "l_partkey": 190, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 45, "l_extendedprice": 49058.55, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-05-11", "l_commitdate": "1993-04-19", "l_receiptdate": "1993-05-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": " about the close " }
-{ "l_orderkey": 4679, "l_partkey": 190, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 7, "l_extendedprice": 7631.33, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-05-11", "l_commitdate": "1993-04-11", "l_receiptdate": "1993-05-16", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "kages. bold, regular packa" }
-{ "l_orderkey": 4769, "l_partkey": 35, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 16, "l_extendedprice": 14960.48, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-16", "l_commitdate": "1995-07-05", "l_receiptdate": "1995-07-22", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": " deposits. slyly even asymptote" }
-{ "l_orderkey": 4802, "l_partkey": 40, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 6, "l_extendedprice": 5640.24, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-16", "l_commitdate": "1997-03-25", "l_receiptdate": "1997-04-21", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "unusual accounts wake blithely. b" }
-{ "l_orderkey": 4804, "l_partkey": 128, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 44, "l_extendedprice": 45237.28, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-05-02", "l_commitdate": "1992-03-24", "l_receiptdate": "1992-05-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "aggle quickly among the slyly fi" }
-{ "l_orderkey": 4805, "l_partkey": 150, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 7, "l_extendedprice": 7351.05, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-05-01", "l_commitdate": "1992-07-09", "l_receiptdate": "1992-05-09", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": " requests. regular deposit" }
-{ "l_orderkey": 4807, "l_partkey": 122, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 9, "l_extendedprice": 9199.08, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-23", "l_commitdate": "1997-03-01", "l_receiptdate": "1997-05-15", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "may are blithely. carefully even pinto b" }
-{ "l_orderkey": 4836, "l_partkey": 162, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 22, "l_extendedprice": 23367.52, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-03", "l_commitdate": "1997-02-23", "l_receiptdate": "1997-03-04", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "al pinto beans. care" }
-{ "l_orderkey": 4837, "l_partkey": 42, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 16, "l_extendedprice": 15072.64, "l_discount": 0.09, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-12", "l_commitdate": "1998-06-06", "l_receiptdate": "1998-08-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ing requests are blithely regular instructi" }
-{ "l_orderkey": 4898, "l_partkey": 72, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 44, "l_extendedprice": 42771.08, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-09-13", "l_commitdate": "1994-08-18", "l_receiptdate": "1994-09-16", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "y regular grouches about" }
-{ "l_orderkey": 4928, "l_partkey": 100, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 4, "l_extendedprice": 4000.4, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-10-25", "l_commitdate": "1993-12-24", "l_receiptdate": "1993-11-16", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "bout the slyly final accounts. carefull" }
-{ "l_orderkey": 4929, "l_partkey": 14, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 20, "l_extendedprice": 18280.2, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-12", "l_commitdate": "1996-05-23", "l_receiptdate": "1996-03-20", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": " final pinto beans detect. final," }
-{ "l_orderkey": 4967, "l_partkey": 71, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 50, "l_extendedprice": 48553.5, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-27", "l_commitdate": "1997-05-13", "l_receiptdate": "1997-06-12", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "kages. final, unusual accounts c" }
-{ "l_orderkey": 4996, "l_partkey": 56, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 35, "l_extendedprice": 33461.75, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-10-30", "l_commitdate": "1992-10-27", "l_receiptdate": "1992-11-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "s. unusual, regular dolphins integrate care" }
-{ "l_orderkey": 5028, "l_partkey": 14, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 15, "l_extendedprice": 13710.15, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-07-17", "l_commitdate": "1992-07-16", "l_receiptdate": "1992-08-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "es are quickly final pains. furiously pend" }
-{ "l_orderkey": 5031, "l_partkey": 50, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 15, "l_extendedprice": 14250.75, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-04-01", "l_commitdate": "1995-02-24", "l_receiptdate": "1995-04-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "yly pending theodolites." }
-{ "l_orderkey": 5092, "l_partkey": 164, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 30, "l_extendedprice": 31924.8, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-27", "l_commitdate": "1995-12-08", "l_receiptdate": "1996-01-09", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ss, ironic deposits. furiously stea" }
-{ "l_orderkey": 5153, "l_partkey": 35, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 42, "l_extendedprice": 39271.26, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-03", "l_commitdate": "1995-11-09", "l_receiptdate": "1995-10-11", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "re thinly. ironic" }
-{ "l_orderkey": 5154, "l_partkey": 190, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 11, "l_extendedprice": 11992.09, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-06", "l_commitdate": "1997-06-30", "l_receiptdate": "1997-09-04", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "luffily bold foxes. final" }
-{ "l_orderkey": 5185, "l_partkey": 197, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 37, "l_extendedprice": 40596.03, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-08", "l_commitdate": "1997-09-08", "l_receiptdate": "1997-08-14", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "gainst the courts dazzle care" }
-{ "l_orderkey": 5187, "l_partkey": 11, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 49, "l_extendedprice": 44639.49, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-20", "l_commitdate": "1997-10-12", "l_receiptdate": "1997-10-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "l, regular platelets instead of the foxes w" }
-{ "l_orderkey": 5190, "l_partkey": 56, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 43, "l_extendedprice": 41110.15, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-08-19", "l_commitdate": "1992-06-10", "l_receiptdate": "1992-09-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "encies use fluffily unusual requests? hoc" }
-{ "l_orderkey": 5217, "l_partkey": 80, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 50, "l_extendedprice": 49004.0, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-26", "l_commitdate": "1995-11-21", "l_receiptdate": "1996-01-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "s. express, express accounts c" }
-{ "l_orderkey": 5281, "l_partkey": 114, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 37, "l_extendedprice": 37522.07, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-10", "l_commitdate": "1996-01-31", "l_receiptdate": "1995-11-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ronic dependencies. fluffily final p" }
-{ "l_orderkey": 5284, "l_partkey": 173, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 16, "l_extendedprice": 17170.72, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-17", "l_commitdate": "1995-08-23", "l_receiptdate": "1995-08-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "unts detect furiously even d" }
-{ "l_orderkey": 5286, "l_partkey": 199, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 1, "l_extendedprice": 1099.19, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-25", "l_commitdate": "1997-11-07", "l_receiptdate": "1997-12-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ly! furiously final pack" }
-{ "l_orderkey": 5313, "l_partkey": 17, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 34, "l_extendedprice": 31178.34, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-07", "l_commitdate": "1997-08-12", "l_receiptdate": "1997-08-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ccording to the blithely final account" }
-{ "l_orderkey": 5315, "l_partkey": 35, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 12, "l_extendedprice": 11220.36, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-12-18", "l_commitdate": "1993-01-16", "l_receiptdate": "1993-01-10", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ccounts. furiously ironi" }
-{ "l_orderkey": 5316, "l_partkey": 108, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 29, "l_extendedprice": 29234.9, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-28", "l_commitdate": "1994-04-29", "l_receiptdate": "1994-04-09", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "ckly unusual foxes bo" }
-{ "l_orderkey": 5379, "l_partkey": 199, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 40, "l_extendedprice": 43967.6, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-01", "l_commitdate": "1995-10-19", "l_receiptdate": "1995-10-30", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "carefully final accounts haggle blithely. " }
-{ "l_orderkey": 5441, "l_partkey": 164, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 3, "l_extendedprice": 3192.48, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-08-12", "l_commitdate": "1994-10-14", "l_receiptdate": "1994-09-01", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "are. unusual, " }
-{ "l_orderkey": 5445, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 33, "l_extendedprice": 32672.97, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-21", "l_commitdate": "1993-10-14", "l_receiptdate": "1993-10-28", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "ests. final instructions" }
-{ "l_orderkey": 5446, "l_partkey": 190, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 27, "l_extendedprice": 29435.13, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-07-21", "l_commitdate": "1994-08-25", "l_receiptdate": "1994-08-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ously across the quic" }
-{ "l_orderkey": 5447, "l_partkey": 99, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 31, "l_extendedprice": 30971.79, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-14", "l_commitdate": "1996-05-07", "l_receiptdate": "1996-07-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": " foxes sleep. blithely unusual accounts det" }
-{ "l_orderkey": 5476, "l_partkey": 48, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 13, "l_extendedprice": 12324.52, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-27", "l_commitdate": "1997-12-08", "l_receiptdate": "1997-12-29", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "iously special ac" }
-{ "l_orderkey": 5506, "l_partkey": 140, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 2, "l_extendedprice": 2080.28, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-04", "l_commitdate": "1994-01-13", "l_receiptdate": "1994-02-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "onic theodolites are fluffil" }
-{ "l_orderkey": 5536, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 14, "l_extendedprice": 13861.26, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-18", "l_commitdate": "1998-05-08", "l_receiptdate": "1998-06-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "instructions sleep " }
-{ "l_orderkey": 5572, "l_partkey": 22, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 24, "l_extendedprice": 22128.48, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-10-30", "l_commitdate": "1994-10-02", "l_receiptdate": "1994-11-27", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "ests cajole. evenly ironic exc" }
-{ "l_orderkey": 5664, "l_partkey": 122, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 25, "l_extendedprice": 25553.0, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-10-29", "l_commitdate": "1998-09-23", "l_receiptdate": "1998-11-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "eposits: furiously ironic grouch" }
-{ "l_orderkey": 5670, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 27, "l_extendedprice": 26732.43, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-05-09", "l_commitdate": "1993-05-30", "l_receiptdate": "1993-06-06", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": " ideas promise bli" }
-{ "l_orderkey": 5728, "l_partkey": 44, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 47, "l_extendedprice": 44369.88, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-13", "l_commitdate": "1995-01-25", "l_receiptdate": "1994-12-25", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "nd the bravely final deposits. final ideas" }
-{ "l_orderkey": 5735, "l_partkey": 60, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 41, "l_extendedprice": 39362.46, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-23", "l_commitdate": "1995-02-10", "l_receiptdate": "1995-01-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "lthily ruthless i" }
-{ "l_orderkey": 5826, "l_partkey": 144, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 4, "l_extendedprice": 4176.56, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-31", "l_commitdate": "1998-09-10", "l_receiptdate": "1998-08-27", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": " packages across the fluffily spec" }
-{ "l_orderkey": 5829, "l_partkey": 40, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 4, "l_extendedprice": 3760.16, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-01", "l_commitdate": "1997-02-17", "l_receiptdate": "1997-03-22", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ithely; accounts cajole ideas. regular foxe" }
-{ "l_orderkey": 5856, "l_partkey": 4, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 1, "l_extendedprice": 904.0, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-29", "l_commitdate": "1995-01-07", "l_receiptdate": "1995-01-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "tly. special deposits wake blithely even" }
-{ "l_orderkey": 5926, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 8, "l_extendedprice": 7920.72, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-07-17", "l_commitdate": "1994-07-20", "l_receiptdate": "1994-08-11", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "gle furiously express foxes. bo" }
-{ "l_orderkey": 5927, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 44, "l_extendedprice": 43563.96, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-29", "l_commitdate": "1997-11-21", "l_receiptdate": "1997-12-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "rding to the special, final decoy" }
-{ "l_orderkey": 5955, "l_partkey": 140, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 14, "l_extendedprice": 14561.96, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-06-22", "l_commitdate": "1995-05-23", "l_receiptdate": "1995-06-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": " unusual, bold theodolit" }
-{ "l_orderkey": 5959, "l_partkey": 135, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 49, "l_extendedprice": 50721.37, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-07-16", "l_commitdate": "1992-08-09", "l_receiptdate": "1992-08-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "usual packages haggle slyly pi" }
-{ "l_orderkey": 5988, "l_partkey": 172, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 41, "l_extendedprice": 43958.97, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-01-20", "l_commitdate": "1994-02-06", "l_receiptdate": "1994-02-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "the pending, express reque" }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/results/temp-dataset/delete-from-loaded-dataset/delete-from-loaded-dataset.1.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temp-dataset/delete-from-loaded-dataset/delete-from-loaded-dataset.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temp-dataset/delete-from-loaded-dataset/delete-from-loaded-dataset.1.adm
deleted file mode 100644
index 0c2fa2d..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/temp-dataset/delete-from-loaded-dataset/delete-from-loaded-dataset.1.adm
+++ /dev/null
@@ -1,25 +0,0 @@
-{ "l_orderkey": 1, "l_partkey": 156, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 17, "l_extendedprice": 17954.55, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-13", "l_commitdate": "1996-02-12", "l_receiptdate": "1996-03-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "egular courts above the" }
-{ "l_orderkey": 1, "l_partkey": 68, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 36, "l_extendedprice": 34850.16, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-12", "l_commitdate": "1996-02-28", "l_receiptdate": "1996-04-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "ly final dependencies: slyly bold " }
-{ "l_orderkey": 1, "l_partkey": 64, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 8, "l_extendedprice": 7712.48, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-29", "l_commitdate": "1996-03-05", "l_receiptdate": "1996-01-31", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "riously. regular, express dep" }
-{ "l_orderkey": 1, "l_partkey": 3, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 28, "l_extendedprice": 25284.0, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-21", "l_commitdate": "1996-03-30", "l_receiptdate": "1996-05-16", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "lites. fluffily even de" }
-{ "l_orderkey": 1, "l_partkey": 25, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 24, "l_extendedprice": 22200.48, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-30", "l_commitdate": "1996-03-14", "l_receiptdate": "1996-04-01", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": " pending foxes. slyly re" }
-{ "l_orderkey": 1, "l_partkey": 16, "l_suppkey": 3, "l_linenumber": 6, "l_quantity": 32, "l_extendedprice": 29312.32, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-30", "l_commitdate": "1996-02-07", "l_receiptdate": "1996-02-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "arefully slyly ex" }
-{ "l_orderkey": 2, "l_partkey": 107, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 38, "l_extendedprice": 38269.8, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-28", "l_commitdate": "1997-01-14", "l_receiptdate": "1997-02-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ven requests. deposits breach a" }
-{ "l_orderkey": 3, "l_partkey": 5, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 45, "l_extendedprice": 40725.0, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-02", "l_commitdate": "1994-01-04", "l_receiptdate": "1994-02-23", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ongside of the furiously brave acco" }
-{ "l_orderkey": 3, "l_partkey": 20, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 49, "l_extendedprice": 45080.98, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-11-09", "l_commitdate": "1993-12-20", "l_receiptdate": "1993-11-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": " unusual accounts. eve" }
-{ "l_orderkey": 3, "l_partkey": 129, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 27, "l_extendedprice": 27786.24, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-16", "l_commitdate": "1993-11-22", "l_receiptdate": "1994-01-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "nal foxes wake. " }
-{ "l_orderkey": 3, "l_partkey": 30, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 2, "l_extendedprice": 1860.06, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-12-04", "l_commitdate": "1994-01-07", "l_receiptdate": "1994-01-01", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "y. fluffily pending d" }
-{ "l_orderkey": 3, "l_partkey": 184, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 28, "l_extendedprice": 30357.04, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-12-14", "l_commitdate": "1994-01-10", "l_receiptdate": "1994-01-01", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "ages nag slyly pending" }
-{ "l_orderkey": 3, "l_partkey": 63, "l_suppkey": 8, "l_linenumber": 6, "l_quantity": 26, "l_extendedprice": 25039.56, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-29", "l_commitdate": "1993-12-18", "l_receiptdate": "1993-11-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ges sleep after the caref" }
-{ "l_orderkey": 4, "l_partkey": 89, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 30, "l_extendedprice": 29672.4, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-10", "l_commitdate": "1995-12-14", "l_receiptdate": "1996-01-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "- quickly regular packages sleep. idly" }
-{ "l_orderkey": 5, "l_partkey": 109, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 15, "l_extendedprice": 15136.5, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-10-31", "l_commitdate": "1994-08-31", "l_receiptdate": "1994-11-20", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ts wake furiously " }
-{ "l_orderkey": 5, "l_partkey": 124, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 26, "l_extendedprice": 26627.12, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-10-16", "l_commitdate": "1994-09-25", "l_receiptdate": "1994-10-19", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "sts use slyly quickly special instruc" }
-{ "l_orderkey": 5, "l_partkey": 38, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 50, "l_extendedprice": 46901.5, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-08-08", "l_commitdate": "1994-10-13", "l_receiptdate": "1994-08-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "eodolites. fluffily unusual" }
-{ "l_orderkey": 6, "l_partkey": 140, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 37, "l_extendedprice": 38485.18, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-04-27", "l_commitdate": "1992-05-15", "l_receiptdate": "1992-05-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "p furiously special foxes" }
-{ "l_orderkey": 7, "l_partkey": 183, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 12, "l_extendedprice": 12998.16, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-07", "l_commitdate": "1996-03-13", "l_receiptdate": "1996-06-03", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "ss pinto beans wake against th" }
-{ "l_orderkey": 7, "l_partkey": 146, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 9, "l_extendedprice": 9415.26, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-01", "l_commitdate": "1996-03-02", "l_receiptdate": "1996-02-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "es. instructions" }
-{ "l_orderkey": 7, "l_partkey": 95, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 46, "l_extendedprice": 45774.14, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-15", "l_commitdate": "1996-03-27", "l_receiptdate": "1996-02-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": " unusual reques" }
-{ "l_orderkey": 7, "l_partkey": 164, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 28, "l_extendedprice": 29796.48, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-21", "l_commitdate": "1996-04-08", "l_receiptdate": "1996-04-20", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": ". slyly special requests haggl" }
-{ "l_orderkey": 7, "l_partkey": 152, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 38, "l_extendedprice": 39981.7, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-11", "l_commitdate": "1996-02-24", "l_receiptdate": "1996-02-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "ns haggle carefully ironic deposits. bl" }
-{ "l_orderkey": 7, "l_partkey": 80, "l_suppkey": 10, "l_linenumber": 6, "l_quantity": 35, "l_extendedprice": 34302.8, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-16", "l_commitdate": "1996-02-23", "l_receiptdate": "1996-01-22", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "jole. excuses wake carefully alongside of " }
-{ "l_orderkey": 7, "l_partkey": 158, "l_suppkey": 3, "l_linenumber": 7, "l_quantity": 5, "l_extendedprice": 5290.75, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-10", "l_commitdate": "1996-03-26", "l_receiptdate": "1996-02-13", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ithely regula" }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/results/temp-dataset/drop-empty-secondary-indexes/drop-empty-secondary-indexes.1.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temp-dataset/drop-empty-secondary-indexes/drop-empty-secondary-indexes.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temp-dataset/drop-empty-secondary-indexes/drop-empty-secondary-indexes.1.adm
deleted file mode 100644
index e69de29..0000000

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/results/temp-dataset/drop-index/drop-index.1.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temp-dataset/drop-index/drop-index.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temp-dataset/drop-index/drop-index.1.adm
deleted file mode 100644
index f3c688b..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/temp-dataset/drop-index/drop-index.1.adm
+++ /dev/null
@@ -1 +0,0 @@
-{ "unique1": 84, "unique2": 10, "two": 0, "four": 0, "ten": 4, "twenty": 4, "onePercent": 84, "tenPercent": 4, "twentyPercent": 4, "fiftyPercent": 0, "unique3": 84, "evenOnePercent": 168, "oddOnePercent": 169, "stringu1": "DGAAAAXXXXXXXXXXXXXXXXXXX", "stringu2": "KAAAAAXXXXXXXXXXXXXXXXXXX", "string4": "OOOOXXXXXXXXXXXXXXXXXXXXXX" }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-app/src/test/resources/runtimets/results/temp-dataset/empty-load-with-index/empty-load-with-index.1.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temp-dataset/empty-load-with-index/empty-load-with-index.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temp-dataset/empty-load-with-index/empty-load-with-index.1.adm
deleted file mode 100644
index 8344a9f..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/temp-dataset/empty-load-with-index/empty-load-with-index.1.adm
+++ /dev/null
@@ -1 +0,0 @@
-{ "l_orderkey": 1, "l_partkey": 156, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 17.0, "l_extendedprice": 17954.55, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-13", "l_commitdate": "1996-02-12", "l_receiptdate": "1996-03-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "egular courts above the" }


[03/12] asterixdb git commit: [ASTERIXDB-2171][STO] Remove Temporary Datasets

Posted by mh...@apache.org.
http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_secondary_index_recovery/secondary_index_recovery.3.script.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_secondary_index_recovery/secondary_index_recovery.3.script.aql b/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_secondary_index_recovery/secondary_index_recovery.3.script.aql
deleted file mode 100644
index 4583455..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_secondary_index_recovery/secondary_index_recovery.3.script.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.
-#
-kill_cc_and_nc.sh
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_secondary_index_recovery/secondary_index_recovery.4.script.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_secondary_index_recovery/secondary_index_recovery.4.script.aql b/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_secondary_index_recovery/secondary_index_recovery.4.script.aql
deleted file mode 100644
index d844d13..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_secondary_index_recovery/secondary_index_recovery.4.script.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.
-#
-stop_and_start.sh

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_secondary_index_recovery/secondary_index_recovery.5.errddl.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_secondary_index_recovery/secondary_index_recovery.5.errddl.aql b/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_secondary_index_recovery/secondary_index_recovery.5.errddl.aql
deleted file mode 100644
index bcc8223..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_secondary_index_recovery/secondary_index_recovery.5.errddl.aql
+++ /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.
- */
-use dataverse SampleDV;
-drop index SampleDS.SampleDSix;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_secondary_index_recovery/secondary_index_recovery.6.script.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_secondary_index_recovery/secondary_index_recovery.6.script.aql b/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_secondary_index_recovery/secondary_index_recovery.6.script.aql
deleted file mode 100644
index f43dec7..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/queries/recovery_ddl/temp_secondary_index_recovery/secondary_index_recovery.6.script.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.
-#
-stop_and_delete.sh

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/results/recovery_ddl/temp_delete_after_recovery/delete_after_recovery.1.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/results/recovery_ddl/temp_delete_after_recovery/delete_after_recovery.1.adm b/asterixdb/asterix-installer/src/test/resources/transactionts/results/recovery_ddl/temp_delete_after_recovery/delete_after_recovery.1.adm
deleted file mode 100644
index b1646d9..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/results/recovery_ddl/temp_delete_after_recovery/delete_after_recovery.1.adm
+++ /dev/null
@@ -1 +0,0 @@
-129088

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/results/recovery_ddl/temp_insert_after_recovery/insert_after_recovery.1.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/results/recovery_ddl/temp_insert_after_recovery/insert_after_recovery.1.adm b/asterixdb/asterix-installer/src/test/resources/transactionts/results/recovery_ddl/temp_insert_after_recovery/insert_after_recovery.1.adm
deleted file mode 100644
index 78b4996..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/results/recovery_ddl/temp_insert_after_recovery/insert_after_recovery.1.adm
+++ /dev/null
@@ -1 +0,0 @@
-258176

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/results/recovery_ddl/temp_load_after_recovery/load_after_recovery.1.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/results/recovery_ddl/temp_load_after_recovery/load_after_recovery.1.adm b/asterixdb/asterix-installer/src/test/resources/transactionts/results/recovery_ddl/temp_load_after_recovery/load_after_recovery.1.adm
deleted file mode 100644
index 78b4996..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/results/recovery_ddl/temp_load_after_recovery/load_after_recovery.1.adm
+++ /dev/null
@@ -1 +0,0 @@
-258176

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recover_after_abort/temp_primary_index_only/create_and_start.sh
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recover_after_abort/temp_primary_index_only/create_and_start.sh b/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recover_after_abort/temp_primary_index_only/create_and_start.sh
deleted file mode 100755
index 789914b..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recover_after_abort/temp_primary_index_only/create_and_start.sh
+++ /dev/null
@@ -1,18 +0,0 @@
-#!/usr/bin/env bash
-# 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.
-$MANAGIX_HOME/bin/managix create -n asterix -c $MANAGIX_HOME/clusters/local/local.xml;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recover_after_abort/temp_primary_index_only/kill_cc_and_nc.sh
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recover_after_abort/temp_primary_index_only/kill_cc_and_nc.sh b/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recover_after_abort/temp_primary_index_only/kill_cc_and_nc.sh
deleted file mode 100755
index 4b876be..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recover_after_abort/temp_primary_index_only/kill_cc_and_nc.sh
+++ /dev/null
@@ -1,18 +0,0 @@
-#!/usr/bin/env bash
-# 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.
-ps -ef | awk '/java.*org\.apache\.hyracks\.control\.[cn]c\.[CN]CDriver/ {print $2}' | xargs -n 1 kill -9

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recover_after_abort/temp_primary_index_only/stop_and_delete.sh
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recover_after_abort/temp_primary_index_only/stop_and_delete.sh b/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recover_after_abort/temp_primary_index_only/stop_and_delete.sh
deleted file mode 100755
index eb1c01e..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recover_after_abort/temp_primary_index_only/stop_and_delete.sh
+++ /dev/null
@@ -1,20 +0,0 @@
-#!/usr/bin/env bash
-# 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.
-$MANAGIX_HOME/bin/managix stop -n asterix;
-$MANAGIX_HOME/bin/managix delete -n asterix;
-

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recover_after_abort/temp_primary_index_only/stop_and_start.sh
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recover_after_abort/temp_primary_index_only/stop_and_start.sh b/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recover_after_abort/temp_primary_index_only/stop_and_start.sh
deleted file mode 100755
index 2fb80ce..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recover_after_abort/temp_primary_index_only/stop_and_start.sh
+++ /dev/null
@@ -1,20 +0,0 @@
-#!/usr/bin/env bash
-# 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.
-$MANAGIX_HOME/bin/managix stop -n asterix;
-$MANAGIX_HOME/bin/managix start -n asterix;
-

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recover_after_abort/temp_primary_plus_default_secondary_index/create_and_start.sh
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recover_after_abort/temp_primary_plus_default_secondary_index/create_and_start.sh b/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recover_after_abort/temp_primary_plus_default_secondary_index/create_and_start.sh
deleted file mode 100755
index 789914b..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recover_after_abort/temp_primary_plus_default_secondary_index/create_and_start.sh
+++ /dev/null
@@ -1,18 +0,0 @@
-#!/usr/bin/env bash
-# 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.
-$MANAGIX_HOME/bin/managix create -n asterix -c $MANAGIX_HOME/clusters/local/local.xml;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recover_after_abort/temp_primary_plus_default_secondary_index/kill_cc_and_nc.sh
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recover_after_abort/temp_primary_plus_default_secondary_index/kill_cc_and_nc.sh b/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recover_after_abort/temp_primary_plus_default_secondary_index/kill_cc_and_nc.sh
deleted file mode 100755
index 4b876be..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recover_after_abort/temp_primary_plus_default_secondary_index/kill_cc_and_nc.sh
+++ /dev/null
@@ -1,18 +0,0 @@
-#!/usr/bin/env bash
-# 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.
-ps -ef | awk '/java.*org\.apache\.hyracks\.control\.[cn]c\.[CN]CDriver/ {print $2}' | xargs -n 1 kill -9

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recover_after_abort/temp_primary_plus_default_secondary_index/stop_and_delete.sh
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recover_after_abort/temp_primary_plus_default_secondary_index/stop_and_delete.sh b/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recover_after_abort/temp_primary_plus_default_secondary_index/stop_and_delete.sh
deleted file mode 100755
index eb1c01e..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recover_after_abort/temp_primary_plus_default_secondary_index/stop_and_delete.sh
+++ /dev/null
@@ -1,20 +0,0 @@
-#!/usr/bin/env bash
-# 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.
-$MANAGIX_HOME/bin/managix stop -n asterix;
-$MANAGIX_HOME/bin/managix delete -n asterix;
-

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recover_after_abort/temp_primary_plus_default_secondary_index/stop_and_start.sh
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recover_after_abort/temp_primary_plus_default_secondary_index/stop_and_start.sh b/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recover_after_abort/temp_primary_plus_default_secondary_index/stop_and_start.sh
deleted file mode 100755
index 2fb80ce..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recover_after_abort/temp_primary_plus_default_secondary_index/stop_and_start.sh
+++ /dev/null
@@ -1,20 +0,0 @@
-#!/usr/bin/env bash
-# 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.
-$MANAGIX_HOME/bin/managix stop -n asterix;
-$MANAGIX_HOME/bin/managix start -n asterix;
-

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recover_after_abort/temp_primary_plus_keyword_secondary_index/create_and_start.sh
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recover_after_abort/temp_primary_plus_keyword_secondary_index/create_and_start.sh b/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recover_after_abort/temp_primary_plus_keyword_secondary_index/create_and_start.sh
deleted file mode 100755
index 789914b..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recover_after_abort/temp_primary_plus_keyword_secondary_index/create_and_start.sh
+++ /dev/null
@@ -1,18 +0,0 @@
-#!/usr/bin/env bash
-# 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.
-$MANAGIX_HOME/bin/managix create -n asterix -c $MANAGIX_HOME/clusters/local/local.xml;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recover_after_abort/temp_primary_plus_keyword_secondary_index/kill_cc_and_nc.sh
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recover_after_abort/temp_primary_plus_keyword_secondary_index/kill_cc_and_nc.sh b/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recover_after_abort/temp_primary_plus_keyword_secondary_index/kill_cc_and_nc.sh
deleted file mode 100755
index 4b876be..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recover_after_abort/temp_primary_plus_keyword_secondary_index/kill_cc_and_nc.sh
+++ /dev/null
@@ -1,18 +0,0 @@
-#!/usr/bin/env bash
-# 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.
-ps -ef | awk '/java.*org\.apache\.hyracks\.control\.[cn]c\.[CN]CDriver/ {print $2}' | xargs -n 1 kill -9

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recover_after_abort/temp_primary_plus_keyword_secondary_index/stop_and_delete.sh
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recover_after_abort/temp_primary_plus_keyword_secondary_index/stop_and_delete.sh b/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recover_after_abort/temp_primary_plus_keyword_secondary_index/stop_and_delete.sh
deleted file mode 100755
index eb1c01e..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recover_after_abort/temp_primary_plus_keyword_secondary_index/stop_and_delete.sh
+++ /dev/null
@@ -1,20 +0,0 @@
-#!/usr/bin/env bash
-# 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.
-$MANAGIX_HOME/bin/managix stop -n asterix;
-$MANAGIX_HOME/bin/managix delete -n asterix;
-

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recover_after_abort/temp_primary_plus_keyword_secondary_index/stop_and_start.sh
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recover_after_abort/temp_primary_plus_keyword_secondary_index/stop_and_start.sh b/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recover_after_abort/temp_primary_plus_keyword_secondary_index/stop_and_start.sh
deleted file mode 100755
index 2fb80ce..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recover_after_abort/temp_primary_plus_keyword_secondary_index/stop_and_start.sh
+++ /dev/null
@@ -1,20 +0,0 @@
-#!/usr/bin/env bash
-# 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.
-$MANAGIX_HOME/bin/managix stop -n asterix;
-$MANAGIX_HOME/bin/managix start -n asterix;
-

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recover_after_abort/temp_primary_plus_multiple_secondary_indices/create_and_start.sh
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recover_after_abort/temp_primary_plus_multiple_secondary_indices/create_and_start.sh b/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recover_after_abort/temp_primary_plus_multiple_secondary_indices/create_and_start.sh
deleted file mode 100755
index 789914b..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recover_after_abort/temp_primary_plus_multiple_secondary_indices/create_and_start.sh
+++ /dev/null
@@ -1,18 +0,0 @@
-#!/usr/bin/env bash
-# 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.
-$MANAGIX_HOME/bin/managix create -n asterix -c $MANAGIX_HOME/clusters/local/local.xml;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recover_after_abort/temp_primary_plus_multiple_secondary_indices/kill_cc_and_nc.sh
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recover_after_abort/temp_primary_plus_multiple_secondary_indices/kill_cc_and_nc.sh b/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recover_after_abort/temp_primary_plus_multiple_secondary_indices/kill_cc_and_nc.sh
deleted file mode 100755
index 4b876be..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recover_after_abort/temp_primary_plus_multiple_secondary_indices/kill_cc_and_nc.sh
+++ /dev/null
@@ -1,18 +0,0 @@
-#!/usr/bin/env bash
-# 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.
-ps -ef | awk '/java.*org\.apache\.hyracks\.control\.[cn]c\.[CN]CDriver/ {print $2}' | xargs -n 1 kill -9

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recover_after_abort/temp_primary_plus_multiple_secondary_indices/stop_and_delete.sh
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recover_after_abort/temp_primary_plus_multiple_secondary_indices/stop_and_delete.sh b/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recover_after_abort/temp_primary_plus_multiple_secondary_indices/stop_and_delete.sh
deleted file mode 100755
index eb1c01e..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recover_after_abort/temp_primary_plus_multiple_secondary_indices/stop_and_delete.sh
+++ /dev/null
@@ -1,20 +0,0 @@
-#!/usr/bin/env bash
-# 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.
-$MANAGIX_HOME/bin/managix stop -n asterix;
-$MANAGIX_HOME/bin/managix delete -n asterix;
-

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recover_after_abort/temp_primary_plus_multiple_secondary_indices/stop_and_start.sh
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recover_after_abort/temp_primary_plus_multiple_secondary_indices/stop_and_start.sh b/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recover_after_abort/temp_primary_plus_multiple_secondary_indices/stop_and_start.sh
deleted file mode 100755
index 2fb80ce..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recover_after_abort/temp_primary_plus_multiple_secondary_indices/stop_and_start.sh
+++ /dev/null
@@ -1,20 +0,0 @@
-#!/usr/bin/env bash
-# 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.
-$MANAGIX_HOME/bin/managix stop -n asterix;
-$MANAGIX_HOME/bin/managix start -n asterix;
-

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recover_after_abort/temp_primary_plus_ngram_index/create_and_start.sh
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recover_after_abort/temp_primary_plus_ngram_index/create_and_start.sh b/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recover_after_abort/temp_primary_plus_ngram_index/create_and_start.sh
deleted file mode 100755
index 789914b..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recover_after_abort/temp_primary_plus_ngram_index/create_and_start.sh
+++ /dev/null
@@ -1,18 +0,0 @@
-#!/usr/bin/env bash
-# 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.
-$MANAGIX_HOME/bin/managix create -n asterix -c $MANAGIX_HOME/clusters/local/local.xml;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recover_after_abort/temp_primary_plus_ngram_index/kill_cc_and_nc.sh
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recover_after_abort/temp_primary_plus_ngram_index/kill_cc_and_nc.sh b/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recover_after_abort/temp_primary_plus_ngram_index/kill_cc_and_nc.sh
deleted file mode 100755
index 4b876be..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recover_after_abort/temp_primary_plus_ngram_index/kill_cc_and_nc.sh
+++ /dev/null
@@ -1,18 +0,0 @@
-#!/usr/bin/env bash
-# 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.
-ps -ef | awk '/java.*org\.apache\.hyracks\.control\.[cn]c\.[CN]CDriver/ {print $2}' | xargs -n 1 kill -9

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recover_after_abort/temp_primary_plus_ngram_index/stop_and_delete.sh
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recover_after_abort/temp_primary_plus_ngram_index/stop_and_delete.sh b/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recover_after_abort/temp_primary_plus_ngram_index/stop_and_delete.sh
deleted file mode 100755
index eb1c01e..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recover_after_abort/temp_primary_plus_ngram_index/stop_and_delete.sh
+++ /dev/null
@@ -1,20 +0,0 @@
-#!/usr/bin/env bash
-# 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.
-$MANAGIX_HOME/bin/managix stop -n asterix;
-$MANAGIX_HOME/bin/managix delete -n asterix;
-

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recover_after_abort/temp_primary_plus_ngram_index/stop_and_start.sh
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recover_after_abort/temp_primary_plus_ngram_index/stop_and_start.sh b/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recover_after_abort/temp_primary_plus_ngram_index/stop_and_start.sh
deleted file mode 100755
index 2fb80ce..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recover_after_abort/temp_primary_plus_ngram_index/stop_and_start.sh
+++ /dev/null
@@ -1,20 +0,0 @@
-#!/usr/bin/env bash
-# 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.
-$MANAGIX_HOME/bin/managix stop -n asterix;
-$MANAGIX_HOME/bin/managix start -n asterix;
-

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recover_after_abort/temp_primary_plus_rtree_index/create_and_start.sh
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recover_after_abort/temp_primary_plus_rtree_index/create_and_start.sh b/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recover_after_abort/temp_primary_plus_rtree_index/create_and_start.sh
deleted file mode 100755
index 789914b..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recover_after_abort/temp_primary_plus_rtree_index/create_and_start.sh
+++ /dev/null
@@ -1,18 +0,0 @@
-#!/usr/bin/env bash
-# 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.
-$MANAGIX_HOME/bin/managix create -n asterix -c $MANAGIX_HOME/clusters/local/local.xml;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recover_after_abort/temp_primary_plus_rtree_index/kill_cc_and_nc.sh
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recover_after_abort/temp_primary_plus_rtree_index/kill_cc_and_nc.sh b/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recover_after_abort/temp_primary_plus_rtree_index/kill_cc_and_nc.sh
deleted file mode 100755
index 4b876be..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recover_after_abort/temp_primary_plus_rtree_index/kill_cc_and_nc.sh
+++ /dev/null
@@ -1,18 +0,0 @@
-#!/usr/bin/env bash
-# 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.
-ps -ef | awk '/java.*org\.apache\.hyracks\.control\.[cn]c\.[CN]CDriver/ {print $2}' | xargs -n 1 kill -9

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recover_after_abort/temp_primary_plus_rtree_index/stop_and_delete.sh
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recover_after_abort/temp_primary_plus_rtree_index/stop_and_delete.sh b/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recover_after_abort/temp_primary_plus_rtree_index/stop_and_delete.sh
deleted file mode 100755
index eb1c01e..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recover_after_abort/temp_primary_plus_rtree_index/stop_and_delete.sh
+++ /dev/null
@@ -1,20 +0,0 @@
-#!/usr/bin/env bash
-# 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.
-$MANAGIX_HOME/bin/managix stop -n asterix;
-$MANAGIX_HOME/bin/managix delete -n asterix;
-

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recover_after_abort/temp_primary_plus_rtree_index/stop_and_start.sh
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recover_after_abort/temp_primary_plus_rtree_index/stop_and_start.sh b/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recover_after_abort/temp_primary_plus_rtree_index/stop_and_start.sh
deleted file mode 100755
index 2fb80ce..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recover_after_abort/temp_primary_plus_rtree_index/stop_and_start.sh
+++ /dev/null
@@ -1,20 +0,0 @@
-#!/usr/bin/env bash
-# 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.
-$MANAGIX_HOME/bin/managix stop -n asterix;
-$MANAGIX_HOME/bin/managix start -n asterix;
-

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recovery_ddl/temp_dataset_recovery/create_and_start.sh
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recovery_ddl/temp_dataset_recovery/create_and_start.sh b/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recovery_ddl/temp_dataset_recovery/create_and_start.sh
deleted file mode 100755
index 789914b..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recovery_ddl/temp_dataset_recovery/create_and_start.sh
+++ /dev/null
@@ -1,18 +0,0 @@
-#!/usr/bin/env bash
-# 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.
-$MANAGIX_HOME/bin/managix create -n asterix -c $MANAGIX_HOME/clusters/local/local.xml;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recovery_ddl/temp_dataset_recovery/kill_cc_and_nc.sh
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recovery_ddl/temp_dataset_recovery/kill_cc_and_nc.sh b/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recovery_ddl/temp_dataset_recovery/kill_cc_and_nc.sh
deleted file mode 100755
index 4b876be..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recovery_ddl/temp_dataset_recovery/kill_cc_and_nc.sh
+++ /dev/null
@@ -1,18 +0,0 @@
-#!/usr/bin/env bash
-# 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.
-ps -ef | awk '/java.*org\.apache\.hyracks\.control\.[cn]c\.[CN]CDriver/ {print $2}' | xargs -n 1 kill -9

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recovery_ddl/temp_dataset_recovery/stop_and_delete.sh
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recovery_ddl/temp_dataset_recovery/stop_and_delete.sh b/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recovery_ddl/temp_dataset_recovery/stop_and_delete.sh
deleted file mode 100755
index eb1c01e..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recovery_ddl/temp_dataset_recovery/stop_and_delete.sh
+++ /dev/null
@@ -1,20 +0,0 @@
-#!/usr/bin/env bash
-# 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.
-$MANAGIX_HOME/bin/managix stop -n asterix;
-$MANAGIX_HOME/bin/managix delete -n asterix;
-

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recovery_ddl/temp_dataset_recovery/stop_and_start.sh
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recovery_ddl/temp_dataset_recovery/stop_and_start.sh b/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recovery_ddl/temp_dataset_recovery/stop_and_start.sh
deleted file mode 100755
index 2fb80ce..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recovery_ddl/temp_dataset_recovery/stop_and_start.sh
+++ /dev/null
@@ -1,20 +0,0 @@
-#!/usr/bin/env bash
-# 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.
-$MANAGIX_HOME/bin/managix stop -n asterix;
-$MANAGIX_HOME/bin/managix start -n asterix;
-

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recovery_ddl/temp_delete_after_recovery/create_and_start.sh
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recovery_ddl/temp_delete_after_recovery/create_and_start.sh b/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recovery_ddl/temp_delete_after_recovery/create_and_start.sh
deleted file mode 100755
index 789914b..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recovery_ddl/temp_delete_after_recovery/create_and_start.sh
+++ /dev/null
@@ -1,18 +0,0 @@
-#!/usr/bin/env bash
-# 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.
-$MANAGIX_HOME/bin/managix create -n asterix -c $MANAGIX_HOME/clusters/local/local.xml;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recovery_ddl/temp_delete_after_recovery/kill_cc_and_nc.sh
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recovery_ddl/temp_delete_after_recovery/kill_cc_and_nc.sh b/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recovery_ddl/temp_delete_after_recovery/kill_cc_and_nc.sh
deleted file mode 100755
index 4b876be..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recovery_ddl/temp_delete_after_recovery/kill_cc_and_nc.sh
+++ /dev/null
@@ -1,18 +0,0 @@
-#!/usr/bin/env bash
-# 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.
-ps -ef | awk '/java.*org\.apache\.hyracks\.control\.[cn]c\.[CN]CDriver/ {print $2}' | xargs -n 1 kill -9

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recovery_ddl/temp_delete_after_recovery/stop_and_delete.sh
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recovery_ddl/temp_delete_after_recovery/stop_and_delete.sh b/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recovery_ddl/temp_delete_after_recovery/stop_and_delete.sh
deleted file mode 100755
index eb1c01e..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recovery_ddl/temp_delete_after_recovery/stop_and_delete.sh
+++ /dev/null
@@ -1,20 +0,0 @@
-#!/usr/bin/env bash
-# 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.
-$MANAGIX_HOME/bin/managix stop -n asterix;
-$MANAGIX_HOME/bin/managix delete -n asterix;
-

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recovery_ddl/temp_delete_after_recovery/stop_and_start.sh
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recovery_ddl/temp_delete_after_recovery/stop_and_start.sh b/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recovery_ddl/temp_delete_after_recovery/stop_and_start.sh
deleted file mode 100755
index 2fb80ce..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recovery_ddl/temp_delete_after_recovery/stop_and_start.sh
+++ /dev/null
@@ -1,20 +0,0 @@
-#!/usr/bin/env bash
-# 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.
-$MANAGIX_HOME/bin/managix stop -n asterix;
-$MANAGIX_HOME/bin/managix start -n asterix;
-

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recovery_ddl/temp_insert_after_recovery/create_and_start.sh
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recovery_ddl/temp_insert_after_recovery/create_and_start.sh b/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recovery_ddl/temp_insert_after_recovery/create_and_start.sh
deleted file mode 100755
index 789914b..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recovery_ddl/temp_insert_after_recovery/create_and_start.sh
+++ /dev/null
@@ -1,18 +0,0 @@
-#!/usr/bin/env bash
-# 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.
-$MANAGIX_HOME/bin/managix create -n asterix -c $MANAGIX_HOME/clusters/local/local.xml;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recovery_ddl/temp_insert_after_recovery/kill_cc_and_nc.sh
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recovery_ddl/temp_insert_after_recovery/kill_cc_and_nc.sh b/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recovery_ddl/temp_insert_after_recovery/kill_cc_and_nc.sh
deleted file mode 100755
index 4b876be..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recovery_ddl/temp_insert_after_recovery/kill_cc_and_nc.sh
+++ /dev/null
@@ -1,18 +0,0 @@
-#!/usr/bin/env bash
-# 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.
-ps -ef | awk '/java.*org\.apache\.hyracks\.control\.[cn]c\.[CN]CDriver/ {print $2}' | xargs -n 1 kill -9

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recovery_ddl/temp_insert_after_recovery/stop_and_delete.sh
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recovery_ddl/temp_insert_after_recovery/stop_and_delete.sh b/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recovery_ddl/temp_insert_after_recovery/stop_and_delete.sh
deleted file mode 100755
index eb1c01e..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recovery_ddl/temp_insert_after_recovery/stop_and_delete.sh
+++ /dev/null
@@ -1,20 +0,0 @@
-#!/usr/bin/env bash
-# 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.
-$MANAGIX_HOME/bin/managix stop -n asterix;
-$MANAGIX_HOME/bin/managix delete -n asterix;
-

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recovery_ddl/temp_insert_after_recovery/stop_and_start.sh
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recovery_ddl/temp_insert_after_recovery/stop_and_start.sh b/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recovery_ddl/temp_insert_after_recovery/stop_and_start.sh
deleted file mode 100755
index 2fb80ce..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recovery_ddl/temp_insert_after_recovery/stop_and_start.sh
+++ /dev/null
@@ -1,20 +0,0 @@
-#!/usr/bin/env bash
-# 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.
-$MANAGIX_HOME/bin/managix stop -n asterix;
-$MANAGIX_HOME/bin/managix start -n asterix;
-

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recovery_ddl/temp_load_after_recovery/create_and_start.sh
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recovery_ddl/temp_load_after_recovery/create_and_start.sh b/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recovery_ddl/temp_load_after_recovery/create_and_start.sh
deleted file mode 100755
index 789914b..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recovery_ddl/temp_load_after_recovery/create_and_start.sh
+++ /dev/null
@@ -1,18 +0,0 @@
-#!/usr/bin/env bash
-# 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.
-$MANAGIX_HOME/bin/managix create -n asterix -c $MANAGIX_HOME/clusters/local/local.xml;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recovery_ddl/temp_load_after_recovery/kill_cc_and_nc.sh
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recovery_ddl/temp_load_after_recovery/kill_cc_and_nc.sh b/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recovery_ddl/temp_load_after_recovery/kill_cc_and_nc.sh
deleted file mode 100755
index 4b876be..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recovery_ddl/temp_load_after_recovery/kill_cc_and_nc.sh
+++ /dev/null
@@ -1,18 +0,0 @@
-#!/usr/bin/env bash
-# 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.
-ps -ef | awk '/java.*org\.apache\.hyracks\.control\.[cn]c\.[CN]CDriver/ {print $2}' | xargs -n 1 kill -9

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recovery_ddl/temp_load_after_recovery/stop_and_delete.sh
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recovery_ddl/temp_load_after_recovery/stop_and_delete.sh b/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recovery_ddl/temp_load_after_recovery/stop_and_delete.sh
deleted file mode 100755
index eb1c01e..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recovery_ddl/temp_load_after_recovery/stop_and_delete.sh
+++ /dev/null
@@ -1,20 +0,0 @@
-#!/usr/bin/env bash
-# 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.
-$MANAGIX_HOME/bin/managix stop -n asterix;
-$MANAGIX_HOME/bin/managix delete -n asterix;
-

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recovery_ddl/temp_load_after_recovery/stop_and_start.sh
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recovery_ddl/temp_load_after_recovery/stop_and_start.sh b/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recovery_ddl/temp_load_after_recovery/stop_and_start.sh
deleted file mode 100755
index 2fb80ce..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recovery_ddl/temp_load_after_recovery/stop_and_start.sh
+++ /dev/null
@@ -1,20 +0,0 @@
-#!/usr/bin/env bash
-# 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.
-$MANAGIX_HOME/bin/managix stop -n asterix;
-$MANAGIX_HOME/bin/managix start -n asterix;
-

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recovery_ddl/temp_secondary_index_recovery/create_and_start.sh
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recovery_ddl/temp_secondary_index_recovery/create_and_start.sh b/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recovery_ddl/temp_secondary_index_recovery/create_and_start.sh
deleted file mode 100755
index 789914b..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recovery_ddl/temp_secondary_index_recovery/create_and_start.sh
+++ /dev/null
@@ -1,18 +0,0 @@
-#!/usr/bin/env bash
-# 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.
-$MANAGIX_HOME/bin/managix create -n asterix -c $MANAGIX_HOME/clusters/local/local.xml;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recovery_ddl/temp_secondary_index_recovery/kill_cc_and_nc.sh
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recovery_ddl/temp_secondary_index_recovery/kill_cc_and_nc.sh b/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recovery_ddl/temp_secondary_index_recovery/kill_cc_and_nc.sh
deleted file mode 100755
index 4b876be..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recovery_ddl/temp_secondary_index_recovery/kill_cc_and_nc.sh
+++ /dev/null
@@ -1,18 +0,0 @@
-#!/usr/bin/env bash
-# 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.
-ps -ef | awk '/java.*org\.apache\.hyracks\.control\.[cn]c\.[CN]CDriver/ {print $2}' | xargs -n 1 kill -9

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recovery_ddl/temp_secondary_index_recovery/stop_and_delete.sh
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recovery_ddl/temp_secondary_index_recovery/stop_and_delete.sh b/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recovery_ddl/temp_secondary_index_recovery/stop_and_delete.sh
deleted file mode 100755
index eb1c01e..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recovery_ddl/temp_secondary_index_recovery/stop_and_delete.sh
+++ /dev/null
@@ -1,20 +0,0 @@
-#!/usr/bin/env bash
-# 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.
-$MANAGIX_HOME/bin/managix stop -n asterix;
-$MANAGIX_HOME/bin/managix delete -n asterix;
-

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recovery_ddl/temp_secondary_index_recovery/stop_and_start.sh
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recovery_ddl/temp_secondary_index_recovery/stop_and_start.sh b/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recovery_ddl/temp_secondary_index_recovery/stop_and_start.sh
deleted file mode 100755
index 2fb80ce..0000000
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/scripts/recovery_ddl/temp_secondary_index_recovery/stop_and_start.sh
+++ /dev/null
@@ -1,20 +0,0 @@
-#!/usr/bin/env bash
-# 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.
-$MANAGIX_HOME/bin/managix stop -n asterix;
-$MANAGIX_HOME/bin/managix start -n asterix;
-

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/2c04ae07/asterixdb/asterix-installer/src/test/resources/transactionts/testsuite.xml
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-installer/src/test/resources/transactionts/testsuite.xml b/asterixdb/asterix-installer/src/test/resources/transactionts/testsuite.xml
index bd68075..38179b2 100644
--- a/asterixdb/asterix-installer/src/test/resources/transactionts/testsuite.xml
+++ b/asterixdb/asterix-installer/src/test/resources/transactionts/testsuite.xml
@@ -158,41 +158,6 @@
     </test-case>
 
     <test-case FilePath="recover_after_abort">
-      <compilation-unit name="temp_primary_index_only">
-        <output-dir compare="Text">primary_index_only</output-dir>
-        <expected-error>org.apache.hyracks.algebricks.common.exceptions.AlgebricksException</expected-error>
-      </compilation-unit>
-    </test-case>
-
-    <test-case FilePath="recover_after_abort">
-      <compilation-unit name="temp_primary_plus_default_secondary_index">
-        <output-dir compare="Text">primary_plus_default_secondary_index</output-dir>
-        <expected-error>org.apache.hyracks.algebricks.common.exceptions.AlgebricksException</expected-error>
-      </compilation-unit>
-    </test-case>
-
-    <test-case FilePath="recover_after_abort">
-      <compilation-unit name="temp_primary_plus_rtree_index">
-        <output-dir compare="Text">primary_plus_rtree_index</output-dir>
-        <expected-error>org.apache.hyracks.algebricks.common.exceptions.AlgebricksException</expected-error>
-      </compilation-unit>
-    </test-case>
-
-    <test-case FilePath="recover_after_abort">
-      <compilation-unit name="temp_primary_plus_keyword_secondary_index">
-        <output-dir compare="Text">primary_plus_keyword_secondary_index</output-dir>
-        <expected-error>org.apache.hyracks.algebricks.common.exceptions.AlgebricksException: Could not find dataset Fragile in dataverse recovery</expected-error>
-      </compilation-unit>
-    </test-case>
-
-    <test-case FilePath="recover_after_abort">
-      <compilation-unit name="temp_primary_plus_ngram_index">
-        <output-dir compare="Text">primary_plus_ngram_index</output-dir>
-        <expected-error>org.apache.hyracks.algebricks.common.exceptions.AlgebricksException</expected-error>
-      </compilation-unit>
-    </test-case>
-
-    <test-case FilePath="recover_after_abort">
       <compilation-unit name="primary_plus_multiple_secondary_indices"><!-- The only exception here is during the kill command which is in a different JVM, hence not caught -->
         <output-dir compare="Text">primary_plus_multiple_secondary_indices</output-dir>
         <!-- <expected-error>org.apache.hyracks.algebricks.common.exceptions.AlgebricksException</expected-error> -->
@@ -249,45 +214,6 @@
         <output-dir compare="Text">function_recovery</output-dir>
       </compilation-unit>
     </test-case>
-
-    <test-case FilePath="recovery_ddl">
-      <compilation-unit name="temp_dataset_recovery">
-        <output-dir compare="Text">dataset_recovery</output-dir>
-        <!-- <expected-error>org.apache.hyracks.algebricks.common.exceptions.AlgebricksException</expected-error> -->
-      </compilation-unit>
-    </test-case>
-
-    <test-case FilePath="recovery_ddl">
-      <compilation-unit name="temp_delete_after_recovery">
-        <output-dir compare="Text">delete_after_recovery</output-dir>
-        <expected-error>org.apache.hyracks.algebricks.common.exceptions.AlgebricksException: Cannot find dataset Fragile_raw in dataverse recovery</expected-error>
-        <expected-error>org.apache.hyracks.algebricks.common.exceptions.AlgebricksException: Could not find dataset Fragile_raw in dataverse recovery</expected-error>
-      </compilation-unit>
-    </test-case>
-
-    <test-case FilePath="recovery_ddl">
-      <compilation-unit name="temp_insert_after_recovery">
-        <output-dir compare="Text">insert_after_recovery</output-dir>
-        <expected-error>org.apache.hyracks.algebricks.common.exceptions.AlgebricksException</expected-error>
-        <expected-error>org.apache.hyracks.algebricks.common.exceptions.AlgebricksException: Could not find dataset Fragile in dataverse recovery</expected-error>
-      </compilation-unit>
-    </test-case>
-
-    <test-case FilePath="recovery_ddl">
-      <compilation-unit name="temp_load_after_recovery">
-        <output-dir compare="Text">load_after_recovery</output-dir>
-        <expected-error>org.apache.hyracks.algebricks.common.exceptions.AlgebricksException</expected-error>
-        <expected-error>org.apache.hyracks.algebricks.common.exceptions.AlgebricksException: Could not find dataset Fragile_raw in dataverse recovery</expected-error>
-      </compilation-unit>
-    </test-case>
-
-    <test-case FilePath="recovery_ddl">
-      <compilation-unit name="temp_secondary_index_recovery">
-        <output-dir compare="Text">secondary_index_recovery</output-dir>
-        <!-- <expected-error>org.apache.hyracks.algebricks.common.exceptions.AlgebricksException</expected-error> -->
-      </compilation-unit>
-    </test-case>
-
   </test-group>
 
 </test-suite>