You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@asterixdb.apache.org by "Murtadha Hubail (Code Review)" <do...@asterixdb.incubator.apache.org> on 2019/04/05 17:31:30 UTC

Change in asterixdb[master]: [NO ISSUE][FUN] Add Function To Return Secondary Index Tuples

Murtadha Hubail has uploaded a new change for review.

  https://asterix-gerrit.ics.uci.edu/3325

Change subject: [NO ISSUE][FUN] Add Function To Return Secondary Index Tuples
......................................................................

[NO ISSUE][FUN] Add Function To Return Secondary Index Tuples

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

Details:
- Add a new private function (dump_index) that can be used
  to query all tuples of a secondary index. This function
  can be used for testing to validate the expected contents
  of secondary indexes.
- Add test case.

Change-Id: Iefae2a798126b9efb049596417b93499d075a608
---
A asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/DumpIndexDatasource.java
A asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/DumpIndexFunction.java
A asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/DumpIndexReader.java
A asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/DumpIndexRewriter.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/util/MetadataBuiltinFunctions.java
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/dump_index/dump_index.1.ddl.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/dump_index/dump_index.2.update.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/dump_index/dump_index.3.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/results/misc/dump_index/dump_index.3.adm
M asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
M asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/dataflow/RecordDataFlowController.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryIndexOperationsHelper.java
12 files changed, 378 insertions(+), 1 deletion(-)


  git pull ssh://asterix-gerrit.ics.uci.edu:29418/asterixdb refs/changes/25/3325/1

diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/DumpIndexDatasource.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/DumpIndexDatasource.java
new file mode 100644
index 0000000..5981787
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/DumpIndexDatasource.java
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.app.function;
+
+import org.apache.asterix.metadata.api.IDatasourceFunction;
+import org.apache.asterix.metadata.declared.DataSourceId;
+import org.apache.asterix.metadata.declared.FunctionDataSource;
+import org.apache.asterix.metadata.declared.MetadataProvider;
+import org.apache.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.properties.INodeDomain;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
+import org.apache.hyracks.storage.am.common.dataflow.IndexDataflowHelperFactory;
+
+public class DumpIndexDatasource extends FunctionDataSource {
+
+    private final IndexDataflowHelperFactory indexDataflowHelperFactory;
+    private final RecordDescriptor recDesc;
+    private final IBinaryComparatorFactory[] comparatorFactories;
+
+    public DumpIndexDatasource(INodeDomain domain, IndexDataflowHelperFactory indexDataflowHelperFactory,
+            RecordDescriptor recDesc, IBinaryComparatorFactory[] comparatorFactories) throws AlgebricksException {
+        super(new DataSourceId(DumpIndexRewriter.DUMP_INDEX.getNamespace(), DumpIndexRewriter.DUMP_INDEX.getName()),
+                domain);
+        this.indexDataflowHelperFactory = indexDataflowHelperFactory;
+        this.recDesc = recDesc;
+        this.comparatorFactories = comparatorFactories;
+    }
+
+    @Override
+    protected IDatasourceFunction createFunction(MetadataProvider metadataProvider,
+            AlgebricksAbsolutePartitionConstraint locations) {
+        return new DumpIndexFunction(metadataProvider.getClusterLocations(), indexDataflowHelperFactory, recDesc,
+                comparatorFactories);
+    }
+}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/DumpIndexFunction.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/DumpIndexFunction.java
new file mode 100644
index 0000000..2fdbef3
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/DumpIndexFunction.java
@@ -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.
+ */
+package org.apache.asterix.app.function;
+
+import org.apache.asterix.external.api.IRecordReader;
+import org.apache.asterix.metadata.declared.AbstractDatasourceFunction;
+import org.apache.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
+import org.apache.hyracks.api.application.INCServiceContext;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.common.api.IIndexDataflowHelper;
+import org.apache.hyracks.storage.am.common.dataflow.IndexDataflowHelperFactory;
+
+public class DumpIndexFunction extends AbstractDatasourceFunction {
+
+    private static final long serialVersionUID = 1L;
+    private final IndexDataflowHelperFactory indexDataflowHelperFactory;
+    private final RecordDescriptor recDesc;
+    private final IBinaryComparatorFactory[] comparatorFactories;
+
+    public DumpIndexFunction(AlgebricksAbsolutePartitionConstraint locations,
+            IndexDataflowHelperFactory indexDataflowHelperFactory, RecordDescriptor recDesc,
+            IBinaryComparatorFactory[] comparatorFactories) {
+        super(locations);
+        this.indexDataflowHelperFactory = indexDataflowHelperFactory;
+        this.recDesc = recDesc;
+        this.comparatorFactories = comparatorFactories;
+    }
+
+    @Override
+    public IRecordReader<char[]> createRecordReader(IHyracksTaskContext ctx, int partition)
+            throws HyracksDataException {
+        INCServiceContext serviceCtx = ctx.getJobletContext().getServiceContext();
+        final IIndexDataflowHelper indexDataflowHelper = indexDataflowHelperFactory.create(serviceCtx, partition);
+        return new DumpIndexReader(indexDataflowHelper, recDesc, comparatorFactories);
+    }
+}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/DumpIndexReader.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/DumpIndexReader.java
new file mode 100644
index 0000000..a3ba10c
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/DumpIndexReader.java
@@ -0,0 +1,108 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.app.function;
+
+import java.io.DataInputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import org.apache.asterix.external.api.IRawRecord;
+import org.apache.asterix.external.input.record.CharArrayRecord;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.dataflow.common.comm.util.ByteBufferInputStream;
+import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+import org.apache.hyracks.storage.am.btree.impls.RangePredicate;
+import org.apache.hyracks.storage.am.common.api.IIndexDataflowHelper;
+import org.apache.hyracks.storage.am.common.impls.NoOpIndexAccessParameters;
+import org.apache.hyracks.storage.common.IIndex;
+import org.apache.hyracks.storage.common.IIndexAccessor;
+import org.apache.hyracks.storage.common.IIndexCursor;
+import org.apache.hyracks.storage.common.MultiComparator;
+
+public class DumpIndexReader extends FunctionReader {
+
+    private final CharArrayRecord record;
+    private final IIndexCursor searchCursor;
+    private final RecordDescriptor secondaryRecDesc;
+    private final StringBuilder recordBuilder = new StringBuilder();
+    private final ByteBufferInputStream bbis = new ByteBufferInputStream();
+    private final DataInputStream dis = new DataInputStream(bbis);
+    private final IIndexDataflowHelper indexDataflowHelper;
+    private final IIndexAccessor accessor;
+
+    public DumpIndexReader(IIndexDataflowHelper indexDataflowHelper, RecordDescriptor secondaryRecDesc,
+            IBinaryComparatorFactory[] comparatorFactories) throws HyracksDataException {
+        this.indexDataflowHelper = indexDataflowHelper;
+        this.secondaryRecDesc = secondaryRecDesc;
+        indexDataflowHelper.open();
+        IIndex indexInstance = indexDataflowHelper.getIndexInstance();
+        accessor = indexInstance.createAccessor(NoOpIndexAccessParameters.INSTANCE);
+        searchCursor = accessor.createSearchCursor(false);
+        MultiComparator searchMultiComparator = MultiComparator.create(comparatorFactories);
+        RangePredicate rangePredicate =
+                new RangePredicate(null, null, true, true, searchMultiComparator, searchMultiComparator, null, null);
+        accessor.search(searchCursor, rangePredicate);
+        record = new CharArrayRecord();
+    }
+
+    @Override
+    public boolean hasNext() throws Exception {
+        return searchCursor.hasNext();
+    }
+
+    @Override
+    public IRawRecord<char[]> next() throws IOException, InterruptedException {
+        searchCursor.next();
+        ITupleReference tuple = searchCursor.getTuple();
+        buildJsonRecord(tuple);
+        record.reset();
+        record.append(recordBuilder.toString().toCharArray());
+        record.endRecord();
+        return record;
+    }
+
+    @Override
+    public void close() throws IOException {
+        bbis.close();
+        dis.close();
+        if (searchCursor != null) {
+            searchCursor.close();
+            searchCursor.destroy();
+        }
+        if (accessor != null) {
+            accessor.destroy();
+        }
+        indexDataflowHelper.close();
+    }
+
+    private void buildJsonRecord(ITupleReference tuple) throws HyracksDataException {
+        recordBuilder.setLength(0);
+        recordBuilder.append("{");
+        for (int j = 0; j < tuple.getFieldCount(); ++j) {
+            recordBuilder.append("\"field-").append(j + 1).append("\":");
+            bbis.setByteBuffer(ByteBuffer.wrap(tuple.getFieldData(j)), tuple.getFieldStart(j));
+            recordBuilder.append(secondaryRecDesc.getFields()[j].deserialize(dis));
+            recordBuilder.append(",");
+        }
+        recordBuilder.deleteCharAt(recordBuilder.length() - 1);
+        recordBuilder.append("}");
+    }
+}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/DumpIndexRewriter.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/DumpIndexRewriter.java
new file mode 100644
index 0000000..c91e9b0
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/DumpIndexRewriter.java
@@ -0,0 +1,71 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.app.function;
+
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.functions.FunctionConstants;
+import org.apache.asterix.metadata.declared.MetadataProvider;
+import org.apache.asterix.metadata.entities.Dataset;
+import org.apache.asterix.metadata.entities.Index;
+import org.apache.asterix.metadata.utils.SecondaryIndexOperationsHelper;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.storage.am.common.dataflow.IndexDataflowHelperFactory;
+
+public class DumpIndexRewriter extends FunctionRewriter {
+
+    public static final FunctionIdentifier DUMP_INDEX =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "dump-index", 3);
+    public static final DumpIndexRewriter INSTANCE = new DumpIndexRewriter(DUMP_INDEX);
+
+    private DumpIndexRewriter(FunctionIdentifier functionId) {
+        super(functionId);
+    }
+
+    @Override
+    public DumpIndexDatasource toDatasource(IOptimizationContext context, AbstractFunctionCallExpression f)
+            throws AlgebricksException {
+        String dataverseName = getString(f.getArguments(), 0);
+        String datasetName = getString(f.getArguments(), 1);
+        String indexName = getString(f.getArguments(), 2);
+        MetadataProvider metadataProvider = (MetadataProvider) context.getMetadataProvider();
+        final Dataset dataset = metadataProvider.findDataset(dataverseName, datasetName);
+        if (dataset == null) {
+            throw new CompilationException(ErrorCode.UNKNOWN_DATASET_IN_DATAVERSE, f.getSourceLocation(), datasetName,
+                    dataverseName);
+        }
+        Index index = metadataProvider.getIndex(dataverseName, datasetName, indexName);
+        if (index == null) {
+            throw new CompilationException(ErrorCode.UNKNOWN_INDEX, f.getSourceLocation(), index);
+        }
+        if (index.isPrimaryIndex()) {
+            throw new IllegalStateException("primary indexes are not supported");
+        }
+        SecondaryIndexOperationsHelper secondaryIndexHelper = SecondaryIndexOperationsHelper
+                .createIndexOperationsHelper(dataset, index, metadataProvider, f.getSourceLocation());
+        IndexDataflowHelperFactory indexDataflowHelperFactory =
+                new IndexDataflowHelperFactory(metadataProvider.getStorageComponentProvider().getStorageManager(),
+                        secondaryIndexHelper.getSecondaryFileSplitProvider());
+        return new DumpIndexDatasource(context.getComputationNodeDomain(), indexDataflowHelperFactory,
+                secondaryIndexHelper.getSecondaryRecDesc(), secondaryIndexHelper.getSecondaryComparatorFactories());
+    }
+}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/util/MetadataBuiltinFunctions.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/util/MetadataBuiltinFunctions.java
index 738d6b4..94e1c4e 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/util/MetadataBuiltinFunctions.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/util/MetadataBuiltinFunctions.java
@@ -22,6 +22,7 @@
 import org.apache.asterix.app.function.CompletedRequestsRewriter;
 import org.apache.asterix.app.function.DatasetResourcesRewriter;
 import org.apache.asterix.app.function.DatasetRewriter;
+import org.apache.asterix.app.function.DumpIndexRewriter;
 import org.apache.asterix.app.function.FeedRewriter;
 import org.apache.asterix.app.function.JobSummariesRewriter;
 import org.apache.asterix.app.function.PingRewriter;
@@ -73,6 +74,11 @@
         BuiltinFunctions.addUnnestFun(CompletedRequestsRewriter.COMPLETED_REQUESTS, true);
         BuiltinFunctions.addDatasourceFunction(CompletedRequestsRewriter.COMPLETED_REQUESTS,
                 CompletedRequestsRewriter.INSTANCE);
+        // Dump index function
+        BuiltinFunctions.addPrivateFunction(DumpIndexRewriter.DUMP_INDEX,
+                (expression, env, mp) -> RecordUtil.FULLY_OPEN_RECORD_TYPE, true);
+        BuiltinFunctions.addUnnestFun(DumpIndexRewriter.DUMP_INDEX, false);
+        BuiltinFunctions.addDatasourceFunction(DumpIndexRewriter.DUMP_INDEX, DumpIndexRewriter.INSTANCE);
     }
 
     private MetadataBuiltinFunctions() {
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/dump_index/dump_index.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/dump_index/dump_index.1.ddl.sqlpp
new file mode 100644
index 0000000..2e97872
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/dump_index/dump_index.1.ddl.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.
+ */
+DROP DATAVERSE test IF EXISTS;
+CREATE  DATAVERSE test;
+USE test;
+CREATE TYPE t1 AS {id:int, name:string?};
+CREATE DATASET ds(t1) PRIMARY KEY id;
+CREATE INDEX name_idx ON ds(name);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/dump_index/dump_index.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/dump_index/dump_index.2.update.sqlpp
new file mode 100644
index 0000000..21ce2b6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/dump_index/dump_index.2.update.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.
+ */
+USE test;
+INSERT INTO ds ([{"id":1, "name": "name1"}, {"id":2, "name": "name2"}]);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/dump_index/dump_index.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/dump_index/dump_index.3.query.sqlpp
new file mode 100644
index 0000000..551a3a7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/dump_index/dump_index.3.query.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.
+ */
+SET `import-private-functions` `true`;
+SELECT VALUE DUMP_INDEX("test", "ds", "name_idx");
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/dump_index/dump_index.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/dump_index/dump_index.3.adm
new file mode 100644
index 0000000..b665a33
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/dump_index/dump_index.3.adm
@@ -0,0 +1 @@
+[ { "field-1": "name1", "field-2": 1 }, { "field-1": "name2", "field-2": 2 } ]
\ 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 69bc647..3344d73 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
@@ -5762,6 +5762,11 @@
         <output-dir compare="Text">completed_requests</output-dir>
       </compilation-unit>
     </test-case>
+    <test-case FilePath="misc">
+      <compilation-unit name="dump_index">
+        <output-dir compare="Text">dump_index</output-dir>
+      </compilation-unit>
+    </test-case>
   </test-group>
   <test-group name="index">
     <test-group name="index/validations">
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/dataflow/RecordDataFlowController.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/dataflow/RecordDataFlowController.java
index 9d72581..c028965 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/dataflow/RecordDataFlowController.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/dataflow/RecordDataFlowController.java
@@ -24,6 +24,7 @@
 import org.apache.hyracks.api.comm.IFrameWriter;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.util.CleanupUtils;
 import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
 
 public class RecordDataFlowController<T> extends AbstractDataFlowController {
@@ -54,9 +55,10 @@
                 tupleForwarder.addTuple(tb);
             }
             tupleForwarder.complete();
-            recordReader.close();
         } catch (Exception e) {
             throw HyracksDataException.create(e);
+        } finally {
+            CleanupUtils.close(recordReader, null);
         }
     }
 
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryIndexOperationsHelper.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryIndexOperationsHelper.java
index 7df723b..aaea111 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryIndexOperationsHelper.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryIndexOperationsHelper.java
@@ -503,4 +503,16 @@
     public void setExternalFiles(List<ExternalFile> externalFiles) {
         this.externalFiles = externalFiles;
     }
+
+    public RecordDescriptor getSecondaryRecDesc() {
+        return secondaryRecDesc;
+    }
+
+    public IBinaryComparatorFactory[] getSecondaryComparatorFactories() {
+        return secondaryComparatorFactories;
+    }
+
+    public IFileSplitProvider getSecondaryFileSplitProvider() {
+        return secondaryFileSplitProvider;
+    }
 }

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3325
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: newchange
Gerrit-Change-Id: Iefae2a798126b9efb049596417b93499d075a608
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>

Change in asterixdb[master]: [NO ISSUE][FUN] Add Function To Return Secondary Index Tuples

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][FUN] Add Function To Return Secondary Index Tuples
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-spidersilk-tests/472/ (14/16)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3325
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Iefae2a798126b9efb049596417b93499d075a608
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][FUN] Add Function To Return Secondary Index Tuples

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][FUN] Add Function To Return Secondary Index Tuples
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-asterix-app-openjdk11/891/ (3/16)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3325
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Iefae2a798126b9efb049596417b93499d075a608
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][FUN] Add Function To Return Secondary Index Tuples

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][FUN] Add Function To Return Secondary Index Tuples
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-cancellation-test/5541/ (11/16)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3325
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Iefae2a798126b9efb049596417b93499d075a608
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][FUN] Add Function To Return Secondary Index Tuples

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][FUN] Add Function To Return Secondary Index Tuples
......................................................................


Patch Set 2: Contrib+1

BAD Compatibility Tests Successful

https://asterix-jenkins.ics.uci.edu/job/asterixbad-compat/4138/ : SUCCESS

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3325
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Iefae2a798126b9efb049596417b93499d075a608
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][FUN] Add Function To Return Secondary Index Tuples

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][FUN] Add Function To Return Secondary Index Tuples
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-verify-txnlog/701/ (1/16)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3325
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Iefae2a798126b9efb049596417b93499d075a608
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][FUN] Add Function To Return Secondary Index Tuples

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][FUN] Add Function To Return Secondary Index Tuples
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-source-assemblies/5743/ (15/16)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3325
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Iefae2a798126b9efb049596417b93499d075a608
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][FUN] Add Function To Return Secondary Index Tuples

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][FUN] Add Function To Return Secondary Index Tuples
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-sonar/9490/ (16/16)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3325
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Iefae2a798126b9efb049596417b93499d075a608
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][FUN] Add Function To Return Secondary Index Tuples

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][FUN] Add Function To Return Secondary Index Tuples
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-stabilization-f69489-compat/816/ (2/16)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3325
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Iefae2a798126b9efb049596417b93499d075a608
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][FUN] Add Function To Return Secondary Index Tuples

Posted by "Anon. E. Moose (Code Review)" <do...@asterixdb.incubator.apache.org>.
Anon. E. Moose #1000171 has posted comments on this change.

Change subject: [NO ISSUE][FUN] Add Function To Return Secondary Index Tuples
......................................................................


Patch Set 2:

Analytics Compatibility Tests Failed
https://cbjenkins.page.link/ueCVVapR1thfvNM36 : UNSTABLE

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3325
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Iefae2a798126b9efb049596417b93499d075a608
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][FUN] Add Function To Return Secondary Index Tuples

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][FUN] Add Function To Return Secondary Index Tuples
......................................................................


Patch Set 1: Integration-Tests+1

Integration Tests Successful

https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-integration-tests/8297/ : SUCCESS

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3325
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Iefae2a798126b9efb049596417b93499d075a608
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][FUN] Add Function To Return Secondary Index Tuples

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][FUN] Add Function To Return Secondary Index Tuples
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-source-format/5495/ (6/16)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3325
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Iefae2a798126b9efb049596417b93499d075a608
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][FUN] Add Function To Return Secondary Index Tuples

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][FUN] Add Function To Return Secondary Index Tuples
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-asterix-app-openjdk11/904/ (5/16)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3325
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Iefae2a798126b9efb049596417b93499d075a608
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][FUN] Add Function To Return Secondary Index Tuples

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][FUN] Add Function To Return Secondary Index Tuples
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-source-assemblies/5756/ (15/16)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3325
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Iefae2a798126b9efb049596417b93499d075a608
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][FUN] Add Function To Return Secondary Index Tuples

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][FUN] Add Function To Return Secondary Index Tuples
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-sonar/9477/ (14/16)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3325
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Iefae2a798126b9efb049596417b93499d075a608
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][FUN] Add Function To Return Secondary Index Tuples

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][FUN] Add Function To Return Secondary Index Tuples
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/5446/ (7/16)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3325
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Iefae2a798126b9efb049596417b93499d075a608
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][FUN] Add Function To Return Secondary Index Tuples

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][FUN] Add Function To Return Secondary Index Tuples
......................................................................


Patch Set 2:

BAD Compatibility Tests Started https://asterix-jenkins.ics.uci.edu/job/asterixbad-compat/4138/

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3325
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Iefae2a798126b9efb049596417b93499d075a608
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][FUN] Add Function To Return Secondary Index Tuples

Posted by "Anon. E. Moose (Code Review)" <do...@asterixdb.incubator.apache.org>.
Anon. E. Moose #1000171 has posted comments on this change.

Change subject: [NO ISSUE][FUN] Add Function To Return Secondary Index Tuples
......................................................................


Patch Set 2: Contrib-2

Analytics Compatibility Tests Failed
https://cbjenkins.page.link/UoD1eZEJnd1BBKTW6 : UNSTABLE

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3325
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Iefae2a798126b9efb049596417b93499d075a608
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][FUN] Add Function To Return Secondary Index Tuples

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][FUN] Add Function To Return Secondary Index Tuples
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-verify-storage/6116/ (10/16)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3325
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Iefae2a798126b9efb049596417b93499d075a608
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][FUN] Add Function To Return Secondary Index Tuples

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][FUN] Add Function To Return Secondary Index Tuples
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-verify-txnlog/714/ (1/16)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3325
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Iefae2a798126b9efb049596417b93499d075a608
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][FUN] Add Function To Return Secondary Index Tuples

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][FUN] Add Function To Return Secondary Index Tuples
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-verify-no-installer-app/5700/ (4/16)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3325
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Iefae2a798126b9efb049596417b93499d075a608
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][FUN] Add Function To Return Secondary Index Tuples

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][FUN] Add Function To Return Secondary Index Tuples
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-ssl-compression/310/ (10/16)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3325
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Iefae2a798126b9efb049596417b93499d075a608
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][FUN] Add Function To Return Secondary Index Tuples

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][FUN] Add Function To Return Secondary Index Tuples
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-ensure-ancestor/3547/ (9/16)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3325
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Iefae2a798126b9efb049596417b93499d075a608
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][FUN] Add Function To Return Secondary Index Tuples

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][FUN] Add Function To Return Secondary Index Tuples
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-verify-asterix-app/5905/ (3/16)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3325
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Iefae2a798126b9efb049596417b93499d075a608
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][FUN] Add Function To Return Secondary Index Tuples

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][FUN] Add Function To Return Secondary Index Tuples
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/5433/ (12/16)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3325
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Iefae2a798126b9efb049596417b93499d075a608
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][FUN] Add Function To Return Secondary Index Tuples

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][FUN] Add Function To Return Secondary Index Tuples
......................................................................


Patch Set 2: Integration-Tests+1

Integration Tests Successful

https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-integration-tests/8310/ : SUCCESS

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3325
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Iefae2a798126b9efb049596417b93499d075a608
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][FUN] Add Function To Return Secondary Index Tuples

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][FUN] Add Function To Return Secondary Index Tuples
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-ensure-ancestor/3534/ (8/16)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3325
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Iefae2a798126b9efb049596417b93499d075a608
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][FUN] Add Function To Return Secondary Index Tuples

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][FUN] Add Function To Return Secondary Index Tuples
......................................................................


Patch Set 1:

BAD Compatibility Tests Started https://asterix-jenkins.ics.uci.edu/job/asterixbad-compat/4133/

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3325
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Iefae2a798126b9efb049596417b93499d075a608
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][FUN] Add Function To Return Secondary Index Tuples

Posted by "Anon. E. Moose (Code Review)" <do...@asterixdb.incubator.apache.org>.
Anon. E. Moose #1000171 has posted comments on this change.

Change subject: [NO ISSUE][FUN] Add Function To Return Secondary Index Tuples
......................................................................


Patch Set 1: Contrib+1

Analytics Compatibility Tests Successful
https://cbjenkins.page.link/dJzMd4AZ5s1y9UTA8 : SUCCESS

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3325
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Iefae2a798126b9efb049596417b93499d075a608
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][FUN] Add Function To Return Secondary Index Tuples

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][FUN] Add Function To Return Secondary Index Tuples
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-ssl-compression/320/ (12/16)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3325
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Iefae2a798126b9efb049596417b93499d075a608
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][FUN] Add Function To Return Secondary Index Tuples

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][FUN] Add Function To Return Secondary Index Tuples
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-cancellation-test/5528/ (7/16)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3325
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Iefae2a798126b9efb049596417b93499d075a608
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][FUN] Add Function To Return Secondary Index Tuples

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][FUN] Add Function To Return Secondary Index Tuples
......................................................................


Patch Set 1: Contrib+1

BAD Compatibility Tests Successful

https://asterix-jenkins.ics.uci.edu/job/asterixbad-compat/4133/ : SUCCESS

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3325
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Iefae2a798126b9efb049596417b93499d075a608
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][FUN] Add Function To Return Secondary Index Tuples

Posted by "Anon. E. Moose (Code Review)" <do...@asterixdb.incubator.apache.org>.
Anon. E. Moose #1000171 has posted comments on this change.

Change subject: [NO ISSUE][FUN] Add Function To Return Secondary Index Tuples
......................................................................


Patch Set 1:

Analytics Compatibility Compilation Successful
https://cbjenkins.page.link/UEBH2j8XjTFsyTPn6 : SUCCESS

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3325
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Iefae2a798126b9efb049596417b93499d075a608
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][FUN] Add Function To Return Secondary Index Tuples

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][FUN] Add Function To Return Secondary Index Tuples
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-verify-no-installer-app/5710/ (6/16)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3325
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Iefae2a798126b9efb049596417b93499d075a608
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][FUN] Add Function To Return Secondary Index Tuples

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][FUN] Add Function To Return Secondary Index Tuples
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-notopic/11021/ (4/16)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3325
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Iefae2a798126b9efb049596417b93499d075a608
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][FUN] Add Function To Return Secondary Index Tuples

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][FUN] Add Function To Return Secondary Index Tuples
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-stabilization-f69489-compat/806/ (11/16)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3325
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Iefae2a798126b9efb049596417b93499d075a608
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][FUN] Add Function To Return Secondary Index Tuples

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][FUN] Add Function To Return Secondary Index Tuples
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-spidersilk-tests/462/ (2/16)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3325
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Iefae2a798126b9efb049596417b93499d075a608
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][FUN] Add Function To Return Secondary Index Tuples

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][FUN] Add Function To Return Secondary Index Tuples
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-source-format/5508/ (8/16)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3325
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Iefae2a798126b9efb049596417b93499d075a608
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][FUN] Add Function To Return Secondary Index Tuples

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][FUN] Add Function To Return Secondary Index Tuples
......................................................................


Patch Set 2:

Integration Tests Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-integration-tests/8310/

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3325
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Iefae2a798126b9efb049596417b93499d075a608
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][FUN] Add Function To Return Secondary Index Tuples

Posted by "Ali Alsuliman (Code Review)" <do...@asterixdb.incubator.apache.org>.
Ali Alsuliman has posted comments on this change.

Change subject: [NO ISSUE][FUN] Add Function To Return Secondary Index Tuples
......................................................................


Patch Set 2: Code-Review+2

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3325
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Iefae2a798126b9efb049596417b93499d075a608
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][FUN] Add Function To Return Secondary Index Tuples

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][FUN] Add Function To Return Secondary Index Tuples
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-notopic/11008/ (5/16)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3325
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Iefae2a798126b9efb049596417b93499d075a608
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][FUN] Add Function To Return Secondary Index Tuples

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][FUN] Add Function To Return Secondary Index Tuples
......................................................................


Patch Set 1:

Integration Tests Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-integration-tests/8297/

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3325
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Iefae2a798126b9efb049596417b93499d075a608
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][FUN] Add Function To Return Secondary Index Tuples

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][FUN] Add Function To Return Secondary Index Tuples
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-asterix-app-sql-execution/5535/ (13/16)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3325
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Iefae2a798126b9efb049596417b93499d075a608
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][FUN] Add Function To Return Secondary Index Tuples

Posted by "Murtadha Hubail (Code Review)" <do...@asterixdb.incubator.apache.org>.
Murtadha Hubail has submitted this change and it was merged.

Change subject: [NO ISSUE][FUN] Add Function To Return Secondary Index Tuples
......................................................................


[NO ISSUE][FUN] Add Function To Return Secondary Index Tuples

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

Details:
- Add a new private function (dump_index) that can be used
  to query all tuples of a secondary index. This function
  can be used for testing to validate the expected contents
  of secondary indexes.
- Add test case.

Change-Id: Iefae2a798126b9efb049596417b93499d075a608
Reviewed-on: https://asterix-gerrit.ics.uci.edu/3325
Tested-by: Jenkins <je...@fulliautomatix.ics.uci.edu>
Integration-Tests: Jenkins <je...@fulliautomatix.ics.uci.edu>
Contrib: Jenkins <je...@fulliautomatix.ics.uci.edu>
Reviewed-by: Ali Alsuliman <al...@gmail.com>
---
A asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/DumpIndexDatasource.java
A asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/DumpIndexFunction.java
A asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/DumpIndexReader.java
A asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/DumpIndexRewriter.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/util/MetadataBuiltinFunctions.java
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/dump_index/dump_index.1.ddl.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/dump_index/dump_index.2.update.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/dump_index/dump_index.3.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/results/misc/dump_index/dump_index.3.adm
M asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
M asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/dataflow/RecordDataFlowController.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryIndexOperationsHelper.java
12 files changed, 378 insertions(+), 1 deletion(-)

Approvals:
  Ali Alsuliman: Looks good to me, approved
  Jenkins: Verified; ; Verified

Objections:
  Anon. E. Moose #1000171: Violations found
  Jenkins: Violations found



diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/DumpIndexDatasource.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/DumpIndexDatasource.java
new file mode 100644
index 0000000..5981787
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/DumpIndexDatasource.java
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.app.function;
+
+import org.apache.asterix.metadata.api.IDatasourceFunction;
+import org.apache.asterix.metadata.declared.DataSourceId;
+import org.apache.asterix.metadata.declared.FunctionDataSource;
+import org.apache.asterix.metadata.declared.MetadataProvider;
+import org.apache.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.properties.INodeDomain;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
+import org.apache.hyracks.storage.am.common.dataflow.IndexDataflowHelperFactory;
+
+public class DumpIndexDatasource extends FunctionDataSource {
+
+    private final IndexDataflowHelperFactory indexDataflowHelperFactory;
+    private final RecordDescriptor recDesc;
+    private final IBinaryComparatorFactory[] comparatorFactories;
+
+    public DumpIndexDatasource(INodeDomain domain, IndexDataflowHelperFactory indexDataflowHelperFactory,
+            RecordDescriptor recDesc, IBinaryComparatorFactory[] comparatorFactories) throws AlgebricksException {
+        super(new DataSourceId(DumpIndexRewriter.DUMP_INDEX.getNamespace(), DumpIndexRewriter.DUMP_INDEX.getName()),
+                domain);
+        this.indexDataflowHelperFactory = indexDataflowHelperFactory;
+        this.recDesc = recDesc;
+        this.comparatorFactories = comparatorFactories;
+    }
+
+    @Override
+    protected IDatasourceFunction createFunction(MetadataProvider metadataProvider,
+            AlgebricksAbsolutePartitionConstraint locations) {
+        return new DumpIndexFunction(metadataProvider.getClusterLocations(), indexDataflowHelperFactory, recDesc,
+                comparatorFactories);
+    }
+}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/DumpIndexFunction.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/DumpIndexFunction.java
new file mode 100644
index 0000000..2fdbef3
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/DumpIndexFunction.java
@@ -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.
+ */
+package org.apache.asterix.app.function;
+
+import org.apache.asterix.external.api.IRecordReader;
+import org.apache.asterix.metadata.declared.AbstractDatasourceFunction;
+import org.apache.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
+import org.apache.hyracks.api.application.INCServiceContext;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.common.api.IIndexDataflowHelper;
+import org.apache.hyracks.storage.am.common.dataflow.IndexDataflowHelperFactory;
+
+public class DumpIndexFunction extends AbstractDatasourceFunction {
+
+    private static final long serialVersionUID = 1L;
+    private final IndexDataflowHelperFactory indexDataflowHelperFactory;
+    private final RecordDescriptor recDesc;
+    private final IBinaryComparatorFactory[] comparatorFactories;
+
+    public DumpIndexFunction(AlgebricksAbsolutePartitionConstraint locations,
+            IndexDataflowHelperFactory indexDataflowHelperFactory, RecordDescriptor recDesc,
+            IBinaryComparatorFactory[] comparatorFactories) {
+        super(locations);
+        this.indexDataflowHelperFactory = indexDataflowHelperFactory;
+        this.recDesc = recDesc;
+        this.comparatorFactories = comparatorFactories;
+    }
+
+    @Override
+    public IRecordReader<char[]> createRecordReader(IHyracksTaskContext ctx, int partition)
+            throws HyracksDataException {
+        INCServiceContext serviceCtx = ctx.getJobletContext().getServiceContext();
+        final IIndexDataflowHelper indexDataflowHelper = indexDataflowHelperFactory.create(serviceCtx, partition);
+        return new DumpIndexReader(indexDataflowHelper, recDesc, comparatorFactories);
+    }
+}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/DumpIndexReader.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/DumpIndexReader.java
new file mode 100644
index 0000000..a3ba10c
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/DumpIndexReader.java
@@ -0,0 +1,108 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.app.function;
+
+import java.io.DataInputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import org.apache.asterix.external.api.IRawRecord;
+import org.apache.asterix.external.input.record.CharArrayRecord;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.dataflow.common.comm.util.ByteBufferInputStream;
+import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+import org.apache.hyracks.storage.am.btree.impls.RangePredicate;
+import org.apache.hyracks.storage.am.common.api.IIndexDataflowHelper;
+import org.apache.hyracks.storage.am.common.impls.NoOpIndexAccessParameters;
+import org.apache.hyracks.storage.common.IIndex;
+import org.apache.hyracks.storage.common.IIndexAccessor;
+import org.apache.hyracks.storage.common.IIndexCursor;
+import org.apache.hyracks.storage.common.MultiComparator;
+
+public class DumpIndexReader extends FunctionReader {
+
+    private final CharArrayRecord record;
+    private final IIndexCursor searchCursor;
+    private final RecordDescriptor secondaryRecDesc;
+    private final StringBuilder recordBuilder = new StringBuilder();
+    private final ByteBufferInputStream bbis = new ByteBufferInputStream();
+    private final DataInputStream dis = new DataInputStream(bbis);
+    private final IIndexDataflowHelper indexDataflowHelper;
+    private final IIndexAccessor accessor;
+
+    public DumpIndexReader(IIndexDataflowHelper indexDataflowHelper, RecordDescriptor secondaryRecDesc,
+            IBinaryComparatorFactory[] comparatorFactories) throws HyracksDataException {
+        this.indexDataflowHelper = indexDataflowHelper;
+        this.secondaryRecDesc = secondaryRecDesc;
+        indexDataflowHelper.open();
+        IIndex indexInstance = indexDataflowHelper.getIndexInstance();
+        accessor = indexInstance.createAccessor(NoOpIndexAccessParameters.INSTANCE);
+        searchCursor = accessor.createSearchCursor(false);
+        MultiComparator searchMultiComparator = MultiComparator.create(comparatorFactories);
+        RangePredicate rangePredicate =
+                new RangePredicate(null, null, true, true, searchMultiComparator, searchMultiComparator, null, null);
+        accessor.search(searchCursor, rangePredicate);
+        record = new CharArrayRecord();
+    }
+
+    @Override
+    public boolean hasNext() throws Exception {
+        return searchCursor.hasNext();
+    }
+
+    @Override
+    public IRawRecord<char[]> next() throws IOException, InterruptedException {
+        searchCursor.next();
+        ITupleReference tuple = searchCursor.getTuple();
+        buildJsonRecord(tuple);
+        record.reset();
+        record.append(recordBuilder.toString().toCharArray());
+        record.endRecord();
+        return record;
+    }
+
+    @Override
+    public void close() throws IOException {
+        bbis.close();
+        dis.close();
+        if (searchCursor != null) {
+            searchCursor.close();
+            searchCursor.destroy();
+        }
+        if (accessor != null) {
+            accessor.destroy();
+        }
+        indexDataflowHelper.close();
+    }
+
+    private void buildJsonRecord(ITupleReference tuple) throws HyracksDataException {
+        recordBuilder.setLength(0);
+        recordBuilder.append("{");
+        for (int j = 0; j < tuple.getFieldCount(); ++j) {
+            recordBuilder.append("\"field-").append(j + 1).append("\":");
+            bbis.setByteBuffer(ByteBuffer.wrap(tuple.getFieldData(j)), tuple.getFieldStart(j));
+            recordBuilder.append(secondaryRecDesc.getFields()[j].deserialize(dis));
+            recordBuilder.append(",");
+        }
+        recordBuilder.deleteCharAt(recordBuilder.length() - 1);
+        recordBuilder.append("}");
+    }
+}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/DumpIndexRewriter.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/DumpIndexRewriter.java
new file mode 100644
index 0000000..c91e9b0
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/DumpIndexRewriter.java
@@ -0,0 +1,71 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.app.function;
+
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.functions.FunctionConstants;
+import org.apache.asterix.metadata.declared.MetadataProvider;
+import org.apache.asterix.metadata.entities.Dataset;
+import org.apache.asterix.metadata.entities.Index;
+import org.apache.asterix.metadata.utils.SecondaryIndexOperationsHelper;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.storage.am.common.dataflow.IndexDataflowHelperFactory;
+
+public class DumpIndexRewriter extends FunctionRewriter {
+
+    public static final FunctionIdentifier DUMP_INDEX =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "dump-index", 3);
+    public static final DumpIndexRewriter INSTANCE = new DumpIndexRewriter(DUMP_INDEX);
+
+    private DumpIndexRewriter(FunctionIdentifier functionId) {
+        super(functionId);
+    }
+
+    @Override
+    public DumpIndexDatasource toDatasource(IOptimizationContext context, AbstractFunctionCallExpression f)
+            throws AlgebricksException {
+        String dataverseName = getString(f.getArguments(), 0);
+        String datasetName = getString(f.getArguments(), 1);
+        String indexName = getString(f.getArguments(), 2);
+        MetadataProvider metadataProvider = (MetadataProvider) context.getMetadataProvider();
+        final Dataset dataset = metadataProvider.findDataset(dataverseName, datasetName);
+        if (dataset == null) {
+            throw new CompilationException(ErrorCode.UNKNOWN_DATASET_IN_DATAVERSE, f.getSourceLocation(), datasetName,
+                    dataverseName);
+        }
+        Index index = metadataProvider.getIndex(dataverseName, datasetName, indexName);
+        if (index == null) {
+            throw new CompilationException(ErrorCode.UNKNOWN_INDEX, f.getSourceLocation(), index);
+        }
+        if (index.isPrimaryIndex()) {
+            throw new IllegalStateException("primary indexes are not supported");
+        }
+        SecondaryIndexOperationsHelper secondaryIndexHelper = SecondaryIndexOperationsHelper
+                .createIndexOperationsHelper(dataset, index, metadataProvider, f.getSourceLocation());
+        IndexDataflowHelperFactory indexDataflowHelperFactory =
+                new IndexDataflowHelperFactory(metadataProvider.getStorageComponentProvider().getStorageManager(),
+                        secondaryIndexHelper.getSecondaryFileSplitProvider());
+        return new DumpIndexDatasource(context.getComputationNodeDomain(), indexDataflowHelperFactory,
+                secondaryIndexHelper.getSecondaryRecDesc(), secondaryIndexHelper.getSecondaryComparatorFactories());
+    }
+}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/util/MetadataBuiltinFunctions.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/util/MetadataBuiltinFunctions.java
index 738d6b4..94e1c4e 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/util/MetadataBuiltinFunctions.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/util/MetadataBuiltinFunctions.java
@@ -22,6 +22,7 @@
 import org.apache.asterix.app.function.CompletedRequestsRewriter;
 import org.apache.asterix.app.function.DatasetResourcesRewriter;
 import org.apache.asterix.app.function.DatasetRewriter;
+import org.apache.asterix.app.function.DumpIndexRewriter;
 import org.apache.asterix.app.function.FeedRewriter;
 import org.apache.asterix.app.function.JobSummariesRewriter;
 import org.apache.asterix.app.function.PingRewriter;
@@ -73,6 +74,11 @@
         BuiltinFunctions.addUnnestFun(CompletedRequestsRewriter.COMPLETED_REQUESTS, true);
         BuiltinFunctions.addDatasourceFunction(CompletedRequestsRewriter.COMPLETED_REQUESTS,
                 CompletedRequestsRewriter.INSTANCE);
+        // Dump index function
+        BuiltinFunctions.addPrivateFunction(DumpIndexRewriter.DUMP_INDEX,
+                (expression, env, mp) -> RecordUtil.FULLY_OPEN_RECORD_TYPE, true);
+        BuiltinFunctions.addUnnestFun(DumpIndexRewriter.DUMP_INDEX, false);
+        BuiltinFunctions.addDatasourceFunction(DumpIndexRewriter.DUMP_INDEX, DumpIndexRewriter.INSTANCE);
     }
 
     private MetadataBuiltinFunctions() {
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/dump_index/dump_index.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/dump_index/dump_index.1.ddl.sqlpp
new file mode 100644
index 0000000..2e97872
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/dump_index/dump_index.1.ddl.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.
+ */
+DROP DATAVERSE test IF EXISTS;
+CREATE  DATAVERSE test;
+USE test;
+CREATE TYPE t1 AS {id:int, name:string?};
+CREATE DATASET ds(t1) PRIMARY KEY id;
+CREATE INDEX name_idx ON ds(name);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/dump_index/dump_index.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/dump_index/dump_index.2.update.sqlpp
new file mode 100644
index 0000000..21ce2b6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/dump_index/dump_index.2.update.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.
+ */
+USE test;
+INSERT INTO ds ([{"id":1, "name": "name1"}, {"id":2, "name": "name2"}]);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/dump_index/dump_index.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/dump_index/dump_index.3.query.sqlpp
new file mode 100644
index 0000000..551a3a7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/dump_index/dump_index.3.query.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.
+ */
+SET `import-private-functions` `true`;
+SELECT VALUE DUMP_INDEX("test", "ds", "name_idx");
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/dump_index/dump_index.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/dump_index/dump_index.3.adm
new file mode 100644
index 0000000..b665a33
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/dump_index/dump_index.3.adm
@@ -0,0 +1 @@
+[ { "field-1": "name1", "field-2": 1 }, { "field-1": "name2", "field-2": 2 } ]
\ 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 a2f30ad..567b739 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
@@ -5745,6 +5745,11 @@
         <output-dir compare="Text">completed_requests</output-dir>
       </compilation-unit>
     </test-case>
+    <test-case FilePath="misc">
+      <compilation-unit name="dump_index">
+        <output-dir compare="Text">dump_index</output-dir>
+      </compilation-unit>
+    </test-case>
   </test-group>
   <test-group name="index">
     <test-group name="index/validations">
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/dataflow/RecordDataFlowController.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/dataflow/RecordDataFlowController.java
index 9d72581..c028965 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/dataflow/RecordDataFlowController.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/dataflow/RecordDataFlowController.java
@@ -24,6 +24,7 @@
 import org.apache.hyracks.api.comm.IFrameWriter;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.util.CleanupUtils;
 import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
 
 public class RecordDataFlowController<T> extends AbstractDataFlowController {
@@ -54,9 +55,10 @@
                 tupleForwarder.addTuple(tb);
             }
             tupleForwarder.complete();
-            recordReader.close();
         } catch (Exception e) {
             throw HyracksDataException.create(e);
+        } finally {
+            CleanupUtils.close(recordReader, null);
         }
     }
 
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryIndexOperationsHelper.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryIndexOperationsHelper.java
index 7df723b..aaea111 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryIndexOperationsHelper.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryIndexOperationsHelper.java
@@ -503,4 +503,16 @@
     public void setExternalFiles(List<ExternalFile> externalFiles) {
         this.externalFiles = externalFiles;
     }
+
+    public RecordDescriptor getSecondaryRecDesc() {
+        return secondaryRecDesc;
+    }
+
+    public IBinaryComparatorFactory[] getSecondaryComparatorFactories() {
+        return secondaryComparatorFactories;
+    }
+
+    public IFileSplitProvider getSecondaryFileSplitProvider() {
+        return secondaryFileSplitProvider;
+    }
 }

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3325
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: merged
Gerrit-Change-Id: Iefae2a798126b9efb049596417b93499d075a608
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>

Change in asterixdb[master]: [NO ISSUE][FUN] Add Function To Return Secondary Index Tuples

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][FUN] Add Function To Return Secondary Index Tuples
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-asterix-app-sql-execution/5522/ (13/16)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3325
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Iefae2a798126b9efb049596417b93499d075a608
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][FUN] Add Function To Return Secondary Index Tuples

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][FUN] Add Function To Return Secondary Index Tuples
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-verify-asterix-app/5892/ (16/16)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3325
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Iefae2a798126b9efb049596417b93499d075a608
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][FUN] Add Function To Return Secondary Index Tuples

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][FUN] Add Function To Return Secondary Index Tuples
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-verify-storage/6103/ (9/16)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3325
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Iefae2a798126b9efb049596417b93499d075a608
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][FUN] Add Function To Return Secondary Index Tuples

Posted by "Anon. E. Moose (Code Review)" <do...@asterixdb.incubator.apache.org>.
Anon. E. Moose #1000171 has posted comments on this change.

Change subject: [NO ISSUE][FUN] Add Function To Return Secondary Index Tuples
......................................................................


Patch Set 2:

Analytics Compatibility Compilation Successful
https://cbjenkins.page.link/voZYNocWf6fAfsNo9 : SUCCESS

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3325
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Iefae2a798126b9efb049596417b93499d075a608
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No