[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();