[MULTIPLE ISSUES] Fix columnar delete issues

- user model changes: no
- storage format changes: no
- interface changes: no

Details:
- ASTERIXDB-3199: Fixes issue when merging components
  contain only antimatter tuples
- ASTERIXDB-3200: Fix NPE for operator deep-copy with
  meta record
- ASTERIXDB-3201: Fix NPE when merging an empty component

Also, this patch introduces ColumnValueException, which is
thrown in case of a columnar-related error. The exception
collects several pieces of information related to an
encountered error.

Change-Id: I9a28b7fe7b7b89636dd10b62769b2e78d7979654
Reviewed-on: https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/17554
Integration-Tests: Jenkins <jenkins@fulliautomatix.ics.uci.edu>
Tested-by: Jenkins <jenkins@fulliautomatix.ics.uci.edu>
Reviewed-by: Wail Alkowaileet <wael.y.k@gmail.com>
Reviewed-by: Murtadha Al Hubail <mhubail@apache.org>
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/pushdown/ExpectedSchemaBuilder.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/pushdown/ExpectedSchemaBuilder.java
index 3f83d1a..36a77c9 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/pushdown/ExpectedSchemaBuilder.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/pushdown/ExpectedSchemaBuilder.java
@@ -70,7 +70,8 @@
     }
 
     public DataProjectionFiltrationInfo createProjectionInfo(LogicalVariable recordVariable) {
-        return createProjectionInfo(recordVariable, Collections.emptyMap(), Collections.emptyMap(), null, null);
+        return createProjectionInfo(recordVariable, Collections.emptyMap(), Collections.emptyMap(), null,
+                Collections.emptyMap());
     }
 
     public DataProjectionFiltrationInfo createProjectionInfo(LogicalVariable recordVariable,
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/common/AsterixHyracksIntegrationUtil.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/common/AsterixHyracksIntegrationUtil.java
index ccedd08..c2b5236 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/common/AsterixHyracksIntegrationUtil.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/common/AsterixHyracksIntegrationUtil.java
@@ -101,15 +101,14 @@
      * main method to run a simple 2 node cluster in-process
      * suggested VM arguments: <code>-enableassertions -Xmx2048m -Dfile.encoding=UTF-8</code>
      *
-     * @param args
-     *            unused
+     * @param args unused
      */
     public static void main(String[] args) throws Exception {
         TestUtils.redirectLoggingToConsole();
         AsterixHyracksIntegrationUtil integrationUtil = new AsterixHyracksIntegrationUtil();
         try {
             integrationUtil.run(Boolean.getBoolean("cleanup.start"), Boolean.getBoolean("cleanup.shutdown"),
-                    System.getProperty("conf.path", DEFAULT_CONF_FILE));
+                    getConfPath());
         } catch (Exception e) {
             LOGGER.fatal("Unexpected exception", e);
             System.exit(1);
@@ -444,4 +443,12 @@
                     TestPrimaryIndexOperationTrackerFactory.class);
         }
     }
+
+    private static String getConfPath() {
+        String providedPath = System.getProperty("conf.path");
+        if (providedPath == null) {
+            return DEFAULT_CONF_FILE;
+        }
+        return joinPath(RESOURCES_PATH, providedPath);
+    }
 }
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/SqlppSinglePartitionExecutionTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/SqlppSinglePartitionExecutionTest.java
new file mode 100644
index 0000000..d0823c7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/SqlppSinglePartitionExecutionTest.java
@@ -0,0 +1,85 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.asterix.common.api.INcApplicationContext;
+import org.apache.asterix.test.common.TestExecutor;
+import org.apache.asterix.testframework.context.TestCaseContext;
+import org.apache.hyracks.control.nc.NodeControllerService;
+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++ specific tests on a cluster with a single partition
+ */
+@RunWith(Parameterized.class)
+public class SqlppSinglePartitionExecutionTest {
+    protected static final String TEST_CONFIG_FILE_NAME = "src/test/resources/cc-single-partition.conf";
+
+    @BeforeClass
+    public static void setUp() throws Exception {
+        final TestExecutor testExecutor = new TestExecutor();
+        LangExecutionUtil.setUp(TEST_CONFIG_FILE_NAME, testExecutor);
+        setNcEndpoints(testExecutor);
+    }
+
+    @AfterClass
+    public static void tearDown() throws Exception {
+        LangExecutionUtil.tearDown();
+    }
+
+    @Parameters(name = "SqlppSinglePartitionExecutionTest {index}: {0}")
+    public static Collection<Object[]> tests() throws Exception {
+        return LangExecutionUtil.tests("only_sqlpp.xml", "testsuite_single_partition_sqlpp.xml");
+    }
+
+    protected TestCaseContext tcCtx;
+
+    public SqlppSinglePartitionExecutionTest(TestCaseContext tcCtx) {
+        this.tcCtx = tcCtx;
+    }
+
+    @Test
+    public void test() throws Exception {
+        LangExecutionUtil.test(tcCtx);
+    }
+
+    private static void setNcEndpoints(TestExecutor testExecutor) {
+        final NodeControllerService[] ncs = ExecutionTestUtil.integrationUtil.ncs;
+        final Map<String, InetSocketAddress> ncEndPoints = new HashMap<>();
+        final String ip = InetAddress.getLoopbackAddress().getHostAddress();
+        for (NodeControllerService nc : ncs) {
+            final String nodeId = nc.getId();
+            final INcApplicationContext appCtx = (INcApplicationContext) nc.getApplicationContext();
+            int apiPort = appCtx.getExternalProperties().getNcApiPort();
+            ncEndPoints.put(nodeId, InetSocketAddress.createUnresolved(ip, apiPort));
+        }
+        testExecutor.setNcEndPoints(ncEndPoints);
+    }
+}
diff --git a/asterixdb/asterix-app/src/test/resources/cc-single-partition.conf b/asterixdb/asterix-app/src/test/resources/cc-single-partition.conf
new file mode 100644
index 0000000..dab765b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/cc-single-partition.conf
@@ -0,0 +1,54 @@
+; Licensed to the Apache Software Foundation (ASF) under one
+; or more contributor license agreements.  See the NOTICE file
+; distributed with this work for additional information
+; regarding copyright ownership.  The ASF licenses this file
+; to you under the Apache License, Version 2.0 (the
+; "License"); you may not use this file except in compliance
+; with the License.  You may obtain a copy of the License at
+;
+;   http://www.apache.org/licenses/LICENSE-2.0
+;
+; Unless required by applicable law or agreed to in writing,
+; software distributed under the License is distributed on an
+; "AS IS" BASIS, WITHOUT 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
+nc.api.port=19004
+#jvm.args=-agentlib:jdwp=transport=dt_socket,server=y,suspend=y,address=5006
+
+[nc]
+credential.file=src/test/resources/security/passwd
+python.cmd.autolocate=true
+python.env=FOO=BAR=BAZ,BAR=BAZ
+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.size=128MB
+storage.memorycomponent.globalbudget=512MB
+
+[cc]
+address = 127.0.0.1
+app.class=org.apache.asterix.hyracks.bootstrap.CCApplication
+heartbeat.period=2000
+heartbeat.max.misses=25
+credential.file=src/test/resources/security/passwd
+
+[common]
+log.dir = logs/
+log.level = INFO
+compiler.framesize=32KB
+compiler.sortmemory=320KB
+compiler.groupmemory=160KB
+compiler.joinmemory=256KB
+compiler.textsearchmemory=160KB
+compiler.windowmemory=192KB
+messaging.frame.size=4096
+messaging.frame.count=512
+storage.buffercache.pagesize=32KB
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/column-pushdown/meta.001.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/column-pushdown/meta.001.sqlpp
new file mode 100644
index 0000000..e5e90c1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/column-pushdown/meta.001.sqlpp
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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 test IF EXISTS;
+ CREATE DATAVERSE test;
+ USE test;
+
+ CREATE TYPE CH2Type AS {
+ };
+
+ CREATE TYPE CH2MetaType AS {
+     uid:int
+ };
+
+ CREATE DATASET `customer`(CH2Type) WITH META(CH2MetaType) PRIMARY KEY META().uid WITH {"storage-format": {"format": "column"}};
+ CREATE DATASET `district`(CH2Type) WITH META(CH2MetaType) PRIMARY KEY META().uid WITH {"storage-format": {"format": "column"}};
+ CREATE DATASET `history`(CH2Type) WITH META(CH2MetaType) PRIMARY KEY META().uid WITH {"storage-format": {"format": "column"}};
+ CREATE DATASET `item`(CH2Type) WITH META(CH2MetaType) PRIMARY KEY META().uid WITH {"storage-format": {"format": "column"}};
+ CREATE DATASET `neworder`(CH2Type) WITH META(CH2MetaType) PRIMARY KEY META().uid WITH {"storage-format": {"format": "column"}};
+ CREATE DATASET `orders`(CH2Type) WITH META(CH2MetaType) PRIMARY KEY META().uid WITH {"storage-format": {"format": "column"}};
+ CREATE DATASET `stock`(CH2Type) WITH META(CH2MetaType) PRIMARY KEY META().uid WITH {"storage-format": {"format": "column"}};
+ CREATE DATASET `warehouse`(CH2Type) WITH META(CH2MetaType) PRIMARY KEY META().uid WITH {"storage-format": {"format": "column"}};
+ CREATE DATASET `supplier`(CH2Type) WITH META(CH2MetaType) PRIMARY KEY META().uid WITH {"storage-format": {"format": "column"}};
+ CREATE DATASET `nation`(CH2Type) WITH META(CH2MetaType) PRIMARY KEY META().uid WITH {"storage-format": {"format": "column"}};
+ CREATE DATASET `region`(CH2Type) WITH META(CH2MetaType) PRIMARY KEY META().uid WITH {"storage-format": {"format": "column"}};
+
+SELECT s.s_i_id, SUM(s.s_order_cnt) as ordercount
+FROM   nation n, supplier su, stock s
+WHERE  s.s_w_id * s.s_i_id MOD 10000 = su.su_suppkey
+  AND  su.su_nationkey = n.n_nationkey
+  AND  n.n_name = 'Germany'
+GROUP BY s.s_i_id
+HAVING SUM(s.s_order_cnt) > (
+     SELECT VALUE SUM(s1.s_order_cnt) * 0.00005
+     FROM nation n1, supplier su1, stock s1
+     WHERE s1.s_w_id * s1.s_i_id MOD 10000 = su1.su_suppkey
+       AND su1.su_nationkey = n1.n_nationkey
+       AND n1.n_name = 'Germany')[0]
+ORDER BY ordercount DESC;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/column-pushdown/meta.001.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/column-pushdown/meta.001.plan
new file mode 100644
index 0000000..01a8496
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/column-pushdown/meta.001.plan
@@ -0,0 +1,119 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$194(DESC) ]  |PARTITIONED|
+          -- STABLE_SORT [$$194(DESC)]  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- NESTED_LOOP  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- SORT_GROUP_BY[$$223]  |PARTITIONED|
+                              {
+                                -- AGGREGATE  |LOCAL|
+                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
+                              }
+                        -- HASH_PARTITION_EXCHANGE [$$223]  |PARTITIONED|
+                          -- SORT_GROUP_BY[$$178]  |PARTITIONED|
+                                  {
+                                    -- AGGREGATE  |LOCAL|
+                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                  }
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- HYBRID_HASH_JOIN [$$192][$$191]  |PARTITIONED|
+                                    -- HASH_PARTITION_EXCHANGE [$$192]  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- HYBRID_HASH_JOIN [$$207][$$206]  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                -- ASSIGN  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- REPLICATE  |PARTITIONED|
+                                                      -- HASH_PARTITION_EXCHANGE [$$212]  |PARTITIONED|
+                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                          -- STREAM_SELECT  |PARTITIONED|
+                                                            -- ASSIGN  |PARTITIONED|
+                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                  -- DATASOURCE_SCAN (test.nation)  |PARTITIONED|
+                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                -- ASSIGN  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- REPLICATE  |PARTITIONED|
+                                                      -- HASH_PARTITION_EXCHANGE [$$211]  |PARTITIONED|
+                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                          -- ASSIGN  |PARTITIONED|
+                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- DATASOURCE_SCAN (test.supplier)  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                    -- HASH_PARTITION_EXCHANGE [$$191]  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- REPLICATE  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- DATASOURCE_SCAN (test.stock)  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                    -- BROADCAST_EXCHANGE  |PARTITIONED|
+                      -- STREAM_PROJECT  |UNPARTITIONED|
+                        -- ASSIGN  |UNPARTITIONED|
+                          -- AGGREGATE  |UNPARTITIONED|
+                            -- STREAM_PROJECT  |UNPARTITIONED|
+                              -- ASSIGN  |UNPARTITIONED|
+                                -- AGGREGATE  |UNPARTITIONED|
+                                  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                                    -- AGGREGATE  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- HYBRID_HASH_JOIN [$$210][$$209]  |PARTITIONED|
+                                            -- HASH_PARTITION_EXCHANGE [$$210]  |PARTITIONED|
+                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- HYBRID_HASH_JOIN [$$212][$$211]  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- REPLICATE  |PARTITIONED|
+                                                        -- HASH_PARTITION_EXCHANGE [$$212]  |PARTITIONED|
+                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                            -- STREAM_SELECT  |PARTITIONED|
+                                                              -- ASSIGN  |PARTITIONED|
+                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- DATASOURCE_SCAN (test.nation)  |PARTITIONED|
+                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- REPLICATE  |PARTITIONED|
+                                                        -- HASH_PARTITION_EXCHANGE [$$211]  |PARTITIONED|
+                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                            -- ASSIGN  |PARTITIONED|
+                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                  -- DATASOURCE_SCAN (test.supplier)  |PARTITIONED|
+                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                            -- HASH_PARTITION_EXCHANGE [$$209]  |PARTITIONED|
+                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                -- ASSIGN  |PARTITIONED|
+                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                    -- ASSIGN  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- REPLICATE  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- DATASOURCE_SCAN (test.stock)  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/column/delete/003/003.001.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/column/delete/003/003.001.ddl.sqlpp
new file mode 100644
index 0000000..b51e2a2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/column/delete/003/003.001.ddl.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.
+ */
+
+-- Test merging components that only contain antimatter tuples
+
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+
+USE test;
+
+
+CREATE TYPE OpenType AS {
+    myId: int
+};
+
+CREATE DATASET ColumnDataset(OpenType)
+PRIMARY KEY myId WITH {
+    "storage-format": {"format" : "column"}
+};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/column/delete/003/003.002.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/column/delete/003/003.002.update.sqlpp
new file mode 100644
index 0000000..c7c9145
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/column/delete/003/003.002.update.sqlpp
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+-- Produce relatively large component
+UPSERT INTO ColumnDataset (
+    SELECT VALUE object_concat({"myId": gen_id},
+        {  "coordinates": {"coordinates": [1.1],"type": "string"  },  "created_at": "string",  "entities": {"urls": [{  "display_url": "string",  "expanded_url": "string",  "indices": [1],  "url": "string"}],"user_mentions": [{  "id": 1,  "id_str": "string",  "indices": [1],  "name": "string",  "screen_name": "string"}]  },  "favorite_count": 1,  "favorited": true,  "filter_level": "string",  "geo": {"coordinates": [1.1],"type": "string"  },  "id": "0000000",  "id_str": "string",  "in_reply_to_screen_name": "string",  "in_reply_to_status_id": 1,  "in_reply_to_status_id_str": "string",  "in_reply_to_user_id": 1,  "in_reply_to_user_id_str": "string",  "is_quote_status": true,  "lang": "string",  "place": {"bounding_box": {  "coordinates": [[[1.1]]],  "type": "string"},"country": "string","country_code": "string","full_name": "string","id": "string","name": "string","place_type": "string","url": "string"  },  "possibly_sensitive": true,  "quoted_status": {"created_at": "string","entities": {"user_mentions": [{  "id": 1,  "id_str": "string",  "indices": [1],  "name": "string",  "screen_name": "string"}]},"favorite_count": 1,"favorited": true,"filter_level": "string","id": 1,"id_str": "string","in_reply_to_screen_name": "string","in_reply_to_status_id": 1,"in_reply_to_status_id_str": "string","in_reply_to_user_id": 1,"in_reply_to_user_id_str": "string","is_quote_status": true,"lang": "string","retweet_count": 1,"retweeted": true,"source": "string","text": "string","truncated": true,"user": {  "contributors_enabled": true,  "created_at": "string",  "default_profile": true,  "default_profile_image": true,  "description": "string",  "favourites_count": 1,  "followers_count": 1,  "friends_count": 1,  "geo_enabled": true,  "id": 1,  "id_str": "string",  "is_translator": true,  "lang": "string",  "listed_count": 1,  "name": "string",  "profile_background_color": "string",  "profile_background_image_url": "string",  "profile_background_image_url_https": "string",  "profile_background_tile": true,  "profile_banner_url": "string",  "profile_image_url": "string",  "profile_image_url_https": "string",  "profile_link_color": "string",  "profile_sidebar_border_color": "string",  "profile_sidebar_fill_color": "string",  "profile_text_color": "string",  "profile_use_background_image": true,  "protected": true,  "screen_name": "string",  "statuses_count": 1,  "verified": true}  },  "quoted_status_id": 1,  "quoted_status_id_str": "string",  "retweet_count": 1,  "retweeted": true,  "source": "string",  "text": "string",  "timestamp_ms": "string",  "truncated": true,  "user": {"contributors_enabled": true,"created_at": "string","default_profile": true,"default_profile_image": true,"description": "string","favourites_count": 1,"followers_count": 1,"friends_count": 1,"geo_enabled": true,"id": 1,"id_str": "string","is_translator": true,"lang": "string","listed_count": 1,"location": "string","name": "string","profile_background_color": "string","profile_background_image_url": "string","profile_background_image_url_https": "string","profile_background_tile": true,"profile_banner_url": "string","profile_image_url": "string","profile_image_url_https": "string","profile_link_color": "string","profile_sidebar_border_color": "string","profile_sidebar_fill_color": "string","profile_text_color": "string","profile_use_background_image": true,"protected": true,"screen_name": "string","statuses_count": 1,"time_zone": "string","url": "string","utc_offset": 1,"verified": true  }}
+    )
+    FROM range(0, 29999) gen_id
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/column/delete/003/003.003.get.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/column/delete/003/003.003.get.http
new file mode 100644
index 0000000..57d830a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/column/delete/003/003.003.get.http
@@ -0,0 +1,19 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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=test&datasetName=ColumnDataset
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/column/delete/003/003.004.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/column/delete/003/003.004.update.sqlpp
new file mode 100644
index 0000000..6698638
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/column/delete/003/003.004.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.
+ */
+
+USE test;
+
+-- Delete 1 record and flush
+DELETE FROM ColumnDataset c
+WHERE c.myId < 1
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/column/delete/003/003.005.get.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/column/delete/003/003.005.get.http
new file mode 100644
index 0000000..57d830a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/column/delete/003/003.005.get.http
@@ -0,0 +1,19 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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=test&datasetName=ColumnDataset
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/column/delete/003/003.006.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/column/delete/003/003.006.update.sqlpp
new file mode 100644
index 0000000..5f09d5a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/column/delete/003/003.006.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.
+ */
+
+USE test;
+
+-- Delete 1 record and flush
+DELETE FROM ColumnDataset c
+WHERE c.myId < 2
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/column/delete/003/003.007.get.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/column/delete/003/003.007.get.http
new file mode 100644
index 0000000..57d830a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/column/delete/003/003.007.get.http
@@ -0,0 +1,19 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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=test&datasetName=ColumnDataset
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/column/delete/003/003.008.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/column/delete/003/003.008.update.sqlpp
new file mode 100644
index 0000000..2d4da3d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/column/delete/003/003.008.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.
+ */
+
+USE test;
+
+-- Delete 1 record and flush. This should trigger a merge against the antimatter's components
+DELETE FROM ColumnDataset c
+WHERE c.myId < 3
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/column/delete/003/003.009.get.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/column/delete/003/003.009.get.http
new file mode 100644
index 0000000..57d830a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/column/delete/003/003.009.get.http
@@ -0,0 +1,19 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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=test&datasetName=ColumnDataset
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/column/delete/003/003.010.sleep.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/column/delete/003/003.010.sleep.sqlpp
new file mode 100644
index 0000000..e50b429
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/column/delete/003/003.010.sleep.sqlpp
@@ -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.
+ */
+
+2000
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/column/delete/003/003.011.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/column/delete/003/003.011.query.sqlpp
new file mode 100644
index 0000000..d52cd6c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/column/delete/003/003.011.query.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 test;
+
+SELECT VALUE COUNT(*)
+FROM ColumnDataset c
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/column/delete/003/003.012.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/column/delete/003/003.012.query.sqlpp
new file mode 100644
index 0000000..cb9f7f6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/column/delete/003/003.012.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 COUNT(*)
+FROM ColumnDataset c
+WHERE c.myId < 3
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/column/delete/004/004.001.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/column/delete/004/004.001.ddl.sqlpp
new file mode 100644
index 0000000..3bc2966
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/column/delete/004/004.001.ddl.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.
+ */
+
+-- Test merging components that only contain antimatter tuples
+
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+
+USE test;
+
+
+CREATE TYPE OpenType AS {
+    id: int
+};
+
+CREATE DATASET ColumnDataset(OpenType)
+PRIMARY KEY id WITH {
+    "storage-format": {"format" : "column"}
+};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/column/delete/004/004.002.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/column/delete/004/004.002.update.sqlpp
new file mode 100644
index 0000000..39572a3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/column/delete/004/004.002.update.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.
+ */
+
+USE test;
+
+INSERT INTO ColumnDataset (
+    {"id": 1, "geo" : {"coordinates": [1, 2]}},
+    {"id": 2, "geo" : {"coordinates": [3, 4]}},
+    {"id": 3, "geo" : {"coordinates": [5, 6]}},
+    {"id": 4, "geo" : {"coordinates": [7, 8]}}
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/column/delete/004/004.003.get.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/column/delete/004/004.003.get.http
new file mode 100644
index 0000000..57d830a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/column/delete/004/004.003.get.http
@@ -0,0 +1,19 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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=test&datasetName=ColumnDataset
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/column/delete/004/004.004.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/column/delete/004/004.004.update.sqlpp
new file mode 100644
index 0000000..c850a84
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/column/delete/004/004.004.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 test;
+
+-- delete all records
+DELETE FROM ColumnDataset c
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/column/delete/004/004.005.get.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/column/delete/004/004.005.get.http
new file mode 100644
index 0000000..57d830a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/column/delete/004/004.005.get.http
@@ -0,0 +1,19 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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=test&datasetName=ColumnDataset
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/column/delete/004/004.006.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/column/delete/004/004.006.update.sqlpp
new file mode 100644
index 0000000..d82b5d0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/column/delete/004/004.006.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 test;
+
+-- Compact to get an empty component
+COMPACT DATASET ColumnDataset;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/column/delete/004/004.007.get.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/column/delete/004/004.007.get.http
new file mode 100644
index 0000000..57d830a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/column/delete/004/004.007.get.http
@@ -0,0 +1,19 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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=test&datasetName=ColumnDataset
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/column/delete/004/004.008.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/column/delete/004/004.008.update.sqlpp
new file mode 100644
index 0000000..c334562
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/column/delete/004/004.008.update.sqlpp
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+-- Insert new records and then flush
+INSERT INTO ColumnDataset (
+    {"id": 1, "geo" : {"coordinates": [1, 2]}},
+    {"id": 2, "geo" : {"coordinates": [3, 4]}},
+    {"id": 3, "geo" : {"coordinates": [5, 6]}},
+    {"id": 4, "geo" : {"coordinates": [7, 8]}}
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/column/delete/004/004.009.get.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/column/delete/004/004.009.get.http
new file mode 100644
index 0000000..57d830a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/column/delete/004/004.009.get.http
@@ -0,0 +1,19 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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=test&datasetName=ColumnDataset
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/column/delete/004/004.010.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/column/delete/004/004.010.update.sqlpp
new file mode 100644
index 0000000..1479603
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/column/delete/004/004.010.update.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 test;
+-- Compact empty component with filled component
+COMPACT DATASET ColumnDataset;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/column/delete/004/004.011.get.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/column/delete/004/004.011.get.http
new file mode 100644
index 0000000..57d830a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/column/delete/004/004.011.get.http
@@ -0,0 +1,19 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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=test&datasetName=ColumnDataset
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/column/delete/004/004.012.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/column/delete/004/004.012.query.sqlpp
new file mode 100644
index 0000000..e62ac12
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/column/delete/004/004.012.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 c
+FROM ColumnDataset c
+ORDER BY c.id
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/column/delete/003/003.003.regexadm b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/delete/003/003.003.regexadm
new file mode 100644
index 0000000..16c1eaf
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/delete/003/003.003.regexadm
@@ -0,0 +1 @@
+\Q{"keys":"myId","type":{"type":"org.apache.asterix.om.types.ARecordType","name":"OpenType","open":true,"fields":[{"myId":{"type":"AInt64"}}]},"splits":[\E.*\Q]}\E
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/column/delete/003/003.005.regexadm b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/delete/003/003.005.regexadm
new file mode 100644
index 0000000..16c1eaf
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/delete/003/003.005.regexadm
@@ -0,0 +1 @@
+\Q{"keys":"myId","type":{"type":"org.apache.asterix.om.types.ARecordType","name":"OpenType","open":true,"fields":[{"myId":{"type":"AInt64"}}]},"splits":[\E.*\Q]}\E
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/column/delete/003/003.007.regexadm b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/delete/003/003.007.regexadm
new file mode 100644
index 0000000..16c1eaf
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/delete/003/003.007.regexadm
@@ -0,0 +1 @@
+\Q{"keys":"myId","type":{"type":"org.apache.asterix.om.types.ARecordType","name":"OpenType","open":true,"fields":[{"myId":{"type":"AInt64"}}]},"splits":[\E.*\Q]}\E
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/column/delete/003/003.009.regexadm b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/delete/003/003.009.regexadm
new file mode 100644
index 0000000..16c1eaf
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/delete/003/003.009.regexadm
@@ -0,0 +1 @@
+\Q{"keys":"myId","type":{"type":"org.apache.asterix.om.types.ARecordType","name":"OpenType","open":true,"fields":[{"myId":{"type":"AInt64"}}]},"splits":[\E.*\Q]}\E
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/column/delete/003/003.011.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/delete/003/003.011.adm
new file mode 100644
index 0000000..414c6bb7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/delete/003/003.011.adm
@@ -0,0 +1 @@
+29997
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/column/delete/003/003.012.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/delete/003/003.012.adm
new file mode 100644
index 0000000..c227083
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/delete/003/003.012.adm
@@ -0,0 +1 @@
+0
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/column/delete/004/004.003.regexadm b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/delete/004/004.003.regexadm
new file mode 100644
index 0000000..9f1dc3a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/delete/004/004.003.regexadm
@@ -0,0 +1 @@
+\Q{"keys":"id","type":{"type":"org.apache.asterix.om.types.ARecordType","name":"OpenType","open":true,"fields":[{"id":{"type":"AInt64"}}]},"splits":[\E.*\Q]}\E
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/column/delete/004/004.005.regexadm b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/delete/004/004.005.regexadm
new file mode 100644
index 0000000..9f1dc3a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/delete/004/004.005.regexadm
@@ -0,0 +1 @@
+\Q{"keys":"id","type":{"type":"org.apache.asterix.om.types.ARecordType","name":"OpenType","open":true,"fields":[{"id":{"type":"AInt64"}}]},"splits":[\E.*\Q]}\E
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/column/delete/004/004.007.regexadm b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/delete/004/004.007.regexadm
new file mode 100644
index 0000000..9f1dc3a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/delete/004/004.007.regexadm
@@ -0,0 +1 @@
+\Q{"keys":"id","type":{"type":"org.apache.asterix.om.types.ARecordType","name":"OpenType","open":true,"fields":[{"id":{"type":"AInt64"}}]},"splits":[\E.*\Q]}\E
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/column/delete/004/004.009.regexadm b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/delete/004/004.009.regexadm
new file mode 100644
index 0000000..9f1dc3a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/delete/004/004.009.regexadm
@@ -0,0 +1 @@
+\Q{"keys":"id","type":{"type":"org.apache.asterix.om.types.ARecordType","name":"OpenType","open":true,"fields":[{"id":{"type":"AInt64"}}]},"splits":[\E.*\Q]}\E
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/column/delete/004/004.011.regexadm b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/delete/004/004.011.regexadm
new file mode 100644
index 0000000..9f1dc3a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/delete/004/004.011.regexadm
@@ -0,0 +1 @@
+\Q{"keys":"id","type":{"type":"org.apache.asterix.om.types.ARecordType","name":"OpenType","open":true,"fields":[{"id":{"type":"AInt64"}}]},"splits":[\E.*\Q]}\E
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/column/delete/004/004.012.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/delete/004/004.012.adm
new file mode 100644
index 0000000..83bc984
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/column/delete/004/004.012.adm
@@ -0,0 +1,4 @@
+{ "id": 1, "geo": { "coordinates": [ 1, 2 ] } }
+{ "id": 2, "geo": { "coordinates": [ 3, 4 ] } }
+{ "id": 3, "geo": { "coordinates": [ 5, 6 ] } }
+{ "id": 4, "geo": { "coordinates": [ 7, 8 ] } }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_single_partition_sqlpp.xml b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_single_partition_sqlpp.xml
new file mode 100644
index 0000000..dbc5a1f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_single_partition_sqlpp.xml
@@ -0,0 +1,33 @@
+<?xml version="1.0" encoding="UTF-8" standalone="yes"?>
+<!--
+ ! Licensed to the Apache Software Foundation (ASF) under one
+ ! or more contributor license agreements.  See the NOTICE file
+ ! distributed with this work for additional information
+ ! regarding copyright ownership.  The ASF licenses this file
+ ! to you under the Apache License, Version 2.0 (the
+ ! "License"); you may not use this file except in compliance
+ ! with the License.  You may obtain a copy of the License at
+ !
+ !   http://www.apache.org/licenses/LICENSE-2.0
+ !
+ ! Unless required by applicable law or agreed to in writing,
+ ! software distributed under the License is distributed on an
+ ! "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ ! KIND, either express or implied.  See the License for the
+ ! specific language governing permissions and limitations
+ ! under the License.
+ !-->
+<test-suite xmlns="urn:xml.testframework.asterix.apache.org" ResultOffsetPath="results" QueryOffsetPath="queries_sqlpp" QueryFileExtension=".sqlpp">
+  <test-group name="column">
+    <test-case FilePath="column">
+      <compilation-unit name="delete/003">
+        <output-dir compare="Text">delete/003</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="column">
+      <compilation-unit name="delete/004">
+        <output-dir compare="Text">delete/004</output-dir>
+      </compilation-unit>
+    </test-case>
+  </test-group>
+</test-suite>
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 af71986..059bc62 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
@@ -16122,6 +16122,16 @@
       </compilation-unit>
     </test-case>
     <test-case FilePath="column">
+      <compilation-unit name="delete/003">
+        <output-dir compare="Text">delete/003</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="column">
+      <compilation-unit name="delete/004">
+        <output-dir compare="Text">delete/004</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="column">
       <compilation-unit name="meta/meta_after_gby">
         <output-dir compare="Text">meta/meta_after_gby</output-dir>
       </compilation-unit>
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/assembler/AssemblerState.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/assembler/AssemblerState.java
index a950337..ccdd950 100644
--- a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/assembler/AssemblerState.java
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/assembler/AssemblerState.java
@@ -18,6 +18,10 @@
  */
 package org.apache.asterix.column.assembler;
 
+import org.apache.hyracks.storage.am.lsm.btree.column.error.ColumnarValueException;
+
+import com.fasterxml.jackson.databind.node.ObjectNode;
+
 public class AssemblerState {
     private EndOfRepeatedGroupAssembler currentGroup;
 
@@ -43,4 +47,14 @@
     public boolean isInGroup() {
         return currentGroup != null;
     }
+
+    public void appendStateInfo(ColumnarValueException e) {
+        ObjectNode stateNode = e.createNode(getClass().getSimpleName());
+        if (isInGroup()) {
+            stateNode.put("inGroup", true);
+            currentGroup.reader.appendReaderInformation(stateNode.putObject("endOfGroupReader"));
+        } else {
+            stateNode.put("inGroup", false);
+        }
+    }
 }
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/assembler/PrimitiveValueAssembler.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/assembler/PrimitiveValueAssembler.java
index eb88778..8dcf024 100644
--- a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/assembler/PrimitiveValueAssembler.java
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/assembler/PrimitiveValueAssembler.java
@@ -21,8 +21,12 @@
 import org.apache.asterix.column.assembler.value.IValueGetter;
 import org.apache.asterix.column.values.IColumnValuesReader;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.lsm.btree.column.error.ColumnarValueException;
+
+import com.fasterxml.jackson.databind.node.ObjectNode;
 
 public class PrimitiveValueAssembler extends AbstractPrimitiveValueAssembler {
+    int counter = 0;
 
     PrimitiveValueAssembler(int level, AssemblerInfo info, IColumnValuesReader reader, IValueGetter primitiveValue) {
         super(level, info, reader, primitiveValue);
@@ -31,7 +35,7 @@
     @Override
     public int next(AssemblerState state) throws HyracksDataException {
         if (!reader.next()) {
-            throw new IllegalStateException("no more values, column index: " + getColumnIndex());
+            throw createException();
         } else if (reader.isNull() && (isDelegate() || reader.getLevel() + 1 == level)) {
             addNullToAncestor(reader.getLevel());
         } else if (reader.isValue()) {
@@ -44,4 +48,16 @@
         //Go to next value
         return -1;
     }
+
+    private ColumnarValueException createException() {
+        ColumnarValueException e = new ColumnarValueException();
+
+        ObjectNode assemblerNode = e.createNode(getClass().getSimpleName());
+        assemblerNode.put("isDelegate", isDelegate());
+
+        ObjectNode readerNode = assemblerNode.putObject("assemblerReader");
+        reader.appendReaderInformation(readerNode);
+
+        return e;
+    }
 }
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/assembler/RepeatedPrimitiveValueAssembler.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/assembler/RepeatedPrimitiveValueAssembler.java
index a10fc61..6588972 100644
--- a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/assembler/RepeatedPrimitiveValueAssembler.java
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/assembler/RepeatedPrimitiveValueAssembler.java
@@ -21,6 +21,9 @@
 import org.apache.asterix.column.assembler.value.IValueGetter;
 import org.apache.asterix.column.values.IColumnValuesReader;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.lsm.btree.column.error.ColumnarValueException;
+
+import com.fasterxml.jackson.databind.node.ObjectNode;
 
 class RepeatedPrimitiveValueAssembler extends AbstractPrimitiveValueAssembler {
     private boolean arrayDelegate;
@@ -63,7 +66,7 @@
 
     private void next() throws HyracksDataException {
         if (!reader.next()) {
-            throw new IllegalStateException("no more values, column index: " + getColumnIndex());
+            throw createException();
         } else if (reader.isNull() && (arrayDelegate || reader.getLevel() + 1 == level)) {
             /*
              * There are two cases here for where the null belongs to:
@@ -81,4 +84,16 @@
             addValueToParent();
         }
     }
+
+    private ColumnarValueException createException() {
+        ColumnarValueException e = new ColumnarValueException();
+        ObjectNode assemblerNode = e.createNode(getClass().getSimpleName());
+        assemblerNode.put("isDelegate", isDelegate());
+        assemblerNode.put("isArrayDelegate", arrayDelegate);
+
+        ObjectNode readerNode = assemblerNode.putObject("assemblerReader");
+        reader.appendReaderInformation(readerNode);
+
+        return e;
+    }
 }
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/operation/lsm/merge/MergeColumnTupleWriter.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/operation/lsm/merge/MergeColumnTupleWriter.java
index 41a8263..dfd7d6f 100644
--- a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/operation/lsm/merge/MergeColumnTupleWriter.java
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/operation/lsm/merge/MergeColumnTupleWriter.java
@@ -34,6 +34,9 @@
 import org.apache.hyracks.storage.am.lsm.btree.column.api.AbstractColumnTupleWriter;
 import org.apache.hyracks.storage.am.lsm.btree.column.api.IColumnTupleIterator;
 import org.apache.hyracks.storage.am.lsm.btree.column.api.IColumnWriteMultiPageOp;
+import org.apache.hyracks.storage.am.lsm.btree.column.error.ColumnarValueException;
+
+import com.fasterxml.jackson.databind.node.ObjectNode;
 
 public class MergeColumnTupleWriter extends AbstractColumnTupleWriter {
     private final MergeColumnWriteMetadata columnMetadata;
@@ -45,6 +48,7 @@
     private final ColumnBatchWriter writer;
     private final int maxNumberOfTuples;
     private int primaryKeysEstimatedSize;
+    private int numberOfAntiMatter;
 
     public MergeColumnTupleWriter(MergeColumnWriteMetadata columnMetadata, int pageSize, int maxNumberOfTuples,
             double tolerance) {
@@ -66,6 +70,7 @@
             primaryKeyWriters[i] = columnMetadata.getWriter(i);
         }
         orderedColumns = new PriorityQueue<>(Comparator.comparingInt(x -> -x.getEstimatedSize()));
+        numberOfAntiMatter = 0;
     }
 
     @Override
@@ -103,19 +108,48 @@
     @Override
     public void writeTuple(ITupleReference tuple) throws HyracksDataException {
         MergeColumnTupleReference columnTuple = (MergeColumnTupleReference) tuple;
+        // +1 to avoid having -0, where the '-' is an antimatter indicator
         int componentIndex = columnTuple.getComponentIndex();
         int skipCount = columnTuple.getAndResetSkipCount();
         if (skipCount > 0) {
-            writtenComponents.add(-componentIndex, skipCount);
+            writtenComponents.add(setAntimatterIndicator(componentIndex), skipCount);
         }
-        if (columnTuple.isAntimatter()) {
-            writtenComponents.add(-componentIndex);
-        } else {
+
+        if (!columnTuple.isAntimatter()) {
+            // anti matters contain only the primary keys, and none of the other columns
             writtenComponents.add(componentIndex);
+        } else {
+            // counter for logging purposes
+            numberOfAntiMatter++;
         }
+
         writePrimaryKeys(columnTuple);
     }
 
+    @Override
+    public int flush(ByteBuffer pageZero) throws HyracksDataException {
+        int numberOfColumns = columnMetadata.getNumberOfColumns();
+        int numberOfPrimaryKeys = columnMetadata.getNumberOfPrimaryKeys();
+        if (writtenComponents.getSize() > 0) {
+            writeNonKeyColumns();
+            writtenComponents.reset();
+        }
+        for (int i = numberOfPrimaryKeys; i < numberOfColumns; i++) {
+            orderedColumns.add(columnMetadata.getWriter(i));
+        }
+        writer.setPageZeroBuffer(pageZero, numberOfColumns, numberOfPrimaryKeys);
+        int allocatedSpace = writer.writePrimaryKeyColumns(primaryKeyWriters);
+        allocatedSpace += writer.writeColumns(orderedColumns);
+
+        numberOfAntiMatter = 0;
+        return allocatedSpace;
+    }
+
+    @Override
+    public void close() {
+        columnMetadata.close();
+    }
+
     private void writePrimaryKeys(MergeColumnTupleReference columnTuple) throws HyracksDataException {
         int primaryKeySize = 0;
         for (int i = 0; i < columnMetadata.getNumberOfPrimaryKeys(); i++) {
@@ -132,7 +166,7 @@
             int componentIndex = writtenComponents.getBlockValue(i);
             if (componentIndex < 0) {
                 //Skip writing values of deleted tuples
-                componentIndex = -componentIndex;
+                componentIndex = clearAntimatterIndicator(componentIndex);
                 skipReaders(componentIndex, writtenComponents.getBlockSize(i));
                 continue;
             }
@@ -141,11 +175,27 @@
             for (int j = columnMetadata.getNumberOfPrimaryKeys(); j < columnMetadata.getNumberOfColumns(); j++) {
                 IColumnValuesReader columnReader = componentTuple.getReader(j);
                 IColumnValuesWriter columnWriter = columnMetadata.getWriter(j);
-                columnReader.write(columnWriter, count);
+                writeColumn(i, componentIndex, columnReader, columnWriter, count);
             }
         }
     }
 
+    private void writeColumn(int blockIndex, int componentIndex, IColumnValuesReader columnReader,
+            IColumnValuesWriter columnWriter, int count) throws HyracksDataException {
+        try {
+            columnReader.write(columnWriter, count);
+        } catch (ColumnarValueException e) {
+            ObjectNode node = e.createNode(getClass().getSimpleName());
+            node.put("numberOfWrittenPrimaryKeys", primaryKeyWriters[0].getCount());
+            node.put("writtenComponents", writtenComponents.toString());
+            node.put("blockIndex", blockIndex);
+            node.put("componentIndex", componentIndex);
+            node.put("count", count);
+            node.put("numberOFAntiMatters", numberOfAntiMatter);
+            throw e;
+        }
+    }
+
     private void skipReaders(int componentIndex, int count) throws HyracksDataException {
         MergeColumnTupleReference componentTuple = componentsTuples[componentIndex];
         for (int j = columnMetadata.getNumberOfPrimaryKeys(); j < columnMetadata.getNumberOfColumns(); j++) {
@@ -154,28 +204,6 @@
         }
     }
 
-    @Override
-    public int flush(ByteBuffer pageZero) throws HyracksDataException {
-        int numberOfColumns = columnMetadata.getNumberOfColumns();
-        int numberOfPrimaryKeys = columnMetadata.getNumberOfPrimaryKeys();
-        if (writtenComponents.getSize() > 0) {
-            writeNonKeyColumns();
-            writtenComponents.reset();
-        }
-        for (int i = numberOfPrimaryKeys; i < numberOfColumns; i++) {
-            orderedColumns.add(columnMetadata.getWriter(i));
-        }
-        writer.setPageZeroBuffer(pageZero, numberOfColumns, numberOfPrimaryKeys);
-        int allocatedSpace = writer.writePrimaryKeyColumns(primaryKeyWriters);
-        allocatedSpace += writer.writeColumns(orderedColumns);
-        return allocatedSpace;
-    }
-
-    @Override
-    public void close() {
-        columnMetadata.close();
-    }
-
     private void writeAllColumns(MergeColumnTupleReference columnTuple) throws HyracksDataException {
         /*
          * The last tuple from one of the components was reached. Since we are going to the next leaf, we will not be
@@ -184,9 +212,18 @@
          */
         int skipCount = columnTuple.getAndResetSkipCount();
         if (skipCount > 0) {
-            writtenComponents.add(-columnTuple.getComponentIndex(), skipCount);
+            writtenComponents.add(setAntimatterIndicator(columnTuple.getComponentIndex()), skipCount);
         }
         writeNonKeyColumns();
         writtenComponents.reset();
     }
+
+    private static int setAntimatterIndicator(int componentIndex) {
+        // This is to avoid -0, where the '-' is the antimatter indicator
+        return -(componentIndex + 1);
+    }
+
+    private static int clearAntimatterIndicator(int componentIndex) {
+        return -componentIndex - 1;
+    }
 }
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/operation/query/ColumnAssembler.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/operation/query/ColumnAssembler.java
index 23a7856..30d01d5 100644
--- a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/operation/query/ColumnAssembler.java
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/operation/query/ColumnAssembler.java
@@ -29,6 +29,9 @@
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.data.std.api.IValueReference;
+import org.apache.hyracks.storage.am.lsm.btree.column.error.ColumnarValueException;
+
+import com.fasterxml.jackson.databind.node.ObjectNode;
 
 public final class ColumnAssembler {
     private final AbstractPrimitiveValueAssembler[] assemblers;
@@ -36,6 +39,7 @@
     private final AssemblerState state;
     private int numberOfTuples;
     private int tupleIndex;
+    private int numberOfSkips;
 
     public ColumnAssembler(AbstractSchemaNode node, ARecordType declaredType, QueryColumnMetadata columnMetadata,
             IColumnValuesReaderFactory readerFactory, IValueGetterFactory valueGetterFactory)
@@ -50,6 +54,7 @@
     public void reset(int numberOfTuples) {
         this.numberOfTuples = numberOfTuples;
         tupleIndex = 0;
+        numberOfSkips = 0;
     }
 
     public void resetColumn(AbstractBytesInputStream stream, int ordinal) throws HyracksDataException {
@@ -75,7 +80,15 @@
         int index = 0;
         while (index < assemblers.length) {
             AbstractPrimitiveValueAssembler assembler = assemblers[index];
-            int groupIndex = assembler.next(state);
+            int groupIndex;
+
+            try {
+                groupIndex = assembler.next(state);
+            } catch (ColumnarValueException e) {
+                appendInformation(e);
+                throw e;
+            }
+
             if (groupIndex != AbstractPrimitiveValueAssembler.NEXT_ASSEMBLER) {
                 index = groupIndex;
             } else {
@@ -93,6 +106,7 @@
     }
 
     public int skip(int count) throws HyracksDataException {
+        numberOfSkips += count;
         tupleIndex += count;
         for (int i = 0; i < assemblers.length; i++) {
             assemblers[i].skip(count);
@@ -103,4 +117,12 @@
     public void setAt(int index) throws HyracksDataException {
         skip(index - tupleIndex);
     }
+
+    private void appendInformation(ColumnarValueException e) {
+        ObjectNode assemblerNode = e.createNode(getClass().getSimpleName());
+        assemblerNode.put("tupleIndex", tupleIndex);
+        assemblerNode.put("numberOfTuples", numberOfTuples);
+        assemblerNode.put("numberOfSkips", numberOfSkips);
+        state.appendStateInfo(e);
+    }
 }
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/operation/query/QueryColumnMetadata.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/operation/query/QueryColumnMetadata.java
index 77bf945..97cc740 100644
--- a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/operation/query/QueryColumnMetadata.java
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/operation/query/QueryColumnMetadata.java
@@ -218,21 +218,18 @@
             filterAccessorProvider.reset();
             columnFilterEvaluator = columnFilterEvaluatorFactory.create(filterAccessorProvider, evaluatorContext);
             filterColumnReaders = filterAccessorProvider.getFilterColumnReaders();
+
         }
 
+        // log normalized filter
+        logFilter(normalizedFilterEvaluator, normalizedEvaluatorFactory.toString());
+        // log requested schema
+        logSchema(clippedRoot, SchemaStringBuilderVisitor.RECORD_SCHEMA, fieldNamesDictionary);
+
         // Primary key readers
         PrimitiveColumnValuesReader[] primaryKeyReaders =
                 createPrimaryKeyReaders(input, readerFactory, numberOfPrimaryKeys);
 
-        if (LOGGER.isInfoEnabled() && normalizedFilterEvaluator != TrueColumnFilterEvaluator.INSTANCE) {
-            String filterString = normalizedFilterEvaluator == FalseColumnFilterEvaluator.INSTANCE ? "SKIP_ALL"
-                    : LogRedactionUtil.userData(normalizedEvaluatorFactory.toString());
-            LOGGER.info("Filter: {}", filterString);
-        }
-
-        // log requested schema
-        logSchema(clippedRoot, SchemaStringBuilderVisitor.RECORD_SCHEMA, fieldNamesDictionary);
-
         return new QueryColumnMetadata(datasetType, null, primaryKeyReaders, serializedMetadata, fieldNamesDictionary,
                 clippedRoot, readerFactory, valueGetterFactory, normalizedFilterEvaluator, filterValueAccessors,
                 columnFilterEvaluator, filterColumnReaders);
@@ -261,6 +258,14 @@
         return primaryKeyReaders;
     }
 
+    protected static void logFilter(IColumnFilterEvaluator normalizedFilterEvaluator, String filterExpression) {
+        if (LOGGER.isInfoEnabled() && normalizedFilterEvaluator != TrueColumnFilterEvaluator.INSTANCE) {
+            String filterString = normalizedFilterEvaluator == FalseColumnFilterEvaluator.INSTANCE ? "SKIP_ALL"
+                    : LogRedactionUtil.userData(filterExpression);
+            LOGGER.info("Filter: {}", filterString);
+        }
+    }
+
     protected static void logSchema(ObjectSchemaNode root, String schemaSource,
             FieldNamesDictionary fieldNamesDictionary) throws HyracksDataException {
         if (LOGGER.isInfoEnabled()) {
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/operation/query/QueryColumnWithMetaMetadata.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/operation/query/QueryColumnWithMetaMetadata.java
index e63358b..c23dbcf 100644
--- a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/operation/query/QueryColumnWithMetaMetadata.java
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/operation/query/QueryColumnWithMetaMetadata.java
@@ -170,15 +170,17 @@
             filterColumnReaders = filterAccessorProvider.getFilterColumnReaders();
         }
 
-        // Primary key readers
-        PrimitiveColumnValuesReader[] primaryKeyReaders =
-                createPrimaryKeyReaders(input, readerFactory, numberOfPrimaryKeys);
-
+        // log normalized filter
+        logFilter(normalizedFilterEvaluator, normalizedEvaluatorFactory.toString());
         // log requested schema for record
         logSchema(clippedRoot, SchemaStringBuilderVisitor.RECORD_SCHEMA, fieldNamesDictionary);
         // log requested schema for meta-record
         logSchema(metaClippedRoot, SchemaStringBuilderVisitor.META_RECORD_SCHEMA, fieldNamesDictionary);
 
+        // Primary key readers
+        PrimitiveColumnValuesReader[] primaryKeyReaders =
+                createPrimaryKeyReaders(input, readerFactory, numberOfPrimaryKeys);
+
         return new QueryColumnWithMetaMetadata(datasetType, metaType, primaryKeyReaders, serializedMetadata,
                 fieldNamesDictionary, clippedRoot, metaClippedRoot, readerFactory, valueGetterFactory,
                 normalizedFilterEvaluator, filterValueAccessors, columnFilterEvaluator, filterColumnReaders);
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/tuple/MergeColumnTupleReference.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/tuple/MergeColumnTupleReference.java
index 61030e1..7657009 100644
--- a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/tuple/MergeColumnTupleReference.java
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/tuple/MergeColumnTupleReference.java
@@ -33,6 +33,8 @@
 import org.apache.hyracks.storage.am.lsm.btree.column.impls.btree.ColumnBTreeReadLeafFrame;
 
 public final class MergeColumnTupleReference extends AbstractAsterixColumnTupleReference {
+    // NoOP callback is for empty pages only
+    private static final IEndOfPageCallBack EMPTY_PAGE_CALLBACK = createNoOpCallBack();
     private final IColumnValuesReader[] columnReaders;
     private int skipCount;
     private IEndOfPageCallBack endOfPageCallBack;
@@ -41,6 +43,7 @@
             MergeColumnReadMetadata columnMetadata, IColumnReadMultiPageOp multiPageOp) {
         super(componentIndex, frame, columnMetadata, multiPageOp);
         this.columnReaders = columnMetadata.getColumnReaders();
+        endOfPageCallBack = EMPTY_PAGE_CALLBACK;
     }
 
     @Override
@@ -112,4 +115,14 @@
     public void registerEndOfPageCallBack(IEndOfPageCallBack endOfPageCallBack) {
         this.endOfPageCallBack = endOfPageCallBack;
     }
+
+    private static IEndOfPageCallBack createNoOpCallBack() {
+        return columnTuple -> {
+            if (!columnTuple.isEmpty()) {
+                // safeguard against unset proper call back for non-empty pages
+                throw new NullPointerException("endOfPageCallBack is null");
+            }
+        };
+    }
+
 }
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/IColumnValuesReader.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/IColumnValuesReader.java
index 7ec6e8a..fcb21c0 100644
--- a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/IColumnValuesReader.java
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/IColumnValuesReader.java
@@ -23,6 +23,8 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.data.std.api.IValueReference;
 
+import com.fasterxml.jackson.databind.node.ObjectNode;
+
 public interface IColumnValuesReader extends Comparable<IColumnValuesReader> {
     /**
      * Reset the reader
@@ -139,4 +141,11 @@
      * @param count the number of values should be skipped
      */
     void skip(int count) throws HyracksDataException;
+
+    /**
+     * Appends readers information for debugging
+     *
+     * @param node container for the reader's information
+     */
+    void appendReaderInformation(ObjectNode node);
 }
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/reader/AbstractColumnValuesReader.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/reader/AbstractColumnValuesReader.java
index 41dfea2..7fd206a 100644
--- a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/reader/AbstractColumnValuesReader.java
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/reader/AbstractColumnValuesReader.java
@@ -33,6 +33,8 @@
 import org.apache.hyracks.data.std.api.IValueReference;
 import org.apache.parquet.bytes.BytesUtils;
 
+import com.fasterxml.jackson.databind.node.ObjectNode;
+
 abstract class AbstractColumnValuesReader implements IColumnValuesReader {
     protected final AbstractValueReader valueReader;
     protected final int columnIndex;
@@ -47,6 +49,10 @@
     private boolean nullLevel;
     private boolean allMissing;
 
+    // Logging members
+    private int numberOfEncounteredMissing;
+    private int numberOfEncounteredNull;
+
     AbstractColumnValuesReader(AbstractValueReader valueReader, int columnIndex, int maxLevel, boolean primaryKey) {
         this.valueReader = valueReader;
         this.columnIndex = columnIndex;
@@ -61,15 +67,13 @@
         }
 
         valueIndex++;
-        try {
-            int actualLevel = definitionLevels.readInt();
-            //Check whether the level is for a null value
-            nullLevel = ColumnValuesUtil.isNull(nullBitMask, actualLevel);
-            //Clear the null bit to allow repeated value readers determine the correct delimiter for null values
-            level = ColumnValuesUtil.clearNullBit(nullBitMask, actualLevel);
-        } catch (IOException e) {
-            throw HyracksDataException.create(e);
-        }
+        int actualLevel = definitionLevels.readInt();
+        //Check whether the level is for a null value
+        nullLevel = ColumnValuesUtil.isNull(nullBitMask, actualLevel);
+        //Clear the null bit to allow repeated value readers determine the correct delimiter for null values
+        level = ColumnValuesUtil.clearNullBit(nullBitMask, actualLevel);
+        numberOfEncounteredMissing += isMissing() ? 1 : 0;
+        numberOfEncounteredNull += isNull() ? 1 : 0;
     }
 
     abstract void resetValues();
@@ -77,6 +81,8 @@
     @Override
     public final void reset(AbstractBytesInputStream in, int tupleCount) throws HyracksDataException {
         valueIndex = 0;
+        numberOfEncounteredMissing = 0;
+        numberOfEncounteredNull = 0;
         if (in.available() == 0) {
             allMissing = true;
             level = 0;
@@ -168,4 +174,17 @@
             next();
         }
     }
+
+    protected void appendCommon(ObjectNode node) {
+        node.put("typeTag", getTypeTag().toString());
+        node.put("columnIndex", columnIndex);
+        node.put("valueIndex", valueIndex);
+        node.put("valueCount", valueCount);
+        node.put("allMissing", allMissing);
+        node.put("level", level);
+        node.put("maxLevel", maxLevel);
+        node.put("nullBitMask", nullBitMask);
+        node.put("numberOfEncounteredMissing", numberOfEncounteredMissing);
+        node.put("numberOfEncounteredNull", numberOfEncounteredNull);
+    }
 }
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/reader/PrimitiveColumnValuesReader.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/reader/PrimitiveColumnValuesReader.java
index 8fdf1b5..c6d2b1e 100644
--- a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/reader/PrimitiveColumnValuesReader.java
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/reader/PrimitiveColumnValuesReader.java
@@ -25,6 +25,9 @@
 import org.apache.asterix.column.values.reader.value.AbstractValueReader;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.data.std.api.IValueReference;
+import org.apache.hyracks.storage.am.lsm.btree.column.error.ColumnarValueException;
+
+import com.fasterxml.jackson.databind.node.ObjectNode;
 
 /**
  * Reader for a non-repeated primitive value
@@ -52,13 +55,9 @@
             return false;
         }
 
-        try {
-            nextLevel();
-            if (primaryKey || level == maxLevel) {
-                valueReader.nextValue();
-            }
-        } catch (IOException e) {
-            throw HyracksDataException.create(e);
+        nextLevel();
+        if (primaryKey || level == maxLevel) {
+            valueReader.nextValue();
         }
         return true;
     }
@@ -96,7 +95,9 @@
     @Override
     public void write(IColumnValuesWriter writer, boolean callNext) throws HyracksDataException {
         if (callNext && !next()) {
-            throw new IllegalStateException("No more values");
+            ColumnarValueException e = new ColumnarValueException();
+            appendReaderInformation(e.createNode(getClass().getSimpleName()));
+            throw e;
         }
 
         writer.writeLevel(level);
@@ -122,4 +123,11 @@
             nextLevel();
         }
     }
+
+    @Override
+    public void appendReaderInformation(ObjectNode node) {
+        appendCommon(node);
+        node.put("isPrimaryKeyColumn", primaryKey);
+    }
+
 }
diff --git a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/reader/RepeatedPrimitiveColumnValuesReader.java b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/reader/RepeatedPrimitiveColumnValuesReader.java
index 9848878..4f9fdec 100644
--- a/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/reader/RepeatedPrimitiveColumnValuesReader.java
+++ b/asterixdb/asterix-column/src/main/java/org/apache/asterix/column/values/reader/RepeatedPrimitiveColumnValuesReader.java
@@ -19,11 +19,15 @@
 package org.apache.asterix.column.values.reader;
 
 import java.io.IOException;
+import java.util.Arrays;
 
 import org.apache.asterix.column.values.IColumnValuesWriter;
 import org.apache.asterix.column.values.reader.value.AbstractValueReader;
 import org.apache.asterix.om.types.ATypeTag;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.lsm.btree.column.error.ColumnarValueException;
+
+import com.fasterxml.jackson.databind.node.ObjectNode;
 
 /**
  * For primitive values that belong to an {@link ATypeTag#ARRAY} and {@link ATypeTag#MULTISET}
@@ -93,7 +97,9 @@
     public void write(IColumnValuesWriter writer, boolean callNext) throws HyracksDataException {
         //We always call next as repeated values cannot be primary keys
         if (!next()) {
-            throw new IllegalStateException("No more values");
+            ColumnarValueException e = new ColumnarValueException();
+            appendReaderInformation(e.createNode(getClass().getSimpleName()));
+            throw e;
         }
 
         if (isRepeatedValue()) {
@@ -123,6 +129,18 @@
         return isDelimiter() && delimiterIndex == delimiters.length - 1;
     }
 
+    @Override
+    public void skip(int count) throws HyracksDataException {
+        for (int i = 0; i < count; i++) {
+            next();
+            if (isRepeatedValue()) {
+                while (!isLastDelimiter()) {
+                    next();
+                }
+            }
+        }
+    }
+
     private void consumeDelimiterIfAny() {
         if (isDelimiter()) {
             delimiterIndex++;
@@ -137,14 +155,11 @@
     }
 
     @Override
-    public void skip(int count) throws HyracksDataException {
-        for (int i = 0; i < count; i++) {
-            next();
-            if (isRepeatedValue()) {
-                while (!isLastDelimiter()) {
-                    next();
-                }
-            }
-        }
+    public void appendReaderInformation(ObjectNode node) {
+        appendCommon(node);
+        node.put("delimiters", Arrays.toString(delimiters));
+        node.put("levelToDelimiterMap", Arrays.toString(levelToDelimiterMap));
+        node.put("delimiterIndex", delimiterIndex);
+        node.put("isDelimiter", isDelimiter());
     }
 }
diff --git a/asterixdb/asterix-column/src/test/java/org/apache/asterix/column/values/reader/AbstractDummyColumnValuesReader.java b/asterixdb/asterix-column/src/test/java/org/apache/asterix/column/values/reader/AbstractDummyColumnValuesReader.java
index 51c360a..a3c6984 100644
--- a/asterixdb/asterix-column/src/test/java/org/apache/asterix/column/values/reader/AbstractDummyColumnValuesReader.java
+++ b/asterixdb/asterix-column/src/test/java/org/apache/asterix/column/values/reader/AbstractDummyColumnValuesReader.java
@@ -29,6 +29,8 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.data.std.api.IValueReference;
 
+import com.fasterxml.jackson.databind.node.ObjectNode;
+
 public abstract class AbstractDummyColumnValuesReader implements IColumnValuesReader {
     private final ATypeTag typeTag;
     protected final int columnIndex;
@@ -154,4 +156,12 @@
             next();
         }
     }
+
+    protected void appendCommon(ObjectNode node) {
+        node.put("columnIndex", columnIndex);
+        node.put("valueIndex", valueIndex);
+        node.put("valueCount", valueCount);
+        node.put("level", level);
+        node.put("maxLevel", maxLevel);
+    }
 }
diff --git a/asterixdb/asterix-column/src/test/java/org/apache/asterix/column/values/reader/DummyPrimitiveColumnValueReader.java b/asterixdb/asterix-column/src/test/java/org/apache/asterix/column/values/reader/DummyPrimitiveColumnValueReader.java
index f888db4..6410daa 100644
--- a/asterixdb/asterix-column/src/test/java/org/apache/asterix/column/values/reader/DummyPrimitiveColumnValueReader.java
+++ b/asterixdb/asterix-column/src/test/java/org/apache/asterix/column/values/reader/DummyPrimitiveColumnValueReader.java
@@ -25,6 +25,8 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.data.std.api.IValueReference;
 
+import com.fasterxml.jackson.databind.node.ObjectNode;
+
 public class DummyPrimitiveColumnValueReader extends AbstractDummyColumnValuesReader {
     DummyPrimitiveColumnValueReader(ATypeTag typeTag, RunLengthIntArray defLevels, List<IValueReference> values,
             int columnIndex, int maxLevel) {
@@ -73,4 +75,10 @@
     public int getNumberOfDelimiters() {
         return 0;
     }
+
+    @Override
+    public void appendReaderInformation(ObjectNode node) {
+        appendCommon(node);
+        node.put("isPrimaryKeyColumn", false);
+    }
 }
diff --git a/asterixdb/asterix-column/src/test/java/org/apache/asterix/column/values/reader/DummyRepeatedPrimitiveColumnValueReader.java b/asterixdb/asterix-column/src/test/java/org/apache/asterix/column/values/reader/DummyRepeatedPrimitiveColumnValueReader.java
index 06c94c3..d9ae969 100644
--- a/asterixdb/asterix-column/src/test/java/org/apache/asterix/column/values/reader/DummyRepeatedPrimitiveColumnValueReader.java
+++ b/asterixdb/asterix-column/src/test/java/org/apache/asterix/column/values/reader/DummyRepeatedPrimitiveColumnValueReader.java
@@ -18,6 +18,7 @@
  */
 package org.apache.asterix.column.values.reader;
 
+import java.util.Arrays;
 import java.util.List;
 
 import org.apache.asterix.column.util.RunLengthIntArray;
@@ -25,6 +26,8 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.data.std.api.IValueReference;
 
+import com.fasterxml.jackson.databind.node.ObjectNode;
+
 public class DummyRepeatedPrimitiveColumnValueReader extends AbstractDummyColumnValuesReader {
     private final int[] delimiters;
     private final int[] levelToDelimiterMap;
@@ -104,4 +107,13 @@
         }
         delimiterIndex = levelToDelimiterMap[level];
     }
+
+    @Override
+    public void appendReaderInformation(ObjectNode node) {
+        appendCommon(node);
+        node.put("delimiters", Arrays.toString(delimiters));
+        node.put("levelToDelimiterMap", Arrays.toString(levelToDelimiterMap));
+        node.put("delimiterIndex", delimiterIndex);
+        node.put("isDelimiter", isDelimiter());
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/error/ColumnarValueException.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/error/ColumnarValueException.java
new file mode 100644
index 0000000..476e1a1
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/error/ColumnarValueException.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.hyracks.storage.am.lsm.btree.column.error;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.node.ObjectNode;
+
+/**
+ * This exception is thrown when an error is encountered in columnar collections
+ * It contains a string builder to collect information about the encountered error
+ */
+public class ColumnarValueException extends IllegalStateException {
+    private static final long serialVersionUID = 1513428477557736034L;
+    private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
+    private final ObjectNode node;
+
+    @Override
+    public String getMessage() {
+        return node.toString();
+    }
+
+    public ColumnarValueException() {
+        node = OBJECT_MAPPER.createObjectNode();
+    }
+
+    public ObjectNode createNode(String fieldName) {
+        return node.putObject(fieldName);
+    }
+
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/lsm/tuples/AbstractColumnTupleReference.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/lsm/tuples/AbstractColumnTupleReference.java
index 310b0b6..d3b4673 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/lsm/tuples/AbstractColumnTupleReference.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/lsm/tuples/AbstractColumnTupleReference.java
@@ -172,6 +172,10 @@
         return frame.getTupleCount();
     }
 
+    protected final boolean isEmpty() {
+        return frame.getTupleCount() == 0;
+    }
+
     @Override
     public final void next() throws HyracksDataException {
         onNext();