You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@asterixdb.apache.org by gg...@apache.org on 2021/09/10 17:03:36 UTC

[asterixdb] branch master updated: [ASTERIXDB-2958][IDX] Fix array index ingestion for datasets with meta fields

This is an automated email from the ASF dual-hosted git repository.

ggalvizo pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/asterixdb.git


The following commit(s) were added to refs/heads/master by this push:
     new 032d694  [ASTERIXDB-2958][IDX] Fix array index ingestion for datasets with meta fields
032d694 is described below

commit 032d694548ad3972afdab5855a0422886d8704ec
Author: ggalvizo <gg...@uci.edu>
AuthorDate: Thu Sep 9 19:16:37 2021 -0700

    [ASTERIXDB-2958][IDX] Fix array index ingestion for datasets with meta fields
    
    - user model changes: no
    - storage format changes: no
    - interface changes: no
    
    Loading job for array index creation has been fixed to project away the
    meta field before performing the UNNEST on the record itself. This fixes
    the issue where the index was not being populated when an array index
    was created on a dataset with a meta field.
    
    Change-Id: I19859eb8a3becf44438afb1ad66b1ae672d21ff3
    Reviewed-on: https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/13084
    Reviewed-by: Glenn Galvizo <gg...@uci.edu>
    Reviewed-by: Dmitry Lychagin <dm...@couchbase.com>
    Integration-Tests: Jenkins <je...@fulliautomatix.ics.uci.edu>
    Tested-by: Jenkins <je...@fulliautomatix.ics.uci.edu>
---
 asterixdb/asterix-app/data/yelp-checkin/feeds.csv  |  3 ++
 .../using-feed-new-index.1.ddl.sqlpp               | 55 ++++++++++++++++++++
 .../using-feed-new-index.2.update.sqlpp            | 26 ++++++++++
 .../using-feed-new-index.3.ddl.sqlpp               | 23 +++++++++
 .../using-feed-new-index.4.query.sqlpp             | 26 ++++++++++
 .../using-feed-old-index.1.ddl.sqlpp               | 58 ++++++++++++++++++++++
 .../using-feed-old-index.2.update.sqlpp            | 26 ++++++++++
 .../using-feed-old-index.3.query.sqlpp             | 26 ++++++++++
 .../open/using-feed/using-feed.1.adm               |  1 +
 .../test/resources/runtimets/testsuite_sqlpp.xml   | 10 ++++
 .../SecondaryArrayIndexBTreeOperationsHelper.java  | 47 ++++++++++++++----
 11 files changed, 291 insertions(+), 10 deletions(-)

diff --git a/asterixdb/asterix-app/data/yelp-checkin/feeds.csv b/asterixdb/asterix-app/data/yelp-checkin/feeds.csv
new file mode 100644
index 0000000..b7bd79c
--- /dev/null
+++ b/asterixdb/asterix-app/data/yelp-checkin/feeds.csv
@@ -0,0 +1,3 @@
+1,"{ ""business_id"": ""--1UhMGODdWsrMastO9DZw"", ""dates"": [ ""2016-04-26 19:49:16"", ""2016-08-30 18:36:57"", ""2016-10-15 02:45:18"", ""2016-11-18 01:54:50"", ""2017-04-20 18:39:06"", ""2017-05-03 17:58:02"", ""2019-03-19 22:04:48"" ] }"
+2,"{ ""business_id"": ""--EF5N7P70J_UYBTPypYlA"", ""dates"": [ ""2018-05-25 19:52:07"", ""2018-09-18 16:09:44"", ""2019-10-18 21:29:09"" ] }"
+3,"{ ""business_id"": ""--Ni3oJ4VOqfOEu7Sj2Vzg"", ""dates"": [ ""2019-06-07 17:54:58"" ] }"
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/using-feed-new-index/using-feed-new-index.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/using-feed-new-index/using-feed-new-index.1.ddl.sqlpp
new file mode 100644
index 0000000..3180b30
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/using-feed-new-index/using-feed-new-index.1.ddl.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.
+ */
+
+/*
+ * Description: Verify that we are able to correctly perform the following types of queries using an array->atomic index with a feed (+ meta field).
+ *     1) UNNEST query, performing a point search.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    business_id: string
+};
+
+CREATE TYPE MetaType AS {
+    _id: string
+};
+
+CREATE DATASET YelpCheckin(CheckinType)
+WITH META(MetaType)
+PRIMARY KEY business_id;
+
+CREATE FEED YelpCheckinStream WITH {
+ "adapter-name" : "localfs",
+ "reader" : "localfs",
+ "parser" : "record-with-metadata",
+ "type-name" : "CheckinType",
+ "meta-type-name" : "MetaType",
+ "path" : "asterix_nc1://data/yelp-checkin/feeds.csv",
+ "format": "csv",
+ "delimiter": ",",
+ "record-format": "adm",
+ "record-index" : "1",
+ "key-indexes" : "0",
+ "key-indicators" : "1",
+ "header": "false"
+};
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/using-feed-new-index/using-feed-new-index.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/using-feed-new-index/using-feed-new-index.2.update.sqlpp
new file mode 100644
index 0000000..13a634b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/using-feed-new-index/using-feed-new-index.2.update.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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 TestYelp;
+
+SET `wait-for-completion-feed` "true";
+CONNECT FEED YelpCheckinStream
+TO DATASET YelpCheckin;
+
+START FEED YelpCheckinStream;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/using-feed-new-index/using-feed-new-index.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/using-feed-new-index/using-feed-new-index.3.ddl.sqlpp
new file mode 100644
index 0000000..4a74895
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/using-feed-new-index/using-feed-new-index.3.ddl.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 TestYelp;
+
+CREATE INDEX checkinDatesIdx
+ON YelpCheckin ( UNNEST dates : string );
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/using-feed-new-index/using-feed-new-index.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/using-feed-new-index/using-feed-new-index.4.query.sqlpp
new file mode 100644
index 0000000..9078e62
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/using-feed-new-index/using-feed-new-index.4.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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 TestYelp;
+SET `compiler.arrayindex` "true";
+
+FROM    YelpCheckin C
+UNNEST  C.dates D
+WHERE   D > ""
+SELECT  COUNT(*);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/using-feed-old-index/using-feed-old-index.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/using-feed-old-index/using-feed-old-index.1.ddl.sqlpp
new file mode 100644
index 0000000..a19d87d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/using-feed-old-index/using-feed-old-index.1.ddl.sqlpp
@@ -0,0 +1,58 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description: Verify that we are able to correctly perform the following types of queries using an array->atomic index with a feed (+ meta field).
+ *     1) UNNEST query, performing a point search.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+    business_id: string
+};
+
+CREATE TYPE MetaType AS {
+    _id: string
+};
+
+CREATE DATASET YelpCheckin(CheckinType)
+WITH META(MetaType)
+PRIMARY KEY business_id;
+
+CREATE INDEX checkinDatesIdx
+ON YelpCheckin ( UNNEST dates : string );
+
+CREATE FEED YelpCheckinStream WITH {
+ "adapter-name" : "localfs",
+ "reader" : "localfs",
+ "parser" : "record-with-metadata",
+ "type-name" : "CheckinType",
+ "meta-type-name" : "MetaType",
+ "path" : "asterix_nc1://data/yelp-checkin/feeds.csv",
+ "format": "csv",
+ "delimiter": ",",
+ "record-format": "adm",
+ "record-index" : "1",
+ "key-indexes" : "0",
+ "key-indicators" : "1",
+ "header": "false"
+};
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/using-feed-old-index/using-feed-old-index.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/using-feed-old-index/using-feed-old-index.2.update.sqlpp
new file mode 100644
index 0000000..13a634b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/using-feed-old-index/using-feed-old-index.2.update.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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 TestYelp;
+
+SET `wait-for-completion-feed` "true";
+CONNECT FEED YelpCheckinStream
+TO DATASET YelpCheckin;
+
+START FEED YelpCheckinStream;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/using-feed-old-index/using-feed-old-index.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/using-feed-old-index/using-feed-old-index.3.query.sqlpp
new file mode 100644
index 0000000..9078e62
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/using-feed-old-index/using-feed-old-index.3.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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 TestYelp;
+SET `compiler.arrayindex` "true";
+
+FROM    YelpCheckin C
+UNNEST  C.dates D
+WHERE   D > ""
+SELECT  COUNT(*);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/open/using-feed/using-feed.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/open/using-feed/using-feed.1.adm
new file mode 100644
index 0000000..fb7e3a9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/open/using-feed/using-feed.1.adm
@@ -0,0 +1 @@
+{ "$1": 11 }
\ No newline at end of file
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 6a94c3e..6879938 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
@@ -8451,6 +8451,16 @@
           <output-dir compare="Text">with-filter-fields</output-dir>
         </compilation-unit>
       </test-case>
+      <test-case FilePath="array-index/select-unnest-queries/open">
+        <compilation-unit name="using-feed-new-index">
+          <output-dir compare="Text">using-feed</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/select-unnest-queries/open">
+        <compilation-unit name="using-feed-old-index">
+          <output-dir compare="Text">using-feed</output-dir>
+        </compilation-unit>
+      </test-case>
     </test-group>
     <test-group name="array-index/select-quantified-queries">
       <test-case FilePath="array-index/select-quantified-queries">
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryArrayIndexBTreeOperationsHelper.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryArrayIndexBTreeOperationsHelper.java
index 8ce63f1..5d80dca 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryArrayIndexBTreeOperationsHelper.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryArrayIndexBTreeOperationsHelper.java
@@ -152,16 +152,13 @@ public class SecondaryArrayIndexBTreeOperationsHelper extends SecondaryTreeIndex
         ITypeTraitProvider typeTraitProvider = metadataProvider.getDataFormat().getTypeTraitProvider();
         IBinaryComparatorFactoryProvider comparatorFactoryProvider =
                 metadataProvider.getDataFormat().getBinaryComparatorFactoryProvider();
-        // Record column is 0 for external datasets, numPrimaryKeys for internal ones
-        int recordColumn = dataset.getDatasetType() == DatasetType.INTERNAL ? numPrimaryKeys : 0;
         boolean isOverridingKeyFieldTypes = arrayIndexDetails.isOverridingKeyFieldTypes();
         int flattenedListPos = 0;
         for (Index.ArrayIndexElement e : arrayIndexDetails.getElementList()) {
             for (int i = 0; i < e.getProjectList().size(); i++) {
-                ARecordType sourceType = (e.getSourceIndicator() == 0) ? itemType : metaType;
-                addSKEvalFactories(isOverridingKeyFieldTypes ? enforcedItemType : sourceType, flattenedListPos, false);
+                addSKEvalFactories(isOverridingKeyFieldTypes ? enforcedItemType : itemType, flattenedListPos, false);
                 Pair<IAType, Boolean> keyTypePair = ArrayIndexUtil.getNonNullableOpenFieldType(e.getTypeList().get(i),
-                        e.getUnnestList(), e.getProjectList().get(i), sourceType);
+                        e.getUnnestList(), e.getProjectList().get(i), itemType);
                 IAType keyType = keyTypePair.first;
                 anySecondaryKeyIsNullable = anySecondaryKeyIsNullable || keyTypePair.second;
                 ISerializerDeserializer keySerde = serdeProvider.getSerializerDeserializer(keyType);
@@ -251,8 +248,8 @@ public class SecondaryArrayIndexBTreeOperationsHelper extends SecondaryTreeIndex
     }
 
     /**
-     * The following job spec is produced: (key provider) -> (PIDX scan) -> (cast)? -> ((unnest) -> (assign))* ->
-     * (select)? -> (sort)? -> (bulk load) -> (sink)
+     * The following job spec is produced: (key provider) -> (PIDX scan) -> (cast)? -> (assign)? ->
+     * ((unnest) -> (assign))* -> (select)? -> (sort)? -> (bulk load) -> (sink)
      */
     @Override
     public JobSpecification buildLoadingJobSpec() throws AlgebricksException {
@@ -275,6 +272,26 @@ public class SecondaryArrayIndexBTreeOperationsHelper extends SecondaryTreeIndex
                 sourceOp = targetOp;
             }
 
+            // We do not index meta fields. Project away meta fields if they exist.
+            if (dataset.hasMetaPart()) {
+                int[] outColumns = new int[] { primaryRecDesc.getFieldCount() };
+                int[] projectionList = new int[primaryRecDesc.getFieldCount() - 1];
+                for (int i = 0; i < projectionList.length - 1; i++) {
+                    projectionList[i] = i;
+                }
+                projectionList[projectionList.length - 1] = primaryRecDesc.getFieldCount() - 2;
+                ISerializerDeserializer[] fields = new ISerializerDeserializer[primaryRecDesc.getFieldCount() - 1];
+                ITypeTraits[] typeTraits = new ITypeTraits[primaryRecDesc.getFieldCount() - 1];
+                for (int i = 0; i < primaryRecDesc.getFieldCount() - 1; i++) {
+                    fields[i] = primaryRecDesc.getFields()[i];
+                    typeTraits[i] = primaryRecDesc.getTypeTraits()[i];
+                }
+                targetOp = createGenericAssignOp(spec, new ArrayList<>(), new RecordDescriptor(fields, typeTraits),
+                        outColumns, projectionList);
+                spec.connect(new OneToOneConnectorDescriptor(spec), sourceOp, 0, targetOp, 0);
+                sourceOp = targetOp;
+            }
+
             // Perform the unnest work.
             final Mutable<IOperatorDescriptor> sourceOpRef = new MutableObject<>(sourceOp);
             final Mutable<IOperatorDescriptor> targetOpRef = new MutableObject<>(targetOp);
@@ -508,7 +525,7 @@ public class SecondaryArrayIndexBTreeOperationsHelper extends SecondaryTreeIndex
         IScalarEvaluatorFactory sef = metadataProvider.getDataFormat().getFieldAccessEvaluatorFactory(
                 metadataProvider.getFunctionManager(), recordType, filterFieldName, numPrimaryKeys, sourceLoc);
         evalFactoryAndRecDescStackBuilder.addFilter(sef,
-                Index.getNonNullableKeyFieldType(filterFieldName, itemType).first);
+                Index.getNonNullableKeyFieldType(filterFieldName, recordType).first);
     }
 
     class EvalFactoryAndRecDescInvoker implements ArrayIndexUtil.TypeTrackerCommandExecutor {
@@ -690,8 +707,18 @@ public class SecondaryArrayIndexBTreeOperationsHelper extends SecondaryTreeIndex
         public Stack<RecordDescriptor> buildRecDescStack() throws AlgebricksException {
             int initialUnnestEvalTypesSize = unnestEvalTypes.size();
             Deque<RecordDescriptor> resultantAsDeque = new ArrayDeque<>();
-            resultantAsDeque.addFirst(primaryRecDesc);
-            resultantAsDeque.addFirst(createUnnestRecDesc(primaryRecDesc, unnestEvalTypes.remove()));
+            RecordDescriptor recDescBeforeFirstUnnest = primaryRecDesc;
+            if (dataset.hasMetaPart()) {
+                ISerializerDeserializer[] fields = new ISerializerDeserializer[primaryRecDesc.getFieldCount() - 1];
+                ITypeTraits[] typeTraits = new ITypeTraits[primaryRecDesc.getFieldCount() - 1];
+                for (int i = 0; i < primaryRecDesc.getFieldCount() - 1; i++) {
+                    fields[i] = primaryRecDesc.getFields()[i];
+                    typeTraits[i] = primaryRecDesc.getTypeTraits()[i];
+                }
+                recDescBeforeFirstUnnest = new RecordDescriptor(fields, typeTraits);
+            }
+            resultantAsDeque.addFirst(recDescBeforeFirstUnnest);
+            resultantAsDeque.addFirst(createUnnestRecDesc(recDescBeforeFirstUnnest, unnestEvalTypes.remove()));
             for (int i = 0; i < initialUnnestEvalTypesSize - 1; i++) {
                 resultantAsDeque.addFirst(createAssignRecDesc(resultantAsDeque.getFirst(), i == 0));
                 resultantAsDeque.addFirst(createUnnestRecDesc(resultantAsDeque.getFirst(), unnestEvalTypes.remove()));