You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@asterixdb.apache.org by mh...@apache.org on 2018/11/07 07:15:52 UTC

[07/10] asterixdb git commit: [ASTERIXDB-2422][STO] Introduce compressed storage

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/io/PersistedResourceRegistry.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/io/PersistedResourceRegistry.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/io/PersistedResourceRegistry.java
index a7034e2..d4c217b 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/io/PersistedResourceRegistry.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/io/PersistedResourceRegistry.java
@@ -59,6 +59,7 @@ import org.apache.asterix.metadata.utils.SecondaryCorrelatedTreeIndexOperationsH
 import org.apache.asterix.om.pointables.nonvisitor.AIntervalPointable;
 import org.apache.asterix.om.pointables.nonvisitor.AListPointable;
 import org.apache.asterix.om.pointables.nonvisitor.ARecordPointable;
+import org.apache.asterix.runtime.compression.CompressionManager;
 import org.apache.asterix.runtime.utils.RuntimeComponentsProvider;
 import org.apache.asterix.transaction.management.opcallbacks.PrimaryIndexOperationTrackerFactory;
 import org.apache.asterix.transaction.management.opcallbacks.SecondaryIndexOperationTrackerFactory;
@@ -248,6 +249,9 @@ public class PersistedResourceRegistry implements IPersistedResourceRegistry {
         REGISTERED_CLASSES.put("UTF8NGramTokenFactory", UTF8NGramTokenFactory.class);
         REGISTERED_CLASSES.put("UTF8WordTokenFactory", UTF8WordTokenFactory.class);
         REGISTERED_CLASSES.put("RTreePolicyType", RTreePolicyType.class);
+
+        //ICompressorDecompressorFactory
+        CompressionManager.registerCompressorDecompressorsFactoryClasses(REGISTERED_CLASSES);
     }
 
     @Override
@@ -265,6 +269,30 @@ public class PersistedResourceRegistry implements IPersistedResourceRegistry {
     }
 
     @Override
+    public IJsonSerializable deserializeOrDefault(JsonNode json, Class<? extends IJsonSerializable> defaultClass)
+            throws HyracksDataException {
+        if (json != null) {
+            return deserialize(json);
+        }
+
+        return deserializeDefault(defaultClass);
+    }
+
+    private IJsonSerializable deserializeDefault(Class<? extends IJsonSerializable> defaultClass)
+            throws HyracksDataException {
+        //Ensure it is registered
+        final String resourceId = getResourceId(defaultClass);
+        try {
+            Class<? extends IJsonSerializable> clazz = getResourceClass(resourceId);
+            //Using static method (fromJson)
+            Method method = clazz.getMethod(DESERIALIZATION_METHOD, IPersistedResourceRegistry.class, JsonNode.class);
+            return (IJsonSerializable) method.invoke(null, this, null);
+        } catch (NoSuchMethodException | IllegalAccessException | InvocationTargetException e) {
+            throw HyracksDataException.create(e);
+        }
+    }
+
+    @Override
     public ObjectNode getClassIdentifier(Class<? extends IJsonSerializable> clazz, long version) {
         final ObjectNode objectNode = JSON_MAPPER.createObjectNode();
         objectNode.put(IPersistedResourceRegistry.TYPE_FIELD_ID, getResourceId(clazz));

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/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 cffa178..3638e55 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
@@ -554,6 +554,8 @@ public class QueryTranslator extends AbstractLangTranslator implements IStatemen
         String nodegroupName = ngNameId == null ? null : ngNameId.getValue();
         String compactionPolicy = dd.getCompactionPolicy();
         Map<String, String> compactionPolicyProperties = dd.getCompactionPolicyProperties();
+        String compressionScheme = metadataProvider.getCompressionManager()
+                .getDdlOrDefaultCompressionScheme(dd.getDatasetCompressionScheme());
         boolean defaultCompactionPolicy = compactionPolicy == null;
         MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
         boolean bActiveTxn = true;
@@ -652,7 +654,7 @@ public class QueryTranslator extends AbstractLangTranslator implements IStatemen
             dataset = new Dataset(dataverseName, datasetName, itemTypeDataverseName, itemTypeName,
                     metaItemTypeDataverseName, metaItemTypeName, ngName, compactionPolicy, compactionPolicyProperties,
                     datasetDetails, dd.getHints(), dsType, DatasetIdFactory.generateDatasetId(),
-                    MetadataUtil.PENDING_ADD_OP);
+                    MetadataUtil.PENDING_ADD_OP, compressionScheme);
             MetadataManager.INSTANCE.addDataset(metadataProvider.getMetadataTxnContext(), dataset);
             if (dd.getDatasetType() == DatasetType.INTERNAL) {
                 JobSpecification jobSpec = DatasetUtil.createDatasetJobSpec(dataset, metadataProvider);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/SqlppExecutionWithCompresisionTest.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/SqlppExecutionWithCompresisionTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/SqlppExecutionWithCompresisionTest.java
new file mode 100644
index 0000000..af50bde
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/SqlppExecutionWithCompresisionTest.java
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.test.runtime;
+
+import java.util.Collection;
+
+import org.apache.asterix.test.common.TestExecutor;
+import org.apache.asterix.testframework.context.TestCaseContext;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+
+/**
+ * Runs the SQL++ storage runtime tests with the Snappy compression.
+ */
+@RunWith(Parameterized.class)
+public class SqlppExecutionWithCompresisionTest {
+    protected static final String TEST_CONFIG_FILE_NAME = "src/test/resources/cc-compression.conf";
+
+    @BeforeClass
+    public static void setUp() throws Exception {
+        LangExecutionUtil.setUp(TEST_CONFIG_FILE_NAME, new TestExecutor());
+    }
+
+    @AfterClass
+    public static void tearDown() throws Exception {
+        LangExecutionUtil.tearDown();
+    }
+
+    @Parameters(name = "SqlppExecutionWithCompressionTest {index}: {0}")
+    public static Collection<Object[]> tests() throws Exception {
+        return LangExecutionUtil.tests("only_sqlpp.xml", "testsuite_sqlpp.xml");
+    }
+
+    protected TestCaseContext tcCtx;
+
+    public SqlppExecutionWithCompresisionTest(TestCaseContext tcCtx) {
+        this.tcCtx = tcCtx;
+    }
+
+    @Test
+    public void test() throws Exception {
+        LangExecutionUtil.test(tcCtx);
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/cc-compression.conf
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/cc-compression.conf b/asterixdb/asterix-app/src/test/resources/cc-compression.conf
new file mode 100644
index 0000000..904707a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/cc-compression.conf
@@ -0,0 +1,57 @@
+; Licensed to the Apache Software Foundation (ASF) under one
+; or more contributor license agreements.  See the NOTICE file
+; distributed with this work for additional information
+; regarding copyright ownership.  The ASF licenses this file
+; to you under the Apache License, Version 2.0 (the
+; "License"); you may not use this file except in compliance
+; with the License.  You may obtain a copy of the License at
+;
+;   http://www.apache.org/licenses/LICENSE-2.0
+;
+; Unless required by applicable law or agreed to in writing,
+; software distributed under the License is distributed on an
+; "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+; KIND, either express or implied.  See the License for the
+; specific language governing permissions and limitations
+; under the License.
+
+[nc/asterix_nc1]
+txn.log.dir=target/tmp/asterix_nc1/txnlog
+core.dump.dir=target/tmp/asterix_nc1/coredump
+iodevices=target/tmp/asterix_nc1/iodevice1,../asterix-server/target/tmp/asterix_nc1/iodevice2
+nc.api.port=19004
+#jvm.args=-agentlib:jdwp=transport=dt_socket,server=y,suspend=y,address=5006
+
+[nc/asterix_nc2]
+ncservice.port=9091
+txn.log.dir=target/tmp/asterix_nc2/txnlog
+core.dump.dir=target/tmp/asterix_nc2/coredump
+iodevices=target/tmp/asterix_nc2/iodevice1,../asterix-server/target/tmp/asterix_nc2/iodevice2
+nc.api.port=19005
+#jvm.args=-agentlib:jdwp=transport=dt_socket,server=y,suspend=y,address=5007
+
+[nc]
+address=127.0.0.1
+command=asterixnc
+app.class=org.apache.asterix.hyracks.bootstrap.NCApplication
+jvm.args=-Xmx4096m -Dnode.Resolver="org.apache.asterix.external.util.IdentitiyResolverFactory"
+storage.buffercache.pagesize=32KB
+storage.buffercache.size=21MB
+storage.memorycomponent.globalbudget=512MB
+storage.compression.block=snappy
+
+[cc]
+address = 127.0.0.1
+app.class=org.apache.asterix.hyracks.bootstrap.CCApplication
+heartbeat.period=2000
+heartbeat.max.misses=25
+
+[common]
+log.level = INFO
+compiler.framesize=32KB
+compiler.sortmemory=320KB
+compiler.groupmemory=160KB
+compiler.joinmemory=256KB
+compiler.textsearchmemory=160KB
+messaging.frame.size=4096
+messaging.frame.count=512

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/compression/incompressible-pages/large-page/large-page.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/compression/incompressible-pages/large-page/large-page.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/compression/incompressible-pages/large-page/large-page.1.ddl.sqlpp
new file mode 100644
index 0000000..ad8ef0b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/compression/incompressible-pages/large-page/large-page.1.ddl.sqlpp
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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 incompressible pages
+ * Expected Res : Success
+ * Date         : 24 Jul 2018
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+create type CompressedType as closed {
+  id: int,
+  array_doubles: [double]
+};
+
+create dataset Compressed(CompressedType)
+primary key id

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/compression/incompressible-pages/large-page/large-page.2.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/compression/incompressible-pages/large-page/large-page.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/compression/incompressible-pages/large-page/large-page.2.update.sqlpp
new file mode 100644
index 0000000..3c4c3c6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/compression/incompressible-pages/large-page/large-page.2.update.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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 test;
+
+set `compiler.sortmemory` "64MB";
+
+load dataset Compressed using localfs
+(("path" = "asterix_nc1://data/compression/incompressible_large_object.adm"), ("format" = "adm"));
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/compression/incompressible-pages/large-page/large-page.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/compression/incompressible-pages/large-page/large-page.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/compression/incompressible-pages/large-page/large-page.3.query.sqlpp
new file mode 100644
index 0000000..01959ce
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/compression/incompressible-pages/large-page/large-page.3.query.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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 test;
+
+select value array_count(c.array_doubles)
+from Compressed as c;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/compression/incompressible-pages/small-page/small-page.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/compression/incompressible-pages/small-page/small-page.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/compression/incompressible-pages/small-page/small-page.1.ddl.sqlpp
new file mode 100644
index 0000000..ad8ef0b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/compression/incompressible-pages/small-page/small-page.1.ddl.sqlpp
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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 incompressible pages
+ * Expected Res : Success
+ * Date         : 24 Jul 2018
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+create type CompressedType as closed {
+  id: int,
+  array_doubles: [double]
+};
+
+create dataset Compressed(CompressedType)
+primary key id

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/compression/incompressible-pages/small-page/small-page.2.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/compression/incompressible-pages/small-page/small-page.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/compression/incompressible-pages/small-page/small-page.2.update.sqlpp
new file mode 100644
index 0000000..6c414c7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/compression/incompressible-pages/small-page/small-page.2.update.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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 test;
+
+set `compiler.sortmemory` "64MB";
+
+load dataset Compressed using localfs
+(("path" = "asterix_nc1://data/compression/incompressible_object.adm"), ("format" = "adm"));
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/compression/incompressible-pages/small-page/small-page.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/compression/incompressible-pages/small-page/small-page.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/compression/incompressible-pages/small-page/small-page.3.query.sqlpp
new file mode 100644
index 0000000..01959ce
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/compression/incompressible-pages/small-page/small-page.3.query.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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 test;
+
+select value array_count(c.array_doubles)
+from Compressed as c;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/compression/invalid-compression-scheme/invalid-compression-scheme.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/compression/invalid-compression-scheme/invalid-compression-scheme.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/compression/invalid-compression-scheme/invalid-compression-scheme.1.ddl.sqlpp
new file mode 100644
index 0000000..92be222
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/compression/invalid-compression-scheme/invalid-compression-scheme.1.ddl.sqlpp
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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 providing invalid compression scheme
+ * Expected Res : Error: Throws Unknown compression scheme zip. Supported schemes are [lz4,lz4hc,snappy,none]
+ * Date         : 24 Jul 2018
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+create type CompressedType as {
+  id: int
+};
+
+create dataset Compressed(CompressedType)
+primary key id
+with {"storage-block-compression": {"scheme": "zip"}};

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/compression/scheme-none/scheme-none.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/compression/scheme-none/scheme-none.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/compression/scheme-none/scheme-none.1.ddl.sqlpp
new file mode 100644
index 0000000..2885755
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/compression/scheme-none/scheme-none.1.ddl.sqlpp
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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 compression without compression
+ * Expected Res : Success
+ * Date         : 24 Jul 2018
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+{
+  id : bigint,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP1(DBLPType)
+primary key id
+with {"storage-block-compression": {"scheme": "none"}};

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/compression/scheme-none/scheme-none.2.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/compression/scheme-none/scheme-none.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/compression/scheme-none/scheme-none.2.update.sqlpp
new file mode 100644
index 0000000..86eacdf
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/compression/scheme-none/scheme-none.2.update.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/* scan and print a delimited text file */
+
+use test;
+
+
+load  dataset DBLP1 using localfs ((`path`=`asterix_nc1://data/dblp-small/dblp-small-id.txt`),(`format`=`delimited-text`),(`delimiter`=`:`));
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/compression/scheme-none/scheme-none.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/compression/scheme-none/scheme-none.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/compression/scheme-none/scheme-none.3.query.sqlpp
new file mode 100644
index 0000000..02f8073
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/compression/scheme-none/scheme-none.3.query.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/* scan and print a delimited text file */
+
+use test;
+
+
+select element paper
+from  DBLP1 as paper
+order by paper.id
+;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/compression/scheme-snappy/scheme-snappy.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/compression/scheme-snappy/scheme-snappy.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/compression/scheme-snappy/scheme-snappy.1.ddl.sqlpp
new file mode 100644
index 0000000..a27a0be
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/compression/scheme-snappy/scheme-snappy.1.ddl.sqlpp
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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 compression with snappy
+ * Expected Res : Success
+ * Date         : 24 Jul 2018
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+{
+  id : bigint,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP1(DBLPType)
+primary key id
+with {"storage-block-compression": {"scheme": "snappy"}};
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/compression/scheme-snappy/scheme-snappy.2.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/compression/scheme-snappy/scheme-snappy.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/compression/scheme-snappy/scheme-snappy.2.update.sqlpp
new file mode 100644
index 0000000..86eacdf
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/compression/scheme-snappy/scheme-snappy.2.update.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/* scan and print a delimited text file */
+
+use test;
+
+
+load  dataset DBLP1 using localfs ((`path`=`asterix_nc1://data/dblp-small/dblp-small-id.txt`),(`format`=`delimited-text`),(`delimiter`=`:`));
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/compression/scheme-snappy/scheme-snappy.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/compression/scheme-snappy/scheme-snappy.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/compression/scheme-snappy/scheme-snappy.3.query.sqlpp
new file mode 100644
index 0000000..02f8073
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/compression/scheme-snappy/scheme-snappy.3.query.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/* scan and print a delimited text file */
+
+use test;
+
+
+select element paper
+from  DBLP1 as paper
+order by paper.id
+;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cross-dataverse/cross-dv02/cross-dv02.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cross-dataverse/cross-dv02/cross-dv02.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cross-dataverse/cross-dv02/cross-dv02.3.query.sqlpp
index e9b2b51..d71e6f5 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cross-dataverse/cross-dv02/cross-dv02.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cross-dataverse/cross-dv02/cross-dv02.3.query.sqlpp
@@ -24,7 +24,19 @@
  * Date         : 28th Aug 2012
  */
 
-select element l
+select l.DataverseName,
+  l.DatasetName,
+  l.DatatypeDataverseName,
+  l.DatatypeName,
+  l.DatasetType,
+  l.GroupName,
+  l.CompactionPolicy,
+  l.CompactionPolicyProperties,
+  l.InternalDetails,
+  l.`Hints`,
+  l.Timestamp,
+  l.DatasetId,
+  l.PendingOp
 from  `Metadata`.`Dataset` as l
 where ((l.DataverseName = 'student') or (l.DataverseName = 'teacher'))
 order by l.DatasetName

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cross-dataverse/cross-dv04/cross-dv04.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cross-dataverse/cross-dv04/cross-dv04.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cross-dataverse/cross-dv04/cross-dv04.3.query.sqlpp
index b9c51a3..5435d9e 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cross-dataverse/cross-dv04/cross-dv04.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cross-dataverse/cross-dv04/cross-dv04.3.query.sqlpp
@@ -27,7 +27,19 @@
  * Date         : 28th Aug 2012
  */
 
-select element l
+select l.DataverseName,
+  l.DatasetName,
+  l.DatatypeDataverseName,
+  l.DatatypeName,
+  l.DatasetType,
+  l.GroupName,
+  l.CompactionPolicy,
+  l.CompactionPolicyProperties,
+  l.InternalDetails,
+  l.`Hints`,
+  l.Timestamp,
+  l.DatasetId,
+  l.PendingOp
 from  `Metadata`.`Dataset` as l
 where ((l.DataverseName = 'student') or (l.DataverseName = 'teacher'))
 order by l.DatasetName

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cross-dataverse/cross-dv19/cross-dv19.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cross-dataverse/cross-dv19/cross-dv19.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cross-dataverse/cross-dv19/cross-dv19.3.query.sqlpp
index a74c316..c7ef7cd 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cross-dataverse/cross-dv19/cross-dv19.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cross-dataverse/cross-dv19/cross-dv19.3.query.sqlpp
@@ -22,7 +22,20 @@
  * Date         : Sep 20 2012
  */
 
-select element l
+select l.DataverseName,
+  l.DatasetName,
+  l.DatatypeDataverseName,
+  l.DatatypeName,
+  l.DatasetType,
+  l.GroupName,
+  l.CompactionPolicy,
+  l.CompactionPolicyProperties,
+  l.ExternalDetails,
+  l.InternalDetails,
+  l.`Hints`,
+  l.Timestamp,
+  l.DatasetId,
+  l.PendingOp
 from  `Metadata`.`Dataset` as l
 where ((l.DataverseName = 'test1') or (l.DataverseName = 'test2') or (l.DataverseName = 'TwitterData'))
 order by l.DataverseName, l.DatasetName, l.DatatypeDataverseName

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl-with-clause/missing-non-optional/missing-non-optional.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl-with-clause/missing-non-optional/missing-non-optional.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl-with-clause/missing-non-optional/missing-non-optional.1.ddl.sqlpp
new file mode 100644
index 0000000..246a380
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl-with-clause/missing-non-optional/missing-non-optional.1.ddl.sqlpp
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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
+ * Expected Res : Excpetion: WITH_FIELD_MUST_CONTAIN_SUB_FIELD
+ * Date         : 1 Nov 2018
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+create type CompressedType as closed {
+  id: int,
+  array_doubles: [double]
+};
+
+create dataset Compressed(CompressedType)
+primary key id
+with {
+  "merge-policy": {
+    "parameters": { "max-mergable-component-size": 16384, "max-tolerance-component-count": 3 }
+  }
+};

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl-with-clause/type-mismatch/type-mismatch.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl-with-clause/type-mismatch/type-mismatch.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl-with-clause/type-mismatch/type-mismatch.1.ddl.sqlpp
new file mode 100644
index 0000000..4fea63f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl-with-clause/type-mismatch/type-mismatch.1.ddl.sqlpp
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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
+ * Expected Res : Excpetion: WITH_FIELD_MUST_BE_OF_TYPE
+ * Date         : 1 Nov 2018
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+create type CompressedType as closed {
+  id: int,
+  array_doubles: [double]
+};
+
+create dataset Compressed(CompressedType)
+primary key id
+with {
+  "merge-policy": {
+    "name": "correlated-prefix",
+    "parameters": { "max-mergable-component-size": "16384", "max-tolerance-component-count": 3 }
+  }
+};

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl-with-clause/unsupported-field/unsupported-field.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl-with-clause/unsupported-field/unsupported-field.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl-with-clause/unsupported-field/unsupported-field.1.ddl.sqlpp
new file mode 100644
index 0000000..2119c61
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl-with-clause/unsupported-field/unsupported-field.1.ddl.sqlpp
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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
+ * Expected Res : Excpetion: UNSUPPORTED_WITH_FIELD
+ * Date         : 1 Nov 2018
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+create type CompressedType as closed {
+  id: int,
+  array_doubles: [double]
+};
+
+create dataset Compressed(CompressedType)
+primary key id
+with {
+  "merge-policy": {
+    "name": "correlated-prefix",
+    "parameters": { "max-mergable-component-size": 16384, "max-tolerance-component-count": 3 }
+  },
+  "unknown-field": "is not accepted"
+};

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl-with-clause/unsupported-subfield/unsupported-subfield.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl-with-clause/unsupported-subfield/unsupported-subfield.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl-with-clause/unsupported-subfield/unsupported-subfield.1.ddl.sqlpp
new file mode 100644
index 0000000..81f7b51
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl-with-clause/unsupported-subfield/unsupported-subfield.1.ddl.sqlpp
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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
+ * Expected Res : Excpetion: WITH_FIELD_MUST_CONTAIN_SUB_FIELD
+ * Date         : 1 Nov 2018
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+create type CompressedType as closed {
+  id: int,
+  array_doubles: [double]
+};
+
+create dataset Compressed(CompressedType)
+primary key id
+with {
+  "merge-policy": {
+    "name": "correlated-prefix",
+    "parameters": { "max-mergable-component-size": 16384, "max-tolerance-component-count": 3 },
+    "unknown-subfield": "is not accepted"
+  }
+};

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.1.ddl.sqlpp
new file mode 100644
index 0000000..06c5840
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.1.ddl.sqlpp
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop  dataverse tpch1 if exists;
+create  dataverse tpch1;
+
+drop  dataverse tpch2 if exists;
+create  dataverse tpch2;
+
+
+create type tpch1.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 type tpch2.OrderType as
+ closed {
+  o_orderkey : bigint,
+  o_custkey : bigint,
+  o_orderstatus : string,
+  o_totalprice : double,
+  o_orderdate : string,
+  o_orderpriority : string,
+  o_clerk : string,
+  o_shippriority : bigint,
+  o_comment : string
+};
+
+create dataset tpch1.LineItem(LineItemType)
+primary key l_orderkey,l_linenumber
+with {"storage-block-compression": {"scheme": "snappy"}};
+
+create  dataset tpch2.Orders(OrderType)
+primary key o_orderkey
+with {"storage-block-compression": {"scheme": "snappy"}};

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.10.post.http
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.10.post.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.10.post.http
new file mode 100644
index 0000000..156c15e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.10.post.http
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/admin/rebalance?nodes="asterix_nc1,asterix_nc2"

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.11.get.http
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.11.get.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.11.get.http
new file mode 100644
index 0000000..e5fe873
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.11.get.http
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/connector?dataverseName=tpch1&datasetName=LineItem

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.12.get.http
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.12.get.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.12.get.http
new file mode 100644
index 0000000..6e444ef
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.12.get.http
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/connector?dataverseName=tpch2&datasetName=Orders

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.13.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.13.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.13.query.sqlpp
new file mode 100644
index 0000000..a30c374
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.13.query.sqlpp
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+
+WITH tmp AS
+(
+    SELECT DISTINCT l_orderkey
+    FROM  tpch1.LineItem
+    WHERE l_commitdate < l_receiptdate
+)
+
+SELECT o.o_orderpriority AS order_priority, count(*) AS count
+FROM tpch2.Orders o
+JOIN tmp t
+ON t.l_orderkey = o.o_orderkey
+WHERE o.o_orderdate >= '1993-07-01' AND o.o_orderdate < '1993-10-01'
+GROUP BY o.o_orderpriority
+ORDER BY o.o_orderpriority
+;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.14.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.14.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.14.query.sqlpp
new file mode 100644
index 0000000..9b19e48
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.14.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+select DatasetName, GroupName, rebalanceCount
+from Metadata.`Dataset`
+where DatasetName = "LineItem";

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.15.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.15.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.15.query.sqlpp
new file mode 100644
index 0000000..0505471
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.15.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+select DatasetName, GroupName, rebalanceCount
+from Metadata.`Dataset`
+where DatasetName = "Orders";

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.16.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.16.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.16.query.sqlpp
new file mode 100644
index 0000000..7edd168
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.16.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+select DatasetName, GroupName, rebalanceCount
+from Metadata.`Dataset`
+where DatasetName = "Dataset";

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.2.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.2.update.sqlpp
new file mode 100644
index 0000000..373ab5e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.2.update.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+load  dataset tpch1.LineItem using localfs ((`path`=`asterix_nc1://data/tpch0.001/lineitem.tbl`),
+      (`format`=`delimited-text`), (`delimiter`=`|`));
+
+load  dataset tpch2.Orders using localfs ((`path`=`asterix_nc1://data/tpch0.001/orders.tbl`),
+      (`format`=`delimited-text`), (`delimiter`=`|`));

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.3.query.sqlpp
new file mode 100644
index 0000000..a30c374
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.3.query.sqlpp
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+
+WITH tmp AS
+(
+    SELECT DISTINCT l_orderkey
+    FROM  tpch1.LineItem
+    WHERE l_commitdate < l_receiptdate
+)
+
+SELECT o.o_orderpriority AS order_priority, count(*) AS count
+FROM tpch2.Orders o
+JOIN tmp t
+ON t.l_orderkey = o.o_orderkey
+WHERE o.o_orderdate >= '1993-07-01' AND o.o_orderdate < '1993-10-01'
+GROUP BY o.o_orderpriority
+ORDER BY o.o_orderpriority
+;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.4.post.http
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.4.post.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.4.post.http
new file mode 100644
index 0000000..4e2b902
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.4.post.http
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/admin/rebalance?nodes="asterix_nc1"

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.5.get.http
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.5.get.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.5.get.http
new file mode 100644
index 0000000..e5fe873
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.5.get.http
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/connector?dataverseName=tpch1&datasetName=LineItem

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.6.get.http
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.6.get.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.6.get.http
new file mode 100644
index 0000000..6e444ef
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.6.get.http
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/connector?dataverseName=tpch2&datasetName=Orders

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.7.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.7.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.7.query.sqlpp
new file mode 100644
index 0000000..a30c374
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.7.query.sqlpp
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+
+WITH tmp AS
+(
+    SELECT DISTINCT l_orderkey
+    FROM  tpch1.LineItem
+    WHERE l_commitdate < l_receiptdate
+)
+
+SELECT o.o_orderpriority AS order_priority, count(*) AS count
+FROM tpch2.Orders o
+JOIN tmp t
+ON t.l_orderkey = o.o_orderkey
+WHERE o.o_orderdate >= '1993-07-01' AND o.o_orderdate < '1993-10-01'
+GROUP BY o.o_orderpriority
+ORDER BY o.o_orderpriority
+;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.8.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.8.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.8.query.sqlpp
new file mode 100644
index 0000000..9b19e48
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.8.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+select DatasetName, GroupName, rebalanceCount
+from Metadata.`Dataset`
+where DatasetName = "LineItem";

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.9.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.9.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.9.query.sqlpp
new file mode 100644
index 0000000..0505471
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.9.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+select DatasetName, GroupName, rebalanceCount
+from Metadata.`Dataset`
+where DatasetName = "Orders";

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_compressed/single_dataset_compressed.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_compressed/single_dataset_compressed.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_compressed/single_dataset_compressed.1.ddl.sqlpp
new file mode 100644
index 0000000..df26d68
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_compressed/single_dataset_compressed.1.ddl.sqlpp
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+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  dataset LineItem(LineItemType)
+primary key l_orderkey,l_linenumber
+with {"storage-block-compression": {"scheme": "snappy"}};
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_compressed/single_dataset_compressed.10.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_compressed/single_dataset_compressed.10.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_compressed/single_dataset_compressed.10.query.sqlpp
new file mode 100644
index 0000000..ca42f3c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_compressed/single_dataset_compressed.10.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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 tpch;
+
+select value count(*) from LineItem;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_compressed/single_dataset_compressed.11.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_compressed/single_dataset_compressed.11.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_compressed/single_dataset_compressed.11.query.sqlpp
new file mode 100644
index 0000000..9b19e48
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_compressed/single_dataset_compressed.11.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+select DatasetName, GroupName, rebalanceCount
+from Metadata.`Dataset`
+where DatasetName = "LineItem";

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_compressed/single_dataset_compressed.2.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_compressed/single_dataset_compressed.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_compressed/single_dataset_compressed.2.update.sqlpp
new file mode 100644
index 0000000..8a59946
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_compressed/single_dataset_compressed.2.update.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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 tpch;
+
+load  dataset LineItem using localfs ((`path`=`asterix_nc1://data/tpch0.001/lineitem.tbl`),
+      (`format`=`delimited-text`),(`delimiter`=`|`));

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_compressed/single_dataset_compressed.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_compressed/single_dataset_compressed.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_compressed/single_dataset_compressed.3.query.sqlpp
new file mode 100644
index 0000000..ca42f3c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_compressed/single_dataset_compressed.3.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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 tpch;
+
+select value count(*) from LineItem;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_compressed/single_dataset_compressed.4.post.http
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_compressed/single_dataset_compressed.4.post.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_compressed/single_dataset_compressed.4.post.http
new file mode 100644
index 0000000..737c1fe
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_compressed/single_dataset_compressed.4.post.http
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/admin/rebalance?dataverseName=tpch&datasetName=LineItem&nodes="asterix_nc1"

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/5aeba9b4/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_compressed/single_dataset_compressed.5.get.http
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_compressed/single_dataset_compressed.5.get.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_compressed/single_dataset_compressed.5.get.http
new file mode 100644
index 0000000..360a01f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_compressed/single_dataset_compressed.5.get.http
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/connector?dataverseName=tpch&datasetName=LineItem