You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@asterixdb.apache.org by wy...@apache.org on 2022/02/25 19:00:16 UTC

[asterixdb] branch master updated: [ASTERIXDB-3003][STO] Enable batch point lookups and stateful cursors

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

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


The following commit(s) were added to refs/heads/master by this push:
     new c1c5bec  [ASTERIXDB-3003][STO] Enable batch point lookups and stateful cursors
c1c5bec is described below

commit c1c5bec0dd3cc5de8e1713f23dece7936d309ba1
Author: Wail Alkowaileet <wa...@gmail.com>
AuthorDate: Fri Feb 18 13:50:10 2022 -0800

    [ASTERIXDB-3003][STO] Enable batch point lookups and stateful cursors
    
    - user model changes: yes
    - storage format changes: no
    - interface changes: yes
    
    Details:
    - Allow the user to enable stateful and batch point lookups
    - A compiler flag 'compiler.batch.lookup' can be set to enable
      batch and stateful point-lookups
    - The stateful cursors are used to search for subsequent points
      without the need for searching the BTree's interior nodes for
      each point lookup
    - The cursor is kept open, and the page is pinned until:
      - the page is consumed (i.e., no more points can be found in the page)
      - or the cursor is closed
    - Introduce IDiskBTreeStatefulPointSearchCursor interface for different
      DiskBTree implementations
    
    Change-Id: I76107f4eb6de620399e2a1142684a7b8dad4e0a1
    Reviewed-on: https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/14804
    Integration-Tests: Jenkins <je...@fulliautomatix.ics.uci.edu>
    Tested-by: Jenkins <je...@fulliautomatix.ics.uci.edu>
    Reviewed-by: Wail Alkowaileet <wa...@gmail.com>
    Reviewed-by: Murtadha Hubail <mh...@apache.org>
---
 .../operators/physical/BTreeSearchPOperator.java   |  13 +-
 .../SqlppBatchPointLookupExecutionTest.java        |  85 ++
 .../src/test/resources/cc-batch-lookup.conf        |  65 ++
 .../test/resources/runtimets/only_batch_lookup.xml |  23 +
 .../api/cluster_state_1/cluster_state_1.1.regexadm |   1 +
 .../cluster_state_1_full.1.regexadm                |   1 +
 .../cluster_state_1_less.1.regexadm                |   1 +
 .../runtimets/testsuite_sqlpp_batch_lookup.xml     | 862 +++++++++++++++++++++
 .../asterix/common/config/CompilerProperties.java  |  12 +-
 .../common/config/OptimizationConfUtil.java        |   3 +
 .../physical/OneToOneExchangePOperator.java        |   5 +-
 .../algebricks/core/config/AlgebricksConfig.java   |   1 +
 .../rewriter/base/PhysicalOptimizationConfig.java  |   9 +
 .../rules/IsolateHyracksOperatorsRule.java         |   1 +
 .../api/IDiskBTreeStatefulPointSearchCursor.java   |  36 +
 .../hyracks/storage/am/btree/impls/DiskBTree.java  |  34 +-
 .../am/btree/impls/DiskBTreePointSearchCursor.java |  60 +-
 .../impls/LSMBTreeBatchPointSearchCursor.java      |  29 +-
 .../lsm/btree/impls/LSMBTreePointSearchCursor.java |   2 +-
 19 files changed, 1175 insertions(+), 68 deletions(-)

diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/BTreeSearchPOperator.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/BTreeSearchPOperator.java
index 06a6687..553fb4c 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/BTreeSearchPOperator.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/BTreeSearchPOperator.java
@@ -62,9 +62,11 @@ import org.apache.hyracks.algebricks.core.algebra.properties.PhysicalRequirement
 import org.apache.hyracks.algebricks.core.algebra.properties.StructuralPropertiesVector;
 import org.apache.hyracks.algebricks.core.algebra.properties.UnorderedPartitionedProperty;
 import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenContext;
+import org.apache.hyracks.algebricks.core.rewriter.base.PhysicalOptimizationConfig;
 import org.apache.hyracks.api.dataflow.IOperatorDescriptor;
 import org.apache.hyracks.api.dataflow.value.IMissingWriterFactory;
 import org.apache.hyracks.storage.am.common.api.ITupleFilterFactory;
+import org.apache.hyracks.storage.am.lsm.btree.impls.LSMBTreeBatchPointSearchCursor;
 
 /**
  * Contributes the runtime operator for an unnest-map representing a BTree search.
@@ -157,7 +159,8 @@ public class BTreeSearchPOperator extends IndexSearchPOperator {
                 nonMatchWriterFactory, dataset, jobGenParams.getIndexName(), lowKeyIndexes, highKeyIndexes,
                 jobGenParams.isLowKeyInclusive(), jobGenParams.isHighKeyInclusive(), propagateFilter,
                 nonFilterWriterFactory, minFilterFieldIndexes, maxFilterFieldIndexes, tupleFilterFactory, outputLimit,
-                unnestMap.getGenerateCallBackProceedResultVar(), isPrimaryIndexPointSearch(op));
+                unnestMap.getGenerateCallBackProceedResultVar(),
+                isPrimaryIndexPointSearch(op, context.getPhysicalOptimizationConfig()));
         IOperatorDescriptor opDesc = btreeSearch.first;
         opDesc.setSourceLocation(unnestMap.getSourceLocation());
 
@@ -168,8 +171,12 @@ public class BTreeSearchPOperator extends IndexSearchPOperator {
         builder.contributeGraphEdge(srcExchange, 0, unnestMap, 0);
     }
 
-    private boolean isPrimaryIndexPointSearch(ILogicalOperator op) {
-        if (!isEqCondition || !isPrimaryIndex || !lowKeyVarList.equals(highKeyVarList)) {
+    /**
+     * Check whether we can use {@link LSMBTreeBatchPointSearchCursor} to perform point-lookups on the primary index
+     */
+    private boolean isPrimaryIndexPointSearch(ILogicalOperator op, PhysicalOptimizationConfig config) {
+        if (!config.isBatchLookupEnabled() || !isEqCondition || !isPrimaryIndex
+                || !lowKeyVarList.equals(highKeyVarList)) {
             return false;
         }
         Index searchIndex = ((DataSourceIndex) idx).getIndex();
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/SqlppBatchPointLookupExecutionTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/SqlppBatchPointLookupExecutionTest.java
new file mode 100644
index 0000000..2608447
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/SqlppBatchPointLookupExecutionTest.java
@@ -0,0 +1,85 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.test.runtime;
+
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.asterix.common.api.INcApplicationContext;
+import org.apache.asterix.test.common.TestExecutor;
+import org.apache.asterix.testframework.context.TestCaseContext;
+import org.apache.hyracks.control.nc.NodeControllerService;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+
+/**
+ * Runs the SQL++ with Batched point-lookups enabled.
+ */
+@RunWith(Parameterized.class)
+public class SqlppBatchPointLookupExecutionTest {
+    protected static final String TEST_CONFIG_FILE_NAME = "src/test/resources/cc-batch-lookup.conf";
+
+    @BeforeClass
+    public static void setUp() throws Exception {
+        final TestExecutor testExecutor = new TestExecutor();
+        LangExecutionUtil.setUp(TEST_CONFIG_FILE_NAME, testExecutor);
+        setNcEndpoints(testExecutor);
+    }
+
+    @AfterClass
+    public static void tearDown() throws Exception {
+        LangExecutionUtil.tearDown();
+    }
+
+    @Parameters(name = "SqlppBatchPointLookupExecutionTest {index}: {0}")
+    public static Collection<Object[]> tests() throws Exception {
+        return LangExecutionUtil.tests("only_batch_lookup.xml", "testsuite_sqlpp_batch_lookup.xml");
+    }
+
+    protected TestCaseContext tcCtx;
+
+    public SqlppBatchPointLookupExecutionTest(TestCaseContext tcCtx) {
+        this.tcCtx = tcCtx;
+    }
+
+    @Test
+    public void test() throws Exception {
+        LangExecutionUtil.test(tcCtx);
+    }
+
+    private static void setNcEndpoints(TestExecutor testExecutor) {
+        final NodeControllerService[] ncs = ExecutionTestUtil.integrationUtil.ncs;
+        final Map<String, InetSocketAddress> ncEndPoints = new HashMap<>();
+        final String ip = InetAddress.getLoopbackAddress().getHostAddress();
+        for (NodeControllerService nc : ncs) {
+            final String nodeId = nc.getId();
+            final INcApplicationContext appCtx = (INcApplicationContext) nc.getApplicationContext();
+            int apiPort = appCtx.getExternalProperties().getNcApiPort();
+            ncEndPoints.put(nodeId, InetSocketAddress.createUnresolved(ip, apiPort));
+        }
+        testExecutor.setNcEndPoints(ncEndPoints);
+    }
+}
diff --git a/asterixdb/asterix-app/src/test/resources/cc-batch-lookup.conf b/asterixdb/asterix-app/src/test/resources/cc-batch-lookup.conf
new file mode 100644
index 0000000..6e10481
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/cc-batch-lookup.conf
@@ -0,0 +1,65 @@
+; Licensed to the Apache Software Foundation (ASF) under one
+; or more contributor license agreements.  See the NOTICE file
+; distributed with this work for additional information
+; regarding copyright ownership.  The ASF licenses this file
+; to you under the Apache License, Version 2.0 (the
+; "License"); you may not use this file except in compliance
+; with the License.  You may obtain a copy of the License at
+;
+;   http://www.apache.org/licenses/LICENSE-2.0
+;
+; Unless required by applicable law or agreed to in writing,
+; software distributed under the License is distributed on an
+; "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+; KIND, either express or implied.  See the License for the
+; specific language governing permissions and limitations
+; under the License.
+
+[nc/asterix_nc1]
+txn.log.dir=target/tmp/asterix_nc1/txnlog
+core.dump.dir=target/tmp/asterix_nc1/coredump
+iodevices=target/tmp/asterix_nc1/iodevice1,
+iodevices=../asterix-server/target/tmp/asterix_nc1/iodevice2
+nc.api.port=19004
+#jvm.args=-agentlib:jdwp=transport=dt_socket,server=y,suspend=y,address=5006
+
+[nc/asterix_nc2]
+ncservice.port=9091
+txn.log.dir=target/tmp/asterix_nc2/txnlog
+core.dump.dir=target/tmp/asterix_nc2/coredump
+iodevices=target/tmp/asterix_nc2/iodevice1,../asterix-server/target/tmp/asterix_nc2/iodevice2
+nc.api.port=19005
+#jvm.args=-agentlib:jdwp=transport=dt_socket,server=y,suspend=y,address=5007
+
+[nc]
+credential.file=src/test/resources/security/passwd
+python.cmd.autolocate=true
+python.env=FOO=BAR=BAZ,BAR=BAZ
+address=127.0.0.1
+command=asterixnc
+app.class=org.apache.asterix.hyracks.bootstrap.NCApplication
+jvm.args=-Xmx4096m -Dnode.Resolver="org.apache.asterix.external.util.IdentitiyResolverFactory"
+storage.buffercache.pagesize=32KB
+storage.buffercache.size=128MB
+storage.memorycomponent.globalbudget=512MB
+
+[cc]
+address = 127.0.0.1
+app.class=org.apache.asterix.hyracks.bootstrap.CCApplication
+heartbeat.period=2000
+heartbeat.max.misses=25
+credential.file=src/test/resources/security/passwd
+
+[common]
+log.dir = logs/
+log.level = INFO
+compiler.framesize=32KB
+compiler.sortmemory=320KB
+compiler.groupmemory=160KB
+compiler.joinmemory=256KB
+compiler.textsearchmemory=160KB
+compiler.windowmemory=192KB
+compiler.batch.lookup=true
+messaging.frame.size=4096
+messaging.frame.count=512
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/only_batch_lookup.xml b/asterixdb/asterix-app/src/test/resources/runtimets/only_batch_lookup.xml
new file mode 100644
index 0000000..334dd52
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/only_batch_lookup.xml
@@ -0,0 +1,23 @@
+<?xml version="1.0" encoding="UTF-8" standalone="yes"?>
+<!--
+ ! Licensed to the Apache Software Foundation (ASF) under one
+ ! or more contributor license agreements.  See the NOTICE file
+ ! distributed with this work for additional information
+ ! regarding copyright ownership.  The ASF licenses this file
+ ! to you under the Apache License, Version 2.0 (the
+ ! "License"); you may not use this file except in compliance
+ ! with the License.  You may obtain a copy of the License at
+ !
+ !   http://www.apache.org/licenses/LICENSE-2.0
+ !
+ ! Unless required by applicable law or agreed to in writing,
+ ! software distributed under the License is distributed on an
+ ! "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ ! KIND, either express or implied.  See the License for the
+ ! specific language governing permissions and limitations
+ ! under the License.
+ !-->
+<test-suite xmlns="urn:xml.testframework.asterix.apache.org" ResultOffsetPath="results" QueryOffsetPath="queries_sqlpp" QueryFileExtension=".sqlpp">
+  <test-group name="failed">
+  </test-group>
+</test-suite>
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1/cluster_state_1.1.regexadm b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1/cluster_state_1.1.regexadm
index bf736b4..70092cf 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1/cluster_state_1.1.regexadm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1/cluster_state_1.1.regexadm
@@ -9,6 +9,7 @@
     "active\.stop\.timeout" : 3600,
     "active\.suspend\.timeout" : 3600,
     "compiler\.arrayindex" : true,
+    "compiler.batch.lookup" : false,
     "compiler\.external\.field\.pushdown" : true,
     "compiler\.framesize" : 32768,
     "compiler\.groupmemory" : 163840,
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_full/cluster_state_1_full.1.regexadm b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_full/cluster_state_1_full.1.regexadm
index d52cedd..3faac6f 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_full/cluster_state_1_full.1.regexadm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_full/cluster_state_1_full.1.regexadm
@@ -9,6 +9,7 @@
     "active\.stop\.timeout" : 3600,
     "active\.suspend\.timeout" : 3600,
     "compiler\.arrayindex" : true,
+    "compiler.batch.lookup" : false,
     "compiler\.external\.field\.pushdown" : true,
     "compiler\.framesize" : 32768,
     "compiler\.groupmemory" : 163840,
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_less/cluster_state_1_less.1.regexadm b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_less/cluster_state_1_less.1.regexadm
index 4f5267f..e42306c 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_less/cluster_state_1_less.1.regexadm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_less/cluster_state_1_less.1.regexadm
@@ -9,6 +9,7 @@
     "active\.stop\.timeout" : 3600,
     "active\.suspend\.timeout" : 3600,
     "compiler\.arrayindex" : true,
+    "compiler.batch.lookup" : false,
     "compiler\.external\.field\.pushdown" : true,
     "compiler\.framesize" : 32768,
     "compiler\.groupmemory" : 163840,
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp_batch_lookup.xml b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp_batch_lookup.xml
new file mode 100644
index 0000000..fd8b886
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp_batch_lookup.xml
@@ -0,0 +1,862 @@
+<?xml version="1.0" encoding="UTF-8" standalone="yes"?>
+<!--
+ ! Licensed to the Apache Software Foundation (ASF) under one
+ ! or more contributor license agreements.  See the NOTICE file
+ ! distributed with this work for additional information
+ ! regarding copyright ownership.  The ASF licenses this file
+ ! to you under the Apache License, Version 2.0 (the
+ ! "License"); you may not use this file except in compliance
+ ! with the License.  You may obtain a copy of the License at
+ !
+ !   http://www.apache.org/licenses/LICENSE-2.0
+ !
+ ! Unless required by applicable law or agreed to in writing,
+ ! software distributed under the License is distributed on an
+ ! "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ ! KIND, either express or implied.  See the License for the
+ ! specific language governing permissions and limitations
+ ! under the License.
+ !-->
+<test-suite xmlns="urn:xml.testframework.asterix.apache.org" ResultOffsetPath="results" QueryOffsetPath="queries_sqlpp" QueryFileExtension=".sqlpp">
+  <test-group name="batched-lookups">
+    <test-case FilePath="dml">
+      <compilation-unit name="compact-dataset-and-its-indexes">
+        <output-dir compare="Text">compact-dataset-and-its-indexes</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="using-constant-merge-policy">
+        <output-dir compare="Text">using-constant-merge-policy</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="using-prefix-merge-policy">
+        <output-dir compare="Text">using-prefix-merge-policy</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="using-concurrent-merge-policy">
+        <output-dir compare="Text">using-concurrent-merge-policy</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="using-correlated-prefix-merge-policy">
+        <output-dir compare="Text">using-correlated-prefix-merge-policy</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="using-correlated-prefix-merge-policy-with-feed">
+        <output-dir compare="Text">using-correlated-prefix-merge-policy</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="using-no-merge-policy">
+        <output-dir compare="Text">using-no-merge-policy</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="delete-from-loaded-dataset-with-index">
+        <output-dir compare="Text">delete-from-loaded-dataset-with-index</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="load-with-index">
+        <output-dir compare="Text">load-with-index</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="scan-delete-btree-secondary-index-nullable">
+        <output-dir compare="Text">scan-delete-btree-secondary-index-nullable</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="scan-delete-btree-correlated-secondary-index-nullable">
+        <output-dir compare="Text">scan-delete-btree-secondary-index-nullable</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="scan-insert-btree-correlated-secondary-index-nullable">
+        <output-dir compare="Text">scan-delete-btree-secondary-index-nullable</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="scan-delete-btree-secondary-index-open">
+        <output-dir compare="Text">scan-delete-btree-secondary-index-open</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="scan-delete-btree-correlated-secondary-index-open">
+        <output-dir compare="Text">scan-delete-btree-secondary-index-open</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="scan-delete-rtree-secondary-index-open">
+        <output-dir compare="Text">scan-delete-rtree-secondary-index-open</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="scan-delete-rtree-correlated-secondary-index-open">
+        <output-dir compare="Text">scan-delete-rtree-secondary-index-open</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="scan-insert-btree-correlated-secondary-index-open">
+        <output-dir compare="Text">scan-delete-btree-secondary-index-open</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="scan-insert-rtree-correlated-secondary-index-open">
+        <output-dir compare="Text">scan-insert-rtree-secondary-index-open</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="fulltext">
+      <compilation-unit name="fulltext-index-01">
+        <output-dir compare="Text">fulltext-index-01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="fulltext">
+      <compilation-unit name="fulltext-index-02">
+        <output-dir compare="Text">fulltext-index-02</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="fulltext">
+      <compilation-unit name="fulltext-index-08">
+        <output-dir compare="Text">fulltext-index-08</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="fulltext">
+      <compilation-unit name="fulltext-index-09">
+        <output-dir compare="Text">fulltext-index-09</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="fulltext">
+      <compilation-unit name="stopwords-full-text-filter-1">
+        <output-dir compare="Text">stopwords-full-text-filter-1</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-join">
+      <compilation-unit name="btree-index-nested-loop-join">
+        <output-dir compare="Text">btree-index-nested-loop-join</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-join">
+      <compilation-unit name="btree-pidx-to-sidx-idxonly-equi-join_01">
+        <output-dir compare="Text">btree-pidx-to-sidx-idxonly-equi-join_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-join">
+      <compilation-unit name="btree-secondary-equi-join_02">
+        <output-dir compare="Text">btree-secondary-equi-join_02</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-join">
+      <compilation-unit name="btree-secondary-equi-join_03">
+        <output-dir compare="Text">btree-secondary-equi-join_03</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-join">
+      <compilation-unit name="btree-secondary-equi-join_04">
+        <output-dir compare="Text">btree-secondary-equi-join_04</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-join">
+      <compilation-unit name="btree-sidx-idxonly-to-pidx-equi-join_01">
+        <output-dir compare="Text">btree-sidx-idxonly-to-pidx-equi-join_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-join">
+      <compilation-unit name="btree-sidx-idxonly-to-sidx-idxonly-equi-join_01">
+        <output-dir compare="Text">btree-sidx-idxonly-to-sidx-idxonly-equi-join_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-join">
+      <compilation-unit name="btree-sidx-non-idxonly-to-pidx-equi-join_01">
+        <output-dir compare="Text">btree-sidx-non-idxonly-to-pidx-equi-join_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-join">
+      <compilation-unit name="btree-sidx-non-idxonly-to-sidx-idxonly-equi-join_01">
+        <output-dir compare="Text">btree-sidx-non-idxonly-to-sidx-idxonly-equi-join_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-join" check-warnings="true">
+      <compilation-unit name="hints-indexnl-params">
+        <output-dir compare="Text">hints-indexnl-params</output-dir>
+        <expected-warn>ASX1132: Invalid specification for hint indexnl. ASX1001: Syntax error: In line 1 &gt;&gt;(8, idx_tenk2_1k_2k)&lt;&lt; Encountered &lt;INTEGER_LITERAL&gt; "8" at column 2.  (in line 35, at column 21)</expected-warn>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-join">
+      <compilation-unit name="rtree-spatial-intersect-point_05">
+        <output-dir compare="Text">rtree-spatial-intersect-point_05</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-selection">
+      <compilation-unit name="btree-index-composite-key-03">
+        <output-dir compare="Text">btree-index-composite-key-03</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-selection">
+      <compilation-unit name="btree-index-composite-key-mixed-intervals">
+        <output-dir compare="Text">btree-index-composite-key-mixed-intervals</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-selection">
+      <compilation-unit name="btree-index-rewrite-multiple">
+        <output-dir compare="Text">btree-index-rewrite-multiple</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-selection">
+      <compilation-unit name="btree-sidx-composite-idxonly-01">
+        <output-dir compare="Text">btree-sidx-composite-idxonly-01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-selection">
+      <compilation-unit name="btree-sidx-composite-idxonly-02">
+        <output-dir compare="Text">btree-sidx-composite-idxonly-02</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-selection">
+      <compilation-unit name="btree-sidx-composite-idxonly-03">
+        <output-dir compare="Text">btree-sidx-composite-idxonly-03</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-selection">
+      <compilation-unit name="btree-sidx-composite-idxonly-04">
+        <output-dir compare="Text">btree-sidx-composite-idxonly-04</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-selection">
+      <compilation-unit name="btree-sidx-idxonly-01">
+        <output-dir compare="Text">btree-sidx-idxonly-01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-selection">
+      <compilation-unit name="cust-index-age-nullable">
+        <output-dir compare="Text">cust-index-age-nullable</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-selection">
+      <compilation-unit name="intersection-with-between">
+        <output-dir compare="Text">intersection-with-between</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-selection">
+      <compilation-unit name="inverted-index-ngram-edit-distance-large-data">
+        <output-dir compare="Text">inverted-index-ngram-edit-distance-large-data</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-selection">
+      <compilation-unit name="inverted-index-olist-edit-distance">
+        <output-dir compare="Text">inverted-index-olist-edit-distance</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-selection">
+      <compilation-unit name="inverted-index-olist-jaccard">
+        <output-dir compare="Text">inverted-index-olist-jaccard</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-selection">
+      <compilation-unit name="inverted-index-ulist-jaccard">
+        <output-dir compare="Text">inverted-index-ulist-jaccard</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-selection">
+      <compilation-unit name="orders-index-custkey-conjunctive-open">
+        <output-dir compare="Text">orders-index-custkey-conjunctive-open</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-selection">
+      <compilation-unit name="orders-index-custkey-conjunctive">
+        <output-dir compare="Text">orders-index-custkey-conjunctive</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-selection">
+      <compilation-unit name="orders-index-custkey-open">
+        <output-dir compare="Text">orders-index-custkey-open</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-selection">
+      <compilation-unit name="orders-index-custkey">
+        <output-dir compare="Text">orders-index-custkey</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-selection">
+      <compilation-unit name="range-search-open">
+        <output-dir compare="Text">range-search-open</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-selection">
+      <compilation-unit name="range-search">
+        <output-dir compare="Text">range-search</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-selection">
+      <compilation-unit name="rtree-sidx-non-idxonly-02">
+        <output-dir compare="Text">rtree-sidx-non-idxonly-02</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-selection">
+      <compilation-unit name="dataset-with-meta">
+        <output-dir compare="Text">dataset-with-meta</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-selection" check-warnings="true">
+      <compilation-unit name="hints-skip-index">
+        <output-dir compare="Text">hints-skip-index</output-dir>
+        <expected-warn>ASX1132: Invalid specification for hint skip-index. ASX1001: Syntax error: In line 1 &gt;&gt;(13, idx_1k)&lt;&lt; Encountered &lt;INTEGER_LITERAL&gt; "13" at column 2.  (in line 32, at column 19)</expected-warn>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-selection" check-warnings="true">
+      <compilation-unit name="hints-use-index">
+        <output-dir compare="Text">hints-use-index</output-dir>
+        <expected-warn>ASX1132: Invalid specification for hint use-index. ASX1001: Syntax error: In line 1 &gt;&gt;(18, idx_1k_2k)&lt;&lt; Encountered &lt;INTEGER_LITERAL&gt; "18" at column 2.  (in line 33, at column 15)</expected-warn>
+        <expected-warn>ASX1132: Invalid specification for hint use-index. ASX1001: Syntax error: In line 1 &gt;&gt;()&lt;&lt; Encountered ")" at column 2.  (in line 33, at column 15)</expected-warn>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-selection">
+      <compilation-unit name="verify">
+        <output-dir compare="Text">verify</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="open-index-enforced/index-join">
+      <compilation-unit name="ngram-jaccard">
+        <output-dir compare="Text">ngram-jaccard</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="open-index-enforced/index-join">
+      <compilation-unit name="ngram-jaccard-inline">
+        <output-dir compare="Text">ngram-jaccard-inline</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="open-index-enforced/index-selection">
+      <compilation-unit name="btree-index-composite-key-mixed-intervals">
+        <output-dir compare="Text">btree-index-composite-key-mixed-intervals</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="open-index-enforced/index-selection">
+      <compilation-unit name="btree-index-rewrite-multiple">
+        <output-dir compare="Text">btree-index-rewrite-multiple</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="open-index-enforced/index-selection">
+      <compilation-unit name="orders-index-custkey">
+        <output-dir compare="Text">orders-index-custkey</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="open-index-enforced/index-selection">
+      <compilation-unit name="orders-index-custkey-conjunctive">
+        <output-dir compare="Text">orders-index-custkey-conjunctive</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="open-index-enforced/index-selection">
+      <compilation-unit name="range-search">
+        <output-dir compare="Text">range-search</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="open-index-non-enforced/index-selection">
+      <compilation-unit name="btree-index-01">
+        <output-dir compare="Text">btree-index-01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="open-index-non-enforced/index-selection">
+      <compilation-unit name="btree-index-02">
+        <output-dir compare="Text">btree-index-02</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="open-index-non-enforced/index-selection">
+      <compilation-unit name="btree-index-04">
+        <output-dir compare="Text">btree-index-04</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="open-index-non-enforced/correlated-index-selection">
+      <compilation-unit name="btree-index-01">
+        <output-dir compare="Text">btree-index-01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="open-index-non-enforced/index-join">
+      <compilation-unit name="btree-equi-join-01">
+        <output-dir compare="Text">btree-equi-join-01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="nested-open-index/index-join">
+      <compilation-unit name="ngram-jaccard">
+        <output-dir compare="Text">ngram-jaccard</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="nested-open-index/index-join">
+      <compilation-unit name="ngram-jaccard-inline">
+        <output-dir compare="Text">ngram-jaccard-inline</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="nested-open-index/index-join">
+      <compilation-unit name="btree-secondary-non-enforced-equi-join">
+        <output-dir compare="Text">btree-secondary-non-enforced-equi-join</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="nested-open-index/index-join">
+      <compilation-unit name="btree-secondary-non-enforced-equi-join-2">
+        <output-dir compare="Text">btree-secondary-non-enforced-equi-join-2</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="nested-open-index/index-selection">
+      <compilation-unit name="btree-index-composite-key-mixed-intervals">
+        <output-dir compare="Text">btree-index-composite-key-mixed-intervals</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="nested-open-index/index-selection">
+      <compilation-unit name="btree-index-rewrite-multiple">
+        <output-dir compare="Text">btree-index-rewrite-multiple</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="nested-open-index/index-selection">
+      <compilation-unit name="orders-index-custkey">
+        <output-dir compare="Text">orders-index-custkey</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="nested-open-index/index-selection">
+      <compilation-unit name="orders-index-custkey-conjunctive">
+        <output-dir compare="Text">orders-index-custkey-conjunctive</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="nested-open-index/index-selection">
+      <compilation-unit name="range-search">
+        <output-dir compare="Text">range-search</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="nested-open-index/index-selection">
+      <compilation-unit name="non-enforced-01">
+        <output-dir compare="Text">non-enforced-01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="nested-open-index/index-selection">
+      <compilation-unit name="non-enforced-02">
+        <output-dir compare="Text">non-enforced-02</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="nested-open-index/index-selection">
+      <compilation-unit name="non-enforced-04">
+        <output-dir compare="Text">non-enforced-04</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="nested-index/index-selection">
+      <compilation-unit name="btree-index-composite-key-mixed-intervals">
+        <output-dir compare="Text">btree-index-composite-key-mixed-intervals</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="nested-index/index-selection">
+      <compilation-unit name="btree-index-rewrite-multiple">
+        <output-dir compare="Text">btree-index-rewrite-multiple</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="nested-index/index-selection">
+      <compilation-unit name="cust-index-age-nullable">
+        <output-dir compare="Text">cust-index-age-nullable</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="nested-index/index-selection">
+      <compilation-unit name="inverted-index-olist-edit-distance">
+        <output-dir compare="Text">inverted-index-olist-edit-distance</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="nested-index/index-selection">
+      <compilation-unit name="inverted-index-olist-jaccard">
+        <output-dir compare="Text">inverted-index-olist-jaccard</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="nested-index/index-selection">
+      <compilation-unit name="inverted-index-ulist-jaccard">
+        <output-dir compare="Text">inverted-index-ulist-jaccard</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="nested-index/index-selection">
+      <compilation-unit name="orders-index-custkey">
+        <output-dir compare="Text">orders-index-custkey</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="nested-index/index-selection">
+      <compilation-unit name="orders-index-custkey-conjunctive">
+        <output-dir compare="Text">orders-index-custkey-conjunctive</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="nested-index/index-selection">
+      <compilation-unit name="orders-index-custkey-conjunctive-open">
+        <output-dir compare="Text">orders-index-custkey-conjunctive-open</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="nested-index/index-selection">
+      <compilation-unit name="orders-index-custkey-open">
+        <output-dir compare="Text">orders-index-custkey-open</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="nested-index/index-selection">
+      <compilation-unit name="range-search">
+        <output-dir compare="Text">range-search</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="nested-index/index-selection">
+      <compilation-unit name="range-search-open">
+        <output-dir compare="Text">range-search-open</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="nested-index/index-selection">
+      <compilation-unit name="rtree-secondary-index-optional">
+        <output-dir compare="Text">rtree-secondary-index-open</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="nested-index-dml">
+      <compilation-unit name="compact-dataset-and-its-indexes">
+        <output-dir compare="Text">compact-dataset-and-its-indexes</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="nested-index-dml">
+      <compilation-unit name="delete-from-loaded-dataset-with-index">
+        <output-dir compare="Text">delete-from-loaded-dataset-with-index</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="nested-index-dml">
+      <compilation-unit name="insert-into-loaded-dataset-with-index_01">
+        <output-dir compare="Text">insert-into-loaded-dataset-with-index_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="nested-index-dml">
+      <compilation-unit name="insert-into-loaded-dataset-with-index_02">
+        <output-dir compare="Text">insert-into-loaded-dataset-with-index_02</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="nested-index-dml">
+      <compilation-unit name="load-with-ngram-index">
+        <output-dir compare="Text">load-with-ngram-index</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="nested-index-dml">
+      <compilation-unit name="load-with-rtree-index">
+        <output-dir compare="Text">load-with-rtree-index</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="nested-index-dml">
+      <compilation-unit name="load-with-word-index">
+        <output-dir compare="Text">load-with-word-index</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="nested-index-dml">
+      <compilation-unit name="scan-delete-btree-secondary-index-nullable">
+        <output-dir compare="Text">scan-delete-btree-secondary-index-nullable</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="nested-index-dml">
+      <compilation-unit name="scan-delete-rtree-secondary-index-nullable">
+        <output-dir compare="Text">scan-delete-rtree-secondary-index-nullable</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="nested-index-dml">
+      <compilation-unit name="scan-delete-rtree-secondary-index">
+        <output-dir compare="Text">scan-delete-rtree-secondary-index</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="array-index/select-unnest-queries/closed">
+      <compilation-unit name="use-case-1">
+        <output-dir compare="Text">use-case-1</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="array-index/select-unnest-queries/open">
+      <compilation-unit name="use-case-1">
+        <output-dir compare="Text">use-case-1</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="array-index/select-unnest-queries/closed">
+      <compilation-unit name="use-case-2">
+        <output-dir compare="Text">use-case-2</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="array-index/select-unnest-queries/open">
+      <compilation-unit name="use-case-2">
+        <output-dir compare="Text">use-case-2</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="array-index/select-unnest-queries/closed">
+      <compilation-unit name="use-case-3">
+        <output-dir compare="Text">use-case-3</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="array-index/select-unnest-queries/open">
+      <compilation-unit name="use-case-3">
+        <output-dir compare="Text">use-case-3</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="array-index/select-unnest-queries/closed">
+      <compilation-unit name="use-case-4">
+        <output-dir compare="Text">use-case-4</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="array-index/select-unnest-queries/open">
+      <compilation-unit name="use-case-4">
+        <output-dir compare="Text">use-case-4</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="array-index/select-unnest-queries/closed">
+      <compilation-unit name="with-3-level-record-path">
+        <output-dir compare="Text">with-3-level-record-path</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="array-index/select-unnest-queries/open">
+      <compilation-unit name="with-3-level-record-path">
+        <output-dir compare="Text">with-3-level-record-path</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="array-index/select-unnest-queries/closed">
+      <compilation-unit name="with-composite-sk">
+        <output-dir compare="Text">with-composite-sk</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="array-index/select-unnest-queries/closed">
+      <compilation-unit name="with-composite-pk">
+        <output-dir compare="Text">with-composite-pk</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="array-index/select-unnest-queries/closed">
+      <compilation-unit name="with-filter-fields">
+        <output-dir compare="Text">with-filter-fields</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="array-index/select-unnest-queries/open">
+      <compilation-unit name="using-feed-new-index">
+        <output-dir compare="Text">using-feed</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="array-index/select-quantified-queries">
+      <compilation-unit name="use-case-1">
+        <output-dir compare="Text">use-case-1</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="array-index/select-quantified-queries">
+      <compilation-unit name="use-case-2">
+        <output-dir compare="Text">use-case-2</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="array-index/select-quantified-queries">
+      <compilation-unit name="use-case-3">
+        <output-dir compare="Text">use-case-3</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="array-index/select-quantified-queries">
+      <compilation-unit name="with-composite-pk">
+        <output-dir compare="Text">with-composite-pk</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="array-index/select-quantified-queries">
+      <compilation-unit name="with-open-index">
+        <output-dir compare="Text">with-open-index</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="spatial">
+      <compilation-unit name="cell-aggregation-with-filtering">
+        <output-dir compare="Text">cell-aggregation-with-filtering</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="sql-compat">
+      <compilation-unit name="outer_join_01">
+        <output-dir compare="Text">outer_join_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="statement-params">
+      <compilation-unit name="index_01">
+        <output-dir compare="Text">index_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpcds">
+      <compilation-unit name="q45">
+        <output-dir compare="Text">q45</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-with-index">
+      <compilation-unit name="q10_returned_item">
+        <output-dir compare="Text">q10_returned_item</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-with-index">
+      <compilation-unit name="q10_returned_item_int64">
+        <output-dir compare="Text">q10_returned_item_int64</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-with-index">
+      <compilation-unit name="q12_shipping">
+        <output-dir compare="Text">q12_shipping</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-with-index">
+      <compilation-unit name="q14_promotion_effect">
+        <output-dir compare="Text">q14_promotion_effect</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-with-index">
+      <compilation-unit name="q15_top_supplier">
+        <output-dir compare="Text">q15_top_supplier</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-with-index">
+      <compilation-unit name="q01_pricing_summary_report_nt">
+        <output-dir compare="Text">q01_pricing_summary_report_nt</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-with-index">
+      <compilation-unit name="q03_shipping_priority_nt">
+        <output-dir compare="Text">q03_shipping_priority_nt</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-with-index">
+      <compilation-unit name="q04_order_priority">
+        <output-dir compare="Text">q04_order_priority</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-with-index">
+      <compilation-unit name="q04_order_priority_with_nodegroup">
+        <output-dir compare="Text">q04_order_priority</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-with-index">
+      <compilation-unit name="q05_local_supplier_volume">
+        <output-dir compare="Text">q05_local_supplier_volume</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-with-index">
+      <compilation-unit name="q06_forecast_revenue_change">
+        <output-dir compare="Text">q06_forecast_revenue_change</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-with-index">
+      <compilation-unit name="q07_volume_shipping">
+        <output-dir compare="Text">q07_volume_shipping</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-with-index">
+      <compilation-unit name="q08_national_market_share">
+        <output-dir compare="Text">q08_national_market_share</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-with-index">
+      <compilation-unit name="query-issue810">
+        <output-dir compare="Text">query-issue810</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-with-index">
+      <compilation-unit name="query-issue810-2">
+        <output-dir compare="Text">query-issue810-2</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-with-index">
+      <compilation-unit name="query-issue810-3">
+        <output-dir compare="Text">query-issue810-3</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-with-index">
+      <compilation-unit name="query-issue827-2">
+        <output-dir compare="Text">query-issue827-2</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-leftouterjoin">
+      <compilation-unit name="probe-pidx-with-join-btree-sidx1">
+        <output-dir compare="Text">probe-pidx-with-join-btree-sidx1</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-leftouterjoin">
+      <compilation-unit name="probe-pidx-with-join-btree-sidx2">
+        <output-dir compare="Text">probe-pidx-with-join-btree-sidx2</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-leftouterjoin">
+      <compilation-unit name="probe-pidx-join-btree-sidx3-idxonly">
+        <output-dir compare="Text">probe-pidx-join-btree-sidx3-idxonly</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-leftouterjoin">
+      <compilation-unit name="probe-pidx-with-join-rtree-sidx2">
+        <output-dir compare="Text">probe-pidx-with-join-rtree-sidx2</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-leftouterjoin">
+      <compilation-unit name="probe-pidx-with-join-invidx-sidx1">
+        <output-dir compare="Text">probe-pidx-with-join-invidx-sidx1</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-leftouterjoin">
+      <compilation-unit name="probe-pidx-with-join-invidx-sidx2">
+        <output-dir compare="Text">probe-pidx-with-join-invidx-sidx2</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-leftouterjoin">
+      <compilation-unit name="probe-sidx-btree-idxonly-join-btree-pidx1">
+        <output-dir compare="Text">probe-sidx-btree-idxonly-join-btree-pidx1</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-leftouterjoin">
+      <compilation-unit name="probe-sidx-btree-idxonly-join-btree-sidx1-idxonly">
+        <output-dir compare="Text">probe-sidx-btree-idxonly-join-btree-sidx1-idxonly</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-leftouterjoin">
+      <compilation-unit name="probe-sidx-btree-non-idxonly-join-btree-pidx1">
+        <output-dir compare="Text">probe-sidx-btree-non-idxonly-join-btree-pidx1</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-leftouterjoin">
+      <compilation-unit name="probe-sidx-btree-non-idxonly-join-btree-sidx1-idxonly">
+        <output-dir compare="Text">probe-sidx-btree-non-idxonly-join-btree-sidx1-idxonly</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tinysocial">
+      <compilation-unit name="tinysocial-suite">
+        <output-dir compare="Text">tinysocial-suite</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tinysocial">
+      <compilation-unit name="tinysocial-suite-open">
+        <output-dir compare="Text">tinysocial-suite-open</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="filters">
+      <compilation-unit name="load-with-secondary-btree">
+        <output-dir compare="Text">load-with-secondary-btree</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="filters">
+      <compilation-unit name="load-with-secondary-btree-index-only">
+        <output-dir compare="Text">load-with-secondary-btree-index-only</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="filters">
+      <compilation-unit name="load-with-secondary-inverted-word">
+        <output-dir compare="Text">load-with-secondary-inverted-word</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="filters">
+      <compilation-unit name="load-with-secondary-rtree">
+        <output-dir compare="Text">load-with-secondary-rtree</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="filters">
+      <compilation-unit name="insert-with-correlated-secondary-btree">
+        <output-dir compare="Text">insert-with-secondary-btree</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="union">
+      <compilation-unit name="union_opt_1">
+        <output-dir compare="Text">union_opt_1</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="upsert">
+      <compilation-unit name="primary-correlated-secondary-btree">
+        <output-dir compare="Text">primary-secondary-btree</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="limit">
+      <compilation-unit name="push-limit-to-primary-lookup">
+        <output-dir compare="Text">push-limit-to-primary-lookup</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="limit">
+      <compilation-unit name="push-limit-to-primary-lookup-select">
+        <output-dir compare="Text">push-limit-to-primary-lookup-select</output-dir>
+      </compilation-unit>
+    </test-case>
+  </test-group>
+</test-suite>
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/CompilerProperties.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/CompilerProperties.java
index 5538a0a..9599435 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/CompilerProperties.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/CompilerProperties.java
@@ -103,7 +103,11 @@ public class CompilerProperties extends AbstractProperties {
         COMPILER_ARRAYINDEX(
                 BOOLEAN,
                 AlgebricksConfig.ARRAY_INDEX_DEFAULT,
-                "Enable/disable using array-indexes in queries");
+                "Enable/disable using array-indexes in queries"),
+        COMPILER_BATCH_LOOKUP(
+                BOOLEAN,
+                AlgebricksConfig.BATCH_LOOKUP_DEFAULT,
+                "Enable/disable batch point-lookups when running queries with secondary indexes");
 
         private final IOptionType type;
         private final Object defaultValue;
@@ -173,6 +177,8 @@ public class CompilerProperties extends AbstractProperties {
 
     public static final String COMPILER_EXTERNALSCANMEMORY_KEY = Option.COMPILER_EXTERNALSCANMEMORY.ini();
 
+    public static final String COMPILER_BATCHED_LOOKUP_KEY = Option.COMPILER_BATCH_LOOKUP.ini();
+
     public static final int COMPILER_PARALLELISM_AS_STORAGE = 0;
 
     public CompilerProperties(PropertiesAccessor accessor) {
@@ -246,4 +252,8 @@ public class CompilerProperties extends AbstractProperties {
     public int getExternalScanMemorySize() {
         return accessor.getInt(Option.COMPILER_EXTERNALSCANMEMORY);
     }
+
+    public boolean isBatchLookup() {
+        return accessor.getBoolean(Option.COMPILER_BATCH_LOOKUP);
+    }
 }
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/OptimizationConfUtil.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/OptimizationConfUtil.java
index d498d69..d94fa04 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/OptimizationConfUtil.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/OptimizationConfUtil.java
@@ -77,6 +77,8 @@ public class OptimizationConfUtil {
         int externalScanBufferSize = getExternalScanBufferSize(
                 (String) querySpecificConfig.get(CompilerProperties.COMPILER_EXTERNALSCANMEMORY_KEY),
                 compilerProperties.getExternalScanMemorySize(), sourceLoc);
+        boolean batchLookup = getBoolean(querySpecificConfig, CompilerProperties.COMPILER_BATCHED_LOOKUP_KEY,
+                compilerProperties.isBatchLookup());
 
         PhysicalOptimizationConfig physOptConf = new PhysicalOptimizationConfig();
         physOptConf.setFrameSize(frameSize);
@@ -95,6 +97,7 @@ public class OptimizationConfUtil {
         physOptConf.setMinMemoryAllocation(minMemoryAllocation);
         physOptConf.setArrayIndexEnabled(arrayIndex);
         physOptConf.setExternalScanBufferSize(externalScanBufferSize);
+        physOptConf.setBatchLookup(batchLookup);
         return physOptConf;
     }
 
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/OneToOneExchangePOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/OneToOneExchangePOperator.java
index 083e4d3..2bd78b1 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/OneToOneExchangePOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/OneToOneExchangePOperator.java
@@ -27,7 +27,6 @@ import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogi
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
 import org.apache.hyracks.algebricks.core.algebra.properties.IPhysicalPropertiesVector;
 import org.apache.hyracks.algebricks.core.algebra.properties.PhysicalRequirements;
-import org.apache.hyracks.algebricks.core.algebra.properties.StructuralPropertiesVector;
 import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenContext;
 import org.apache.hyracks.api.dataflow.IConnectorDescriptor;
 import org.apache.hyracks.api.job.IConnectorDescriptorRegistry;
@@ -43,7 +42,9 @@ public class OneToOneExchangePOperator extends AbstractExchangePOperator {
     @Override
     public void computeDeliveredProperties(ILogicalOperator op, IOptimizationContext context) {
         AbstractLogicalOperator op2 = (AbstractLogicalOperator) op.getInputs().get(0).getValue();
-        deliveredProperties = (StructuralPropertiesVector) op2.getDeliveredPhysicalProperties().clone();
+        if (op2.getDeliveredPhysicalProperties() != null) {
+            deliveredProperties = op2.getDeliveredPhysicalProperties().clone();
+        }
     }
 
     @Override
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/config/AlgebricksConfig.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/config/AlgebricksConfig.java
index 86c6066..b626ae6 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/config/AlgebricksConfig.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/config/AlgebricksConfig.java
@@ -37,4 +37,5 @@ public class AlgebricksConfig {
     public static final boolean ARRAY_INDEX_DEFAULT = true;
     public static final int EXTERNAL_SCAN_BUFFER_SIZE =
             StorageUtil.getIntSizeInBytes(8, StorageUtil.StorageUnit.KILOBYTE);
+    public static final boolean BATCH_LOOKUP_DEFAULT = false;
 }
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/PhysicalOptimizationConfig.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/PhysicalOptimizationConfig.java
index 9c9b21d..4d54a10 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/PhysicalOptimizationConfig.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/PhysicalOptimizationConfig.java
@@ -47,6 +47,7 @@ public class PhysicalOptimizationConfig {
     private static final String MIN_MEMORY_ALLOCATION = "MIN_MEMORY_ALLOCATION";
     private static final String ARRAY_INDEX = "ARRAY_INDEX";
     private static final String EXTERNAL_SCAN_BUFFER_SIZE = "EXTERNAL_SCAN_BUFFER_SIZE";
+    private static final String BATCH_LOOKUP = "BATCH_LOOKUP";
 
     private final Properties properties = new Properties();
 
@@ -239,6 +240,14 @@ public class PhysicalOptimizationConfig {
         return getInt(EXTERNAL_SCAN_BUFFER_SIZE, AlgebricksConfig.EXTERNAL_SCAN_BUFFER_SIZE);
     }
 
+    public boolean isBatchLookupEnabled() {
+        return getBoolean(BATCH_LOOKUP, AlgebricksConfig.BATCH_LOOKUP_DEFAULT);
+    }
+
+    public void setBatchLookup(boolean batchedLookup) {
+        setBoolean(BATCH_LOOKUP, batchedLookup);
+    }
+
     public void setExternalScanBufferSize(int bufferSize) {
         setInt(EXTERNAL_SCAN_BUFFER_SIZE, bufferSize);
     }
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/IsolateHyracksOperatorsRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/IsolateHyracksOperatorsRule.java
index e13ec30..129aba3 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/IsolateHyracksOperatorsRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/IsolateHyracksOperatorsRule.java
@@ -132,6 +132,7 @@ public class IsolateHyracksOperatorsRule implements IAlgebraicRewriteRule {
         e.setExecutionMode(inOp.getExecutionMode());
         context.computeAndSetTypeEnvironmentForOperator(e);
         e.recomputeSchema();
+        e.computeDeliveredPhysicalProperties(context);
 
         inOpRef.setValue(e);
     }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/api/IDiskBTreeStatefulPointSearchCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/api/IDiskBTreeStatefulPointSearchCursor.java
new file mode 100644
index 0000000..f436636
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/api/IDiskBTreeStatefulPointSearchCursor.java
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.storage.am.btree.api;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.btree.impls.DiskBTreePointSearchCursor;
+import org.apache.hyracks.storage.am.common.api.ITreeIndexFrame;
+import org.apache.hyracks.storage.common.ISearchPredicate;
+
+/**
+ * Allows stateful {@link DiskBTreePointSearchCursor} to resume the search without closing and reopening the cursor
+ * Note: upon closing a cursor, ensure the search state is cleared
+ */
+public interface IDiskBTreeStatefulPointSearchCursor {
+    int getLastPageId();
+
+    void setCursorToNextKey(ISearchPredicate searchPred) throws HyracksDataException;
+
+    ITreeIndexFrame getFrame();
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/DiskBTree.java b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/DiskBTree.java
index ae6bbaa..179f1da 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/DiskBTree.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/DiskBTree.java
@@ -24,8 +24,8 @@ import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.io.FileReference;
 import org.apache.hyracks.api.util.HyracksConstants;
 import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
-import org.apache.hyracks.storage.am.btree.api.IBTreeFrame;
 import org.apache.hyracks.storage.am.btree.api.IBTreeLeafFrame;
+import org.apache.hyracks.storage.am.btree.api.IDiskBTreeStatefulPointSearchCursor;
 import org.apache.hyracks.storage.am.common.api.IPageManager;
 import org.apache.hyracks.storage.am.common.api.ITreeIndexCursor;
 import org.apache.hyracks.storage.am.common.api.ITreeIndexFrame;
@@ -38,7 +38,6 @@ import org.apache.hyracks.storage.common.IIndexCursorStats;
 import org.apache.hyracks.storage.common.ISearchPredicate;
 import org.apache.hyracks.storage.common.MultiComparator;
 import org.apache.hyracks.storage.common.NoOpIndexCursorStats;
-import org.apache.hyracks.storage.common.buffercache.BufferCache;
 import org.apache.hyracks.storage.common.buffercache.IBufferCache;
 import org.apache.hyracks.storage.common.buffercache.ICachedPage;
 import org.apache.hyracks.storage.common.file.BufferedFileHandle;
@@ -76,7 +75,8 @@ public class DiskBTree extends BTree {
     private void search(ITreeIndexCursor cursor, ISearchPredicate searchPred, BTreeOpContext ctx)
             throws HyracksDataException {
         ctx.reset();
-        ctx.setPred((RangePredicate) searchPred);
+        RangePredicate rangePredicate = (RangePredicate) searchPred;
+        ctx.setPred(rangePredicate);
         ctx.setCursor(cursor);
         if (ctx.getPred().getLowKeyComparator() == null) {
             ctx.getPred().setLowKeyComparator(ctx.getCmp());
@@ -87,25 +87,17 @@ public class DiskBTree extends BTree {
         cursor.setBufferCache(bufferCache);
         cursor.setFileId(getFileId());
 
-        if (cursor instanceof DiskBTreePointSearchCursor) {
-            DiskBTreePointSearchCursor pointCursor = (DiskBTreePointSearchCursor) cursor;
+        if (cursor instanceof IDiskBTreeStatefulPointSearchCursor) {
+            IDiskBTreeStatefulPointSearchCursor pointCursor = (IDiskBTreeStatefulPointSearchCursor) cursor;
             int lastPageId = pointCursor.getLastPageId();
-            if (lastPageId != BufferCache.INVALID_PAGEID) {
-                // check whether the last leaf page contains this key
-                ICachedPage lastPage =
-                        bufferCache.pin(BufferedFileHandle.getDiskPageId(getFileId(), lastPageId), false);
-                ctx.getLeafFrame().setPage(lastPage);
-                if (fitInPage(ctx.getPred().getLowKey(), ctx.getPred().getLowKeyComparator(), ctx.getLeafFrame())) {
-                    // use this page
-                    ctx.getCursorInitialState().setPage(lastPage);
-                    ctx.getCursorInitialState().setPageId(lastPageId);
-                    pointCursor.open(ctx.getCursorInitialState(), searchPred);
+            if (lastPageId != IBufferCache.INVALID_PAGEID) {
+                if (fitInPage(ctx.getPred().getLowKey(), ctx.getPred().getLowKeyComparator(), pointCursor.getFrame())) {
+                    pointCursor.setCursorToNextKey(searchPred);
                     return;
                 } else {
-                    // release the last page and clear the states of this cursor
+                    // release the last page, clear the states of this cursor, and close the cursor
                     // then retry the search from root to leaf
-                    bufferCache.unpin(lastPage);
-                    pointCursor.clearSearchState();
+                    cursor.close();
                 }
             }
         }
@@ -113,7 +105,7 @@ public class DiskBTree extends BTree {
         searchDown(rootNode, rootPage, ctx, cursor);
     }
 
-    private boolean fitInPage(ITupleReference key, MultiComparator comparator, IBTreeFrame frame)
+    private boolean fitInPage(ITupleReference key, MultiComparator comparator, ITreeIndexFrame frame)
             throws HyracksDataException {
         // assume that search keys are sorted (non-decreasing)
         ITupleReference rightmostTuple = frame.getRightmostTuple();
@@ -191,14 +183,14 @@ public class DiskBTree extends BTree {
         }
 
         @Override
-        public DiskBTreeRangeSearchCursor createSearchCursor(boolean exclusive) {
+        public ITreeIndexCursor createSearchCursor(boolean exclusive) {
             IBTreeLeafFrame leafFrame = (IBTreeLeafFrame) btree.getLeafFrameFactory().createFrame();
             return new DiskBTreeRangeSearchCursor(leafFrame, exclusive, (IIndexCursorStats) iap.getParameters()
                     .getOrDefault(HyracksConstants.INDEX_CURSOR_STATS, NoOpIndexCursorStats.INSTANCE));
         }
 
         @Override
-        public BTreeRangeSearchCursor createPointCursor(boolean exclusive, boolean stateful) {
+        public ITreeIndexCursor createPointCursor(boolean exclusive, boolean stateful) {
             IBTreeLeafFrame leafFrame = (IBTreeLeafFrame) btree.getLeafFrameFactory().createFrame();
             return new DiskBTreePointSearchCursor(leafFrame, exclusive, stateful);
         }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/DiskBTreePointSearchCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/DiskBTreePointSearchCursor.java
index 1bf3ecf..8fd9a96 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/DiskBTreePointSearchCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/DiskBTreePointSearchCursor.java
@@ -21,13 +21,16 @@ package org.apache.hyracks.storage.am.btree.impls;
 
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.storage.am.btree.api.IBTreeLeafFrame;
+import org.apache.hyracks.storage.am.btree.api.IDiskBTreeStatefulPointSearchCursor;
+import org.apache.hyracks.storage.am.common.api.ITreeIndexFrame;
 import org.apache.hyracks.storage.am.common.ophelpers.FindTupleMode;
 import org.apache.hyracks.storage.am.common.ophelpers.FindTupleNoExactMatchPolicy;
 import org.apache.hyracks.storage.common.ICursorInitialState;
 import org.apache.hyracks.storage.common.ISearchPredicate;
-import org.apache.hyracks.storage.common.buffercache.BufferCache;
+import org.apache.hyracks.storage.common.buffercache.IBufferCache;
 
-public class DiskBTreePointSearchCursor extends DiskBTreeRangeSearchCursor {
+public class DiskBTreePointSearchCursor extends DiskBTreeRangeSearchCursor
+        implements IDiskBTreeStatefulPointSearchCursor {
     /**
      * A stateful cursor keeps the search state (last search page Id + index) across multiple searches
      * until {@link #clearSearchState()} is called explicity
@@ -36,16 +39,14 @@ public class DiskBTreePointSearchCursor extends DiskBTreeRangeSearchCursor {
 
     private boolean nextHasBeenCalled;
 
-    private int lastPageId = BufferCache.INVALID_PAGEID;
+    private int lastPageId;
     private int lastTupleIndex = 0;
 
     public DiskBTreePointSearchCursor(IBTreeLeafFrame frame, boolean exclusiveLatchNodes, boolean stateful) {
         super(frame, exclusiveLatchNodes);
         this.stateful = stateful;
-    }
-
-    public DiskBTreePointSearchCursor(IBTreeLeafFrame frame, boolean exclusiveLatchNodes) {
-        this(frame, exclusiveLatchNodes, false);
+        lastPageId = IBufferCache.INVALID_PAGEID;
+        lastTupleIndex = 0;
     }
 
     @Override
@@ -59,6 +60,12 @@ public class DiskBTreePointSearchCursor extends DiskBTreeRangeSearchCursor {
     }
 
     @Override
+    public void doClose() throws HyracksDataException {
+        clearSearchState();
+        super.doClose();
+    }
+
+    @Override
     public void doOpen(ICursorInitialState initialState, ISearchPredicate searchPred) throws HyracksDataException {
         // in case open is called multiple times without closing
         if (page != null) {
@@ -71,7 +78,25 @@ public class DiskBTreePointSearchCursor extends DiskBTreeRangeSearchCursor {
         page = initialState.getPage();
         isPageDirty = false;
         frame.setPage(page);
+        setCursorToNextKey(searchPred);
+    }
+
+    @Override
+    public int getLastPageId() {
+        return lastPageId;
+    }
+
+    @Override
+    protected int getLowKeyIndex() throws HyracksDataException {
+        if (stateful) {
+            return frame.findTupleIndex(lowKey, frameTuple, lowKeyCmp, lastTupleIndex);
+        } else {
+            return super.getLowKeyIndex();
+        }
+    }
 
+    @Override
+    public void setCursorToNextKey(ISearchPredicate searchPred) throws HyracksDataException {
         pred = (RangePredicate) searchPred;
         lowKeyCmp = pred.getLowKeyComparator();
         lowKey = pred.getLowKey();
@@ -83,7 +108,7 @@ public class DiskBTreePointSearchCursor extends DiskBTreeRangeSearchCursor {
 
         nextHasBeenCalled = false;
 
-        // only get the low key position
+        // only get the lowKey position
         tupleIndex = getLowKeyIndex();
         if (stateful) {
             lastPageId = pageId;
@@ -95,22 +120,13 @@ public class DiskBTreePointSearchCursor extends DiskBTreeRangeSearchCursor {
         }
     }
 
-    public int getLastPageId() {
-        return lastPageId;
+    private void clearSearchState() {
+        this.lastPageId = IBufferCache.INVALID_PAGEID;
+        this.lastTupleIndex = 0;
     }
 
     @Override
-    protected int getLowKeyIndex() throws HyracksDataException {
-        if (stateful) {
-            return frame.findTupleIndex(lowKey, frameTuple, lowKeyCmp, lastTupleIndex);
-        } else {
-            return super.getLowKeyIndex();
-        }
+    public ITreeIndexFrame getFrame() {
+        return frame;
     }
-
-    public void clearSearchState() {
-        this.lastPageId = BufferCache.INVALID_PAGEID;
-        this.lastTupleIndex = 0;
-    }
-
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeBatchPointSearchCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeBatchPointSearchCursor.java
index 8ab6fb1..73d06d2 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeBatchPointSearchCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeBatchPointSearchCursor.java
@@ -21,14 +21,15 @@ package org.apache.hyracks.storage.am.lsm.btree.impls;
 
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+import org.apache.hyracks.storage.am.btree.impls.BTree.BTreeAccessor;
 import org.apache.hyracks.storage.am.btree.impls.BatchPredicate;
-import org.apache.hyracks.storage.am.btree.impls.DiskBTreePointSearchCursor;
+import org.apache.hyracks.storage.am.common.api.ITreeIndexCursor;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent.LSMComponentType;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
 
 /**
  * This cursor performs point searches for each batch of search keys.
  * Assumption: the search keys must be sorted into the increasing order.
- *
  */
 public class LSMBTreeBatchPointSearchCursor extends LSMBTreePointSearchCursor {
 
@@ -42,7 +43,9 @@ public class LSMBTreeBatchPointSearchCursor extends LSMBTreePointSearchCursor {
         while (!foundTuple && batchPred.hasNext()) {
             batchPred.next();
             if (foundIn >= 0) {
-                btreeCursors[foundIn].close();
+                if (operationalComponents.get(foundIn).getType() == LSMComponentType.MEMORY) {
+                    btreeCursors[foundIn].close();
+                }
                 foundIn = -1;
             }
             foundTuple = super.doHasNext();
@@ -56,6 +59,11 @@ public class LSMBTreeBatchPointSearchCursor extends LSMBTreePointSearchCursor {
     }
 
     @Override
+    protected ITreeIndexCursor createCursor(LSMComponentType type, BTreeAccessor btreeAccessor) {
+        return btreeAccessor.createPointCursor(false, true);
+    }
+
+    @Override
     protected boolean isSearchCandidate(int componentIndex) throws HyracksDataException {
         if (!super.isSearchCandidate(componentIndex)) {
             return false;
@@ -68,21 +76,6 @@ public class LSMBTreeBatchPointSearchCursor extends LSMBTreePointSearchCursor {
                 maxFileterKey, opCtx.getFilterCmp());
     }
 
-    @Override
-    protected void closeCursors() throws HyracksDataException {
-        super.closeCursors();
-        if (btreeCursors != null) {
-            // clear search states of btree cursors
-            for (int i = 0; i < numBTrees; ++i) {
-                if (btreeCursors[i] != null) {
-                    if (btreeCursors[i] instanceof DiskBTreePointSearchCursor) {
-                        ((DiskBTreePointSearchCursor) btreeCursors[i]).clearSearchState();
-                    }
-                }
-            }
-        }
-    }
-
     public int getKeyIndex() {
         return ((BatchPredicate) predicate).getKeyIndex();
     }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreePointSearchCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreePointSearchCursor.java
index 7bb96c3..9740bc4 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreePointSearchCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreePointSearchCursor.java
@@ -231,7 +231,7 @@ public class LSMBTreePointSearchCursor extends EnforcedIndexCursor implements IL
     }
 
     protected ITreeIndexCursor createCursor(LSMComponentType type, BTreeAccessor btreeAccessor) {
-        return btreeAccessor.createPointCursor(false, type == LSMComponentType.DISK);
+        return btreeAccessor.createPointCursor(false, false);
     }
 
     private void destroyAndNullifyCursorAtIndex(int i) throws HyracksDataException {